From 6b07c851f3f33688f0998bcee3de615901a3df2d Mon Sep 17 00:00:00 2001 From: litao Date: Fri, 25 Feb 2022 00:23:20 +0800 Subject: [PATCH 001/145] HDFS-16397. Reconfig slow disk parameters for datanode (#3828) --- .../hdfs/server/datanode/BPServiceActor.java | 20 ++- .../hadoop/hdfs/server/datanode/DNConf.java | 15 ++- .../hadoop/hdfs/server/datanode/DataNode.java | 81 +++++++++++- .../hdfs/server/datanode/FileIoProvider.java | 4 + .../datanode/ProfilingFileIoEvents.java | 37 ++++-- .../datanode/metrics/DataNodeDiskMetrics.java | 35 +++++- .../datanode/TestDataNodeReconfiguration.java | 119 ++++++++++++++++++ .../hadoop/hdfs/tools/TestDFSAdmin.java | 2 +- 8 files changed, 290 insertions(+), 23 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index bf284d1e1dd96..6199626fd2ef6 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hdfs.server.datanode; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY; import static org.apache.hadoop.util.Time.monotonicNow; import java.io.Closeable; @@ -556,7 +558,7 @@ HeartbeatResponse sendHeartBeat(boolean requestBlockReportLease) SlowPeerReports.create(dn.getPeerMetrics().getOutliers()) : SlowPeerReports.EMPTY_REPORT; final SlowDiskReports slowDisks = - outliersReportDue && dn.getDiskMetrics() != null ? + outliersReportDue && dnConf.diskStatsEnabled && dn.getDiskMetrics() != null ? SlowDiskReports.create(dn.getDiskMetrics().getDiskOutliersStats()) : SlowDiskReports.EMPTY_REPORT; @@ -1195,7 +1197,7 @@ static class Scheduler { private final long heartbeatIntervalMs; private final long lifelineIntervalMs; private volatile long blockReportIntervalMs; - private final long outliersReportIntervalMs; + private volatile long outliersReportIntervalMs; Scheduler(long heartbeatIntervalMs, long lifelineIntervalMs, long blockReportIntervalMs, long outliersReportIntervalMs) { @@ -1356,10 +1358,22 @@ long getBlockReportIntervalMs() { } void setBlockReportIntervalMs(long intervalMs) { - Preconditions.checkArgument(intervalMs > 0); + Preconditions.checkArgument(intervalMs > 0, + DFS_BLOCKREPORT_INTERVAL_MSEC_KEY + " should be larger than 0"); this.blockReportIntervalMs = intervalMs; } + void setOutliersReportIntervalMs(long intervalMs) { + Preconditions.checkArgument(intervalMs > 0, + DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY + " should be larger than 0"); + this.outliersReportIntervalMs = intervalMs; + } + + @VisibleForTesting + long getOutliersReportIntervalMs() { + return this.outliersReportIntervalMs; + } + /** * Wrapped for testing. * @return diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java index 563fbde2719de..9b5343321d30b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java @@ -109,8 +109,8 @@ public class DNConf { volatile long blockReportInterval; volatile long blockReportSplitThreshold; volatile boolean peerStatsEnabled; - final boolean diskStatsEnabled; - final long outliersReportIntervalMs; + volatile boolean diskStatsEnabled; + volatile long outliersReportIntervalMs; final long ibrInterval; volatile long initialBlockReportDelayMs; volatile long cacheReportInterval; @@ -511,4 +511,15 @@ void setInitBRDelayMs(String delayMs) { void setPeerStatsEnabled(boolean enablePeerStats) { peerStatsEnabled = enablePeerStats; } + + public void setFileIoProfilingSamplingPercentage(int samplingPercentage) { + diskStatsEnabled = Util.isDiskStatsEnabled(samplingPercentage); + } + + public void setOutliersReportIntervalMs(String reportIntervalMs) { + dn.getConf().set(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, reportIntervalMs); + outliersReportIntervalMs = getConf().getTimeDuration( + DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, + DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 6087d8d753cf1..84382275bd965 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -35,6 +35,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_INTERFACE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY; @@ -48,6 +50,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_RECEIVER_THREADS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_NETWORK_COUNTS_CACHE_MAX_SIZE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OOB_TIMEOUT_DEFAULT; @@ -59,6 +63,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PLUGINS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT; @@ -70,6 +78,7 @@ import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_CREATE; import static org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY; import static org.apache.hadoop.util.ExitUtil.terminate; +import static org.apache.hadoop.util.Preconditions.checkNotNull; import static org.apache.hadoop.util.Time.now; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; @@ -327,7 +336,11 @@ public class DataNode extends ReconfigurableBase DFS_DATANODE_PEER_STATS_ENABLED_KEY, DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY, DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY, - DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY)); + DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY, + DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, + DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, + DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, + DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY)); public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog"); @@ -370,7 +383,7 @@ public static InetSocketAddress createSocketAddr(String target) { DataNodeMetrics metrics; @Nullable private volatile DataNodePeerMetrics peerMetrics; - private DataNodeDiskMetrics diskMetrics; + private volatile DataNodeDiskMetrics diskMetrics; private InetSocketAddress streamingAddr; private LoadingCache> datanodeNetworkCounts; @@ -651,6 +664,11 @@ public String reconfigurePropertyImpl(String property, String newVal) case DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY: case DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY: return reconfSlowPeerParameters(property, newVal); + case DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY: + case DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY: + case DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY: + case DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY: + return reconfSlowDiskParameters(property, newVal); default: break; } @@ -777,6 +795,61 @@ private String reconfSlowPeerParameters(String property, String newVal) } } + private String reconfSlowDiskParameters(String property, String newVal) + throws ReconfigurationException { + String result = null; + try { + LOG.info("Reconfiguring {} to {}", property, newVal); + if (property.equals(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY)) { + checkNotNull(dnConf, "DNConf has not been initialized."); + String reportInterval = (newVal == null ? DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_DEFAULT : + newVal); + result = reportInterval; + dnConf.setOutliersReportIntervalMs(reportInterval); + for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) { + if (bpos != null) { + for (BPServiceActor actor : bpos.getBPServiceActors()) { + actor.getScheduler().setOutliersReportIntervalMs( + dnConf.outliersReportIntervalMs); + } + } + } + } else if (property.equals(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY)) { + checkNotNull(dnConf, "DNConf has not been initialized."); + int samplingPercentage = (newVal == null ? + DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_DEFAULT : + Integer.parseInt(newVal)); + result = Integer.toString(samplingPercentage); + dnConf.setFileIoProfilingSamplingPercentage(samplingPercentage); + if (fileIoProvider != null) { + fileIoProvider.getProfilingEventHook().setSampleRangeMax(samplingPercentage); + } + if (samplingPercentage > 0 && diskMetrics == null) { + diskMetrics = new DataNodeDiskMetrics(this, + dnConf.outliersReportIntervalMs, getConf()); + } else if (samplingPercentage <= 0 && diskMetrics != null) { + diskMetrics.shutdownAndWait(); + } + } else if (property.equals(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY)) { + checkNotNull(diskMetrics, "DataNode disk stats may be disabled."); + long minDisks = (newVal == null ? DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_DEFAULT : + Long.parseLong(newVal)); + result = Long.toString(minDisks); + diskMetrics.setMinOutlierDetectionDisks(minDisks); + } else if (property.equals(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY)) { + checkNotNull(diskMetrics, "DataNode disk stats may be disabled."); + long threshold = (newVal == null ? DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_DEFAULT : + Long.parseLong(newVal)); + result = Long.toString(threshold); + diskMetrics.setLowThresholdMs(threshold); + } + LOG.info("RECONFIGURE* changed {} to {}", property, newVal); + return result; + } catch (IllegalArgumentException e) { + throw new ReconfigurationException(property, newVal, getConf().get(property), e); + } + } + /** * Get a list of the keys of the re-configurable properties in configuration. */ @@ -2372,7 +2445,7 @@ public void shutdown() { if (metrics != null) { metrics.shutdown(); } - if (diskMetrics != null) { + if (dnConf.diskStatsEnabled && diskMetrics != null) { diskMetrics.shutdownAndWait(); } if (dataNodeInfoBeanName != null) { @@ -3942,7 +4015,7 @@ public String getSendPacketDownstreamAvgInfo() { @Override // DataNodeMXBean public String getSlowDisks() { - if (diskMetrics == null) { + if (!dnConf.diskStatsEnabled || diskMetrics == null) { //Disk Stats not enabled return null; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java index 7d6435ea3f2c7..552f5199f1d3a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java @@ -1071,4 +1071,8 @@ private void onFailure(@Nullable FsVolumeSpi volume, long begin) { } profilingEventHook.onFailure(volume, begin); } + + public ProfilingFileIoEvents getProfilingEventHook() { + return profilingEventHook; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProfilingFileIoEvents.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProfilingFileIoEvents.java index e97873341e423..c22401b645f14 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProfilingFileIoEvents.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ProfilingFileIoEvents.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.datanode; +import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -40,8 +41,8 @@ class ProfilingFileIoEvents { static final Logger LOG = LoggerFactory.getLogger(ProfilingFileIoEvents.class); - private final boolean isEnabled; - private final int sampleRangeMax; + private volatile boolean isEnabled; + private volatile int sampleRangeMax; public ProfilingFileIoEvents(@Nullable Configuration conf) { if (conf != null) { @@ -49,15 +50,7 @@ public ProfilingFileIoEvents(@Nullable Configuration conf) { DFSConfigKeys.DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, DFSConfigKeys .DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_DEFAULT); - isEnabled = Util.isDiskStatsEnabled(fileIOSamplingPercentage); - if (fileIOSamplingPercentage > 100) { - LOG.warn(DFSConfigKeys - .DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY + - " value cannot be more than 100. Setting value to 100"); - fileIOSamplingPercentage = 100; - } - sampleRangeMax = (int) ((double) fileIOSamplingPercentage / 100 * - Integer.MAX_VALUE); + setSampleRangeMax(fileIOSamplingPercentage); } else { isEnabled = false; sampleRangeMax = 0; @@ -145,4 +138,26 @@ private DataNodeVolumeMetrics getVolumeMetrics(final FsVolumeSpi volume) { } return null; } + + public void setSampleRangeMax(int fileIOSamplingPercentage) { + isEnabled = Util.isDiskStatsEnabled(fileIOSamplingPercentage); + if (fileIOSamplingPercentage > 100) { + LOG.warn(DFSConfigKeys + .DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY + + " value cannot be more than 100. Setting value to 100"); + fileIOSamplingPercentage = 100; + } + sampleRangeMax = (int) ((double) fileIOSamplingPercentage / 100 * + Integer.MAX_VALUE); + } + + @VisibleForTesting + public boolean getDiskStatsEnabled() { + return isEnabled; + } + + @VisibleForTesting + public int getSampleRangeMax() { + return sampleRangeMax; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java index 9c2151c775be5..409084cfe8be8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeDiskMetrics.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports.DiskOp; import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,6 +43,9 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY; + /** * This class detects and maintains DataNode disk outliers and their * latencies for different ops (metadata, read, write). @@ -68,11 +72,11 @@ public class DataNodeDiskMetrics { /** * Minimum number of disks to run outlier detection. */ - private final long minOutlierDetectionDisks; + private volatile long minOutlierDetectionDisks; /** * Threshold in milliseconds below which a disk is definitely not slow. */ - private final long lowThresholdMs; + private volatile long lowThresholdMs; /** * The number of slow disks that needs to be excluded. */ @@ -269,4 +273,31 @@ public void addSlowDiskForTesting(String slowDiskPath, public List getSlowDisksToExclude() { return slowDisksToExclude; } + + public void setLowThresholdMs(long thresholdMs) { + Preconditions.checkArgument(thresholdMs > 0, + DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY + " should be larger than 0"); + lowThresholdMs = thresholdMs; + this.slowDiskDetector.setLowThresholdMs(thresholdMs); + } + + public long getLowThresholdMs() { + return lowThresholdMs; + } + + public void setMinOutlierDetectionDisks(long minDisks) { + Preconditions.checkArgument(minDisks > 0, + DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY + " should be larger than 0"); + minOutlierDetectionDisks = minDisks; + this.slowDiskDetector.setMinNumResources(minDisks); + } + + public long getMinOutlierDetectionDisks() { + return minOutlierDetectionDisks; + } + + @VisibleForTesting + public OutlierDetector getSlowDiskDetector() { + return this.slowDiskDetector; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java index 2150ea0561ce6..1a9d6024acd0f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java @@ -28,12 +28,18 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_RECEIVER_THREADS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_RECEIVER_THREADS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PEER_METRICS_MIN_OUTLIER_DETECTION_SAMPLES_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; @@ -554,4 +560,117 @@ public void testSlowPeerParameters() throws Exception { DFS_DATANODE_SLOWPEER_LOW_THRESHOLD_MS_DEFAULT); } } + + @Test + public void testSlowDiskParameters() throws ReconfigurationException, IOException { + String[] slowDisksParameters1 = { + DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, + DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY}; + + for (int i = 0; i < NUM_DATA_NODE; i++) { + DataNode dn = cluster.getDataNodes().get(i); + + // Try invalid values. + try { + dn.reconfigureProperty(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, "text"); + } catch (ReconfigurationException expected) { + assertTrue("expecting NumberFormatException", + expected.getCause() instanceof NumberFormatException); + } + + try { + dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, "text"); + } catch (ReconfigurationException expected) { + assertTrue("expecting NumberFormatException", + expected.getCause() instanceof NumberFormatException); + } + + // Enable disk stats, make DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY > 0. + dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, "1"); + for (String parameter : slowDisksParameters1) { + try { + dn.reconfigureProperty(parameter, "text"); + fail("ReconfigurationException expected"); + } catch (ReconfigurationException expected) { + assertTrue("expecting NumberFormatException", + expected.getCause() instanceof NumberFormatException); + } + + try { + dn.reconfigureProperty(parameter, String.valueOf(-1)); + fail("ReconfigurationException expected"); + } catch (ReconfigurationException expected) { + assertTrue("expecting IllegalArgumentException", + expected.getCause() instanceof IllegalArgumentException); + } + } + + // Change and verify properties. + dn.reconfigureProperty(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, "1ms"); + assertEquals(1, dn.getDnConf().outliersReportIntervalMs); + + BlockPoolManager blockPoolManager = new BlockPoolManager(dn); + blockPoolManager.refreshNamenodes(dn.getConf()); + for (BPOfferService bpos : blockPoolManager.getAllNamenodeThreads()) { + if (bpos != null) { + for (BPServiceActor actor : bpos.getBPServiceActors()) { + assertEquals(String.format("%s has wrong value", + DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY), + 1, actor.getScheduler().getOutliersReportIntervalMs()); + } + } + } + + String[] slowDisksParameters2 = { + DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, + DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, + DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY}; + for (String parameter : slowDisksParameters2) { + dn.reconfigureProperty(parameter, "99"); + } + // Assert diskMetrics. + assertEquals(99, dn.getDiskMetrics().getMinOutlierDetectionDisks()); + assertEquals(99, dn.getDiskMetrics().getLowThresholdMs()); + // Assert dnConf. + assertTrue(dn.getDnConf().diskStatsEnabled); + // Assert profilingEventHook. + assertTrue(dn.getFileIoProvider().getProfilingEventHook().getDiskStatsEnabled()); + assertEquals((int) ((double) 99 / 100 * Integer.MAX_VALUE), + dn.getFileIoProvider().getProfilingEventHook().getSampleRangeMax()); + // Assert slowDiskDetector. + assertEquals(99, + dn.getDiskMetrics().getSlowDiskDetector().getMinOutlierDetectionNodes()); + assertEquals(99, + dn.getDiskMetrics().getSlowDiskDetector().getLowThresholdMs()); + + // Revert to default and verify. + dn.reconfigureProperty(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, null); + assertEquals(String.format("expect %s is not configured", + DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY), null, + dn.getConf().get(DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY)); + + dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, null); + assertEquals(String.format("expect %s is not configured", + DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY), null, + dn.getConf().get(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY)); + assertFalse(dn.getFileIoProvider().getProfilingEventHook().getDiskStatsEnabled()); + assertEquals(0, + dn.getFileIoProvider().getProfilingEventHook().getSampleRangeMax()); + + // Enable disk stats, make DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY > 0. + dn.reconfigureProperty(DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, "1"); + dn.reconfigureProperty(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, null); + dn.reconfigureProperty(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY, null); + assertEquals(String.format("expect %s is not configured", + DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY), null, + dn.getConf().get(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY)); + assertEquals(String.format("expect %s is not configured", + DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY), null, + dn.getConf().get(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY)); + assertEquals(DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_DEFAULT, + dn.getDiskMetrics().getSlowDiskDetector().getMinOutlierDetectionNodes()); + assertEquals(DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_DEFAULT, + dn.getDiskMetrics().getSlowDiskDetector().getLowThresholdMs()); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java index 19863de09d6c9..78598760df42c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java @@ -338,7 +338,7 @@ public void testDataNodeGetReconfigurableProperties() throws IOException { final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("datanode", address, outs, errs); - assertEquals(12, outs.size()); + assertEquals(16, outs.size()); assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, outs.get(1)); } From 12fa38d546e2de84fb89bee60aea9345ad35b46a Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Wed, 23 Feb 2022 16:17:53 -0800 Subject: [PATCH 002/145] HADOOP-18139: Allow configuration of zookeeper server principal. Fixes #4024 Signed-off-by: Owen O'Malley --- .../hadoop/fs/CommonConfigurationKeys.java | 2 ++ .../ZKDelegationTokenSecretManager.java | 5 ++++ .../hadoop/util/curator/ZKCuratorManager.java | 29 +++++++++++++++++++ 3 files changed, 36 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 6949c67f278d1..db82498b3c15e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -399,6 +399,8 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { public static final String ZK_ACL_DEFAULT = "world:anyone:rwcda"; /** Authentication for the ZooKeeper ensemble. */ public static final String ZK_AUTH = ZK_PREFIX + "auth"; + /** Principal name for zookeeper servers. */ + public static final String ZK_SERVER_PRINCIPAL = ZK_PREFIX + "server.principal"; /** Address of the ZooKeeper ensemble. */ public static final String ZK_ADDRESS = ZK_PREFIX + "address"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java index 5e5ea8cebc6f7..452565676a028 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java @@ -55,6 +55,7 @@ import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager; import static org.apache.hadoop.util.Time.now; +import org.apache.hadoop.util.curator.ZKCuratorManager; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; @@ -98,6 +99,8 @@ public abstract class ZKDelegationTokenSecretManager authInfos) throws IOException { CuratorFramework client = CuratorFrameworkFactory.builder() .connectString(zkHostPort) + .zookeeperFactory(new HadoopZookeeperFactory( + conf.get(CommonConfigurationKeys.ZK_SERVER_PRINCIPAL))) .sessionTimeoutMs(zkSessionTimeout) .retryPolicy(retryPolicy) .authorization(authInfos) @@ -428,4 +434,27 @@ public void setData(String path, byte[] data, int version) .forPath(path, data)); } } + + public static class HadoopZookeeperFactory implements ZookeeperFactory { + private final String zkPrincipal; + + public HadoopZookeeperFactory(String zkPrincipal) { + this.zkPrincipal = zkPrincipal; + } + + @Override + public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, + Watcher watcher, boolean canBeReadOnly + ) throws Exception { + ZKClientConfig zkClientConfig = new ZKClientConfig(); + if (zkPrincipal != null) { + LOG.info("Configuring zookeeper to use {} as the server principal", + zkPrincipal); + zkClientConfig.setProperty(ZKClientConfig.ZK_SASL_CLIENT_USERNAME, + zkPrincipal); + } + return new ZooKeeper(connectString, sessionTimeout, watcher, + canBeReadOnly, zkClientConfig); + } + } } \ No newline at end of file From 8aa568cea5050f60dd7fbc69b60c78657da032e8 Mon Sep 17 00:00:00 2001 From: litao Date: Mon, 28 Feb 2022 21:03:59 +0800 Subject: [PATCH 003/145] [SPS]: Fix bug for unit test of reconfiguring SPS mode (#3998) --- .../hdfs/server/namenode/TestNameNodeReconfigure.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java index fe555532cbc99..33debdb14923f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java @@ -307,6 +307,8 @@ public void testReconfigureStoragePolicySatisfierEnabled() StoragePolicySatisfierMode.EXTERNAL.toString(), nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT)); + assertNotNull("SPS Manager should be created", + nameNode.getNamesystem().getBlockManager().getSPSManager()); } /** @@ -322,6 +324,8 @@ public void testSatisfyStoragePolicyAfterSatisfierDisabled() StoragePolicySatisfierMode.NONE.toString()); verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, StoragePolicySatisfierMode.NONE, false); + assertNull("SPS Manager should be null", + nameNode.getNamesystem().getBlockManager().getSPSManager()); Path filePath = new Path("/testSPS"); DistributedFileSystem fileSystem = cluster.getFileSystem(); @@ -345,7 +349,7 @@ void verifySPSEnabled(final NameNode nameNode, String property, .getNamesystem().getBlockManager().getSPSManager(); boolean isSPSRunning = spsMgr != null ? spsMgr.isSatisfierRunning() : false; - assertEquals(property + " has wrong value", isSPSRunning, isSPSRunning); + assertEquals(property + " has wrong value", isSatisfierRunning, isSPSRunning); String actual = nameNode.getConf().get(property, DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT); assertEquals(property + " has wrong value", expected, From b2f541a7492a61f4dcc28982a2ac6dec32d49e31 Mon Sep 17 00:00:00 2001 From: Hui Fei Date: Mon, 28 Feb 2022 21:05:19 +0800 Subject: [PATCH 004/145] Revert "[SPS]: Fix bug for unit test of reconfiguring SPS mode (#3998)" (#4038) --- .../hdfs/server/namenode/TestNameNodeReconfigure.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java index 33debdb14923f..fe555532cbc99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java @@ -307,8 +307,6 @@ public void testReconfigureStoragePolicySatisfierEnabled() StoragePolicySatisfierMode.EXTERNAL.toString(), nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT)); - assertNotNull("SPS Manager should be created", - nameNode.getNamesystem().getBlockManager().getSPSManager()); } /** @@ -324,8 +322,6 @@ public void testSatisfyStoragePolicyAfterSatisfierDisabled() StoragePolicySatisfierMode.NONE.toString()); verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, StoragePolicySatisfierMode.NONE, false); - assertNull("SPS Manager should be null", - nameNode.getNamesystem().getBlockManager().getSPSManager()); Path filePath = new Path("/testSPS"); DistributedFileSystem fileSystem = cluster.getFileSystem(); @@ -349,7 +345,7 @@ void verifySPSEnabled(final NameNode nameNode, String property, .getNamesystem().getBlockManager().getSPSManager(); boolean isSPSRunning = spsMgr != null ? spsMgr.isSatisfierRunning() : false; - assertEquals(property + " has wrong value", isSatisfierRunning, isSPSRunning); + assertEquals(property + " has wrong value", isSPSRunning, isSPSRunning); String actual = nameNode.getConf().get(property, DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT); assertEquals(property + " has wrong value", expected, From 5eab9719cbf6b9bddbdb4454a5f8e1ae12495492 Mon Sep 17 00:00:00 2001 From: SevenAddSix Date: Mon, 28 Feb 2022 21:12:18 +0800 Subject: [PATCH 005/145] HDFS-16480. Fix typo: indicies -> indices (#4020) --- .../blockmanagement/ErasureCodingWork.java | 30 +++++++++---------- .../apache/hadoop/hdfs/MiniDFSCluster.java | 2 +- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java index 8de3f381ddffe..6158677654b94 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java @@ -30,8 +30,8 @@ import java.util.Set; class ErasureCodingWork extends BlockReconstructionWork { - private final byte[] liveBlockIndicies; - private final byte[] liveBusyBlockIndicies; + private final byte[] liveBlockIndices; + private final byte[] liveBusyBlockIndices; private final String blockPoolId; public ErasureCodingWork(String blockPoolId, BlockInfo block, @@ -40,18 +40,18 @@ public ErasureCodingWork(String blockPoolId, BlockInfo block, List containingNodes, List liveReplicaStorages, int additionalReplRequired, int priority, - byte[] liveBlockIndicies, byte[] liveBusyBlockIndicies) { + byte[] liveBlockIndices, byte[] liveBusyBlockIndices) { super(block, bc, srcNodes, containingNodes, liveReplicaStorages, additionalReplRequired, priority); this.blockPoolId = blockPoolId; - this.liveBlockIndicies = liveBlockIndicies; - this.liveBusyBlockIndicies = liveBusyBlockIndicies; + this.liveBlockIndices = liveBlockIndices; + this.liveBusyBlockIndices = liveBusyBlockIndices; LOG.debug("Creating an ErasureCodingWork to {} reconstruct ", block); } - byte[] getLiveBlockIndicies() { - return liveBlockIndicies; + byte[] getLiveBlockIndices() { + return liveBlockIndices; } @Override @@ -72,15 +72,15 @@ void chooseTargets(BlockPlacementPolicy blockplacement, */ private boolean hasAllInternalBlocks() { final BlockInfoStriped block = (BlockInfoStriped) getBlock(); - if (liveBlockIndicies.length - + liveBusyBlockIndicies.length < block.getRealTotalBlockNum()) { + if (liveBlockIndices.length + + liveBusyBlockIndices.length < block.getRealTotalBlockNum()) { return false; } BitSet bitSet = new BitSet(block.getTotalBlockNum()); - for (byte index : liveBlockIndicies) { + for (byte index : liveBlockIndices) { bitSet.set(index); } - for (byte busyIndex: liveBusyBlockIndicies) { + for (byte busyIndex: liveBusyBlockIndices) { bitSet.set(busyIndex); } for (int i = 0; i < block.getRealDataBlockNum(); i++) { @@ -147,14 +147,14 @@ void addTaskToDatanode(NumberReplicas numberReplicas) { } else { targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded( new ExtendedBlock(blockPoolId, stripedBlk), getSrcNodes(), targets, - getLiveBlockIndicies(), stripedBlk.getErasureCodingPolicy()); + getLiveBlockIndices(), stripedBlk.getErasureCodingPolicy()); } } private void createReplicationWork(int sourceIndex, DatanodeStorageInfo target) { BlockInfoStriped stripedBlk = (BlockInfoStriped) getBlock(); - final byte blockIndex = liveBlockIndicies[sourceIndex]; + final byte blockIndex = liveBlockIndices[sourceIndex]; final DatanodeDescriptor source = getSrcNodes()[sourceIndex]; final long internBlkLen = StripedBlockUtil.getInternalBlockLength( stripedBlk.getNumBytes(), stripedBlk.getCellSize(), @@ -173,7 +173,7 @@ private List findLeavingServiceSources() { BitSet bitSet = new BitSet(block.getRealTotalBlockNum()); for (int i = 0; i < getSrcNodes().length; i++) { if (getSrcNodes()[i].isInService()) { - bitSet.set(liveBlockIndicies[i]); + bitSet.set(liveBlockIndices[i]); } } // If the block is on the node which is decommissioning or @@ -184,7 +184,7 @@ private List findLeavingServiceSources() { if ((getSrcNodes()[i].isDecommissionInProgress() || (getSrcNodes()[i].isEnteringMaintenance() && getSrcNodes()[i].isAlive())) && - !bitSet.get(liveBlockIndicies[i])) { + !bitSet.get(liveBlockIndices[i])) { srcIndices.add(i); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index 703111ed2f225..3e13e600b5189 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -1068,7 +1068,7 @@ private static void copyKey(Configuration srcConf, Configuration destConf, * Do the rest of the NN configuration for things like shared edits, * as well as directory formatting, etc. for a single nameservice * @param nnCounter the count of the number of namenodes already configured/started. Also, - * acts as the index to the next NN to start (since indicies start at 0). + * acts as the index to the next NN to start (since indices start at 0). * @throws IOException */ private void configureNameService(MiniDFSNNTopology.NSConf nameservice, int nsCounter, From d05655d2adc5c521e5036054bc1e595f4fb1d568 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Mon, 28 Feb 2022 21:44:24 +0530 Subject: [PATCH 006/145] Revert "HADOOP-18082.Add debug log when RPC#Reader gets a Call. (#3891). Contributed by JiangHua Zhu." Exposes a Race Condition. Which leads to test failures in YARN. (HADOOP-18143) This reverts commit 2025243fbf94fb9a0324de7b159a275bc58c84d0. --- .../src/main/java/org/apache/hadoop/ipc/Server.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index c36e1f655f242..c5732c68b1517 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -3029,9 +3029,6 @@ private void internalQueueCall(Call call, boolean blocking) } else { callQueue.add(call); } - - LOG.debug("Call has entered the CallQueue and is waiting to be processed. " + - "Call details: {}", call); long deltaNanos = Time.monotonicNowNanos() - call.timestampNanos; call.getProcessingDetails().set(Timing.ENQUEUE, deltaNanos, TimeUnit.NANOSECONDS); From b56af00114c5647d647703ac9b0b491933903788 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 1 Mar 2022 11:48:10 +0000 Subject: [PATCH 007/145] HADOOP-18075. ABFS: Fix failure caused by listFiles() in ITestAbfsRestOperationException (#4040) Contributed by Sumangala Patki --- .../fs/azurebfs/AzureBlobFileSystem.java | 8 ++--- .../AbfsListStatusRemoteIterator.java | 36 ++++++++++++------- .../ITestAbfsListStatusRemoteIterator.java | 34 ++++++++---------- .../ITestAbfsRestOperationException.java | 5 +-- 4 files changed, 45 insertions(+), 38 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 47866140145e9..ae70b8dc53b02 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -1193,7 +1193,7 @@ public RemoteIterator listStatusIterator(Path path) TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat, listener); AbfsListStatusRemoteIterator abfsLsItr = - new AbfsListStatusRemoteIterator(getFileStatus(path, tracingContext), abfsStore, + new AbfsListStatusRemoteIterator(path, abfsStore, tracingContext); return RemoteIterators.typeCastingRemoteIterator(abfsLsItr); } else { @@ -1368,9 +1368,9 @@ private void checkCheckAccessException(final Path path, * @throws IOException if the exception error code is not on the allowed list. */ @VisibleForTesting - static void checkException(final Path path, - final AzureBlobFileSystemException exception, - final AzureServiceErrorCode... allowedErrorCodesList) throws IOException { + public static void checkException(final Path path, + final AzureBlobFileSystemException exception, + final AzureServiceErrorCode... allowedErrorCodesList) throws IOException { if (exception instanceof AbfsRestOperationException) { AbfsRestOperationException ere = (AbfsRestOperationException) exception; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java index ce6207bf5f20c..3fecb1f059115 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java @@ -32,7 +32,10 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; public class AbfsListStatusRemoteIterator @@ -45,7 +48,7 @@ public class AbfsListStatusRemoteIterator private static final int MAX_QUEUE_SIZE = 10; private static final long POLL_WAIT_TIME_IN_MS = 250; - private final FileStatus fileStatus; + private final Path path; private final ListingSupport listingSupport; private final ArrayBlockingQueue listResultQueue; private final TracingContext tracingContext; @@ -55,13 +58,15 @@ public class AbfsListStatusRemoteIterator private String continuation; private Iterator currIterator; - public AbfsListStatusRemoteIterator(final FileStatus fileStatus, - final ListingSupport listingSupport, TracingContext tracingContext) { - this.fileStatus = fileStatus; + public AbfsListStatusRemoteIterator(final Path path, + final ListingSupport listingSupport, TracingContext tracingContext) + throws IOException { + this.path = path; this.listingSupport = listingSupport; this.tracingContext = tracingContext; listResultQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE); currIterator = Collections.emptyIterator(); + addNextBatchIteratorToQueue(); fetchBatchesAsync(); } @@ -130,9 +135,6 @@ private void asyncOp() { Thread.currentThread().interrupt(); LOG.error("Thread got interrupted: {}", interruptedException); } - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOG.error("Thread got interrupted: {}", e); } finally { synchronized (this) { isAsyncInProgress = false; @@ -141,13 +143,21 @@ private void asyncOp() { } private synchronized void addNextBatchIteratorToQueue() - throws IOException, InterruptedException { + throws IOException { List fileStatuses = new ArrayList<>(); - continuation = listingSupport - .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE, - continuation, tracingContext); - if (!fileStatuses.isEmpty()) { - listResultQueue.put(new AbfsListResult(fileStatuses.iterator())); + try { + try { + continuation = listingSupport.listStatus(path, null, fileStatuses, + FETCH_ALL_FALSE, continuation, tracingContext); + } catch (AbfsRestOperationException ex) { + AzureBlobFileSystem.checkException(path, ex); + } + if (!fileStatuses.isEmpty()) { + listResultQueue.put(new AbfsListResult(fileStatuses.iterator())); + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + LOG.error("Thread interrupted", ie); } if (continuation == null || continuation.isEmpty()) { isIterationComplete = true; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java index 3f50aec659142..ea1d0e26facec 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java @@ -68,10 +68,9 @@ public void testAbfsIteratorWithHasNext() throws Exception { setPageSize(10); final List fileNames = createFilesUnderDirectory(testDir); - ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore()); - RemoteIterator fsItr = new AbfsListStatusRemoteIterator( - getFileSystem().getFileStatus(testDir), listngSupport, - getTestTracingContext(getFileSystem(), true)); + ListingSupport listingSupport = Mockito.spy(getFileSystem().getAbfsStore()); + RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, + listingSupport, getTestTracingContext(getFileSystem(), true)); Assertions.assertThat(fsItr) .describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") @@ -84,7 +83,7 @@ public void testAbfsIteratorWithHasNext() throws Exception { } verifyIteratorResultCount(itrCount, fileNames); int minNumberOfInvocations = TEST_FILES_NUMBER / 10; - verify(listngSupport, Mockito.atLeast(minNumberOfInvocations)) + verify(listingSupport, Mockito.atLeast(minNumberOfInvocations)) .listStatus(any(Path.class), nullable(String.class), anyList(), anyBoolean(), nullable(String.class), @@ -97,10 +96,9 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { setPageSize(10); final List fileNames = createFilesUnderDirectory(testDir); - ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore()); - RemoteIterator fsItr = new AbfsListStatusRemoteIterator( - getFileSystem().getFileStatus(testDir), listngSupport, - getTestTracingContext(getFileSystem(), true)); + ListingSupport listingSupport = Mockito.spy(getFileSystem().getAbfsStore()); + RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, + listingSupport, getTestTracingContext(getFileSystem(), true)); Assertions.assertThat(fsItr) .describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") @@ -114,7 +112,7 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { LambdaTestUtils.intercept(NoSuchElementException.class, fsItr::next); verifyIteratorResultCount(itrCount, fileNames); int minNumberOfInvocations = TEST_FILES_NUMBER / 10; - verify(listngSupport, Mockito.atLeast(minNumberOfInvocations)) + verify(listingSupport, Mockito.atLeast(minNumberOfInvocations)) .listStatus(any(Path.class), nullable(String.class), anyList(), anyBoolean(), nullable(String.class), @@ -169,10 +167,9 @@ public void testWithAbfsIteratorDisabledWithoutHasNext() throws Exception { public void testNextWhenNoMoreElementsPresent() throws Exception { Path testDir = createTestDirectory(); setPageSize(10); - RemoteIterator fsItr = - new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir), - getFileSystem().getAbfsStore(), - getTestTracingContext(getFileSystem(), true)); + RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, + getFileSystem().getAbfsStore(), + getTestTracingContext(getFileSystem(), true)); fsItr = Mockito.spy(fsItr); Mockito.doReturn(false).when(fsItr).hasNext(); @@ -212,12 +209,11 @@ public void testIOException() throws Exception { getFileSystem().mkdirs(testDir); String exceptionMessage = "test exception"; - ListingSupport lsSupport =getMockListingSupport(exceptionMessage); - RemoteIterator fsItr = - new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir), - lsSupport, getTestTracingContext(getFileSystem(), true)); + ListingSupport lsSupport = getMockListingSupport(exceptionMessage); - LambdaTestUtils.intercept(IOException.class, fsItr::next); + LambdaTestUtils.intercept(IOException.class, + () -> new AbfsListStatusRemoteIterator(testDir, lsSupport, + getTestTracingContext(getFileSystem(), true))); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java index a71e7bc815f75..ce9415a81795c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsRestOperationException.java @@ -74,8 +74,9 @@ public void testAbfsRestOperationExceptionFormat() throws IOException { // verify its format String errorMessage = ex.getLocalizedMessage(); String[] errorFields = errorMessage.split(","); - - Assert.assertEquals(6, errorFields.length); + Assertions.assertThat(errorFields) + .describedAs("fields in exception of %s", ex) + .hasSize(6); // Check status message, status code, HTTP Request Type and URL. Assert.assertEquals("Operation failed: \"The specified path does not exist.\"", errorFields[0].trim()); Assert.assertEquals("404", errorFields[1].trim()); From 91997872155e4b4b99c3dea7baccddbf2af9aa19 Mon Sep 17 00:00:00 2001 From: litao Date: Wed, 2 Mar 2022 11:25:50 +0800 Subject: [PATCH 008/145] HDFS-16458. [SPS]: Fix bug for unit test of reconfiguring SPS mode (#4041) --- .../hdfs/server/namenode/TestNameNodeReconfigure.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java index fe555532cbc99..33debdb14923f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java @@ -307,6 +307,8 @@ public void testReconfigureStoragePolicySatisfierEnabled() StoragePolicySatisfierMode.EXTERNAL.toString(), nameNode.getConf().get(DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT)); + assertNotNull("SPS Manager should be created", + nameNode.getNamesystem().getBlockManager().getSPSManager()); } /** @@ -322,6 +324,8 @@ public void testSatisfyStoragePolicyAfterSatisfierDisabled() StoragePolicySatisfierMode.NONE.toString()); verifySPSEnabled(nameNode, DFS_STORAGE_POLICY_SATISFIER_MODE_KEY, StoragePolicySatisfierMode.NONE, false); + assertNull("SPS Manager should be null", + nameNode.getNamesystem().getBlockManager().getSPSManager()); Path filePath = new Path("/testSPS"); DistributedFileSystem fileSystem = cluster.getFileSystem(); @@ -345,7 +349,7 @@ void verifySPSEnabled(final NameNode nameNode, String property, .getNamesystem().getBlockManager().getSPSManager(); boolean isSPSRunning = spsMgr != null ? spsMgr.isSatisfierRunning() : false; - assertEquals(property + " has wrong value", isSPSRunning, isSPSRunning); + assertEquals(property + " has wrong value", isSatisfierRunning, isSPSRunning); String actual = nameNode.getConf().get(property, DFS_STORAGE_POLICY_SATISFIER_MODE_DEFAULT); assertEquals(property + " has wrong value", expected, From 9e475aede6adc5b2e31f7447e3ab94c6db27f93b Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Wed, 2 Mar 2022 11:16:24 +0100 Subject: [PATCH 009/145] YARN-10983. Follow-up changes for YARN-10904. Contributed by Benjamin Teke --- .../scheduler/capacity/AbstractCSQueue.java | 4 ++-- .../QueueAppLifetimeAndLimitSettings.java | 14 +++++------ .../capacity/QueueNodeLabelsSettings.java | 23 ++++++++----------- 3 files changed, 19 insertions(+), 22 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index 020c6011043a4..e924932a36d00 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -331,7 +331,7 @@ protected void setupQueueConfigs(Resource clusterResource) throws // Collect and set the Node label configuration this.queueNodeLabelsSettings = new QueueNodeLabelsSettings(configuration, parent, - getQueuePath(), queueContext.getQueueManager().getConfiguredNodeLabelsForAllQueues()); + queuePath, queueContext.getQueueManager().getConfiguredNodeLabelsForAllQueues()); // Initialize the queue capacities setupConfigurableCapacities(); @@ -379,7 +379,7 @@ protected void setupQueueConfigs(Resource clusterResource) throws // Setup application related limits this.queueAppLifetimeSettings = new QueueAppLifetimeAndLimitSettings(configuration, - this, getQueuePath()); + this, queuePath); } finally { writeLock.unlock(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAppLifetimeAndLimitSettings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAppLifetimeAndLimitSettings.java index e0f4d60fe4c66..2e71d8eef3edb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAppLifetimeAndLimitSettings.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueAppLifetimeAndLimitSettings.java @@ -35,9 +35,9 @@ public class QueueAppLifetimeAndLimitSettings { private int maxParallelApps; public QueueAppLifetimeAndLimitSettings(CapacitySchedulerConfiguration configuration, - AbstractCSQueue q, String queuePath) { + AbstractCSQueue q, QueuePath queuePath) { // Store max parallel apps property - this.maxParallelApps = configuration.getMaxParallelAppsForQueue(queuePath); + this.maxParallelApps = configuration.getMaxParallelAppsForQueue(queuePath.getFullPath()); this.maxApplicationLifetime = getInheritedMaxAppLifetime(q, configuration); this.defaultApplicationLifetime = setupInheritedDefaultAppLifetime(q, queuePath, configuration, maxApplicationLifetime); @@ -48,7 +48,7 @@ private long getInheritedMaxAppLifetime(CSQueue q, CapacitySchedulerConfiguratio long maxAppLifetime = conf.getMaximumLifetimePerQueue(q.getQueuePath()); // If q is the root queue, then get max app lifetime from conf. - if (parentQ == null) { + if (q.getQueuePathObject().isRoot()) { return maxAppLifetime; } @@ -62,16 +62,16 @@ private long getInheritedMaxAppLifetime(CSQueue q, CapacitySchedulerConfiguratio } private long setupInheritedDefaultAppLifetime(CSQueue q, - String queuePath, CapacitySchedulerConfiguration conf, long myMaxAppLifetime) { + QueuePath queuePath, CapacitySchedulerConfiguration conf, long myMaxAppLifetime) { CSQueue parentQ = q.getParent(); - long defaultAppLifetime = conf.getDefaultLifetimePerQueue(queuePath); + long defaultAppLifetime = conf.getDefaultLifetimePerQueue(queuePath.getFullPath()); defaultAppLifetimeWasSpecifiedInConfig = (defaultAppLifetime >= 0 - || (parentQ != null && + || (!queuePath.isRoot() && parentQ.getDefaultAppLifetimeWasSpecifiedInConfig())); // If q is the root queue, then get default app lifetime from conf. - if (parentQ == null) { + if (queuePath.isRoot()) { return defaultAppLifetime; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueNodeLabelsSettings.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueNodeLabelsSettings.java index 8d64e17a5e786..c431d2bb45543 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueNodeLabelsSettings.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueNodeLabelsSettings.java @@ -19,7 +19,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.util.Sets; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; -import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.ROOT; import java.io.IOException; import java.util.Set; @@ -30,14 +29,14 @@ */ public class QueueNodeLabelsSettings { private final CSQueue parent; - private final String queuePath; + private final QueuePath queuePath; private Set accessibleLabels; private Set configuredNodeLabels; private String defaultLabelExpression; public QueueNodeLabelsSettings(CapacitySchedulerConfiguration configuration, CSQueue parent, - String queuePath, + QueuePath queuePath, ConfiguredNodeLabels configuredNodeLabels) throws IOException { this.parent = parent; this.queuePath = queuePath; @@ -54,7 +53,7 @@ private void initializeNodeLabels(CapacitySchedulerConfiguration configuration, } private void initializeAccessibleLabels(CapacitySchedulerConfiguration configuration) { - this.accessibleLabels = configuration.getAccessibleNodeLabels(queuePath); + this.accessibleLabels = configuration.getAccessibleNodeLabels(queuePath.getFullPath()); // Inherit labels from parent if not set if (this.accessibleLabels == null && parent != null) { this.accessibleLabels = parent.getAccessibleNodeLabels(); @@ -62,7 +61,8 @@ private void initializeAccessibleLabels(CapacitySchedulerConfiguration configura } private void initializeDefaultLabelExpression(CapacitySchedulerConfiguration configuration) { - this.defaultLabelExpression = configuration.getDefaultNodeLabelExpression(queuePath); + this.defaultLabelExpression = configuration.getDefaultNodeLabelExpression( + queuePath.getFullPath()); // If the accessible labels is not null and the queue has a parent with a // similar set of labels copy the defaultNodeLabelExpression from the parent if (this.accessibleLabels != null && parent != null @@ -75,21 +75,22 @@ private void initializeDefaultLabelExpression(CapacitySchedulerConfiguration con private void initializeConfiguredNodeLabels(CapacitySchedulerConfiguration configuration, ConfiguredNodeLabels configuredNodeLabelsParam) { if (configuredNodeLabelsParam != null) { - if (queuePath.equals(ROOT)) { + if (queuePath.isRoot()) { this.configuredNodeLabels = configuredNodeLabelsParam.getAllConfiguredLabels(); } else { - this.configuredNodeLabels = configuredNodeLabelsParam.getLabelsByQueue(queuePath); + this.configuredNodeLabels = configuredNodeLabelsParam.getLabelsByQueue( + queuePath.getFullPath()); } } else { // Fallback to suboptimal but correct logic - this.configuredNodeLabels = configuration.getConfiguredNodeLabels(queuePath); + this.configuredNodeLabels = configuration.getConfiguredNodeLabels(queuePath.getFullPath()); } } private void validateNodeLabels() throws IOException { // Check if labels of this queue is a subset of parent queue, only do this // when the queue in question is not root - if (isNotRoot()) { + if (!queuePath.isRoot()) { if (parent.getAccessibleNodeLabels() != null && !parent .getAccessibleNodeLabels().contains(RMNodeLabelsManager.ANY)) { // If parent isn't "*", child shouldn't be "*" too @@ -109,10 +110,6 @@ private void validateNodeLabels() throws IOException { } } - private boolean isNotRoot() { - return parent != null && parent.getParent() != null; - } - public boolean isAccessibleToPartition(String nodePartition) { // if queue's label is *, it can access any node if (accessibleLabels != null && accessibleLabels.contains(RMNodeLabelsManager.ANY)) { From 902a7935e971bbbc979a29de08d73e2ba4b08e47 Mon Sep 17 00:00:00 2001 From: ted12138 <67770300+ted12138@users.noreply.github.com> Date: Wed, 2 Mar 2022 18:25:56 +0800 Subject: [PATCH 010/145] HADOOP-18128. Fix typo issues of outputstream.md (#4025) --- .../src/site/markdown/filesystem/outputstream.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/outputstream.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/outputstream.md index 8d0d4c4354f0b..1498d8db2e23b 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/outputstream.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/outputstream.md @@ -42,7 +42,7 @@ or `FSDataOutputStreamBuilder.build()`. These all return instances of `FSDataOutputStream`, through which data can be written through various `write()` methods. After a stream's `close()` method is called, all data written to the -stream MUST BE persisted to the fileysystem and visible to oll other +stream MUST BE persisted to the filesystem and visible to oll other clients attempting to read data from that path via `FileSystem.open()`. As well as operations to write the data, Hadoop's `OutputStream` implementations @@ -126,7 +126,7 @@ consistent with the data at the filesystem. The output stream returned from a call of `FileSystem.append(path, buffersize, progress)` within a filesystem `FS`, -can be modelled as a stream whose `buffer` is intialized to that of +can be modelled as a stream whose `buffer` is initialized to that of the original file: ```python @@ -184,7 +184,7 @@ This document covers the requirements of such implementations. HDFS's `FileSystem` implementation, `DistributedFileSystem`, returns an instance of `HdfsDataOutputStream`. This implementation has at least two behaviors -which are not explicitly declared by the base Java implmentation +which are not explicitly declared by the base Java implementation 1. Writes are synchronized: more than one thread can write to the same output stream. This is a use pattern which HBase relies on. @@ -648,7 +648,7 @@ in production. 1. `OutputStream.write()` MAY persist the data, synchronously or asynchronously 1. `OutputStream.flush()` flushes data to the destination. There are no strict persistence requirements. -1. `Syncable.hflush()` synchronously sends all outstaning data to the destination +1. `Syncable.hflush()` synchronously sends all outstanding data to the destination filesystem. After returning to the caller, the data MUST be visible to other readers, it MAY be durable. That is: it does not have to be persisted, merely guaranteed to be consistently visible to all clients attempting to open a new stream reading @@ -678,7 +678,7 @@ public void hflush() throws IOException { ``` This is perfectly acceptable as an implementation: the semantics of `hflush()` -are satisifed. +are satisfied. What is not acceptable is downgrading `hsync()` to `hflush()`, as the durability guarantee is no longer met. @@ -863,7 +863,7 @@ local data as can be written to full checksummed blocks of data. That is, the hsync/hflush operations are not guaranteed to write all the pending data until the file is finally closed. -For this reason, the local fileystem accessed via `file://` URLs +For this reason, the local filesystem accessed via `file://` URLs does not support `Syncable` unless `setWriteChecksum(false)` was called on that FileSystem instance so as to disable checksum creation. After which, obviously, checksums are not generated for any file. From 356d337d1e5a97dc72ae209df919cfb99e0dd841 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Wed, 2 Mar 2022 16:19:17 +0100 Subject: [PATCH 011/145] YARN-11042. Fix testQueueSubmitWithACLsEnabledWithQueueMapping in TestAppManager. Contributed by Tamas Domok --- .../hadoop/yarn/server/resourcemanager/TestAppManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index 87147ce62c9a5..8c73a0d5bc15d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -331,7 +331,7 @@ public void testQueueSubmitWithACLsEnabledWithQueueMapping() csConf.set(PREFIX + "root.test.acl_submit_applications", "test"); csConf.set(PREFIX + "root.test.acl_administer_queue", "test"); - asContext.setQueue("test"); + asContext.setQueue("oldQueue"); MockRM newMockRM = new MockRM(csConf); RMContext newMockRMContext = newMockRM.getRMContext(); From f800b65b40e0053a469110eed5f46696948a1737 Mon Sep 17 00:00:00 2001 From: Chao Sun Date: Wed, 2 Mar 2022 17:22:56 -0800 Subject: [PATCH 012/145] Make upstream aware of 3.3.2 release --- .../markdown/release/3.3.2/CHANGELOG.3.3.2.md | 350 ++++++++ .../release/3.3.2/RELEASENOTES.3.3.2.md | 93 ++ .../jdiff/Apache_Hadoop_HDFS_3.3.2.xml | 835 ++++++++++++++++++ hadoop-project-dist/pom.xml | 2 +- 4 files changed, 1279 insertions(+), 1 deletion(-) create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/CHANGELOG.3.3.2.md create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/RELEASENOTES.3.3.2.md create mode 100644 hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.2.xml diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/CHANGELOG.3.3.2.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/CHANGELOG.3.3.2.md new file mode 100644 index 0000000000000..162f9928489ee --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/CHANGELOG.3.3.2.md @@ -0,0 +1,350 @@ + + +# Apache Hadoop Changelog + +## Release 3.3.2 - 2022-02-21 + + + +### IMPORTANT ISSUES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HDFS-15814](https://issues.apache.org/jira/browse/HDFS-15814) | Make some parameters configurable for DataNodeDiskMetrics | Major | hdfs | tomscut | tomscut | + + +### NEW FEATURES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HDFS-15288](https://issues.apache.org/jira/browse/HDFS-15288) | Add Available Space Rack Fault Tolerant BPP | Major | . | Ayush Saxena | Ayush Saxena | +| [HDFS-16048](https://issues.apache.org/jira/browse/HDFS-16048) | RBF: Print network topology on the router web | Minor | . | tomscut | tomscut | +| [HDFS-16337](https://issues.apache.org/jira/browse/HDFS-16337) | Show start time of Datanode on Web | Minor | . | tomscut | tomscut | +| [HADOOP-17979](https://issues.apache.org/jira/browse/HADOOP-17979) | Interface EtagSource to allow FileStatus subclasses to provide etags | Major | fs, fs/azure, fs/s3 | Steve Loughran | Steve Loughran | + + +### IMPROVEMENTS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [YARN-10123](https://issues.apache.org/jira/browse/YARN-10123) | Error message around yarn app -stop/start can be improved to highlight that an implementation at framework level is needed for the stop/start functionality to work | Minor | client, documentation | Siddharth Ahuja | Siddharth Ahuja | +| [HADOOP-17756](https://issues.apache.org/jira/browse/HADOOP-17756) | Increase precommit job timeout from 20 hours to 24 hours. | Major | build | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-16073](https://issues.apache.org/jira/browse/HDFS-16073) | Remove redundant RPC requests for getFileLinkInfo in ClientNamenodeProtocolTranslatorPB | Minor | . | lei w | lei w | +| [HDFS-16074](https://issues.apache.org/jira/browse/HDFS-16074) | Remove an expensive debug string concatenation | Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HDFS-16080](https://issues.apache.org/jira/browse/HDFS-16080) | RBF: Invoking method in all locations should break the loop after successful result | Minor | . | Viraj Jasani | Viraj Jasani | +| [HDFS-16075](https://issues.apache.org/jira/browse/HDFS-16075) | Use empty array constants present in StorageType and DatanodeInfo to avoid creating redundant objects | Major | . | Viraj Jasani | Viraj Jasani | +| [MAPREDUCE-7354](https://issues.apache.org/jira/browse/MAPREDUCE-7354) | Use empty array constants present in TaskCompletionEvent to avoid creating redundant objects | Minor | . | Viraj Jasani | Viraj Jasani | +| [HDFS-16082](https://issues.apache.org/jira/browse/HDFS-16082) | Avoid non-atomic operations on exceptionsSinceLastBalance and failedTimesSinceLastSuccessfulBalance in Balancer | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-16076](https://issues.apache.org/jira/browse/HDFS-16076) | Avoid using slow DataNodes for reading by sorting locations | Major | hdfs | tomscut | tomscut | +| [HDFS-16085](https://issues.apache.org/jira/browse/HDFS-16085) | Move the getPermissionChecker out of the read lock | Minor | . | tomscut | tomscut | +| [YARN-10834](https://issues.apache.org/jira/browse/YARN-10834) | Intra-queue preemption: apps that don't use defined custom resource won't be preempted. | Major | . | Eric Payne | Eric Payne | +| [HADOOP-17777](https://issues.apache.org/jira/browse/HADOOP-17777) | Update clover-maven-plugin version from 3.3.0 to 4.4.1 | Major | . | Wanqiang Ji | Wanqiang Ji | +| [HDFS-16090](https://issues.apache.org/jira/browse/HDFS-16090) | Fine grained locking for datanodeNetworkCounts | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17749](https://issues.apache.org/jira/browse/HADOOP-17749) | Remove lock contention in SelectorPool of SocketIOWithTimeout | Major | common | Xuesen Liang | Xuesen Liang | +| [HADOOP-17775](https://issues.apache.org/jira/browse/HADOOP-17775) | Remove JavaScript package from Docker environment | Major | build | Masatake Iwasaki | Masatake Iwasaki | +| [HADOOP-17402](https://issues.apache.org/jira/browse/HADOOP-17402) | Add GCS FS impl reference to core-default.xml | Major | fs | Rafal Wojdyla | Rafal Wojdyla | +| [HADOOP-17794](https://issues.apache.org/jira/browse/HADOOP-17794) | Add a sample configuration to use ZKDelegationTokenSecretManager in Hadoop KMS | Major | documentation, kms, security | Akira Ajisaka | Akira Ajisaka | +| [HDFS-16122](https://issues.apache.org/jira/browse/HDFS-16122) | Fix DistCpContext#toString() | Minor | . | tomscut | tomscut | +| [HADOOP-12665](https://issues.apache.org/jira/browse/HADOOP-12665) | Document hadoop.security.token.service.use\_ip | Major | documentation | Arpit Agarwal | Akira Ajisaka | +| [YARN-10456](https://issues.apache.org/jira/browse/YARN-10456) | RM PartitionQueueMetrics records are named QueueMetrics in Simon metrics registry | Major | resourcemanager | Eric Payne | Eric Payne | +| [HDFS-15650](https://issues.apache.org/jira/browse/HDFS-15650) | Make the socket timeout for computing checksum of striped blocks configurable | Minor | datanode, ec, erasure-coding | Yushi Hayasaka | Yushi Hayasaka | +| [YARN-10858](https://issues.apache.org/jira/browse/YARN-10858) | [UI2] YARN-10826 breaks Queue view | Major | yarn-ui-v2 | Andras Gyori | Masatake Iwasaki | +| [HADOOP-16290](https://issues.apache.org/jira/browse/HADOOP-16290) | Enable RpcMetrics units to be configurable | Major | ipc, metrics | Erik Krogen | Viraj Jasani | +| [YARN-10860](https://issues.apache.org/jira/browse/YARN-10860) | Make max container per heartbeat configs refreshable | Major | . | Eric Badger | Eric Badger | +| [HADOOP-17813](https://issues.apache.org/jira/browse/HADOOP-17813) | Checkstyle - Allow line length: 100 | Major | . | Akira Ajisaka | Viraj Jasani | +| [HADOOP-17811](https://issues.apache.org/jira/browse/HADOOP-17811) | ABFS ExponentialRetryPolicy doesn't pick up configuration values | Minor | documentation, fs/azure | Brian Frank Loss | Brian Frank Loss | +| [HADOOP-17819](https://issues.apache.org/jira/browse/HADOOP-17819) | Add extensions to ProtobufRpcEngine RequestHeaderProto | Major | common | Hector Sandoval Chaverri | Hector Sandoval Chaverri | +| [HDFS-15936](https://issues.apache.org/jira/browse/HDFS-15936) | Solve BlockSender#sendPacket() does not record SocketTimeout exception | Minor | . | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16153](https://issues.apache.org/jira/browse/HDFS-16153) | Avoid evaluation of LOG.debug statement in QuorumJournalManager | Trivial | . | wangzhaohui | wangzhaohui | +| [HDFS-16154](https://issues.apache.org/jira/browse/HDFS-16154) | TestMiniJournalCluster failing intermittently because of not reseting UserGroupInformation completely | Minor | . | wangzhaohui | wangzhaohui | +| [HADOOP-17837](https://issues.apache.org/jira/browse/HADOOP-17837) | Make it easier to debug UnknownHostExceptions from NetUtils.connect | Minor | . | Bryan Beaudreault | Bryan Beaudreault | +| [HDFS-16175](https://issues.apache.org/jira/browse/HDFS-16175) | Improve the configurable value of Server #PURGE\_INTERVAL\_NANOS | Major | ipc | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16173](https://issues.apache.org/jira/browse/HDFS-16173) | Improve CopyCommands#Put#executor queue configurability | Major | fs | JiangHua Zhu | JiangHua Zhu | +| [HADOOP-17897](https://issues.apache.org/jira/browse/HADOOP-17897) | Allow nested blocks in switch case in checkstyle settings | Minor | build | Masatake Iwasaki | Masatake Iwasaki | +| [HADOOP-17857](https://issues.apache.org/jira/browse/HADOOP-17857) | Check real user ACLs in addition to proxied user ACLs | Major | . | Eric Payne | Eric Payne | +| [HDFS-16210](https://issues.apache.org/jira/browse/HDFS-16210) | RBF: Add the option of refreshCallQueue to RouterAdmin | Major | . | Janus Chow | Janus Chow | +| [HDFS-16221](https://issues.apache.org/jira/browse/HDFS-16221) | RBF: Add usage of refreshCallQueue for Router | Major | . | Janus Chow | Janus Chow | +| [HDFS-16223](https://issues.apache.org/jira/browse/HDFS-16223) | AvailableSpaceRackFaultTolerantBlockPlacementPolicy should use chooseRandomWithStorageTypeTwoTrial() for better performance. | Major | . | Ayush Saxena | Ayush Saxena | +| [HADOOP-17893](https://issues.apache.org/jira/browse/HADOOP-17893) | Improve PrometheusSink for Namenode TopMetrics | Major | metrics | Max Xie | Max Xie | +| [HADOOP-17926](https://issues.apache.org/jira/browse/HADOOP-17926) | Maven-eclipse-plugin is no longer needed since Eclipse can import Maven projects by itself. | Minor | documentation | Rintaro Ikeda | Rintaro Ikeda | +| [YARN-10935](https://issues.apache.org/jira/browse/YARN-10935) | AM Total Queue Limit goes below per-user AM Limit if parent is full. | Major | capacity scheduler, capacityscheduler | Eric Payne | Eric Payne | +| [HADOOP-17939](https://issues.apache.org/jira/browse/HADOOP-17939) | Support building on Apple Silicon | Major | build, common | Dongjoon Hyun | Dongjoon Hyun | +| [HADOOP-17941](https://issues.apache.org/jira/browse/HADOOP-17941) | Update xerces to 2.12.1 | Minor | . | Zhongwei Zhu | Zhongwei Zhu | +| [HDFS-16246](https://issues.apache.org/jira/browse/HDFS-16246) | Print lockWarningThreshold in InstrumentedLock#logWarning and InstrumentedLock#logWaitWarning | Minor | . | tomscut | tomscut | +| [HDFS-16252](https://issues.apache.org/jira/browse/HDFS-16252) | Correct docs for dfs.http.client.retry.policy.spec | Major | . | Stephen O'Donnell | Stephen O'Donnell | +| [HDFS-16241](https://issues.apache.org/jira/browse/HDFS-16241) | Standby close reconstruction thread | Major | . | zhanghuazong | zhanghuazong | +| [HADOOP-17974](https://issues.apache.org/jira/browse/HADOOP-17974) | Fix the import statements in hadoop-aws module | Minor | build, fs/azure | Tamas Domok | | +| [HDFS-16277](https://issues.apache.org/jira/browse/HDFS-16277) | Improve decision in AvailableSpaceBlockPlacementPolicy | Major | block placement | guophilipse | guophilipse | +| [HADOOP-17770](https://issues.apache.org/jira/browse/HADOOP-17770) | WASB : Support disabling buffered reads in positional reads | Major | . | Anoop Sam John | Anoop Sam John | +| [HDFS-16282](https://issues.apache.org/jira/browse/HDFS-16282) | Duplicate generic usage information to hdfs debug command | Minor | tools | daimin | daimin | +| [YARN-1115](https://issues.apache.org/jira/browse/YARN-1115) | Provide optional means for a scheduler to check real user ACLs | Major | capacity scheduler, scheduler | Eric Payne | | +| [HDFS-16279](https://issues.apache.org/jira/browse/HDFS-16279) | Print detail datanode info when process first storage report | Minor | . | tomscut | tomscut | +| [HDFS-16286](https://issues.apache.org/jira/browse/HDFS-16286) | Debug tool to verify the correctness of erasure coding on file | Minor | erasure-coding, tools | daimin | daimin | +| [HDFS-16294](https://issues.apache.org/jira/browse/HDFS-16294) | Remove invalid DataNode#CONFIG\_PROPERTY\_SIMULATED | Major | datanode | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16299](https://issues.apache.org/jira/browse/HDFS-16299) | Fix bug for TestDataNodeVolumeMetrics#verifyDataNodeVolumeMetrics | Minor | . | tomscut | tomscut | +| [HDFS-16301](https://issues.apache.org/jira/browse/HDFS-16301) | Improve BenchmarkThroughput#SIZE naming standardization | Minor | benchmarks, test | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16287](https://issues.apache.org/jira/browse/HDFS-16287) | Support to make dfs.namenode.avoid.read.slow.datanode reconfigurable | Major | . | Haiyang Hu | Haiyang Hu | +| [HDFS-16321](https://issues.apache.org/jira/browse/HDFS-16321) | Fix invalid config in TestAvailableSpaceRackFaultTolerantBPP | Minor | test | guophilipse | guophilipse | +| [HDFS-16315](https://issues.apache.org/jira/browse/HDFS-16315) | Add metrics related to Transfer and NativeCopy for DataNode | Major | . | tomscut | tomscut | +| [HADOOP-17998](https://issues.apache.org/jira/browse/HADOOP-17998) | Allow get command to run with multi threads. | Major | fs | Chengwei Wang | Chengwei Wang | +| [HDFS-16344](https://issues.apache.org/jira/browse/HDFS-16344) | Improve DirectoryScanner.Stats#toString | Major | . | tomscut | tomscut | +| [HADOOP-18023](https://issues.apache.org/jira/browse/HADOOP-18023) | Allow cp command to run with multi threads. | Major | fs | Chengwei Wang | Chengwei Wang | +| [HDFS-16314](https://issues.apache.org/jira/browse/HDFS-16314) | Support to make dfs.namenode.block-placement-policy.exclude-slow-nodes.enabled reconfigurable | Major | . | Haiyang Hu | Haiyang Hu | +| [HADOOP-18026](https://issues.apache.org/jira/browse/HADOOP-18026) | Fix default value of Magic committer | Minor | common | guophilipse | guophilipse | +| [HDFS-16345](https://issues.apache.org/jira/browse/HDFS-16345) | Fix test cases fail in TestBlockStoragePolicy | Major | build | guophilipse | guophilipse | +| [HADOOP-18040](https://issues.apache.org/jira/browse/HADOOP-18040) | Use maven.test.failure.ignore instead of ignoreTestFailure | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17643](https://issues.apache.org/jira/browse/HADOOP-17643) | WASB : Make metadata checks case insensitive | Major | . | Anoop Sam John | Anoop Sam John | +| [HADOOP-18033](https://issues.apache.org/jira/browse/HADOOP-18033) | Upgrade fasterxml Jackson to 2.13.0 | Major | build | Akira Ajisaka | Viraj Jasani | +| [HDFS-16327](https://issues.apache.org/jira/browse/HDFS-16327) | Make dfs.namenode.max.slowpeer.collect.nodes reconfigurable | Major | . | tomscut | tomscut | +| [HDFS-16375](https://issues.apache.org/jira/browse/HDFS-16375) | The FBR lease ID should be exposed to the log | Major | . | tomscut | tomscut | +| [HDFS-16386](https://issues.apache.org/jira/browse/HDFS-16386) | Reduce DataNode load when FsDatasetAsyncDiskService is working | Major | datanode | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16391](https://issues.apache.org/jira/browse/HDFS-16391) | Avoid evaluation of LOG.debug statement in NameNodeHeartbeatService | Trivial | . | wangzhaohui | wangzhaohui | +| [YARN-8234](https://issues.apache.org/jira/browse/YARN-8234) | Improve RM system metrics publisher's performance by pushing events to timeline server in batch | Critical | resourcemanager, timelineserver | Hu Ziqian | Ashutosh Gupta | +| [HADOOP-18052](https://issues.apache.org/jira/browse/HADOOP-18052) | Support Apple Silicon in start-build-env.sh | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-18056](https://issues.apache.org/jira/browse/HADOOP-18056) | DistCp: Filter duplicates in the source paths | Major | . | Ayush Saxena | Ayush Saxena | +| [HADOOP-18065](https://issues.apache.org/jira/browse/HADOOP-18065) | ExecutorHelper.logThrowableFromAfterExecute() is too noisy. | Minor | . | Mukund Thakur | Mukund Thakur | +| [HDFS-16043](https://issues.apache.org/jira/browse/HDFS-16043) | Add markedDeleteBlockScrubberThread to delete blocks asynchronously | Major | hdfs, namanode | Xiangyi Zhu | Xiangyi Zhu | +| [HADOOP-18094](https://issues.apache.org/jira/browse/HADOOP-18094) | Disable S3A auditing by default. | Blocker | fs/s3 | Steve Loughran | Steve Loughran | + + +### BUG FIXES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [YARN-10438](https://issues.apache.org/jira/browse/YARN-10438) | Handle null containerId in ClientRMService#getContainerReport() | Major | resourcemanager | Raghvendra Singh | Shubham Gupta | +| [YARN-10428](https://issues.apache.org/jira/browse/YARN-10428) | Zombie applications in the YARN queue using FAIR + sizebasedweight | Critical | capacityscheduler | Guang Yang | Andras Gyori | +| [HDFS-15916](https://issues.apache.org/jira/browse/HDFS-15916) | DistCp: Backward compatibility: Distcp fails from Hadoop 3 to Hadoop 2 for snapshotdiff | Major | distcp | Srinivasu Majeti | Ayush Saxena | +| [HDFS-15977](https://issues.apache.org/jira/browse/HDFS-15977) | Call explicit\_bzero only if it is available | Major | libhdfs++ | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-14922](https://issues.apache.org/jira/browse/HADOOP-14922) | Build of Mapreduce Native Task module fails with unknown opcode "bswap" | Major | . | Anup Halarnkar | Anup Halarnkar | +| [HADOOP-17700](https://issues.apache.org/jira/browse/HADOOP-17700) | ExitUtil#halt info log should log HaltException | Major | . | Viraj Jasani | Viraj Jasani | +| [YARN-10770](https://issues.apache.org/jira/browse/YARN-10770) | container-executor permission is wrong in SecureContainer.md | Major | documentation | Akira Ajisaka | Siddharth Ahuja | +| [YARN-10691](https://issues.apache.org/jira/browse/YARN-10691) | DominantResourceCalculator isInvalidDivisor should consider only countable resource types | Major | . | Bilwa S T | Bilwa S T | +| [HDFS-16031](https://issues.apache.org/jira/browse/HDFS-16031) | Possible Resource Leak in org.apache.hadoop.hdfs.server.aliasmap#InMemoryAliasMap | Major | . | Narges Shadab | Narges Shadab | +| [MAPREDUCE-7348](https://issues.apache.org/jira/browse/MAPREDUCE-7348) | TestFrameworkUploader#testNativeIO fails | Major | test | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15915](https://issues.apache.org/jira/browse/HDFS-15915) | Race condition with async edits logging due to updating txId outside of the namesystem log | Major | hdfs, namenode | Konstantin Shvachko | Konstantin Shvachko | +| [HDFS-16040](https://issues.apache.org/jira/browse/HDFS-16040) | RpcQueueTime metric counts requeued calls as unique events. | Major | hdfs | Simbarashe Dzinamarira | Simbarashe Dzinamarira | +| [MAPREDUCE-7287](https://issues.apache.org/jira/browse/MAPREDUCE-7287) | Distcp will delete existing file , If we use "-delete and -update" options and distcp file. | Major | distcp | zhengchenyu | zhengchenyu | +| [HDFS-15998](https://issues.apache.org/jira/browse/HDFS-15998) | Fix NullPointException In listOpenFiles | Major | . | Haiyang Hu | Haiyang Hu | +| [HDFS-16050](https://issues.apache.org/jira/browse/HDFS-16050) | Some dynamometer tests fail | Major | test | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17631](https://issues.apache.org/jira/browse/HADOOP-17631) | Configuration ${env.VAR:-FALLBACK} should eval FALLBACK when restrictSystemProps=true | Minor | common | Steve Loughran | Steve Loughran | +| [YARN-10809](https://issues.apache.org/jira/browse/YARN-10809) | testWithHbaseConfAtHdfsFileSystem consistently failing | Major | . | Viraj Jasani | Viraj Jasani | +| [YARN-10803](https://issues.apache.org/jira/browse/YARN-10803) | [JDK 11] TestRMFailoverProxyProvider and TestNoHaRMFailoverProxyProvider fails by ClassCastException | Major | test | Akira Ajisaka | Akira Ajisaka | +| [HDFS-16057](https://issues.apache.org/jira/browse/HDFS-16057) | Make sure the order for location in ENTERING\_MAINTENANCE state | Minor | . | tomscut | tomscut | +| [HDFS-16055](https://issues.apache.org/jira/browse/HDFS-16055) | Quota is not preserved in snapshot INode | Major | hdfs | Siyao Meng | Siyao Meng | +| [HDFS-16068](https://issues.apache.org/jira/browse/HDFS-16068) | WebHdfsFileSystem has a possible connection leak in connection with HttpFS | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [YARN-10767](https://issues.apache.org/jira/browse/YARN-10767) | Yarn Logs Command retrying on Standby RM for 30 times | Major | . | D M Murali Krishna Reddy | D M Murali Krishna Reddy | +| [HADOOP-17760](https://issues.apache.org/jira/browse/HADOOP-17760) | Delete hadoop.ssl.enabled and dfs.https.enable from docs and core-default.xml | Major | documentation | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-13671](https://issues.apache.org/jira/browse/HDFS-13671) | Namenode deletes large dir slowly caused by FoldedTreeSet#removeAndGet | Major | . | Yiqun Lin | Haibin Huang | +| [HDFS-16061](https://issues.apache.org/jira/browse/HDFS-16061) | DFTestUtil.waitReplication can produce false positives | Major | hdfs | Ahmed Hussein | Ahmed Hussein | +| [HDFS-14575](https://issues.apache.org/jira/browse/HDFS-14575) | LeaseRenewer#daemon threads leak in DFSClient | Major | . | Tao Yang | Renukaprasad C | +| [YARN-10826](https://issues.apache.org/jira/browse/YARN-10826) | [UI2] Upgrade Node.js to at least v12.22.1 | Major | yarn-ui-v2 | Akira Ajisaka | Masatake Iwasaki | +| [HADOOP-17769](https://issues.apache.org/jira/browse/HADOOP-17769) | Upgrade JUnit to 4.13.2 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [YARN-10824](https://issues.apache.org/jira/browse/YARN-10824) | Title not set for JHS and NM webpages | Major | . | Rajshree Mishra | Bilwa S T | +| [HDFS-16092](https://issues.apache.org/jira/browse/HDFS-16092) | Avoid creating LayoutFlags redundant objects | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17764](https://issues.apache.org/jira/browse/HADOOP-17764) | S3AInputStream read does not re-open the input stream on the second read retry attempt | Major | fs/s3 | Zamil Majdy | Zamil Majdy | +| [HDFS-16109](https://issues.apache.org/jira/browse/HDFS-16109) | Fix flaky some unit tests since they offen timeout | Minor | test | tomscut | tomscut | +| [HDFS-16108](https://issues.apache.org/jira/browse/HDFS-16108) | Incorrect log placeholders used in JournalNodeSyncer | Minor | . | Viraj Jasani | Viraj Jasani | +| [MAPREDUCE-7353](https://issues.apache.org/jira/browse/MAPREDUCE-7353) | Mapreduce job fails when NM is stopped | Major | . | Bilwa S T | Bilwa S T | +| [HDFS-16121](https://issues.apache.org/jira/browse/HDFS-16121) | Iterative snapshot diff report can generate duplicate records for creates, deletes and Renames | Major | snapshots | Srinivasu Majeti | Shashikant Banerjee | +| [HDFS-15796](https://issues.apache.org/jira/browse/HDFS-15796) | ConcurrentModificationException error happens on NameNode occasionally | Critical | hdfs | Daniel Ma | Daniel Ma | +| [HADOOP-17793](https://issues.apache.org/jira/browse/HADOOP-17793) | Better token validation | Major | . | Artem Smotrakov | Artem Smotrakov | +| [HDFS-16042](https://issues.apache.org/jira/browse/HDFS-16042) | DatanodeAdminMonitor scan should be delay based | Major | datanode | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17803](https://issues.apache.org/jira/browse/HADOOP-17803) | Remove WARN logging from LoggingAuditor when executing a request outside an audit span | Major | fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HDFS-16127](https://issues.apache.org/jira/browse/HDFS-16127) | Improper pipeline close recovery causes a permanent write failure or data loss. | Major | . | Kihwal Lee | Kihwal Lee | +| [HADOOP-17028](https://issues.apache.org/jira/browse/HADOOP-17028) | ViewFS should initialize target filesystems lazily | Major | client-mounts, fs, viewfs | Uma Maheswara Rao G | Abhishek Das | +| [HADOOP-17801](https://issues.apache.org/jira/browse/HADOOP-17801) | No error message reported when bucket doesn't exist in S3AFS | Major | fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HADOOP-17796](https://issues.apache.org/jira/browse/HADOOP-17796) | Upgrade jetty version to 9.4.43 | Major | . | Wei-Chiu Chuang | Renukaprasad C | +| [HDFS-12920](https://issues.apache.org/jira/browse/HDFS-12920) | HDFS default value change (with adding time unit) breaks old version MR tarball work with Hadoop 3.x | Critical | configuration, hdfs | Junping Du | Akira Ajisaka | +| [HDFS-16145](https://issues.apache.org/jira/browse/HDFS-16145) | CopyListing fails with FNF exception with snapshot diff | Major | distcp | Shashikant Banerjee | Shashikant Banerjee | +| [YARN-10813](https://issues.apache.org/jira/browse/YARN-10813) | Set default capacity of root for node labels | Major | . | Andras Gyori | Andras Gyori | +| [HDFS-16144](https://issues.apache.org/jira/browse/HDFS-16144) | Revert HDFS-15372 (Files in snapshots no longer see attribute provider permissions) | Major | . | Stephen O'Donnell | Stephen O'Donnell | +| [HADOOP-17817](https://issues.apache.org/jira/browse/HADOOP-17817) | HADOOP-17817. S3A to raise IOE if both S3-CSE and S3Guard enabled | Major | fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [YARN-9551](https://issues.apache.org/jira/browse/YARN-9551) | TestTimelineClientV2Impl.testSyncCall fails intermittently | Minor | ATSv2, test | Prabhu Joseph | Andras Gyori | +| [HDFS-15175](https://issues.apache.org/jira/browse/HDFS-15175) | Multiple CloseOp shared block instance causes the standby namenode to crash when rolling editlog | Critical | . | Yicong Cai | Wan Chang | +| [YARN-10869](https://issues.apache.org/jira/browse/YARN-10869) | CS considers only the default maximum-allocation-mb/vcore property as a maximum when it creates dynamic queues | Major | capacity scheduler | Benjamin Teke | Benjamin Teke | +| [YARN-10789](https://issues.apache.org/jira/browse/YARN-10789) | RM HA startup can fail due to race conditions in ZKConfigurationStore | Major | . | Tarun Parimi | Tarun Parimi | +| [HADOOP-17812](https://issues.apache.org/jira/browse/HADOOP-17812) | NPE in S3AInputStream read() after failure to reconnect to store | Major | fs/s3 | Bobby Wang | Bobby Wang | +| [YARN-6221](https://issues.apache.org/jira/browse/YARN-6221) | Entities missing from ATS when summary log file info got returned to the ATS before the domain log | Critical | yarn | Sushmitha Sreenivasan | Xiaomin Zhang | +| [MAPREDUCE-7258](https://issues.apache.org/jira/browse/MAPREDUCE-7258) | HistoryServerRest.html#Task\_Counters\_API, modify the jobTaskCounters's itemName from "taskcounterGroup" to "taskCounterGroup". | Minor | documentation | jenny | jenny | +| [HADOOP-17370](https://issues.apache.org/jira/browse/HADOOP-17370) | Upgrade commons-compress to 1.21 | Major | common | Dongjoon Hyun | Akira Ajisaka | +| [HDFS-16151](https://issues.apache.org/jira/browse/HDFS-16151) | Improve the parameter comments related to ProtobufRpcEngine2#Server() | Minor | documentation | JiangHua Zhu | JiangHua Zhu | +| [HADOOP-17844](https://issues.apache.org/jira/browse/HADOOP-17844) | Upgrade JSON smart to 2.4.7 | Major | . | Renukaprasad C | Renukaprasad C | +| [HDFS-16177](https://issues.apache.org/jira/browse/HDFS-16177) | Bug fix for Util#receiveFile | Minor | . | tomscut | tomscut | +| [YARN-10814](https://issues.apache.org/jira/browse/YARN-10814) | YARN shouldn't start with empty hadoop.http.authentication.signature.secret.file | Major | . | Benjamin Teke | Tamas Domok | +| [HADOOP-17858](https://issues.apache.org/jira/browse/HADOOP-17858) | Avoid possible class loading deadlock with VerifierNone initialization | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17869](https://issues.apache.org/jira/browse/HADOOP-17869) | fs.s3a.connection.maximum should be bigger than fs.s3a.threads.max | Major | common | Dongjoon Hyun | Dongjoon Hyun | +| [HADOOP-17886](https://issues.apache.org/jira/browse/HADOOP-17886) | Upgrade ant to 1.10.11 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17874](https://issues.apache.org/jira/browse/HADOOP-17874) | ExceptionsHandler to add terse/suppressed Exceptions in thread-safe manner | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-15129](https://issues.apache.org/jira/browse/HADOOP-15129) | Datanode caches namenode DNS lookup failure and cannot startup | Minor | ipc | Karthik Palaniappan | Chris Nauroth | +| [HADOOP-17870](https://issues.apache.org/jira/browse/HADOOP-17870) | HTTP Filesystem to qualify paths in open()/getFileStatus() | Minor | fs | VinothKumar Raman | VinothKumar Raman | +| [HADOOP-17899](https://issues.apache.org/jira/browse/HADOOP-17899) | Avoid using implicit dependency on junit-jupiter-api | Major | test | Masatake Iwasaki | Masatake Iwasaki | +| [YARN-10901](https://issues.apache.org/jira/browse/YARN-10901) | Permission checking error on an existing directory in LogAggregationFileController#verifyAndCreateRemoteLogDir | Major | nodemanager | Tamas Domok | Tamas Domok | +| [HADOOP-17804](https://issues.apache.org/jira/browse/HADOOP-17804) | Prometheus metrics only include the last set of labels | Major | common | Adam Binford | Adam Binford | +| [HDFS-16207](https://issues.apache.org/jira/browse/HDFS-16207) | Remove NN logs stack trace for non-existent xattr query | Major | namenode | Ahmed Hussein | Ahmed Hussein | +| [HDFS-16187](https://issues.apache.org/jira/browse/HDFS-16187) | SnapshotDiff behaviour with Xattrs and Acls is not consistent across NN restarts with checkpointing | Major | snapshots | Srinivasu Majeti | Shashikant Banerjee | +| [HDFS-16198](https://issues.apache.org/jira/browse/HDFS-16198) | Short circuit read leaks Slot objects when InvalidToken exception is thrown | Major | . | Eungsop Yoo | Eungsop Yoo | +| [YARN-10870](https://issues.apache.org/jira/browse/YARN-10870) | Missing user filtering check -\> yarn.webapp.filter-entity-list-by-user for RM Scheduler page | Major | yarn | Siddharth Ahuja | Gergely Pollák | +| [HADOOP-17891](https://issues.apache.org/jira/browse/HADOOP-17891) | lz4-java and snappy-java should be excluded from relocation in shaded Hadoop libraries | Major | . | L. C. Hsieh | L. C. Hsieh | +| [HADOOP-17919](https://issues.apache.org/jira/browse/HADOOP-17919) | Fix command line example in Hadoop Cluster Setup documentation | Minor | documentation | Rintaro Ikeda | Rintaro Ikeda | +| [YARN-9606](https://issues.apache.org/jira/browse/YARN-9606) | Set sslfactory for AuthenticatedURL() while creating LogsCLI#webServiceClient | Major | . | Bilwa S T | Bilwa S T | +| [HDFS-16233](https://issues.apache.org/jira/browse/HDFS-16233) | Do not use exception handler to implement copy-on-write for EnumCounters | Major | namenode | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HDFS-16235](https://issues.apache.org/jira/browse/HDFS-16235) | Deadlock in LeaseRenewer for static remove method | Major | hdfs | angerszhu | angerszhu | +| [HADOOP-17940](https://issues.apache.org/jira/browse/HADOOP-17940) | Upgrade Kafka to 2.8.1 | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [YARN-10970](https://issues.apache.org/jira/browse/YARN-10970) | Standby RM should expose prom endpoint | Major | resourcemanager | Max Xie | Max Xie | +| [HADOOP-17934](https://issues.apache.org/jira/browse/HADOOP-17934) | NullPointerException when no HTTP response set on AbfsRestOperation | Major | fs/azure | Josh Elser | Josh Elser | +| [HDFS-16181](https://issues.apache.org/jira/browse/HDFS-16181) | [SBN Read] Fix metric of RpcRequestCacheMissAmount can't display when tailEditLog form JN | Critical | . | wangzhaohui | wangzhaohui | +| [HADOOP-17922](https://issues.apache.org/jira/browse/HADOOP-17922) | Lookup old S3 encryption configs for JCEKS | Major | fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HADOOP-17925](https://issues.apache.org/jira/browse/HADOOP-17925) | BUILDING.txt should not encourage to activate docs profile on building binary artifacts | Minor | documentation | Rintaro Ikeda | Masatake Iwasaki | +| [HADOOP-16532](https://issues.apache.org/jira/browse/HADOOP-16532) | Fix TestViewFsTrash to use the correct homeDir. | Minor | test, viewfs | Steve Loughran | Xing Lin | +| [HDFS-16268](https://issues.apache.org/jira/browse/HDFS-16268) | Balancer stuck when moving striped blocks due to NPE | Major | balancer & mover, erasure-coding | Leon Gao | Leon Gao | +| [HDFS-16271](https://issues.apache.org/jira/browse/HDFS-16271) | RBF: NullPointerException when setQuota through routers with quota disabled | Major | . | Chengwei Wang | Chengwei Wang | +| [YARN-10976](https://issues.apache.org/jira/browse/YARN-10976) | Fix resource leak due to Files.walk | Minor | . | lujie | lujie | +| [HADOOP-17932](https://issues.apache.org/jira/browse/HADOOP-17932) | Distcp file length comparison have no effect | Major | common, tools, tools/distcp | yinan zhan | yinan zhan | +| [HDFS-16272](https://issues.apache.org/jira/browse/HDFS-16272) | Int overflow in computing safe length during EC block recovery | Critical | 3.1.1 | daimin | daimin | +| [HADOOP-17953](https://issues.apache.org/jira/browse/HADOOP-17953) | S3A: ITestS3AFileContextStatistics test to lookup global or per-bucket configuration for encryption algorithm | Minor | fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HADOOP-17971](https://issues.apache.org/jira/browse/HADOOP-17971) | Exclude IBM Java security classes from being shaded/relocated | Major | build | Nicholas Marion | Nicholas Marion | +| [HDFS-7612](https://issues.apache.org/jira/browse/HDFS-7612) | TestOfflineEditsViewer.testStored() uses incorrect default value for cacheDir | Major | test | Konstantin Shvachko | Michael Kuchenbecker | +| [HDFS-16269](https://issues.apache.org/jira/browse/HDFS-16269) | [Fix] Improve NNThroughputBenchmark#blockReport operation | Major | benchmarks, namenode | JiangHua Zhu | JiangHua Zhu | +| [HADOOP-17945](https://issues.apache.org/jira/browse/HADOOP-17945) | JsonSerialization raises EOFException reading JSON data stored on google GCS | Major | fs | Steve Loughran | Steve Loughran | +| [HDFS-16259](https://issues.apache.org/jira/browse/HDFS-16259) | Catch and re-throw sub-classes of AccessControlException thrown by any permission provider plugins (eg Ranger) | Major | namenode | Stephen O'Donnell | Stephen O'Donnell | +| [HADOOP-17988](https://issues.apache.org/jira/browse/HADOOP-17988) | Disable JIRA plugin for YETUS on Hadoop | Critical | build | Gautham Banasandra | Gautham Banasandra | +| [HDFS-16311](https://issues.apache.org/jira/browse/HDFS-16311) | Metric metadataOperationRate calculation error in DataNodeVolumeMetrics | Major | . | tomscut | tomscut | +| [HADOOP-18002](https://issues.apache.org/jira/browse/HADOOP-18002) | abfs rename idempotency broken -remove recovery | Major | fs/azure | Steve Loughran | Steve Loughran | +| [HDFS-16182](https://issues.apache.org/jira/browse/HDFS-16182) | numOfReplicas is given the wrong value in BlockPlacementPolicyDefault$chooseTarget can cause DataStreamer to fail with Heterogeneous Storage | Major | namanode | Max Xie | Max Xie | +| [HADOOP-17999](https://issues.apache.org/jira/browse/HADOOP-17999) | No-op implementation of setWriteChecksum and setVerifyChecksum in ViewFileSystem | Major | . | Abhishek Das | Abhishek Das | +| [HDFS-16329](https://issues.apache.org/jira/browse/HDFS-16329) | Fix log format for BlockManager | Minor | . | tomscut | tomscut | +| [HDFS-16330](https://issues.apache.org/jira/browse/HDFS-16330) | Fix incorrect placeholder for Exception logs in DiskBalancer | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-16328](https://issues.apache.org/jira/browse/HDFS-16328) | Correct disk balancer param desc | Minor | documentation, hdfs | guophilipse | guophilipse | +| [HDFS-16334](https://issues.apache.org/jira/browse/HDFS-16334) | Correct NameNode ACL description | Minor | documentation | guophilipse | guophilipse | +| [HDFS-16343](https://issues.apache.org/jira/browse/HDFS-16343) | Add some debug logs when the dfsUsed are not used during Datanode startup | Major | datanode | Mukul Kumar Singh | Mukul Kumar Singh | +| [YARN-10991](https://issues.apache.org/jira/browse/YARN-10991) | Fix to ignore the grouping "[]" for resourcesStr in parseResourcesString method | Minor | distributed-shell | Ashutosh Gupta | Ashutosh Gupta | +| [HADOOP-17975](https://issues.apache.org/jira/browse/HADOOP-17975) | Fallback to simple auth does not work for a secondary DistributedFileSystem instance | Major | ipc | István Fajth | István Fajth | +| [HDFS-16350](https://issues.apache.org/jira/browse/HDFS-16350) | Datanode start time should be set after RPC server starts successfully | Minor | . | Viraj Jasani | Viraj Jasani | +| [YARN-11007](https://issues.apache.org/jira/browse/YARN-11007) | Correct words in YARN documents | Minor | documentation | guophilipse | guophilipse | +| [YARN-10975](https://issues.apache.org/jira/browse/YARN-10975) | EntityGroupFSTimelineStore#ActiveLogParser parses already processed files | Major | timelineserver | Prabhu Joseph | Ravuri Sushma sree | +| [HDFS-16332](https://issues.apache.org/jira/browse/HDFS-16332) | Expired block token causes slow read due to missing handling in sasl handshake | Major | datanode, dfs, dfsclient | Shinya Yoshida | Shinya Yoshida | +| [HDFS-16293](https://issues.apache.org/jira/browse/HDFS-16293) | Client sleeps and holds 'dataQueue' when DataNodes are congested | Major | hdfs-client | Yuanxin Zhu | Yuanxin Zhu | +| [YARN-9063](https://issues.apache.org/jira/browse/YARN-9063) | ATS 1.5 fails to start if RollingLevelDb files are corrupt or missing | Major | timelineserver, timelineservice | Tarun Parimi | Ashutosh Gupta | +| [HDFS-16333](https://issues.apache.org/jira/browse/HDFS-16333) | fix balancer bug when transfer an EC block | Major | balancer & mover, erasure-coding | qinyuren | qinyuren | +| [YARN-11020](https://issues.apache.org/jira/browse/YARN-11020) | [UI2] No container is found for an application attempt with a single AM container | Major | yarn-ui-v2 | Andras Gyori | Andras Gyori | +| [HDFS-16373](https://issues.apache.org/jira/browse/HDFS-16373) | Fix MiniDFSCluster restart in case of multiple namenodes | Major | . | Ayush Saxena | Ayush Saxena | +| [HADOOP-18048](https://issues.apache.org/jira/browse/HADOOP-18048) | [branch-3.3] Dockerfile\_aarch64 build fails with fatal error: Python.h: No such file or directory | Major | . | Siyao Meng | Siyao Meng | +| [HDFS-16377](https://issues.apache.org/jira/browse/HDFS-16377) | Should CheckNotNull before access FsDatasetSpi | Major | . | tomscut | tomscut | +| [YARN-6862](https://issues.apache.org/jira/browse/YARN-6862) | Nodemanager resource usage metrics sometimes are negative | Major | nodemanager | YunFan Zhou | Benjamin Teke | +| [HADOOP-13500](https://issues.apache.org/jira/browse/HADOOP-13500) | Synchronizing iteration of Configuration properties object | Major | conf | Jason Darrell Lowe | Dhananjay Badaya | +| [YARN-10178](https://issues.apache.org/jira/browse/YARN-10178) | Global Scheduler async thread crash caused by 'Comparison method violates its general contract | Major | capacity scheduler | tuyu | Andras Gyori | +| [YARN-11053](https://issues.apache.org/jira/browse/YARN-11053) | AuxService should not use class name as default system classes | Major | auxservices | Cheng Pan | Cheng Pan | +| [HDFS-16395](https://issues.apache.org/jira/browse/HDFS-16395) | Remove useless NNThroughputBenchmark#dummyActionNoSynch() | Major | benchmarks, namenode | JiangHua Zhu | JiangHua Zhu | +| [HADOOP-18045](https://issues.apache.org/jira/browse/HADOOP-18045) | Disable TestDynamometerInfra | Major | test | Akira Ajisaka | Akira Ajisaka | +| [HDFS-14099](https://issues.apache.org/jira/browse/HDFS-14099) | Unknown frame descriptor when decompressing multiple frames in ZStandardDecompressor | Major | . | xuzq | xuzq | +| [HADOOP-18063](https://issues.apache.org/jira/browse/HADOOP-18063) | Remove unused import AbstractJavaKeyStoreProvider in Shell class | Minor | . | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16409](https://issues.apache.org/jira/browse/HDFS-16409) | Fix typo: testHasExeceptionsReturnsCorrectValue -\> testHasExceptionsReturnsCorrectValue | Trivial | . | Ashutosh Gupta | Ashutosh Gupta | +| [HDFS-16408](https://issues.apache.org/jira/browse/HDFS-16408) | Ensure LeaseRecheckIntervalMs is greater than zero | Major | namenode | Jingxuan Fu | Jingxuan Fu | +| [HDFS-16410](https://issues.apache.org/jira/browse/HDFS-16410) | Insecure Xml parsing in OfflineEditsXmlLoader | Minor | . | Ashutosh Gupta | Ashutosh Gupta | +| [HDFS-16420](https://issues.apache.org/jira/browse/HDFS-16420) | Avoid deleting unique data blocks when deleting redundancy striped blocks | Critical | ec, erasure-coding | qinyuren | Jackson Wang | +| [YARN-10561](https://issues.apache.org/jira/browse/YARN-10561) | Upgrade node.js to 12.22.1 and yarn to 1.22.5 in YARN application catalog webapp | Critical | webapp | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-18096](https://issues.apache.org/jira/browse/HADOOP-18096) | Distcp: Sync moves filtered file to home directory rather than deleting | Critical | . | Ayush Saxena | Ayush Saxena | + + +### TESTS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [MAPREDUCE-7342](https://issues.apache.org/jira/browse/MAPREDUCE-7342) | Stop RMService in TestClientRedirect.testRedirect() | Minor | . | Zhengxi Li | Zhengxi Li | +| [MAPREDUCE-7311](https://issues.apache.org/jira/browse/MAPREDUCE-7311) | Fix non-idempotent test in TestTaskProgressReporter | Minor | . | Zhengxi Li | Zhengxi Li | +| [HADOOP-17936](https://issues.apache.org/jira/browse/HADOOP-17936) | TestLocalFSCopyFromLocal.testDestinationFileIsToParentDirectory failure after reverting HADOOP-16878 | Major | . | Chao Sun | Chao Sun | +| [HDFS-15862](https://issues.apache.org/jira/browse/HDFS-15862) | Make TestViewfsWithNfs3.testNfsRenameSingleNN() idempotent | Minor | nfs | Zhengxi Li | Zhengxi Li | + + +### SUB-TASKS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [YARN-10337](https://issues.apache.org/jira/browse/YARN-10337) | TestRMHATimelineCollectors fails on hadoop trunk | Major | test, yarn | Ahmed Hussein | Bilwa S T | +| [HDFS-15457](https://issues.apache.org/jira/browse/HDFS-15457) | TestFsDatasetImpl fails intermittently | Major | hdfs | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17424](https://issues.apache.org/jira/browse/HADOOP-17424) | Replace HTrace with No-Op tracer | Major | . | Siyao Meng | Siyao Meng | +| [HADOOP-17705](https://issues.apache.org/jira/browse/HADOOP-17705) | S3A to add option fs.s3a.endpoint.region to set AWS region | Major | fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HADOOP-17670](https://issues.apache.org/jira/browse/HADOOP-17670) | S3AFS and ABFS to log IOStats at DEBUG mode or optionally at INFO level in close() | Minor | fs/azure, fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HADOOP-17511](https://issues.apache.org/jira/browse/HADOOP-17511) | Add an Audit plugin point for S3A auditing/context | Major | . | Steve Loughran | Steve Loughran | +| [HADOOP-17470](https://issues.apache.org/jira/browse/HADOOP-17470) | Collect more S3A IOStatistics | Major | fs/s3 | Steve Loughran | Steve Loughran | +| [HADOOP-17735](https://issues.apache.org/jira/browse/HADOOP-17735) | Upgrade aws-java-sdk to 1.11.1026 | Major | build, fs/s3 | Steve Loughran | Steve Loughran | +| [HADOOP-17547](https://issues.apache.org/jira/browse/HADOOP-17547) | Magic committer to downgrade abort in cleanup if list uploads fails with access denied | Major | fs/s3 | Steve Loughran | Bogdan Stolojan | +| [HADOOP-17771](https://issues.apache.org/jira/browse/HADOOP-17771) | S3AFS creation fails "Unable to find a region via the region provider chain." | Blocker | fs/s3 | Steve Loughran | Steve Loughran | +| [HDFS-15659](https://issues.apache.org/jira/browse/HDFS-15659) | Set dfs.namenode.redundancy.considerLoad to false in MiniDFSCluster | Major | test | Akira Ajisaka | Ahmed Hussein | +| [HADOOP-17774](https://issues.apache.org/jira/browse/HADOOP-17774) | bytesRead FS statistic showing twice the correct value in S3A | Major | fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HADOOP-17290](https://issues.apache.org/jira/browse/HADOOP-17290) | ABFS: Add Identifiers to Client Request Header | Major | fs/azure | Sumangala Patki | Sumangala Patki | +| [HADOOP-17250](https://issues.apache.org/jira/browse/HADOOP-17250) | ABFS: Random read perf improvement | Major | fs/azure | Sneha Vijayarajan | Mukund Thakur | +| [HADOOP-17596](https://issues.apache.org/jira/browse/HADOOP-17596) | ABFS: Change default Readahead Queue Depth from num(processors) to const | Major | fs/azure | Sumangala Patki | Sumangala Patki | +| [HADOOP-17715](https://issues.apache.org/jira/browse/HADOOP-17715) | ABFS: Append blob tests with non HNS accounts fail | Minor | . | Sneha Varma | Sneha Varma | +| [HADOOP-17714](https://issues.apache.org/jira/browse/HADOOP-17714) | ABFS: testBlobBackCompatibility, testRandomRead & WasbAbfsCompatibility tests fail when triggered with default configs | Minor | test | Sneha Varma | Sneha Varma | +| [HDFS-16140](https://issues.apache.org/jira/browse/HDFS-16140) | TestBootstrapAliasmap fails by BindException | Major | test | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-13887](https://issues.apache.org/jira/browse/HADOOP-13887) | Encrypt S3A data client-side with AWS SDK (S3-CSE) | Minor | fs/s3 | Jeeyoung Kim | Mehakmeet Singh | +| [HADOOP-17458](https://issues.apache.org/jira/browse/HADOOP-17458) | S3A to treat "SdkClientException: Data read has a different length than the expected" as EOFException | Minor | fs/s3 | Steve Loughran | Bogdan Stolojan | +| [HADOOP-17628](https://issues.apache.org/jira/browse/HADOOP-17628) | Distcp contract test is really slow with ABFS and S3A; timing out | Minor | fs/azure, fs/s3, test, tools/distcp | Bilahari T H | Steve Loughran | +| [HADOOP-17822](https://issues.apache.org/jira/browse/HADOOP-17822) | fs.s3a.acl.default not working after S3A Audit feature added | Major | fs/s3 | Steve Loughran | Steve Loughran | +| [HADOOP-17139](https://issues.apache.org/jira/browse/HADOOP-17139) | Re-enable optimized copyFromLocal implementation in S3AFileSystem | Minor | fs/s3 | Sahil Takiar | Bogdan Stolojan | +| [HADOOP-17823](https://issues.apache.org/jira/browse/HADOOP-17823) | S3A Tests to skip if S3Guard and S3-CSE are enabled. | Major | build, fs/s3 | Mehakmeet Singh | Mehakmeet Singh | +| [HDFS-16184](https://issues.apache.org/jira/browse/HDFS-16184) | De-flake TestBlockScanner#testSkipRecentAccessFile | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17677](https://issues.apache.org/jira/browse/HADOOP-17677) | Distcp is unable to determine region with S3 PrivateLink endpoints | Major | fs/s3, tools/distcp | KJ | | +| [HDFS-16192](https://issues.apache.org/jira/browse/HDFS-16192) | ViewDistributedFileSystem#rename wrongly using src in the place of dst. | Major | . | Uma Maheswara Rao G | Uma Maheswara Rao G | +| [HADOOP-17156](https://issues.apache.org/jira/browse/HADOOP-17156) | Clear abfs readahead requests on stream close | Major | fs/azure | Rajesh Balamohan | Mukund Thakur | +| [HADOOP-17618](https://issues.apache.org/jira/browse/HADOOP-17618) | ABFS: Partially obfuscate SAS object IDs in Logs | Major | fs/azure | Sumangala Patki | Sumangala Patki | +| [HADOOP-17894](https://issues.apache.org/jira/browse/HADOOP-17894) | CredentialProviderFactory.getProviders() recursion loading JCEKS file from s3a | Major | conf, fs/s3 | Steve Loughran | Steve Loughran | +| [HADOOP-17126](https://issues.apache.org/jira/browse/HADOOP-17126) | implement non-guava Precondition checkNotNull | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17195](https://issues.apache.org/jira/browse/HADOOP-17195) | Intermittent OutOfMemory error while performing hdfs CopyFromLocal to abfs | Major | fs/azure | Mehakmeet Singh | Mehakmeet Singh | +| [HADOOP-17929](https://issues.apache.org/jira/browse/HADOOP-17929) | implement non-guava Precondition checkArgument | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17198](https://issues.apache.org/jira/browse/HADOOP-17198) | Support S3 Access Points | Major | fs/s3 | Steve Loughran | Bogdan Stolojan | +| [HADOOP-17871](https://issues.apache.org/jira/browse/HADOOP-17871) | S3A CSE: minor tuning | Minor | fs/s3 | Steve Loughran | Mehakmeet Singh | +| [HADOOP-17947](https://issues.apache.org/jira/browse/HADOOP-17947) | Provide alternative to Guava VisibleForTesting | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17930](https://issues.apache.org/jira/browse/HADOOP-17930) | implement non-guava Precondition checkState | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17374](https://issues.apache.org/jira/browse/HADOOP-17374) | AliyunOSS: support ListObjectsV2 | Major | fs/oss | wujinhu | wujinhu | +| [HADOOP-17863](https://issues.apache.org/jira/browse/HADOOP-17863) | ABFS: Fix compiler deprecation warning in TextFileBasedIdentityHandler | Minor | fs/azure | Sumangala Patki | Sumangala Patki | +| [HADOOP-17928](https://issues.apache.org/jira/browse/HADOOP-17928) | s3a: set fs.s3a.downgrade.syncable.exceptions = true by default | Major | fs/s3 | Steve Loughran | Steve Loughran | +| [HDFS-16336](https://issues.apache.org/jira/browse/HDFS-16336) | De-flake TestRollingUpgrade#testRollback | Minor | hdfs, test | Kevin Wikant | Viraj Jasani | +| [HDFS-16171](https://issues.apache.org/jira/browse/HDFS-16171) | De-flake testDecommissionStatus | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17226](https://issues.apache.org/jira/browse/HADOOP-17226) | Failure of ITestAssumeRole.testRestrictedCommitActions | Minor | fs/s3, test | Steve Loughran | Steve Loughran | +| [HADOOP-14334](https://issues.apache.org/jira/browse/HADOOP-14334) | S3 SSEC tests to downgrade when running against a mandatory encryption object store | Minor | fs/s3, test | Steve Loughran | Monthon Klongklaew | +| [HADOOP-16223](https://issues.apache.org/jira/browse/HADOOP-16223) | remove misleading fs.s3a.delegation.tokens.enabled prompt | Minor | fs/s3 | Steve Loughran | | + + +### OTHER: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HDFS-16078](https://issues.apache.org/jira/browse/HDFS-16078) | Remove unused parameters for DatanodeManager.handleLifeline() | Minor | . | tomscut | tomscut | +| [HDFS-16079](https://issues.apache.org/jira/browse/HDFS-16079) | Improve the block state change log | Minor | . | tomscut | tomscut | +| [HDFS-16089](https://issues.apache.org/jira/browse/HDFS-16089) | EC: Add metric EcReconstructionValidateTimeMillis for StripedBlockReconstructor | Minor | . | tomscut | tomscut | +| [HDFS-16298](https://issues.apache.org/jira/browse/HDFS-16298) | Improve error msg for BlockMissingException | Minor | . | tomscut | tomscut | +| [HDFS-16312](https://issues.apache.org/jira/browse/HDFS-16312) | Fix typo for DataNodeVolumeMetrics and ProfilingFileIoEvents | Minor | . | tomscut | tomscut | +| [HADOOP-18005](https://issues.apache.org/jira/browse/HADOOP-18005) | Correct log format for LdapGroupsMapping | Minor | . | tomscut | tomscut | +| [HDFS-16319](https://issues.apache.org/jira/browse/HDFS-16319) | Add metrics doc for ReadLockLongHoldCount and WriteLockLongHoldCount | Minor | . | tomscut | tomscut | +| [HDFS-16326](https://issues.apache.org/jira/browse/HDFS-16326) | Simplify the code for DiskBalancer | Minor | . | tomscut | tomscut | +| [HDFS-16335](https://issues.apache.org/jira/browse/HDFS-16335) | Fix HDFSCommands.md | Minor | . | tomscut | tomscut | +| [HDFS-16339](https://issues.apache.org/jira/browse/HDFS-16339) | Show the threshold when mover threads quota is exceeded | Minor | . | tomscut | tomscut | +| [YARN-10820](https://issues.apache.org/jira/browse/YARN-10820) | Make GetClusterNodesRequestPBImpl thread safe | Major | client | Prabhu Joseph | SwathiChandrashekar | +| [HADOOP-17808](https://issues.apache.org/jira/browse/HADOOP-17808) | ipc.Client not setting interrupt flag after catching InterruptedException | Minor | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17834](https://issues.apache.org/jira/browse/HADOOP-17834) | Bump aliyun-sdk-oss to 3.13.0 | Major | . | Siyao Meng | Siyao Meng | +| [HADOOP-17950](https://issues.apache.org/jira/browse/HADOOP-17950) | Provide replacement for deprecated APIs of commons-io IOUtils | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17955](https://issues.apache.org/jira/browse/HADOOP-17955) | Bump netty to the latest 4.1.68 | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [HADOOP-17946](https://issues.apache.org/jira/browse/HADOOP-17946) | Update commons-lang to latest 3.x | Minor | . | Sean Busbey | Renukaprasad C | +| [HDFS-16323](https://issues.apache.org/jira/browse/HDFS-16323) | DatanodeHttpServer doesn't require handler state map while retrieving filter handlers | Minor | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-13464](https://issues.apache.org/jira/browse/HADOOP-13464) | update GSON to 2.7+ | Minor | build | Sean Busbey | Igor Dvorzhak | +| [HADOOP-18061](https://issues.apache.org/jira/browse/HADOOP-18061) | Update the year to 2022 | Major | . | Ayush Saxena | Ayush Saxena | + + diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/RELEASENOTES.3.3.2.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/RELEASENOTES.3.3.2.md new file mode 100644 index 0000000000000..9948d8ff3222c --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.2/RELEASENOTES.3.3.2.md @@ -0,0 +1,93 @@ + + +# Apache Hadoop 3.3.2 Release Notes + +These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements. + + +--- + +* [HDFS-15288](https://issues.apache.org/jira/browse/HDFS-15288) | *Major* | **Add Available Space Rack Fault Tolerant BPP** + +Added a new BlockPlacementPolicy: "AvailableSpaceRackFaultTolerantBlockPlacementPolicy" which uses the same optimization logic as the AvailableSpaceBlockPlacementPolicy along with spreading the replicas across maximum number of racks, similar to BlockPlacementPolicyRackFaultTolerant. +The BPP can be configured by setting the blockplacement policy class as org.apache.hadoop.hdfs.server.blockmanagement.AvailableSpaceRackFaultTolerantBlockPlacementPolicy + + +--- + +* [HADOOP-17424](https://issues.apache.org/jira/browse/HADOOP-17424) | *Major* | **Replace HTrace with No-Op tracer** + +Dependency on HTrace and TraceAdmin protocol/utility were removed. Tracing functionality is no-op until alternative tracer implementation is added. + + +--- + +* [HDFS-15814](https://issues.apache.org/jira/browse/HDFS-15814) | *Major* | **Make some parameters configurable for DataNodeDiskMetrics** + +**WARNING: No release note provided for this change.** + + +--- + +* [YARN-10820](https://issues.apache.org/jira/browse/YARN-10820) | *Major* | **Make GetClusterNodesRequestPBImpl thread safe** + +Added syncronization so that the "yarn node list" command does not fail intermittently + + +--- + +* [HADOOP-13887](https://issues.apache.org/jira/browse/HADOOP-13887) | *Minor* | **Encrypt S3A data client-side with AWS SDK (S3-CSE)** + +Adds support for client side encryption in AWS S3, +with keys managed by AWS-KMS. + +Read the documentation in encryption.md very, very carefully before +use and consider it unstable. + +S3-CSE is enabled in the existing configuration option +"fs.s3a.server-side-encryption-algorithm": + +fs.s3a.server-side-encryption-algorithm=CSE-KMS +fs.s3a.server-side-encryption.key=\ + +You cannot enable CSE and SSE in the same client, although +you can still enable a default SSE option in the S3 console. + +\* Not compatible with S3Guard. +\* Filesystem list/get status operations subtract 16 bytes from the length + of all files \>= 16 bytes long to compensate for the padding which CSE + adds. +\* The SDK always warns about the specific algorithm chosen being + deprecated. It is critical to use this algorithm for ranged + GET requests to work (i.e. random IO). Ignore. +\* Unencrypted files CANNOT BE READ. + The entire bucket SHOULD be encrypted with S3-CSE. +\* Uploading files may be a bit slower as blocks are now + written sequentially. +\* The Multipart Upload API is disabled when S3-CSE is active. + + +--- + +* [YARN-8234](https://issues.apache.org/jira/browse/YARN-8234) | *Critical* | **Improve RM system metrics publisher's performance by pushing events to timeline server in batch** + +When Timeline Service V1 or V1.5 is used, if "yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.enable-batch" is set to true, ResourceManager sends timeline events in batch. The default value is false. If this functionality is enabled, the maximum number that events published in batch is configured by "yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.batch-size". The default value is 1000. The interval of publishing events can be configured by "yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.interval-seconds". By default, it is set to 60 seconds. + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.2.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.2.xml new file mode 100644 index 0000000000000..b4d954cb53ebd --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.2.xml @@ -0,0 +1,835 @@ + + + + + + + + + + + A distributed implementation of {@link +org.apache.hadoop.fs.FileSystem}. This is loosely modelled after +Google's GFS.

+ +

The most important difference is that unlike GFS, Hadoop DFS files +have strictly one writer at any one time. Bytes are always appended +to the end of the writer's stream. There is no notion of "record appends" +or "mutations" that are then checked or reordered. Writers simply emit +a byte stream. That byte stream is guaranteed to be stored in the +order written.

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method must return as quickly as possible, since it's called + in a critical section of the NameNode's operation. + + @param succeeded Whether authorization succeeded. + @param userName Name of the user executing the request. + @param addr Remote address of the request. + @param cmd The requested command. + @param src Path of affected source file. + @param dst Path of affected destination file (if any). + @param stat File information for operations that change the file's + metadata (permissions, owner, times, etc).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml index 024cf055207c5..84577a20a2b46 100644 --- a/hadoop-project-dist/pom.xml +++ b/hadoop-project-dist/pom.xml @@ -134,7 +134,7 @@ false - 3.3.1 + 3.3.2 -unstable From 22fe79cee3a3dcc4c3976a64145d5f1b016c4416 Mon Sep 17 00:00:00 2001 From: Tamas Domok Date: Thu, 3 Mar 2022 15:54:03 +0100 Subject: [PATCH 013/145] YARN-11076. Upgrade jQuery version in Yarn UI2. (#4046) Change-Id: I3cb1677741df5a1978e83029443d4a2d5d7e3d7f --- .../hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json | 9 ++++++--- .../hadoop-yarn-ui/src/main/webapp/bower.json | 6 +++--- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json index ffe380595bf45..2ce4628116bfe 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower-shrinkwrap.json @@ -9,7 +9,8 @@ "2.2.0": "49e042ca89922ed96b27488c2a98add280ae7123" }, "https://github.com/components/jqueryui.git": { - "1.11.4": "c34f8dbf3ba57b3784b93f26119f436c0e8288e1" + "1.11.4": "c34f8dbf3ba57b3784b93f26119f436c0e8288e1", + "1.12.1": "44ecf3794cc56b65954cc19737234a3119d036cc" }, "https://github.com/dockyard/ember-qunit-notifications.git": { "0.1.0": "a83277aa7a1c0545c66e6d133caebb9a620e71ad" @@ -33,7 +34,8 @@ "4.0.0": "80eb44bec78568b152dc5547269c164cb7b5363c" }, "https://github.com/jquery/jquery-dist.git": { - "2.1.4": "7751e69b615c6eca6f783a81e292a55725af6b85" + "2.1.4": "7751e69b615c6eca6f783a81e292a55725af6b85", + "3.6.0": "e786e3d9707ffd9b0dd330ca135b66344dcef85a" }, "https://github.com/jquery/qunit.git": { "1.19.0": "467e7e34652ad7d5883ce9c568461cf8c5e172a8" @@ -61,6 +63,7 @@ "3.3.0": "ac909550c9544325632542bbea97531cc60bc628" }, "https://github.com/twbs/bootstrap.git": { - "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a" + "3.3.6": "81df608a40bf0629a1dc08e584849bb1e43e0b7a", + "3.4.1": "3.4.1" } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json index 8e8ef9972357a..bdf7a9aedb07d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/bower.json @@ -8,14 +8,14 @@ "ember-load-initializers": "0.1.7", "ember-qunit": "0.4.16", "ember-qunit-notifications": "0.1.0", - "jquery": "2.1.4", + "jquery": "3.6.0", "loader.js": "3.3.0", "qunit": "1.19.0", - "jquery-ui": "1.11.4", + "jquery-ui": "1.12.1", "moment": "2.12.0", "moment-timezone": "0.5.33", "more-js": "0.8.2", - "bootstrap": "3.3.6", + "bootstrap": "3.4.1", "d3": "~3.5.6", "datatables": "~1.10.8", "spin.js": "~2.3.2", From 1c27c69f44e6ad34ad53209e92d4231f00c3ca13 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Thu, 3 Mar 2022 16:38:56 +0100 Subject: [PATCH 014/145] Revert "YARN-10947. Simplify AbstractCSQueue#initializeQueueState. Contributed by Andras Gyori" This reverts commit de526e166c349e49d657c5980901670c086d36b2. With this commit, the patch of https://issues.apache.org/jira/browse/YARN-10850 has been committed. --- .../yarn/client/api/impl/AHSv2ClientImpl.java | 4 +- .../client/api/impl/TestAHSv2ClientImpl.java | 70 ------------------- 2 files changed, 3 insertions(+), 71 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java index 5f909302acc2c..e3653baa2b785 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java @@ -137,7 +137,9 @@ public List getContainers(ApplicationAttemptId ApplicationId appId = applicationAttemptId.getApplicationId(); ApplicationReport appReport = getApplicationReport(appId); Map filters = new HashMap<>(); - filters.put("appattemptId", applicationAttemptId.toString()); + filters.put("infofilters", "SYSTEM_INFO_PARENT_ENTITY eq {\"id\":\"" + + applicationAttemptId.toString() + + "\",\"type\":\"YARN_APPLICATION_ATTEMPT\"}"); List entities = readerClient.getContainerEntities( appId, "ALL", filters, 0, null); List containers = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java index 9664e4071068e..f4d7a5629778a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSv2ClientImpl.java @@ -22,7 +22,6 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport; @@ -46,11 +45,8 @@ import org.junit.Test; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; @@ -111,72 +107,6 @@ public void testGetAppAttemptReport() throws IOException, YarnException { isEqualTo("test original tracking url"); } - @Test - public void testGetContainerByAppAttempt() throws IOException, YarnException { - int applicationId = 1; - ApplicationId appId = ApplicationId.newInstance(0, applicationId); - - ApplicationAttemptId appAttemptId = - ApplicationAttemptId.newInstance(appId, 1); - - ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1); - - when(spyTimelineReaderClient.getContainerEntities( - appId, "ALL", - ImmutableMap.of("appattemptId", appAttemptId.toString()), - 0, null)) - .thenReturn(Arrays.asList(createContainerEntity(containerId))); - - when(spyTimelineReaderClient.getApplicationEntity(appId, "ALL", null)) - .thenReturn(createApplicationTimelineEntity(appId, true, - false)); - - List containerList = client.getContainers(appAttemptId); - - assertThat(containerList.size()).isEqualTo(1); - - assertThat(containerList.get(0).getContainerId().getApplicationAttemptId() - .getApplicationId().getId()).isEqualTo(applicationId); - } - - @Test - public void testGetMultipleContainersByAppAttempt() throws IOException, YarnException { - int numContainers = 4; - int applicationId = 3; - int applicationAttemptId = 3; - ApplicationId appMultiple = ApplicationId.newInstance(0, applicationId); - ApplicationAttemptId appAttemptIdMultiple = - ApplicationAttemptId.newInstance(appMultiple, applicationAttemptId); - - List containerEntities = new ArrayList<>(); - for (int containerId = 0; containerId < numContainers; ++containerId) { - containerEntities.add(createContainerEntity(ContainerId.newContainerId( - appAttemptIdMultiple, containerId))); - } - - when(spyTimelineReaderClient.getContainerEntities( - appMultiple, "ALL", - ImmutableMap.of("appattemptId", appAttemptIdMultiple.toString()), - 0, null)) - .thenReturn(containerEntities); - - when(spyTimelineReaderClient.getApplicationEntity(appMultiple, "ALL", null)) - .thenReturn(createApplicationTimelineEntity(appMultiple, true, - false)); - - List containerListMultiple = client.getContainers(appAttemptIdMultiple); - - assertThat(containerListMultiple.size()).isEqualTo(numContainers); - for (int containerIdx = 0; containerIdx < containerListMultiple.size(); ++containerIdx) { - ContainerReport report = containerListMultiple.get(containerIdx); - assertThat(report.getContainerId().getContainerId()).isEqualTo(containerIdx); - assertThat(report.getContainerId().getApplicationAttemptId().getAttemptId()) - .isEqualTo(applicationAttemptId); - assertThat(report.getContainerId().getApplicationAttemptId().getApplicationId().getId()) - .isEqualTo(applicationId); - } - } - @Test public void testGetAppReport() throws IOException, YarnException { final ApplicationId appId = ApplicationId.newInstance(0, 1); From 379baa5eb65a65df50461f337b88ff13f2134aeb Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Thu, 3 Mar 2022 16:44:12 +0100 Subject: [PATCH 015/145] YARN-10947. Simplify AbstractCSQueue#initializeQueueState. Contributed by Andras Gyori --- .../scheduler/capacity/AbstractCSQueue.java | 56 +---------- .../scheduler/capacity/CSQueueUtils.java | 14 +-- .../scheduler/capacity/QueueStateHelper.java | 98 +++++++++++++++++++ 3 files changed, 101 insertions(+), 67 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueStateHelper.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index e924932a36d00..b0ab336b7ad59 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -349,7 +349,7 @@ protected void setupQueueConfigs(Resource clusterResource) throws // Initialize the queue state based on previous state, configured state // and its parent state - initializeQueueState(); + QueueStateHelper.setQueueState(this); authorizer = YarnAuthorizationProvider.getInstance(configuration); @@ -553,60 +553,6 @@ public QueueCapacityVector getConfiguredCapacityVector( return configuredCapacityVectors.get(label); } - private void initializeQueueState() { - QueueState previousState = getState(); - QueueState configuredState = queueContext.getConfiguration() - .getConfiguredState(getQueuePath()); - QueueState parentState = (parent == null) ? null : parent.getState(); - - // verify that we can not any value for State other than RUNNING/STOPPED - if (configuredState != null && configuredState != QueueState.RUNNING - && configuredState != QueueState.STOPPED) { - throw new IllegalArgumentException("Invalid queue state configuration." - + " We can only use RUNNING or STOPPED."); - } - // If we did not set state in configuration, use Running as default state - QueueState defaultState = QueueState.RUNNING; - - if (previousState == null) { - // If current state of the queue is null, we would inherit the state - // from its parent. If this queue does not has parent, such as root queue, - // we would use the configured state. - if (parentState == null) { - updateQueueState((configuredState == null) ? defaultState - : configuredState); - } else { - if (configuredState == null) { - updateQueueState((parentState == QueueState.DRAINING) ? - QueueState.STOPPED : parentState); - } else if (configuredState == QueueState.RUNNING - && parentState != QueueState.RUNNING) { - throw new IllegalArgumentException( - "The parent queue:" + parent.getQueuePath() - + " cannot be STOPPED as the child queue:" + getQueuePath() - + " is in RUNNING state."); - } else { - updateQueueState(configuredState); - } - } - } else { - // when we get a refreshQueue request from AdminService, - if (previousState == QueueState.RUNNING) { - if (configuredState == QueueState.STOPPED) { - stopQueue(); - } - } else { - if (configuredState == QueueState.RUNNING) { - try { - activateQueue(); - } catch (YarnException ex) { - throw new IllegalArgumentException(ex.getMessage()); - } - } - } - } - } - protected QueueInfo getQueueInfo() { // Deliberately doesn't use lock here, because this method will be invoked // from schedulerApplicationAttempt, to avoid deadlock, sacrifice diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java index 244bb62d508d8..c6d50a1cfb7c5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java @@ -19,11 +19,12 @@ import java.util.Set; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.util.Sets; +import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; -import org.apache.hadoop.yarn.server.resourcemanager.placement.ApplicationPlacementContext; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.utils.Lock; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; @@ -302,15 +303,4 @@ public static void updateAbsoluteCapacitiesByNodeLabels(QueueCapacities queueCap } } } - - public static ApplicationPlacementContext extractQueuePath(String queuePath) { - int parentQueueNameEndIndex = queuePath.lastIndexOf("."); - if (parentQueueNameEndIndex > -1) { - String parent = queuePath.substring(0, parentQueueNameEndIndex).trim(); - String leaf = queuePath.substring(parentQueueNameEndIndex + 1).trim(); - return new ApplicationPlacementContext(leaf, parent); - } else{ - return new ApplicationPlacementContext(queuePath); - } - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueStateHelper.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueStateHelper.java new file mode 100644 index 0000000000000..5ec7d01bce9b1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueStateHelper.java @@ -0,0 +1,98 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hadoop.yarn.api.records.QueueState; +import org.apache.hadoop.yarn.exceptions.YarnException; + +import java.util.Set; + +/** + * Collects all logic that are handling queue state transitions. + */ +public final class QueueStateHelper { + private static final Set VALID_STATE_CONFIGURATIONS = ImmutableSet.of( + QueueState.RUNNING, QueueState.STOPPED); + private static final QueueState DEFAULT_STATE = QueueState.RUNNING; + + private QueueStateHelper() {} + + /** + * Sets the current state of the queue based on its previous state, its parent's state and its + * configured state. + * @param queue the queue whose state is set + */ + public static void setQueueState(AbstractCSQueue queue) { + QueueState previousState = queue.getState(); + QueueState configuredState = queue.getQueueContext().getConfiguration().getConfiguredState( + queue.getQueuePath()); + QueueState parentState = (queue.getParent() == null) ? null : queue.getParent().getState(); + + // verify that we can not any value for State other than RUNNING/STOPPED + if (configuredState != null && !VALID_STATE_CONFIGURATIONS.contains(configuredState)) { + throw new IllegalArgumentException("Invalid queue state configuration." + + " We can only use RUNNING or STOPPED."); + } + + if (previousState == null) { + initializeState(queue, configuredState, parentState); + } else { + reinitializeState(queue, previousState, configuredState); + } + } + + private static void reinitializeState( + AbstractCSQueue queue, QueueState previousState, QueueState configuredState) { + // when we get a refreshQueue request from AdminService, + if (previousState == QueueState.RUNNING) { + if (configuredState == QueueState.STOPPED) { + queue.stopQueue(); + } + } else { + if (configuredState == QueueState.RUNNING) { + try { + queue.activateQueue(); + } catch (YarnException ex) { + throw new IllegalArgumentException(ex.getMessage()); + } + } + } + } + + private static void initializeState( + AbstractCSQueue queue, QueueState configuredState, QueueState parentState) { + QueueState currentState = configuredState == null ? DEFAULT_STATE : configuredState; + + if (parentState != null) { + if (configuredState == QueueState.RUNNING && parentState != QueueState.RUNNING) { + throw new IllegalArgumentException( + "The parent queue:" + queue.getParent().getQueuePath() + + " cannot be STOPPED as the child queue:" + queue.getQueuePath() + + " is in RUNNING state."); + } + + if (configuredState == null) { + currentState = parentState == QueueState.DRAINING ? QueueState.STOPPED : parentState; + } + } + + queue.updateQueueState(currentState); + } +} From 6995374b54e7c5ae436e74103fcf1ad5e2a8f4aa Mon Sep 17 00:00:00 2001 From: Mehakmeet Singh Date: Fri, 4 Mar 2022 00:14:28 +0530 Subject: [PATCH 016/145] HADOOP-18150. Fix ITestAuditManagerDisabled test in S3A. (#4044) Contributed by Mehakmeet Singh --- .../hadoop/fs/s3a/audit/ITestAuditManagerDisabled.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManagerDisabled.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManagerDisabled.java index 033d793a6cbd9..68aaef51d60f6 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManagerDisabled.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditManagerDisabled.java @@ -28,9 +28,10 @@ import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.NOOP_SPAN; import static org.apache.hadoop.fs.s3a.audit.AuditTestSupport.resetAuditOptions; +import static org.apache.hadoop.fs.s3a.audit.S3AAuditConstants.AUDIT_ENABLED; /** - * Verify that by default audit managers are disabled. + * Verify that audit managers are disabled if set to false. */ public class ITestAuditManagerDisabled extends AbstractS3ACostTest { @@ -42,11 +43,12 @@ public ITestAuditManagerDisabled() { public Configuration createConfiguration() { Configuration conf = super.createConfiguration(); resetAuditOptions(conf); + conf.setBoolean(AUDIT_ENABLED, false); return conf; } /** - * The default auditor is the no-op auditor. + * Verify that the auditor is the no-op auditor if auditing is disabled. */ @Test public void testAuditorDisabled() { From 278568203b9c2033743ecca60dbc62d397a85a8d Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Fri, 4 Mar 2022 22:17:48 +0530 Subject: [PATCH 017/145] HDFS-16481. Provide support to set Http and Rpc ports in MiniJournalCluster (#4028). Contributed by Viraj Jasani. --- .../java/org/apache/hadoop/net/NetUtils.java | 26 +++++ .../hdfs/qjournal/MiniJournalCluster.java | 41 +++++++- .../hdfs/qjournal/TestMiniJournalCluster.java | 98 ++++++++++++++++++- 3 files changed, 161 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java index 4b924af03c196..fead87d7907d7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java @@ -1053,6 +1053,32 @@ public static int getFreeSocketPort() { return port; } + /** + * Return free ports. There is no guarantee they will remain free, so + * ports should be used immediately. The number of free ports returned by + * this method should match argument {@code numOfPorts}. Num of ports + * provided in the argument should not exceed 25. + * + * @param numOfPorts Number of free ports to acquire. + * @return Free ports for binding a local socket. + */ + public static Set getFreeSocketPorts(int numOfPorts) { + Preconditions.checkArgument(numOfPorts > 0 && numOfPorts <= 25, + "Valid range for num of ports is between 0 and 26"); + final Set freePorts = new HashSet<>(numOfPorts); + for (int i = 0; i < numOfPorts * 5; i++) { + int port = getFreeSocketPort(); + if (port == 0) { + continue; + } + freePorts.add(port); + if (freePorts.size() == numOfPorts) { + return freePorts; + } + } + throw new IllegalStateException(numOfPorts + " free ports could not be acquired."); + } + /** * Return an @{@link InetAddress} to bind to. If bindWildCardAddress is true * than returns null. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java index d0bbd44f1afbb..1c43b39159a99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.FAKE_NSINFO; import static org.junit.Assert.fail; +import java.io.Closeable; import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; @@ -45,13 +46,16 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; import org.apache.hadoop.test.GenericTestUtils; -public class MiniJournalCluster { +public final class MiniJournalCluster implements Closeable { + public static final String CLUSTER_WAITACTIVE_URI = "waitactive"; public static class Builder { private String baseDir; private int numJournalNodes = 3; private boolean format = true; private final Configuration conf; + private int[] httpPorts = null; + private int[] rpcPorts = null; static { DefaultMetricsSystem.setMiniClusterMode(true); @@ -76,6 +80,16 @@ public Builder format(boolean f) { return this; } + public Builder setHttpPorts(int... ports) { + this.httpPorts = ports; + return this; + } + + public Builder setRpcPorts(int... ports) { + this.rpcPorts = ports; + return this; + } + public MiniJournalCluster build() throws IOException { return new MiniJournalCluster(this); } @@ -99,6 +113,19 @@ private JNInfo(JournalNode node) { private final JNInfo[] nodes; private MiniJournalCluster(Builder b) throws IOException { + + if (b.httpPorts != null && b.httpPorts.length != b.numJournalNodes) { + throw new IllegalArgumentException( + "Num of http ports (" + b.httpPorts.length + ") should match num of JournalNodes (" + + b.numJournalNodes + ")"); + } + + if (b.rpcPorts != null && b.rpcPorts.length != b.numJournalNodes) { + throw new IllegalArgumentException( + "Num of rpc ports (" + b.rpcPorts.length + ") should match num of JournalNodes (" + + b.numJournalNodes + ")"); + } + LOG.info("Starting MiniJournalCluster with " + b.numJournalNodes + " journal nodes"); @@ -173,8 +200,10 @@ private Configuration createConfForNode(Builder b, int idx) { Configuration conf = new Configuration(b.conf); File logDir = getStorageDir(idx); conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, logDir.toString()); - conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "localhost:0"); - conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "localhost:0"); + int httpPort = b.httpPorts != null ? b.httpPorts[idx] : 0; + int rpcPort = b.rpcPorts != null ? b.rpcPorts[idx] : 0; + conf.set(DFSConfigKeys.DFS_JOURNALNODE_RPC_ADDRESS_KEY, "localhost:" + rpcPort); + conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "localhost:" + httpPort); return conf; } @@ -274,4 +303,10 @@ public void setNamenodeSharedEditsConf(String jid) { .DFS_NAMENODE_SHARED_EDITS_DIR_KEY, quorumJournalURI.toString()); } } + + @Override + public void close() throws IOException { + this.shutdown(); + } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java index cace7c92891ab..ccbbc94c99ede 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestMiniJournalCluster.java @@ -22,15 +22,23 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.qjournal.server.JournalNode; -import org.junit.Test; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TestMiniJournalCluster { + + private static final Logger LOG = LoggerFactory.getLogger(TestMiniJournalCluster.class); + @Test public void testStartStop() throws IOException { Configuration conf = new Configuration(); @@ -52,4 +60,92 @@ public void testStartStop() throws IOException { c.shutdown(); } } + + @Test + public void testStartStopWithPorts() throws Exception { + Configuration conf = new Configuration(); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of http ports (1) should match num of JournalNodes (3)", + "MiniJournalCluster port validation failed", + () -> { + new MiniJournalCluster.Builder(conf).setHttpPorts(8481).build(); + }); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of rpc ports (2) should match num of JournalNodes (3)", + "MiniJournalCluster port validation failed", + () -> { + new MiniJournalCluster.Builder(conf).setRpcPorts(8481, 8482).build(); + }); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of rpc ports (1) should match num of JournalNodes (3)", + "MiniJournalCluster port validation failed", + () -> { + new MiniJournalCluster.Builder(conf).setHttpPorts(800, 9000, 10000).setRpcPorts(8481) + .build(); + }); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of http ports (4) should match num of JournalNodes (3)", + "MiniJournalCluster port validation failed", + () -> { + new MiniJournalCluster.Builder(conf).setHttpPorts(800, 9000, 1000, 2000) + .setRpcPorts(8481, 8482, 8483).build(); + }); + + final Set httpAndRpcPorts = NetUtils.getFreeSocketPorts(6); + LOG.info("Free socket ports: {}", httpAndRpcPorts); + + for (Integer httpAndRpcPort : httpAndRpcPorts) { + assertNotEquals("None of the acquired socket port should not be zero", 0, + httpAndRpcPort.intValue()); + } + + final int[] httpPorts = new int[3]; + final int[] rpcPorts = new int[3]; + int httpPortIdx = 0; + int rpcPortIdx = 0; + for (Integer httpAndRpcPort : httpAndRpcPorts) { + if (httpPortIdx < 3) { + httpPorts[httpPortIdx++] = httpAndRpcPort; + } else { + rpcPorts[rpcPortIdx++] = httpAndRpcPort; + } + } + + LOG.info("Http ports selected: {}", httpPorts); + LOG.info("Rpc ports selected: {}", rpcPorts); + + try (MiniJournalCluster miniJournalCluster = new MiniJournalCluster.Builder(conf) + .setHttpPorts(httpPorts) + .setRpcPorts(rpcPorts).build()) { + miniJournalCluster.waitActive(); + URI uri = miniJournalCluster.getQuorumJournalURI("myjournal"); + String[] addrs = uri.getAuthority().split(";"); + assertEquals(3, addrs.length); + + assertEquals(httpPorts[0], miniJournalCluster.getJournalNode(0).getHttpAddress().getPort()); + assertEquals(httpPorts[1], miniJournalCluster.getJournalNode(1).getHttpAddress().getPort()); + assertEquals(httpPorts[2], miniJournalCluster.getJournalNode(2).getHttpAddress().getPort()); + + assertEquals(rpcPorts[0], + miniJournalCluster.getJournalNode(0).getRpcServer().getAddress().getPort()); + assertEquals(rpcPorts[1], + miniJournalCluster.getJournalNode(1).getRpcServer().getAddress().getPort()); + assertEquals(rpcPorts[2], + miniJournalCluster.getJournalNode(2).getRpcServer().getAddress().getPort()); + + JournalNode node = miniJournalCluster.getJournalNode(0); + String dir = node.getConf().get(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY); + assertEquals(new File(MiniDFSCluster.getBaseDirectory() + "journalnode-0").getAbsolutePath(), + dir); + } + } + } From 19561846dfb0821b5d4fc2fbd250467e2b7ddd54 Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Sat, 5 Mar 2022 07:11:22 +0530 Subject: [PATCH 018/145] HADOOP-18151. Switch the baseurl for Centos 8 (#4047) * Centos 8 has reached its End-of-Life and thus its packages are no longer accessible from mirror.centos.org. * This PR switches the baseurl to vault.centos.org where the packages are archived. --- dev-support/docker/Dockerfile_centos_8 | 7 ++++ .../set-vault-as-baseurl-centos.sh | 33 +++++++++++++++++++ .../native/libhdfspp/tests/CMakeLists.txt | 1 - 3 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 dev-support/docker/pkg-resolver/set-vault-as-baseurl-centos.sh diff --git a/dev-support/docker/Dockerfile_centos_8 b/dev-support/docker/Dockerfile_centos_8 index 7b82c4997dee6..8f3b008f7ba03 100644 --- a/dev-support/docker/Dockerfile_centos_8 +++ b/dev-support/docker/Dockerfile_centos_8 @@ -30,6 +30,13 @@ COPY pkg-resolver pkg-resolver RUN chmod a+x pkg-resolver/*.sh pkg-resolver/*.py \ && chmod a+r pkg-resolver/*.json +###### +# Centos 8 has reached its EOL and the packages +# are no longer available on mirror.centos.org site. +# Please see https://www.centos.org/centos-linux-eol/ +###### +RUN pkg-resolver/set-vault-as-baseurl-centos.sh centos:8 + ###### # Install packages from yum ###### diff --git a/dev-support/docker/pkg-resolver/set-vault-as-baseurl-centos.sh b/dev-support/docker/pkg-resolver/set-vault-as-baseurl-centos.sh new file mode 100644 index 0000000000000..4be4cd956b15b --- /dev/null +++ b/dev-support/docker/pkg-resolver/set-vault-as-baseurl-centos.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +# 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. + +if [ $# -lt 1 ]; then + echo "ERROR: Need at least 1 argument, $# were provided" + exit 1 +fi + +if [ "$1" == "centos:7" ] || [ "$1" == "centos:8" ]; then + cd /etc/yum.repos.d/ || exit && + sed -i 's/mirrorlist/#mirrorlist/g' /etc/yum.repos.d/CentOS-* && + sed -i 's|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g' /etc/yum.repos.d/CentOS-* && + yum update -y && + cd /root || exit +else + echo "ERROR: Setting the archived baseurl is only supported for centos 7 and 8 environments" + exit 1 +fi diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt index d7d20ec36007a..1a88b5c81e521 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/CMakeLists.txt @@ -136,7 +136,6 @@ target_link_libraries(hdfs_config_connect_bugs_test common gmock_main bindings_c add_memcheck_test(hdfs_config_connect_bugs hdfs_config_connect_bugs_test) - # # # INTEGRATION TESTS - TESTS THE FULL LIBRARY AGAINST ACTUAL SERVERS From 8f07081789e4690e2c5023c0f07ca6088d937632 Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Sat, 5 Mar 2022 23:05:14 +0530 Subject: [PATCH 019/145] HDFS-16462. Make HDFS get tool cross platform (#4003) --- .../libhdfspp/tests/tools/CMakeLists.txt | 3 + .../libhdfspp/tests/tools/hdfs-get-mock.cc | 56 ++++++++++++ .../libhdfspp/tests/tools/hdfs-get-mock.h | 68 ++++++++++++++ .../libhdfspp/tests/tools/hdfs-tool-tests.cc | 14 +++ .../native/libhdfspp/tools/CMakeLists.txt | 3 +- .../hdfs-copy-to-local/hdfs-copy-to-local.cc | 14 ++- .../hdfs-copy-to-local/hdfs-copy-to-local.h | 5 ++ .../libhdfspp/tools/hdfs-get/CMakeLists.txt | 27 ++++++ .../libhdfspp/tools/hdfs-get/hdfs-get.cc | 25 ++++++ .../libhdfspp/tools/hdfs-get/hdfs-get.h | 47 ++++++++++ .../native/libhdfspp/tools/hdfs-get/main.cc | 52 +++++++++++ .../main/native/libhdfspp/tools/hdfs_get.cc | 88 ------------------- 12 files changed, 308 insertions(+), 94 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/CMakeLists.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/main.cc delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_get.cc diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt index 56755adac3f90..769e5da0f1ccd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt @@ -35,6 +35,7 @@ add_executable(hdfs_tool_tests hdfs-count-mock.cc hdfs-mkdir-mock.cc hdfs-rm-mock.cc + hdfs-get-mock.cc main.cc) target_include_directories(hdfs_tool_tests PRIVATE ../tools @@ -54,6 +55,7 @@ target_include_directories(hdfs_tool_tests PRIVATE ../../tools/hdfs-count ../../tools/hdfs-mkdir ../../tools/hdfs-rm + ../../tools/hdfs-get ../../tools/hdfs-cat) target_link_libraries(hdfs_tool_tests PRIVATE gmock_main @@ -72,5 +74,6 @@ target_link_libraries(hdfs_tool_tests PRIVATE hdfs_count_lib hdfs_mkdir_lib hdfs_rm_lib + hdfs_get_lib hdfs_cat_lib) add_test(hdfs_tool_tests hdfs_tool_tests) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.cc new file mode 100644 index 0000000000000..713564e45b160 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.cc @@ -0,0 +1,56 @@ +/** + * 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. + */ + +#include +#include +#include +#include + +#include +#include + +#include "hdfs-get-mock.h" +#include "hdfs-tool-tests.h" + +namespace hdfs::tools::test { +GetMock::~GetMock() = default; + +void GetMock::SetExpectations( + std::function()> test_case, + const std::vector &args) const { + // Get the pointer to the function that defines the test case + const auto test_case_func = + test_case.target (*)()>(); + ASSERT_NE(test_case_func, nullptr); + + // Set the expected method calls and their corresponding arguments for each + // test case + if (*test_case_func == &CallHelp) { + EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true)); + return; + } + + if (*test_case_func == &Pass2Paths) { + const auto arg1 = args[0]; + const auto arg2 = args[1]; + EXPECT_CALL(*this, HandlePath(arg1, arg2)) + .Times(1) + .WillOnce(testing::Return(true)); + } +} +} // namespace hdfs::tools::test diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.h new file mode 100644 index 0000000000000..535f7153f1f98 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-get-mock.h @@ -0,0 +1,68 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_GET_MOCK +#define LIBHDFSPP_TOOLS_HDFS_GET_MOCK + +#include +#include +#include +#include + +#include + +#include "hdfs-get.h" + +namespace hdfs::tools::test { +/** + * {@class GetMock} is an {@class Get} whereby it mocks the + * HandleHelp and HandlePath methods for testing their functionality. + */ +class GetMock : public hdfs::tools::Get { +public: + /** + * {@inheritdoc} + */ + GetMock(const int argc, char **argv) : Get(argc, argv) {} + + // Abiding to the Rule of 5 + GetMock(const GetMock &) = delete; + GetMock(GetMock &&) = delete; + GetMock &operator=(const GetMock &) = delete; + GetMock &operator=(GetMock &&) = delete; + ~GetMock() override; + + /** + * Defines the methods and the corresponding arguments that are expected + * to be called on this instance of {@link HdfsTool} for the given test case. + * + * @param test_case An {@link std::function} object that points to the + * function defining the test case + * @param args The arguments that are passed to this test case + */ + void SetExpectations(std::function()> test_case, + const std::vector &args = {}) const; + + MOCK_METHOD(bool, HandleHelp, (), (const, override)); + + MOCK_METHOD(bool, HandlePath, (const std::string &, const std::string &), + (const, override)); +}; +} // namespace hdfs::tools::test + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc index 7678834a955ca..50d555aebda0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc @@ -31,6 +31,7 @@ #include "hdfs-df-mock.h" #include "hdfs-disallow-snapshot-mock.h" #include "hdfs-du-mock.h" +#include "hdfs-get-mock.h" #include "hdfs-mkdir-mock.h" #include "hdfs-move-to-local-mock.h" #include "hdfs-rename-snapshot-mock.h" @@ -108,6 +109,11 @@ INSTANTIATE_TEST_SUITE_P( testing::Values(CallHelp, Pass2Paths)); +INSTANTIATE_TEST_SUITE_P( + HdfsGet, HdfsToolBasicTest, + testing::Values(CallHelp, + Pass2Paths)); + INSTANTIATE_TEST_SUITE_P( HdfsMoveToLocal, HdfsToolBasicTest, testing::Values(CallHelp, @@ -173,6 +179,10 @@ INSTANTIATE_TEST_SUITE_P( HdfsCopyToLocal, HdfsToolNegativeTestThrows, testing::Values(Pass3Paths)); +INSTANTIATE_TEST_SUITE_P( + HdfsGet, HdfsToolNegativeTestThrows, + testing::Values(Pass3Paths)); + INSTANTIATE_TEST_SUITE_P( HdfsMoveToLocal, HdfsToolNegativeTestThrows, testing::Values(Pass3Paths)); @@ -220,6 +230,10 @@ INSTANTIATE_TEST_SUITE_P( HdfsCopyToLocal, HdfsToolNegativeTestNoThrow, testing::Values(PassAPath)); +INSTANTIATE_TEST_SUITE_P( + HdfsGet, HdfsToolNegativeTestNoThrow, + testing::Values(PassAPath)); + INSTANTIATE_TEST_SUITE_P( HdfsDeleteSnapshot, HdfsToolNegativeTestNoThrow, testing::Values(PassAPath)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt index aa82c018b3871..bed78b7a5777d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt @@ -60,8 +60,7 @@ add_subdirectory(hdfs-df) add_subdirectory(hdfs-du) -add_executable(hdfs_get hdfs_get.cc) -target_link_libraries(hdfs_get tools_common hdfspp_static) +add_subdirectory(hdfs-get) add_subdirectory(hdfs-copy-to-local) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.cc index 9219b36ffd5da..7affa1fbdc92a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.cc @@ -64,7 +64,8 @@ bool CopyToLocal::ValidateConstraints() const { std::string CopyToLocal::GetDescription() const { std::stringstream desc; - desc << "Usage: hdfs_copyToLocal [OPTION] SRC_FILE DST_FILE" << std::endl + desc << "Usage: hdfs_" << GetToolName() << " [OPTION] SRC_FILE DST_FILE" + << std::endl << std::endl << "Copy SRC_FILE from hdfs to DST_FILE on the local file system." << std::endl @@ -72,16 +73,19 @@ std::string CopyToLocal::GetDescription() const { << " -h display this help and exit" << std::endl << std::endl << "Examples:" << std::endl - << "hdfs_copyToLocal hdfs://localhost.localdomain:8020/dir/file " + << "hdfs_" << GetToolName() + << " hdfs://localhost.localdomain:8020/dir/file " "/home/usr/myfile" << std::endl - << "hdfs_copyToLocal /dir/file /home/usr/dir/file" << std::endl; + << "hdfs_" << GetToolName() << " /dir/file /home/usr/dir/file" + << std::endl; return desc.str(); } bool CopyToLocal::Do() { if (!Initialize()) { - std::cerr << "Unable to initialize HDFS copyToLocal tool" << std::endl; + std::cerr << "Unable to initialize HDFS " << GetToolName() << " tool" + << std::endl; return false; } @@ -129,4 +133,6 @@ bool CopyToLocal::HandlePath(const std::string &source, std::fclose(dst_file); return true; } + +std::string CopyToLocal::GetToolName() const { return "copyToLocal"; } } // namespace hdfs::tools diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.h index 6eb8cf1b922f9..0137f1e614d1c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-copy-to-local/hdfs-copy-to-local.h @@ -81,6 +81,11 @@ class CopyToLocal : public HdfsTool { [[nodiscard]] virtual bool HandlePath(const std::string &source, const std::string &target) const; + /** + * @return The name of the tool. + */ + [[nodiscard]] virtual std::string GetToolName() const; + private: /** * A boost data-structure containing the description of positional arguments diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/CMakeLists.txt new file mode 100644 index 0000000000000..367bca6a0279c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/CMakeLists.txt @@ -0,0 +1,27 @@ +# +# 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. +# + +add_library(hdfs_get_lib STATIC $ hdfs-get.cc) +target_include_directories(hdfs_get_lib PRIVATE ../../tools hdfs-get ${Boost_INCLUDE_DIRS}) +target_link_libraries(hdfs_get_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static hdfs_copyToLocal_lib) + +add_executable(hdfs_get main.cc) +target_include_directories(hdfs_get PRIVATE ../../tools) +target_link_libraries(hdfs_get PRIVATE hdfs_get_lib) + +install(TARGETS hdfs_get RUNTIME DESTINATION bin) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.cc new file mode 100644 index 0000000000000..e45c1d0215b26 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.cc @@ -0,0 +1,25 @@ +/** + * 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. + */ + +#include "hdfs-get.h" + +namespace hdfs::tools { +Get::Get(const int argc, char **argv) : CopyToLocal(argc, argv) {} + +std::string Get::GetToolName() const { return "get"; } +} // namespace hdfs::tools diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.h new file mode 100644 index 0000000000000..8153264a72a3e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/hdfs-get.h @@ -0,0 +1,47 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_GET +#define LIBHDFSPP_TOOLS_HDFS_GET + +#include "hdfs-copy-to-local/hdfs-copy-to-local.h" + +namespace hdfs::tools { +class Get : public CopyToLocal { +public: + /** + * {@inheritdoc} + */ + Get(int argc, char **argv); + + // Abiding to the Rule of 5 + Get(const Get &) = default; + Get(Get &&) = default; + Get &operator=(const Get &) = delete; + Get &operator=(Get &&) = delete; + ~Get() override = default; + +protected: + /** + * {@inheritdoc} + */ + [[nodiscard]] std::string GetToolName() const override; +}; +} // namespace hdfs::tools + +#endif \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/main.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/main.cc new file mode 100644 index 0000000000000..916e60a4631c6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-get/main.cc @@ -0,0 +1,52 @@ +/** + * 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. + */ + +#include +#include +#include + +#include + +#include "hdfs-get.h" + +int main(int argc, char *argv[]) { + const auto result = std::atexit([]() -> void { + // Clean up static data on exit and prevent valgrind memory leaks + google::protobuf::ShutdownProtobufLibrary(); + }); + if (result != 0) { + std::cerr << "Error: Unable to schedule clean-up tasks for HDFS " + "get tool, exiting" + << std::endl; + std::exit(EXIT_FAILURE); + } + + hdfs::tools::Get get(argc, argv); + auto success = false; + + try { + success = get.Do(); + } catch (const std::exception &e) { + std::cerr << "Error: " << e.what() << std::endl; + } + + if (!success) { + std::exit(EXIT_FAILURE); + } + return 0; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_get.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_get.cc deleted file mode 100644 index 16dd72d24bf72..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_get.cc +++ /dev/null @@ -1,88 +0,0 @@ -/* - 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. -*/ - -#include -#include -#include "tools_common.h" - -void usage(){ - std::cout << "Usage: hdfs_get [OPTION] SRC_FILE DST_FILE" - << std::endl - << std::endl << "Copy SRC_FILE from hdfs to DST_FILE on the local file system." - << std::endl - << std::endl << " -h display this help and exit" - << std::endl - << std::endl << "Examples:" - << std::endl << "hdfs_get hdfs://localhost.localdomain:8020/dir/file /home/usr/myfile" - << std::endl << "hdfs_get /dir/file /home/usr/dir/file" - << std::endl; -} - -int main(int argc, char *argv[]) { - if (argc > 4) { - usage(); - exit(EXIT_FAILURE); - } - - int input; - - //Using GetOpt to read in the values - opterr = 0; - while ((input = getopt(argc, argv, "h")) != -1) { - switch (input) - { - case 'h': - usage(); - exit(EXIT_SUCCESS); - case '?': - if (isprint(optopt)) - std::cerr << "Unknown option `-" << (char) optopt << "'." << std::endl; - else - std::cerr << "Unknown option character `" << (char) optopt << "'." << std::endl; - usage(); - exit(EXIT_FAILURE); - default: - exit(EXIT_FAILURE); - } - } - - std::string uri_path = argv[optind]; - std::string dest = argv[optind+1]; - - //Building a URI object from the given uri_path - hdfs::URI uri = hdfs::parse_path_or_exit(uri_path); - - std::shared_ptr fs = hdfs::doConnect(uri, false); - if (!fs) { - std::cerr << "Could not connect the file system. " << std::endl; - exit(EXIT_FAILURE); - } - - std::FILE* dst_file = std::fopen(dest.c_str(), "wb"); - if(!dst_file){ - std::cerr << "Unable to open the destination file: " << dest << std::endl; - exit(EXIT_FAILURE); - } - readFile(fs, uri.get_path(), 0, dst_file, false); - std::fclose(dst_file); - - // Clean up static data and prevent valgrind memory leaks - google::protobuf::ShutdownProtobufLibrary(); - return 0; -} From 851a4dc2d90374dc8ea6c0ea1e319af4756e6727 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Mon, 7 Mar 2022 09:00:22 +0100 Subject: [PATCH 020/145] YARN-11049. MutableConfScheduler is referred as plain String instead of class name. Contributed by Szilard Nemeth --- .../resourcemanager/ResourceManager.java | 4 ++-- .../resourcemanager/webapp/RMWebServices.java | 19 ++++++++++--------- .../webapp/TestRMWebServices.java | 5 +++-- 3 files changed, 15 insertions(+), 13 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index 357cbf3f15291..3bd6a0fe43a8c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -1864,8 +1864,8 @@ static void deleteRMConfStore(Configuration conf) throws Exception { confStore.initialize(conf, conf, rmContext); confStore.format(); } else { - System.out.println("Scheduler Configuration format only " + - "supported by MutableConfScheduler."); + System.out.println(String.format("Scheduler Configuration format only " + + "supported by %s.", MutableConfScheduler.class.getSimpleName())); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java index 1dac043ae16b6..26c921796b3c4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServices.java @@ -2678,7 +2678,7 @@ public Response formatSchedulerConfiguration(@Context HttpServletRequest hsr) } else { return Response.status(Status.BAD_REQUEST) .entity("Scheduler Configuration format only supported by " + - "MutableConfScheduler.").build(); + MutableConfScheduler.class.getSimpleName()).build(); } } @@ -2727,8 +2727,8 @@ public synchronized Response validateAndGetSchedulerConfiguration( .build(); } } else { - String errorMsg = "Configuration change validation only supported by " + - "MutableConfScheduler."; + String errorMsg = String.format("Configuration change validation only supported by %s.", + MutableConfScheduler.class.getSimpleName()); LOG.warn(errorMsg); return Response.status(Status.BAD_REQUEST) .entity(errorMsg) @@ -2823,10 +2823,11 @@ public Response getSchedulerConfiguration(@Context HttpServletRequest hsr) .build(); } else { return Response.status(Status.BAD_REQUEST).entity( - "This API only supports to retrieve scheduler configuration" - + " from a mutable-conf scheduler, underneath scheduler " - + scheduler.getClass().getSimpleName() - + " is not an instance of MutableConfScheduler") + String.format("This API only supports to retrieve scheduler configuration" + + " from a mutable-conf scheduler, underneath scheduler %s" + + " is not an instance of %s", + scheduler.getClass().getSimpleName(), + MutableConfScheduler.class.getSimpleName())) .build(); } } @@ -2858,8 +2859,8 @@ public Response getSchedulerConfigurationVersion(@Context } } else { return Response.status(Status.BAD_REQUEST) - .entity("Configuration Version only supported by " - + "MutableConfScheduler.").build(); + .entity(String.format("Configuration Version only supported by %s.", + MutableConfScheduler.class.getSimpleName())).build(); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java index d5d534395b54b..ce9de643744dd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServices.java @@ -75,6 +75,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.MutableConfScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; @@ -1003,8 +1004,8 @@ public void testValidateAndGetSchedulerConfigurationInvalidScheduler() Assert.assertEquals(Status.BAD_REQUEST .getStatusCode(), response.getStatus()); Assert.assertTrue(response.getEntity().toString() - .contains("Configuration change validation only supported by" - +" MutableConfScheduler.")); + .contains(String.format("Configuration change validation only supported by %s.", + MutableConfScheduler.class.getSimpleName()))); } @Test From da5a774018939616919adf5ec2da91930a8d1010 Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Mon, 7 Mar 2022 23:21:10 +0100 Subject: [PATCH 021/145] HADOOP-17563. Upgrade BouncyCastle to 1.68 (#3980) Contributed by PJ Fanning --- LICENSE-binary | 4 ++-- hadoop-project/pom.xml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index f5e8c707e6e0f..de4e1cb75b356 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -468,8 +468,8 @@ com.microsoft.azure:azure-cosmosdb-gateway:2.4.5 com.microsoft.azure:azure-data-lake-store-sdk:2.3.3 com.microsoft.azure:azure-keyvault-core:1.0.0 com.microsoft.sqlserver:mssql-jdbc:6.2.1.jre7 -org.bouncycastle:bcpkix-jdk15on:1.60 -org.bouncycastle:bcprov-jdk15on:1.60 +org.bouncycastle:bcpkix-jdk15on:1.68 +org.bouncycastle:bcprov-jdk15on:1.68 org.checkerframework:checker-qual:2.5.2 org.codehaus.mojo:animal-sniffer-annotations:1.17 org.jruby.jcodings:jcodings:1.0.13 diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index da9932be019b0..82dc6810d7cc9 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -109,7 +109,7 @@ 27.0-jre 4.2.3 - 1.60 + 1.68 2.0.0-M21 From 66b72406bd8bed28ca32c75e07fc2b682500e92b Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Tue, 8 Mar 2022 13:57:04 +0530 Subject: [PATCH 022/145] HADOOP-18131. Upgrade maven enforcer plugin and relevant dependencies (#4000) Reviewed-by: Akira Ajisaka Reviewed-by: Wei-Chiu Chuang Signed-off-by: Takanobu Asanuma --- .../hadoop-client-check-invariants/pom.xml | 2 +- .../pom.xml | 2 +- .../hadoop-client-integration-tests/pom.xml | 6 +++ .../hadoop-client-minicluster/pom.xml | 22 ++++++-- .../hadoop-cloud-storage/pom.xml | 4 ++ .../hadoop-huaweicloud/pom.xml | 12 +++-- hadoop-common-project/hadoop-common/pom.xml | 16 ++++++ .../hadoop-hdfs-client/pom.xml | 4 ++ hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml | 6 +++ .../hadoop-mapreduce-client-app/pom.xml | 6 +++ .../hadoop-mapreduce-client-core/pom.xml | 6 +++ .../hadoop-mapreduce-client-hs/pom.xml | 6 +++ .../hadoop-mapreduce-client-jobclient/pom.xml | 6 +++ .../hadoop-mapreduce-client/pom.xml | 10 ++++ .../hadoop-mapreduce-examples/pom.xml | 6 +++ hadoop-mapreduce-project/pom.xml | 4 ++ hadoop-maven-plugins/pom.xml | 18 +++++++ hadoop-minicluster/pom.xml | 6 +++ hadoop-project/pom.xml | 12 +++++ hadoop-tools/hadoop-aliyun/pom.xml | 6 +++ hadoop-tools/hadoop-archive-logs/pom.xml | 6 +++ hadoop-tools/hadoop-archives/pom.xml | 6 +++ hadoop-tools/hadoop-aws/pom.xml | 16 ++++++ hadoop-tools/hadoop-datajoin/pom.xml | 6 +++ hadoop-tools/hadoop-distcp/pom.xml | 6 +++ .../hadoop-dynamometer-infra/pom.xml | 6 +++ hadoop-tools/hadoop-extras/pom.xml | 16 ++++++ .../hadoop-federation-balance/pom.xml | 6 +++ hadoop-tools/hadoop-fs2img/pom.xml | 6 +++ hadoop-tools/hadoop-gridmix/pom.xml | 6 +++ hadoop-tools/hadoop-openstack/pom.xml | 6 +++ hadoop-tools/hadoop-rumen/pom.xml | 6 +++ hadoop-tools/hadoop-streaming/pom.xml | 6 +++ hadoop-tools/hadoop-tools-dist/pom.xml | 6 +++ .../pom.xml | 12 +++++ .../hadoop-yarn-services-core/pom.xml | 12 +++++ .../hadoop-yarn/hadoop-yarn-client/pom.xml | 6 +++ .../hadoop-yarn-server-common/pom.xml | 6 +++ .../pom.xml | 12 +++++ .../pom.xml | 6 +++ pom.xml | 51 +++++++++---------- 41 files changed, 332 insertions(+), 37 deletions(-) diff --git a/hadoop-client-modules/hadoop-client-check-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml index 9d4bce1fddccd..b1c00678406d7 100644 --- a/hadoop-client-modules/hadoop-client-check-invariants/pom.xml +++ b/hadoop-client-modules/hadoop-client-check-invariants/pom.xml @@ -56,7 +56,7 @@ org.codehaus.mojo extra-enforcer-rules - 1.0-beta-3 + 1.5.1 diff --git a/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml index 635250ec1ae1f..0e576ac6f0666 100644 --- a/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml +++ b/hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml @@ -60,7 +60,7 @@ org.codehaus.mojo extra-enforcer-rules - 1.0-beta-3 + 1.5.1 diff --git a/hadoop-client-modules/hadoop-client-integration-tests/pom.xml b/hadoop-client-modules/hadoop-client-integration-tests/pom.xml index 967782163f7e8..ba593ebd1b42d 100644 --- a/hadoop-client-modules/hadoop-client-integration-tests/pom.xml +++ b/hadoop-client-modules/hadoop-client-integration-tests/pom.xml @@ -184,6 +184,12 @@ hadoop-hdfs test test-jar + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml index 2a0a5c19230a6..d70198ac428fa 100644 --- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml +++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml @@ -332,6 +332,10 @@ org.apache.hadoop.thirdparty hadoop-shaded-guava + + org.ow2.asm + asm-commons + - com.sun.jersey - jersey-core + com.sun.jersey + jersey-core true @@ -451,9 +455,19 @@ true - com.sun.jersey - jersey-servlet + com.sun.jersey + jersey-servlet true + + + javax.servlet + servlet-api + + + javax.enterprise + cdi-api + + diff --git a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml index a8f45a7f3a222..33d3f9578172f 100644 --- a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml +++ b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml @@ -101,6 +101,10 @@ org.apache.zookeeper zookeeper + + org.projectlombok + lombok + diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml b/hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml index 3d8275943f8f3..b96883b9ac80d 100755 --- a/hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml @@ -100,10 +100,14 @@ hadoop-common provided - - jdk.tools - jdk.tools - + + jdk.tools + jdk.tools + + + org.javassist + javassist + diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index 84da42a3204fc..467c2717562ca 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -141,6 +141,16 @@ com.sun.jersey jersey-servlet compile + + + javax.enterprise + cdi-api + + + javax.servlet + servlet-api + + @@ -205,6 +215,12 @@ org.apache.commons commons-configuration2 compile + + + javax.servlet + servlet-api + + org.apache.commons diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml index d65e6030369b3..c4e65ef811dbf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml @@ -51,6 +51,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> log4j log4j + + org.slf4j + slf4j-ext + diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml index e17602d1f6466..d4d5c1eb33983 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml @@ -63,6 +63,12 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> org.apache.hadoop hadoop-hdfs provided + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml index c55cf8c3108a9..e3b3511c0ce17 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/pom.xml @@ -87,6 +87,12 @@ hadoop-hdfs test-jar test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml index 946dbe5e8bc72..957202a7dd0ba 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml @@ -45,6 +45,12 @@ org.apache.hadoop hadoop-hdfs-client + + + org.json + json + + org.apache.hadoop diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml index baf38d3c84511..79327a780fceb 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/pom.xml @@ -39,6 +39,12 @@ org.apache.hadoop hadoop-hdfs + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml index e98b574d1fe38..75f250e1d7271 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/pom.xml @@ -93,6 +93,12 @@ hadoop-hdfs test test-jar + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml index bf18d3edbc774..d6b453fb3b034 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml @@ -68,6 +68,10 @@ paranamer-ant com.thoughtworks.paranamer + + org.xerial.snappy + snappy-java + @@ -107,6 +111,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml index 48cf27efe437f..11932e04e3784 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/pom.xml @@ -74,6 +74,12 @@ hadoop-hdfs test test-jar + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-mapreduce-project/pom.xml b/hadoop-mapreduce-project/pom.xml index 09b4d42606118..3ce66a10a84f7 100644 --- a/hadoop-mapreduce-project/pom.xml +++ b/hadoop-mapreduce-project/pom.xml @@ -74,6 +74,10 @@ paranamer-ant com.thoughtworks.paranamer + + org.xerial.snappy + snappy-java + diff --git a/hadoop-maven-plugins/pom.xml b/hadoop-maven-plugins/pom.xml index a5473258c35f6..37daeb82cd506 100644 --- a/hadoop-maven-plugins/pom.xml +++ b/hadoop-maven-plugins/pom.xml @@ -39,6 +39,12 @@ org.apache.maven maven-core ${maven.dependency.version} + + + org.sonatype.sisu + sisu-inject-plexus + + org.apache.maven.plugin-tools @@ -98,6 +104,18 @@ org.sonatype.sisu sisu-inject-plexus + + org.apache.maven.plugin-tools + maven-plugin-annotations + + + org.sonatype.aether + aether-api + + + org.sonatype.aether + aether-util + diff --git a/hadoop-minicluster/pom.xml b/hadoop-minicluster/pom.xml index cf1c9417004c4..c0334b3fcc178 100644 --- a/hadoop-minicluster/pom.xml +++ b/hadoop-minicluster/pom.xml @@ -41,6 +41,12 @@ hadoop-hdfs compile test-jar + + + org.ow2.asm + asm-commons + + diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 82dc6810d7cc9..193f898fc7f66 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -858,6 +858,12 @@ com.sun.jersey jersey-core ${jersey.version} + + + org.osgi + org.osgi.core + + com.sun.jersey @@ -1783,6 +1789,12 @@ org.xerial.snappy snappy-java ${snappy-java.version} + + + org.osgi + org.osgi.core + + org.lz4 diff --git a/hadoop-tools/hadoop-aliyun/pom.xml b/hadoop-tools/hadoop-aliyun/pom.xml index c5da3e513949f..570032f370b57 100644 --- a/hadoop-tools/hadoop-aliyun/pom.xml +++ b/hadoop-tools/hadoop-aliyun/pom.xml @@ -117,6 +117,12 @@ org.apache.hadoop hadoop-common provided + + + org.projectlombok + lombok + + diff --git a/hadoop-tools/hadoop-archive-logs/pom.xml b/hadoop-tools/hadoop-archive-logs/pom.xml index 3caa26c52f884..f6154484ad270 100644 --- a/hadoop-tools/hadoop-archive-logs/pom.xml +++ b/hadoop-tools/hadoop-archive-logs/pom.xml @@ -59,6 +59,12 @@ hadoop-hdfs test test-jar + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-archives/pom.xml b/hadoop-tools/hadoop-archives/pom.xml index 966fff7ae7c5b..b16b88d11dada 100644 --- a/hadoop-tools/hadoop-archives/pom.xml +++ b/hadoop-tools/hadoop-archives/pom.xml @@ -79,6 +79,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 4aa38f68bb54f..5583bb7ad05ec 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -413,6 +413,16 @@ org.apache.hadoop hadoop-common provided + + + javax.servlet + servlet-api + + + javax.enterprise + cdi-api + + org.apache.hadoop @@ -472,6 +482,12 @@ hadoop-hdfs test test-jar + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-datajoin/pom.xml b/hadoop-tools/hadoop-datajoin/pom.xml index ec77281b66350..890e60a318513 100644 --- a/hadoop-tools/hadoop-datajoin/pom.xml +++ b/hadoop-tools/hadoop-datajoin/pom.xml @@ -90,6 +90,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-distcp/pom.xml b/hadoop-tools/hadoop-distcp/pom.xml index 55738ef808c28..5194e51d81fbf 100644 --- a/hadoop-tools/hadoop-distcp/pom.xml +++ b/hadoop-tools/hadoop-distcp/pom.xml @@ -85,6 +85,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/pom.xml b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/pom.xml index 7bd4318cf70ce..b31f26163265f 100644 --- a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/pom.xml +++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/pom.xml @@ -60,6 +60,12 @@ hadoop-hdfs test-jar compile + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-extras/pom.xml b/hadoop-tools/hadoop-extras/pom.xml index 160f8b4023efb..d6e0ba07dc4f8 100644 --- a/hadoop-tools/hadoop-extras/pom.xml +++ b/hadoop-tools/hadoop-extras/pom.xml @@ -74,6 +74,16 @@ org.apache.hadoop hadoop-common provided + + + javax.servlet + servlet-api + + + javax.enterprise + cdi-api + + org.apache.hadoop @@ -90,6 +100,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-federation-balance/pom.xml b/hadoop-tools/hadoop-federation-balance/pom.xml index 71f2cb3639137..63269b3878cbd 100644 --- a/hadoop-tools/hadoop-federation-balance/pom.xml +++ b/hadoop-tools/hadoop-federation-balance/pom.xml @@ -85,6 +85,12 @@ org.apache.hadoop hadoop-hdfs provided + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-fs2img/pom.xml b/hadoop-tools/hadoop-fs2img/pom.xml index 9a273621cfa9d..8c5ed0224bf14 100644 --- a/hadoop-tools/hadoop-fs2img/pom.xml +++ b/hadoop-tools/hadoop-fs2img/pom.xml @@ -41,6 +41,12 @@ org.apache.hadoop hadoop-hdfs provided + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-gridmix/pom.xml b/hadoop-tools/hadoop-gridmix/pom.xml index f383ecad64150..21d786c342d6c 100644 --- a/hadoop-tools/hadoop-gridmix/pom.xml +++ b/hadoop-tools/hadoop-gridmix/pom.xml @@ -90,6 +90,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-openstack/pom.xml b/hadoop-tools/hadoop-openstack/pom.xml index e1065883feb9e..1577de28505ad 100644 --- a/hadoop-tools/hadoop-openstack/pom.xml +++ b/hadoop-tools/hadoop-openstack/pom.xml @@ -100,6 +100,12 @@ org.apache.hadoop hadoop-common compile + + + javax.enterprise + cdi-api + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-rumen/pom.xml b/hadoop-tools/hadoop-rumen/pom.xml index fd3b5c9a1b8fa..9a966d55c0ed1 100644 --- a/hadoop-tools/hadoop-rumen/pom.xml +++ b/hadoop-tools/hadoop-rumen/pom.xml @@ -86,6 +86,12 @@ hadoop-hdfs test test-jar + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-streaming/pom.xml b/hadoop-tools/hadoop-streaming/pom.xml index f03eb001450b4..33e6ca8ff7491 100644 --- a/hadoop-tools/hadoop-streaming/pom.xml +++ b/hadoop-tools/hadoop-streaming/pom.xml @@ -96,6 +96,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop diff --git a/hadoop-tools/hadoop-tools-dist/pom.xml b/hadoop-tools/hadoop-tools-dist/pom.xml index cc811fca6949d..73b6ae075d611 100644 --- a/hadoop-tools/hadoop-tools-dist/pom.xml +++ b/hadoop-tools/hadoop-tools-dist/pom.xml @@ -38,6 +38,12 @@ org.apache.hadoop hadoop-streaming compile + + + org.projectlombok + lombok + + org.apache.hadoop diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml index 7e473895ca3a6..7e468bc75684e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/pom.xml @@ -142,6 +142,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + org.apache.hadoop @@ -153,6 +159,12 @@ hadoop-hdfs test test-jar + + + org.ow2.asm + asm-commons + + javax.ws.rs diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml index 5c7a9b7de862c..4daac9fa235ae 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/pom.xml @@ -185,6 +185,12 @@ org.apache.hadoop hadoop-hdfs + + + org.ow2.asm + asm-commons + + @@ -195,6 +201,12 @@ org.apache.commons commons-configuration2 + + + javax.servlet + servlet-api + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml index b7ec64934c420..dbe4e9048b2df 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/pom.xml @@ -42,6 +42,12 @@ org.apache.hadoop hadoop-hdfs test + + + org.ow2.asm + asm-commons + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml index 9b61479528bf1..3bf70bf7560e5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml @@ -125,6 +125,12 @@ org.apache.geronimo.specs geronimo-jcache_1.0_spec + + + org.osgi + org.osgi.core + + org.ehcache diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml index ccd1b13d1572a..17596405caf23 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/pom.xml @@ -44,6 +44,12 @@ hadoop-common test-jar test + + + org.javassist + javassist + + junit @@ -55,6 +61,12 @@ mockito-core 2.8.9 test + + + org.projectlombok + lombok + + org.powermock diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/pom.xml index 11be84e3c11fa..39706647b7a23 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-common/pom.xml @@ -46,6 +46,12 @@ org.apache.hadoop hadoop-yarn-api provided + + + com.google.guava + guava + + diff --git a/pom.xml b/pom.xml index 6e295b838333d..d3e95d990d020 100644 --- a/pom.xml +++ b/pom.xml @@ -100,8 +100,8 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x 1.7 2.4 3.0.2 - 3.0.0-M1 - 1.1.0 + 3.0.0 + 2.0.0 3.0.1 1.5 1.5 @@ -178,107 +178,104 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x - + true Use hadoop-thirdparty shaded instead of curator shaded org.apache.curator.shaded.** - static org.apache.curator.shaded.** - + true Use hadoop-common provided Sets rather than Guava provided Sets org.apache.hadoop.thirdparty.com.google.common.collect.Sets - static org.apache.hadoop.thirdparty.com.google.common.collect.Sets.** + org.apache.hadoop.thirdparty.com.google.common.collect.Sets.** - + true Use hadoop-common provided Lists rather than Guava provided Lists org.apache.hadoop.thirdparty.com.google.common.collect.Lists - static org.apache.hadoop.thirdparty.com.google.common.collect.Lists.** + org.apache.hadoop.thirdparty.com.google.common.collect.Lists.** - + true Use hadoop-annotation provided VisibleForTesting rather than the one provided by Guava org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting - + true Use alternatives to Guava common classes com.google.common.** - static com.google.common.** - + true Use alternative to Guava provided BaseEncoding org.apache.hadoop.thirdparty.com.google.common.io.BaseEncoding - static org.apache.hadoop.thirdparty.com.google.common.io.BaseEncoding.** + org.apache.hadoop.thirdparty.com.google.common.io.BaseEncoding.** - + true Use alternative to Guava provided Optional org.apache.hadoop.thirdparty.com.google.common.base.Optional - static org.apache.hadoop.thirdparty.com.google.common.base.Optional.** + org.apache.hadoop.thirdparty.com.google.common.base.Optional.** - + true Use alternative to Guava provided Function org.apache.hadoop.thirdparty.com.google.common.base.Function - static org.apache.hadoop.thirdparty.com.google.common.base.Function.** + org.apache.hadoop.thirdparty.com.google.common.base.Function.** - + true Use alternative to Guava provided Predicate org.apache.hadoop.thirdparty.com.google.common.base.Predicate - static org.apache.hadoop.thirdparty.com.google.common.base.Predicate.** + org.apache.hadoop.thirdparty.com.google.common.base.Predicate.** - + true Use alternative to Guava provided Supplier org.apache.hadoop.thirdparty.com.google.common.base.Supplier - static org.apache.hadoop.thirdparty.com.google.common.base.Supplier.** + org.apache.hadoop.thirdparty.com.google.common.base.Supplier.** - + true Use alternative to Guava provided ImmutableListMultimap org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableListMultimap - static org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableListMultimap.** + org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableListMultimap.** - + true Use hadoop-common provided Preconditions rather than Guava provided org.apache.hadoop.thirdparty.com.google.common.base.Preconditions - static org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.** + org.apache.hadoop.thirdparty.com.google.common.base.Preconditions.** - + true Use Fasterxml Jackson 2 dependency in place of org.codehaus Jackson 1 org.codehaus.jackson.** - static org.codehaus.jackson.** From 49e4e4d0a5049e61b17ac0890c17aa9fea9ccfb8 Mon Sep 17 00:00:00 2001 From: Stephen O'Donnell Date: Tue, 8 Mar 2022 11:07:11 +0000 Subject: [PATCH 023/145] HDFS-16496. Snapshot diff on snapshotable directory fails with not snapshottable error (#4051) --- .../hdfs/server/namenode/snapshot/SnapshotManager.java | 6 +++--- .../server/namenode/snapshot/TestSnapshotDiffReport.java | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java index 7d192db362c65..9e6fc36377cf5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotManager.java @@ -391,8 +391,8 @@ public INodeDirectory checkAndGetSnapshottableAncestorDir( final INodesInPath iip) throws IOException { final INodeDirectory dir = getSnapshottableAncestorDir(iip); if (dir == null) { - throw new SnapshotException("Directory is neither snapshottable nor" + - " under a snap root!"); + throw new SnapshotException("The path " + iip.getPath() + + " is neither snapshottable nor under a snapshot root!"); } return dir; } @@ -402,7 +402,7 @@ public INodeDirectory getSnapshottableAncestorDir(final INodesInPath iip) final String path = iip.getPath(); final INode inode = iip.getLastINode(); final INodeDirectory dir; - if (inode instanceof INodeDirectory) { + if (inode != null && inode.isDirectory()) { dir = INodeDirectory.valueOf(inode, path); } else { dir = INodeDirectory.valueOf(iip.getINode(-2), iip.getParentPath()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java index efe59ca0d943d..1d50e50d426f2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java @@ -345,7 +345,8 @@ public void testSnapRootDescendantDiffReport() throws Exception { + nonSnapDir.getName() + "'should fail!"); } catch (SnapshotException e) { GenericTestUtils.assertExceptionContains( - "Directory is neither snapshottable nor under a snap root!", e); + "The path " + nonSnapDir + + " is neither snapshottable nor under a snapshot root!", e); } final String invalidName = "invalid"; From 56d807175d87da40f39b6a5c59519da15a38aa30 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Tue, 8 Mar 2022 18:02:51 +0530 Subject: [PATCH 024/145] YARN-11081. TestYarnConfigurationFields consistently keeps failing (#4027) Signed-off-by: Akira Ajisaka --- .../src/main/resources/yarn-default.xml | 13 +++++++++++++ .../ApplicationPlacementAllocatorFactory.java | 3 ++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index 53b6b14cfd21b..39212626ae419 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -4846,4 +4846,17 @@ yarn.resourcemanager.enable-node-untracked-without-include-path false + + + yarn.scheduler.app-placement-allocator.class + + + In the absence of APPLICATION_PLACEMENT_TYPE_CLASS from the RM + application scheduling environments, the value of this config + is used to determine the default implementation of AppPlacementAllocator. + If APPLICATION_PLACEMENT_TYPE_CLASS is absent from the application + scheduling env and this config also has no value present, then + default implementation LocalityAppPlacementAllocator is used. + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ApplicationPlacementAllocatorFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ApplicationPlacementAllocatorFactory.java index a4e5484d81009..71d657debee69 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ApplicationPlacementAllocatorFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ApplicationPlacementAllocatorFactory.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.util.ReflectionUtils; @@ -45,7 +46,7 @@ public static AppPlacementAllocator getAppPlacementAllocator( SchedulerRequestKey schedulerRequestKey, RMContext rmContext) { Class policyClass; try { - if (appPlacementAllocatorName == null) { + if (StringUtils.isEmpty(appPlacementAllocatorName)) { policyClass = ApplicationSchedulingConfig.DEFAULT_APPLICATION_PLACEMENT_TYPE_CLASS; } else { policyClass = Class.forName(appPlacementAllocatorName); From 2ece95064b9bf11ec6a165baf6de7ced4986747f Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Wed, 9 Mar 2022 19:29:53 +0100 Subject: [PATCH 025/145] YARN-10945. Add javadoc to all methods of AbstractCSQueue. Contributed by Andras Gyori --- .../scheduler/capacity/AbstractCSQueue.java | 85 ++++++++++++++++++- 1 file changed, 81 insertions(+), 4 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index b0ab336b7ad59..87fa9dac34a8d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -77,6 +77,10 @@ import static org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.NO_LABEL; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT; +/** + * Provides implementation of {@code CSQueue} methods common for every queue class in Capacity + * Scheduler. + */ public abstract class AbstractCSQueue implements CSQueue { private static final Logger LOG = LoggerFactory.getLogger(AbstractCSQueue.class); @@ -106,11 +110,9 @@ public abstract class AbstractCSQueue implements CSQueue { CSQueueUsageTracker usageTracker; public enum CapacityConfigType { - // FIXME, from what I can see, Percentage mode can almost apply to weighted - // and percentage mode at the same time, there's only small area need to be - // changed, we need to rename "PERCENTAGE" to "PERCENTAGE" and "WEIGHT" NONE, PERCENTAGE, ABSOLUTE_RESOURCE }; + protected CapacityConfigType capacityConfigType = CapacityConfigType.NONE; @@ -175,6 +177,9 @@ private static QueuePath createQueuePath(CSQueue parent, String queueName) { return new QueuePath(parent.getQueuePath(), queueName); } + /** + * Sets up capacity and weight values from configuration. + */ protected void setupConfigurableCapacities() { CSQueueUtils.loadCapacitiesByLabelsFromConf(queuePath, queueCapacities, queueContext.getConfiguration(), this.queueNodeLabelsSettings.getConfiguredNodeLabels()); @@ -272,6 +277,12 @@ public Set getAccessibleNodeLabels() { return queueNodeLabelsSettings.getAccessibleNodeLabels(); } + /** + * Checks whether the user has the required permission to execute the action of {@code QueueACL}. + * @param acl the access type the user is checked for + * @param user UGI of the user + * @return true, if the user has permission, false otherwise + */ @Override public boolean hasAccess(QueueACL acl, UserGroupInformation user) { return authorizer.checkPermission( @@ -319,6 +330,11 @@ public String getDefaultNodeLabelExpression() { return this.queueNodeLabelsSettings.getDefaultLabelExpression(); } + /** + * Initialize queue properties that are based on configuration. + * @param clusterResource overall resource of the cluster + * @throws IOException if configuration is set in a way that is inconsistent + */ protected void setupQueueConfigs(Resource clusterResource) throws IOException { @@ -460,6 +476,11 @@ protected void updateCapacityConfigType() { } } + /** + * Initializes configured minimum and maximum capacity from configuration, if capacity is defined + * in ABSOLUTE node. + * @param clusterResource overall resource of the cluster + */ protected void updateConfigurableResourceLimits(Resource clusterResource) { for (String label : queueNodeLabelsSettings.getConfiguredNodeLabels()) { final Resource minResource = getMinimumAbsoluteResource(getQueuePath(), label); @@ -647,6 +668,13 @@ public Resource getMinimumAllocation() { return queueAllocationSettings.getMinimumAllocation(); } + /** + * Increments resource usage of the queue and all related statistics and metrics that depends on + * it. + * @param clusterResource overall cluster resource + * @param resource resource amount to increment + * @param nodePartition node label + */ void allocateResource(Resource clusterResource, Resource resource, String nodePartition) { writeLock.lock(); @@ -660,6 +688,13 @@ void allocateResource(Resource clusterResource, } } + /** + * Decrements resource usage of the queue and all related statistics and metrics that depends on + * it. + * @param clusterResource overall cluster resource + * @param resource resource amount to decrement + * @param nodePartition node label + */ protected void releaseResource(Resource clusterResource, Resource resource, String nodePartition) { writeLock.lock(); @@ -675,6 +710,10 @@ protected void releaseResource(Resource clusterResource, } } + /** + * Returns whether we should continue to look at all heart beating nodes even + * after the reservation limit was hit. + */ @Private public boolean isReservationsContinueLooking() { return reservationsContinueLooking; @@ -759,6 +798,15 @@ boolean hasChildQueues() { return childQueues != null && !childQueues.isEmpty(); } + /** + * Checks whether this queue has remaining resources left for further container assigment. + * @param clusterResource overall cluster resource + * @param nodePartition node label + * @param currentResourceLimits limit of the queue imposed by its maximum capacity + * @param resourceCouldBeUnreserved reserved resource that could potentially be unreserved + * @param schedulingMode scheduling strategy to handle node labels + * @return true if queue has remaining free resource, false otherwise + */ boolean canAssignToThisQueue(Resource clusterResource, String nodePartition, ResourceLimits currentResourceLimits, Resource resourceCouldBeUnreserved, SchedulingMode schedulingMode) { @@ -932,10 +980,14 @@ boolean hasPendingResourceRequest(String nodePartition, @Override public Priority getDefaultApplicationPriority() { - // TODO add dummy implementation return null; } + /** + * Returns the union of all node labels that could be accessed by this queue based on accessible + * node labels and configured node labels properties. + * @return node labels this queue has access to + */ @Override public Set getNodeLabelsForQueue() { // if queue's label is *, queue can access any labels. Instead of @@ -978,6 +1030,12 @@ public CSAssignment assignContainers(Resource clusterResource, resourceLimits, schedulingMode); } + /** + * Checks whether this queue could accept the container allocation request. + * @param cluster overall cluster resource + * @param request container allocation request + * @return true if queue could accept the container allocation request, false otherwise + */ @Override public boolean accept(Resource cluster, ResourceCommitRequest request) { @@ -1043,6 +1101,10 @@ public void updateQueueState(QueueState queueState) { this.state = queueState; } + /** + * Sets the state of this queue to RUNNING. + * @throws YarnException if its parent queue is not in RUNNING state + */ @Override public void activateQueue() throws YarnException { this.writeLock.lock(); @@ -1064,6 +1126,9 @@ public void activateQueue() throws YarnException { } } + /** + * Stops this queue if no application is currently running on the queue. + */ protected void appFinished() { this.writeLock.lock(); try { @@ -1087,6 +1152,9 @@ public UserWeights getUserWeights() { return userWeights; } + /** + * Recursively sets the state of this queue and the state of its parent to DRAINING. + */ public void recoverDrainingState() { this.writeLock.lock(); try { @@ -1308,10 +1376,19 @@ protected String getCapacityOrWeightString() { } } + /** + * Checks whether this queue is a dynamic queue and could be deleted. + * @return true if the dynamic queue could be deleted, false otherwise + */ public boolean isEligibleForAutoDeletion() { return false; } + /** + * Checks whether this queue is a dynamic queue and there has not been an application submission + * on it for a configured period of time. + * @return true if queue has been idle for a configured period of time, false otherwise + */ public boolean isInactiveDynamicQueue() { long idleDurationSeconds = (Time.monotonicNow() - getLastSubmittedTimestamp())/1000; From db8ae4b65448c506c9234641b2c1f9b8e894dc18 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Wed, 9 Mar 2022 19:36:09 +0100 Subject: [PATCH 026/145] YARN-10918. Simplify method: CapacitySchedulerQueueManager#parseQueue. Contributed by Andras Gyori --- .../scheduler/capacity/CSQueue.java | 7 ++ .../CapacitySchedulerQueueManager.java | 116 +++++++----------- .../scheduler/capacity/PlanQueue.java | 19 +++ .../scheduler/capacity/ReservationQueue.java | 6 + 4 files changed, 74 insertions(+), 74 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java index 90cb4f34ddecc..e2aeaab41808f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueue.java @@ -95,6 +95,13 @@ public interface CSQueue extends SchedulerQueue { */ QueuePath getQueuePathObject(); + /** + * Checks whether the queue is a dynamic queue (created dynamically in the fashion of auto queue + * creation v2). + * @return true, if it is a dynamic queue, false otherwise + */ + boolean isDynamicQueue(); + public PrivilegedEntity getPrivilegedEntity(); Resource getMaximumAllocation(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java index 50e8d5289c8ba..29e5de09d4bd8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java @@ -43,13 +43,11 @@ import org.apache.hadoop.yarn.security.Permission; import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; -import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueStateManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerQueueManager; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; import org.apache.hadoop.yarn.server.resourcemanager.security.AppPriorityACLsManager; import org.apache.hadoop.classification.VisibleForTesting; @@ -231,99 +229,62 @@ public void reinitializeQueues(CapacitySchedulerConfiguration newConf) * @throws IOException */ static CSQueue parseQueue( - CapacitySchedulerQueueContext queueContext, - CapacitySchedulerConfiguration conf, - CSQueue parent, String queueName, - CSQueueStore newQueues, - CSQueueStore oldQueues, + CapacitySchedulerQueueContext queueContext, CapacitySchedulerConfiguration conf, + CSQueue parent, String queueName, CSQueueStore newQueues, CSQueueStore oldQueues, QueueHook hook) throws IOException { CSQueue queue; - String fullQueueName = (parent == null) ? - queueName : - (parent.getQueuePath() + "." + queueName); + String fullQueueName = (parent == null) ? queueName : + (QueuePath.createFromQueues(parent.getQueuePath(), queueName).getFullPath()); String[] staticChildQueueNames = conf.getQueues(fullQueueName); List childQueueNames = staticChildQueueNames != null ? Arrays.asList(staticChildQueueNames) : Collections.emptyList(); - - boolean isReservableQueue = conf.isReservable(fullQueueName); - boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled( - fullQueueName); - // if a queue is eligible for auto queue creation v2 - // it must be a ParentQueue (even if it is empty) - boolean isAutoQueueCreationV2Enabled = conf.isAutoQueueCreationV2Enabled( - fullQueueName); - boolean isDynamicParent = false; - - // Auto created parent queues might not have static children, but they - // must be kept as a ParentQueue CSQueue oldQueue = oldQueues.get(fullQueueName); - if (oldQueue instanceof ParentQueue) { - isDynamicParent = ((ParentQueue) oldQueue).isDynamicQueue(); - } - if (childQueueNames.size() == 0 && !isDynamicParent && - !isAutoQueueCreationV2Enabled) { - if (null == parent) { - throw new IllegalStateException( - "Queue configuration missing child queue names for " + queueName); - } - // Check if the queue will be dynamically managed by the Reservation - // system + boolean isReservableQueue = conf.isReservable(fullQueueName); + boolean isAutoCreateEnabled = conf.isAutoCreateChildQueueEnabled(fullQueueName); + // if a queue is eligible for auto queue creation v2 it must be a ParentQueue + // (even if it is empty) + final boolean isDynamicParent = oldQueue instanceof ParentQueue && oldQueue.isDynamicQueue(); + boolean isAutoQueueCreationEnabledParent = isDynamicParent || conf.isAutoQueueCreationV2Enabled( + fullQueueName) || isAutoCreateEnabled; + + if (childQueueNames.size() == 0 && !isAutoQueueCreationEnabledParent) { + validateParent(parent, queueName); + // Check if the queue will be dynamically managed by the Reservation system if (isReservableQueue) { - queue = new PlanQueue(queueContext, queueName, parent, - oldQueues.get(fullQueueName)); - - //initializing the "internal" default queue, for SLS compatibility - String defReservationId = - queueName + ReservationConstants.DEFAULT_QUEUE_SUFFIX; - - List childQueues = new ArrayList<>(); - ReservationQueue resQueue = new ReservationQueue(queueContext, - defReservationId, (PlanQueue) queue); - try { - resQueue.setEntitlement(new QueueEntitlement(1.0f, 1.0f)); - } catch (SchedulerDynamicEditException e) { - throw new IllegalStateException(e); - } - childQueues.add(resQueue); - ((PlanQueue) queue).setChildQueues(childQueues); - newQueues.add(resQueue); - - } else if (isAutoCreateEnabled) { - queue = new ManagedParentQueue(queueContext, queueName, parent, - oldQueues.get(fullQueueName)); - - } else{ - queue = new LeafQueue(queueContext, queueName, parent, - oldQueues.get(fullQueueName)); - // Used only for unit tests - queue = hook.hook(queue); + queue = new PlanQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); + ReservationQueue defaultResQueue = ((PlanQueue) queue).initializeDefaultInternalQueue(); + newQueues.add(defaultResQueue); + } else { + queue = new LeafQueue(queueContext, queueName, parent, oldQueues.get(fullQueueName)); } - } else{ + + queue = hook.hook(queue); + } else { if (isReservableQueue) { - throw new IllegalStateException( - "Only Leaf Queues can be reservable for " + fullQueueName); + throw new IllegalStateException("Only Leaf Queues can be reservable for " + fullQueueName); } ParentQueue parentQueue; if (isAutoCreateEnabled) { - parentQueue = new ManagedParentQueue(queueContext, queueName, parent, - oldQueues.get(fullQueueName)); - } else{ - parentQueue = new ParentQueue(queueContext, queueName, parent, - oldQueues.get(fullQueueName)); + parentQueue = new ManagedParentQueue(queueContext, queueName, parent, oldQueues.get( + fullQueueName)); + } else { + parentQueue = new ParentQueue(queueContext, queueName, parent, oldQueues.get( + fullQueueName)); } - // Used only for unit tests queue = hook.hook(parentQueue); - List childQueues = new ArrayList<>(); for (String childQueueName : childQueueNames) { - CSQueue childQueue = parseQueue(queueContext, conf, queue, childQueueName, - newQueues, oldQueues, hook); + CSQueue childQueue = parseQueue(queueContext, conf, queue, childQueueName, newQueues, + oldQueues, hook); childQueues.add(childQueue); } - parentQueue.setChildQueues(childQueues); + + if (!childQueues.isEmpty()) { + parentQueue.setChildQueues(childQueues); + } } @@ -721,4 +682,11 @@ private boolean isDanglingDynamicQueue( // that existingQueues contain valid dynamic queues. return !isDynamicQueue(parent); } + + private static void validateParent(CSQueue parent, String queueName) { + if (parent == null) { + throw new IllegalStateException("Queue configuration missing child queue names for " + + queueName); + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java index cca46f50095d7..3bbbffe588192 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/PlanQueue.java @@ -21,8 +21,10 @@ import java.io.IOException; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,6 +125,23 @@ public void reinitialize(CSQueue newlyParsedQueue, } } + public ReservationQueue initializeDefaultInternalQueue() throws IOException { + //initializing the "internal" default queue, for SLS compatibility + String defReservationId = + getQueueName() + ReservationConstants.DEFAULT_QUEUE_SUFFIX; + + ReservationQueue resQueue = new ReservationQueue(queueContext, + defReservationId, this); + try { + resQueue.initializeEntitlements(); + } catch (SchedulerDynamicEditException e) { + throw new IllegalStateException(e); + } + childQueues.add(resQueue); + + return resQueue; + } + private void updateQuotas(float newUserLimit, float newUserLimitFactor, int newMaxAppsForReservation, int newMaxAppsPerUserForReservation) { this.userLimit = newUserLimit; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java index 7b3144b6a8b35..37cf425a83f13 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ReservationQueue.java @@ -22,6 +22,8 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,6 +76,10 @@ public void reinitialize(CSQueue newlyParsedQueue, } } + public void initializeEntitlements() throws SchedulerDynamicEditException { + setEntitlement(new QueueEntitlement(1.0f, 1.0f)); + } + private void updateQuotas(float userLimit, float userLimitFactor, int maxAppsForReservation, int maxAppsPerUserForReservation) { setUserLimit(userLimit); From db36747e831f7045a3e1a0ca390c944981dd141a Mon Sep 17 00:00:00 2001 From: Duo Zhang Date: Thu, 10 Mar 2022 10:15:09 +0800 Subject: [PATCH 027/145] HADOOP-17526 Use Slf4jRequestLog for HttpRequestLog (#4050) Signed-off-by: Wei-Chiu Chuang --- .../src/main/conf/log4j.properties | 63 +++++++++------- .../apache/hadoop/http/HttpRequestLog.java | 72 ++++--------------- .../hadoop/http/HttpRequestLogAppender.java | 62 ---------------- .../hadoop/http/TestHttpRequestLog.java | 27 +++---- .../http/TestHttpRequestLogAppender.java | 37 ---------- .../hadoop/http/TestHttpServerLifecycle.java | 21 ------ 6 files changed, 63 insertions(+), 219 deletions(-) delete mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLogAppender.java delete mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLogAppender.java diff --git a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties index 5a2ca4d922852..54d5c729848c7 100644 --- a/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties +++ b/hadoop-common-project/hadoop-common/src/main/conf/log4j.properties @@ -251,30 +251,45 @@ log4j.appender.NMAUDIT.MaxBackupIndex=${nm.audit.log.maxbackupindex} #log4j.appender.HSAUDIT.DatePattern=.yyyy-MM-dd # Http Server Request Logs -#log4j.logger.http.requests.namenode=INFO,namenoderequestlog -#log4j.appender.namenoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender -#log4j.appender.namenoderequestlog.Filename=${hadoop.log.dir}/jetty-namenode-yyyy_mm_dd.log -#log4j.appender.namenoderequestlog.RetainDays=3 - -#log4j.logger.http.requests.datanode=INFO,datanoderequestlog -#log4j.appender.datanoderequestlog=org.apache.hadoop.http.HttpRequestLogAppender -#log4j.appender.datanoderequestlog.Filename=${hadoop.log.dir}/jetty-datanode-yyyy_mm_dd.log -#log4j.appender.datanoderequestlog.RetainDays=3 - -#log4j.logger.http.requests.resourcemanager=INFO,resourcemanagerrequestlog -#log4j.appender.resourcemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender -#log4j.appender.resourcemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-resourcemanager-yyyy_mm_dd.log -#log4j.appender.resourcemanagerrequestlog.RetainDays=3 - -#log4j.logger.http.requests.jobhistory=INFO,jobhistoryrequestlog -#log4j.appender.jobhistoryrequestlog=org.apache.hadoop.http.HttpRequestLogAppender -#log4j.appender.jobhistoryrequestlog.Filename=${hadoop.log.dir}/jetty-jobhistory-yyyy_mm_dd.log -#log4j.appender.jobhistoryrequestlog.RetainDays=3 - -#log4j.logger.http.requests.nodemanager=INFO,nodemanagerrequestlog -#log4j.appender.nodemanagerrequestlog=org.apache.hadoop.http.HttpRequestLogAppender -#log4j.appender.nodemanagerrequestlog.Filename=${hadoop.log.dir}/jetty-nodemanager-yyyy_mm_dd.log -#log4j.appender.nodemanagerrequestlog.RetainDays=3 +#log4j.appender.AccessNNDRFA=org.apache.log4j.DailyRollingFileAppender +#log4j.appender.AccessNNDRFA.File=${hadoop.log.dir}/jetty-namenode.log +#log4j.appender.AccessNNDRFA.DatePattern=.yyyy-MM-dd +#log4j.appender.AccessNNDRFA.layout=org.apache.log4j.PatternLayout +#log4j.appender.AccessNNDRFA.layout.ConversionPattern=%m%n + +#log4j.logger.http.requests.namenode=INFO,AccessNNDRFA + +#log4j.appender.AccessDNDRFA=org.apache.log4j.DailyRollingFileAppender +#log4j.appender.AccessDNDRFA.File=${hadoop.log.dir}/jetty-datanode.log +#log4j.appender.AccessDNDRFA.DatePattern=.yyyy-MM-dd +#log4j.appender.AccessDNDRFA.layout=org.apache.log4j.PatternLayout +#log4j.appender.AccessDNDRFA.layout.ConversionPattern=%m%n + +#log4j.logger.http.requests.datanode=INFO,AccessDNDRFA + +#log4j.appender.AccessRMDRFA=org.apache.log4j.DailyRollingFileAppender +#log4j.appender.AccessRMDRFA.File=${hadoop.log.dir}/jetty-resourcemanager.log +#log4j.appender.AccessRMDRFA.DatePattern=.yyyy-MM-dd +#log4j.appender.AccessRMDRFA.layout=org.apache.log4j.PatternLayout +#log4j.appender.AccessRMDRFA.layout.ConversionPattern=%m%n + +#log4j.logger.http.requests.resourcemanager=INFO,AccessRMDRFA + +#log4j.appender.AccessJHDRFA=org.apache.log4j.DailyRollingFileAppender +#log4j.appender.AccessJHDRFA.File=${hadoop.log.dir}/jetty-jobhistory.log +#log4j.appender.AccessJHDRFA.DatePattern=.yyyy-MM-dd +#log4j.appender.AccessJHDRFA.layout=org.apache.log4j.PatternLayout +#log4j.appender.AccessJHDRFA.layout.ConversionPattern=%m%n + +#log4j.logger.http.requests.jobhistory=INFO,AccessJHDRFA + +#log4j.appender.AccessNMDRFA=org.apache.log4j.DailyRollingFileAppender +#log4j.appender.AccessNMDRFA.File=${hadoop.log.dir}/jetty-jobhistory.log +#log4j.appender.AccessNMDRFA.DatePattern=.yyyy-MM-dd +#log4j.appender.AccessNMDRFA.layout=org.apache.log4j.PatternLayout +#log4j.appender.AccessNMDRFA.layout.ConversionPattern=%m%n + +#log4j.logger.http.requests.nodemanager=INFO,AccessNMDRFA # WebHdfs request log on datanodes # Specify -Ddatanode.webhdfs.logger=INFO,HTTPDRFA on datanode startup to diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java index b2f18538b6c7d..863afdf1f2e53 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLog.java @@ -17,16 +17,12 @@ */ package org.apache.hadoop.http; +import java.util.Collections; import java.util.HashMap; - -import org.apache.commons.logging.impl.Log4JLogger; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogConfigurationException; -import org.apache.commons.logging.LogFactory; -import org.apache.log4j.Appender; -import org.eclipse.jetty.server.AsyncRequestLogWriter; +import java.util.Map; import org.eclipse.jetty.server.CustomRequestLog; import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.Slf4jRequestLogWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,67 +33,27 @@ public class HttpRequestLog { public static final Logger LOG = LoggerFactory.getLogger(HttpRequestLog.class); - private static final HashMap serverToComponent; + private static final Map serverToComponent; static { - serverToComponent = new HashMap(); - serverToComponent.put("cluster", "resourcemanager"); - serverToComponent.put("hdfs", "namenode"); - serverToComponent.put("node", "nodemanager"); + Map map = new HashMap(); + map.put("cluster", "resourcemanager"); + map.put("hdfs", "namenode"); + map.put("node", "nodemanager"); + serverToComponent = Collections.unmodifiableMap(map); } public static RequestLog getRequestLog(String name) { - String lookup = serverToComponent.get(name); if (lookup != null) { name = lookup; } String loggerName = "http.requests." + name; - String appenderName = name + "requestlog"; - Log logger = LogFactory.getLog(loggerName); - - boolean isLog4JLogger;; - try { - isLog4JLogger = logger instanceof Log4JLogger; - } catch (NoClassDefFoundError err) { - // In some dependent projects, log4j may not even be on the classpath at - // runtime, in which case the above instanceof check will throw - // NoClassDefFoundError. - LOG.debug("Could not load Log4JLogger class", err); - isLog4JLogger = false; - } - if (isLog4JLogger) { - Log4JLogger httpLog4JLog = (Log4JLogger)logger; - org.apache.log4j.Logger httpLogger = httpLog4JLog.getLogger(); - Appender appender = null; - - try { - appender = httpLogger.getAppender(appenderName); - } catch (LogConfigurationException e) { - LOG.warn("Http request log for {} could not be created", loggerName); - throw e; - } - - if (appender == null) { - LOG.info("Http request log for {} is not defined", loggerName); - return null; - } + Slf4jRequestLogWriter writer = new Slf4jRequestLogWriter(); + writer.setLoggerName(loggerName); + return new CustomRequestLog(writer, CustomRequestLog.EXTENDED_NCSA_FORMAT); + } - if (appender instanceof HttpRequestLogAppender) { - HttpRequestLogAppender requestLogAppender - = (HttpRequestLogAppender)appender; - AsyncRequestLogWriter logWriter = new AsyncRequestLogWriter(); - logWriter.setFilename(requestLogAppender.getFilename()); - logWriter.setRetainDays(requestLogAppender.getRetainDays()); - return new CustomRequestLog(logWriter, - CustomRequestLog.EXTENDED_NCSA_FORMAT); - } else { - LOG.warn("Jetty request log for {} was of the wrong class", loggerName); - return null; - } - } else { - LOG.warn("Jetty request log can only be enabled using Log4j"); - return null; - } + private HttpRequestLog() { } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLogAppender.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLogAppender.java deleted file mode 100644 index eda1d1fee402e..0000000000000 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpRequestLogAppender.java +++ /dev/null @@ -1,62 +0,0 @@ -/** - * 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.hadoop.http; - -import org.apache.log4j.spi.LoggingEvent; -import org.apache.log4j.AppenderSkeleton; - -/** - * Log4j Appender adapter for HttpRequestLog - */ -public class HttpRequestLogAppender extends AppenderSkeleton { - - private String filename; - private int retainDays; - - public HttpRequestLogAppender() { - } - - public void setRetainDays(int retainDays) { - this.retainDays = retainDays; - } - - public int getRetainDays() { - return retainDays; - } - - public void setFilename(String filename) { - this.filename = filename; - } - - public String getFilename() { - return filename; - } - - @Override - public void append(LoggingEvent event) { - } - - @Override - public void close() { - } - - @Override - public boolean requiresLayout() { - return false; - } -} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java index d0123e32039c9..58721c4baa8f9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLog.java @@ -17,32 +17,25 @@ */ package org.apache.hadoop.http; -import org.apache.log4j.Logger; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + import org.eclipse.jetty.server.CustomRequestLog; import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.Slf4jRequestLogWriter; import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; - public class TestHttpRequestLog { - @Test - public void testAppenderUndefined() { - RequestLog requestLog = HttpRequestLog.getRequestLog("test"); - assertNull("RequestLog should be null", requestLog); - } - @Test public void testAppenderDefined() { - HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender(); - requestLogAppender.setName("testrequestlog"); - Logger.getLogger("http.requests.test").addAppender(requestLogAppender); RequestLog requestLog = HttpRequestLog.getRequestLog("test"); - Logger.getLogger("http.requests.test").removeAppender(requestLogAppender); assertNotNull("RequestLog should not be null", requestLog); - assertEquals("Class mismatch", - CustomRequestLog.class, requestLog.getClass()); + assertThat(requestLog, instanceOf(CustomRequestLog.class)); + CustomRequestLog crl = (CustomRequestLog) requestLog; + assertThat(crl.getWriter(), instanceOf(Slf4jRequestLogWriter.class)); + assertEquals(CustomRequestLog.EXTENDED_NCSA_FORMAT, crl.getFormatString()); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLogAppender.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLogAppender.java deleted file mode 100644 index e84bee06e6e08..0000000000000 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpRequestLogAppender.java +++ /dev/null @@ -1,37 +0,0 @@ -/** - * 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.hadoop.http; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class TestHttpRequestLogAppender { - - @Test - public void testParameterPropagation() { - - HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender(); - requestLogAppender.setFilename("jetty-namenode-yyyy_mm_dd.log"); - requestLogAppender.setRetainDays(17); - assertEquals("Filename mismatch", "jetty-namenode-yyyy_mm_dd.log", - requestLogAppender.getFilename()); - assertEquals("Retain days mismatch", 17, - requestLogAppender.getRetainDays()); - } -} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java index 40f1b3df08c6e..757ea0c05e7c0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java @@ -68,27 +68,6 @@ public void testStartedServerIsAlive() throws Throwable { stop(server); } - /** - * Test that the server with request logging enabled - * - * @throws Throwable on failure - */ - @Test - public void testStartedServerWithRequestLog() throws Throwable { - HttpRequestLogAppender requestLogAppender = new HttpRequestLogAppender(); - requestLogAppender.setName("httprequestlog"); - requestLogAppender.setFilename( - GenericTestUtils.getTempPath("jetty-name-yyyy_mm_dd.log")); - Logger.getLogger(HttpServer2.class.getName() + ".test").addAppender(requestLogAppender); - HttpServer2 server = null; - server = createTestServer(); - assertNotLive(server); - server.start(); - assertAlive(server); - stop(server); - Logger.getLogger(HttpServer2.class.getName() + ".test").removeAppender(requestLogAppender); - } - /** * Assert that the result of {@link HttpServer2#toString()} contains the specific text * @param server server to examine From 9539ff108aaa420168935a3dbc73c43c1686e252 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 10 Mar 2022 08:02:18 +0100 Subject: [PATCH 028/145] YARN-11052. Improve code quality in TestRMWebServicesNodeLabels. Contributed by Szilard Nemeth --- .../webapp/TestRMWebServicesNodeLabels.java | 938 ++++++++---------- 1 file changed, 424 insertions(+), 514 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java index f1ea0fba0339d..503b4a8b2bbdd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesNodeLabels.java @@ -6,9 +6,9 @@ * 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 - * + *

+ * 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. @@ -26,9 +26,15 @@ import java.io.IOException; import java.io.StringWriter; import java.util.ArrayList; +import java.util.List; +import java.util.Map; import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriBuilder; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.util.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.http.JettyUtils; @@ -55,7 +61,6 @@ import com.google.inject.Guice; import com.google.inject.servlet.ServletModule; import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.UniformInterfaceException; import com.sun.jersey.api.client.WebResource; import com.sun.jersey.api.json.JSONJAXBContext; import com.sun.jersey.api.json.JSONMarshaller; @@ -69,6 +74,28 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase { private static final Logger LOG = LoggerFactory .getLogger(TestRMWebServicesNodeLabels.class); + private static final String NODE_0 = "nid:0"; + private static final String NODE_1 = "nid1:0"; + private static final String NODE_2 = "nid2:0"; + private static final String LABEL_A = "a"; + private static final String LABEL_B = "b"; + private static final String LABEL_X = "x"; + private static final String LABEL_Y = "y"; + private static final String LABEL_Z = "z"; + public static final boolean DEFAULT_NL_EXCLUSIVITY = true; + private static final String PATH_WS = "ws"; + private static final String PATH_V1 = "v1"; + private static final String PATH_NODES = "nodes"; + private static final String PATH_CLUSTER = "cluster"; + private static final String PATH_REPLACE_NODE_TO_LABELS = "replace-node-to-labels"; + private static final String PATH_LABEL_MAPPINGS = "label-mappings"; + private static final String PATH_GET_LABELS = "get-labels"; + private static final String PATH_REPLACE_LABELS = "replace-labels"; + private static final String PATH_REMOVE_LABELS = "remove-node-labels"; + private static final String PATH_GET_NODE_LABELS = "get-node-labels"; + private static final String PATH_GET_NODE_TO_LABELS = "get-node-to-labels"; + private static final String QUERY_USER_NAME = "user.name"; + private static final String PATH_ADD_NODE_LABELS = "add-node-labels"; private static MockRM rm; private static YarnConfiguration conf; @@ -76,6 +103,8 @@ public class TestRMWebServicesNodeLabels extends JerseyTestBase { private static String userName; private static String notUserName; private static RMWebServices rmWebService; + private WebResource resource; + private static class WebServletModule extends ServletModule { @@ -92,7 +121,7 @@ protected void configureServlets() { conf = new YarnConfiguration(); conf.set(YarnConfiguration.YARN_ADMIN_ACL, userName); rm = new MockRM(conf); - rmWebService = new RMWebServices(rm,conf); + rmWebService = new RMWebServices(rm, conf); bind(RMWebServices.class).toInstance(rmWebService); bind(GenericExceptionHandler.class); bind(ResourceManager.class).toInstance(rm); @@ -100,7 +129,7 @@ protected void configureServlets() { TestRMWebServicesAppsModification.TestRMCustomAuthFilter.class); serve("/*").with(GuiceContainer.class); } - }; + } @Override @Before @@ -108,6 +137,7 @@ public void setUp() throws Exception { super.setUp(); GuiceServletConfig.setInjector( Guice.createInjector(new WebServletModule())); + resource = resource(); } public TestRMWebServicesNodeLabels() { @@ -118,528 +148,444 @@ public TestRMWebServicesNodeLabels() { .contextPath("jersey-guice-filter").servletPath("/").build()); } - @Test - public void testNodeLabels() throws JSONException, Exception { - WebResource r = resource(); + private WebResource getClusterWebResource() { + return resource.path(PATH_WS).path(PATH_V1).path(PATH_CLUSTER); + } + + private ClientResponse get(String path) { + return getClusterWebResource() + .path(path) + .queryParam(QUERY_USER_NAME, userName) + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + } + + private ClientResponse get(String path, MultivaluedMapImpl queryParams) { + return getClusterWebResource() + .path(path) + .queryParam(QUERY_USER_NAME, userName) + .queryParams(queryParams) + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + } + + private ClientResponse post(String path, String userName, Object payload, + Class payloadClass) throws Exception { + return getClusterWebResource() + .path(path) + .queryParam(QUERY_USER_NAME, userName) + .accept(MediaType.APPLICATION_JSON) + .entity(toJson(payload, payloadClass), + MediaType.APPLICATION_JSON) + .post(ClientResponse.class); + } + private ClientResponse post(String path, String userName, Object payload, + Class payloadClass, MultivaluedMapImpl queryParams) throws Exception { + WebResource.Builder builder = getClusterWebResource() + .path(path) + .queryParam(QUERY_USER_NAME, userName) + .queryParams(queryParams) + .accept(MediaType.APPLICATION_JSON); + + if (payload != null && payloadClass != null) { + builder.entity(toJson(payload, payloadClass), MediaType.APPLICATION_JSON); + } + return builder.post(ClientResponse.class); + } + + @Test + public void testNodeLabels() throws Exception { ClientResponse response; // Add a label - NodeLabelsInfo nlsifo = new NodeLabelsInfo(); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("a")); - response = - r.path("ws").path("v1").path("cluster") - .path("add-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsifo, NodeLabelsInfo.class), - MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = addNodeLabels(Lists.newArrayList(Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY))); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals(1, nlsifo.getNodeLabels().size()); - for (NodeLabelInfo nl : nlsifo.getNodeLabelsInfo()) { - assertEquals("a", nl.getName()); - assertTrue(nl.getExclusivity()); - } - + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfo(response.getEntity(NodeLabelsInfo.class), Lists.newArrayList( + Pair.of(LABEL_A, true))); + // Add another - nlsifo = new NodeLabelsInfo(); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("b", false)); - response = - r.path("ws").path("v1").path("cluster") - .path("add-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsifo, NodeLabelsInfo.class), - MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = addNodeLabels(Lists.newArrayList(Pair.of(LABEL_B, false))); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals(2, nlsifo.getNodeLabels().size()); - // Verify exclusivity for 'y' as false - for (NodeLabelInfo nl : nlsifo.getNodeLabelsInfo()) { - if (nl.getName().equals("b")) { - assertFalse(nl.getExclusivity()); - } - } - + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + // Verify exclusivity for 'b' as false + assertNodeLabelsInfo(response.getEntity(NodeLabelsInfo.class), + Lists.newArrayList( + Pair.of(LABEL_A, true), + Pair.of(LABEL_B, false))); + // Add labels to a node - MultivaluedMapImpl params = new MultivaluedMapImpl(); - params.add("labels", "a"); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("replace-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); + response = replaceLabelsOnNode(NODE_0, LABEL_A); + assertHttp200(response); // Add labels to another node - params = new MultivaluedMapImpl(); - params.add("labels", "b"); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid1:0") - .path("replace-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); + response = replaceLabelsOnNode(NODE_1, LABEL_B); + assertHttp200(response); // Add labels to another node - params = new MultivaluedMapImpl(); - params.add("labels", "b"); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid2:0") - .path("replace-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); - - // Verify, using get-labels-to-Nodes - response = - r.path("ws").path("v1").path("cluster") - .path("label-mappings").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - LabelsToNodesInfo ltni = response.getEntity(LabelsToNodesInfo.class); - assertEquals(2, ltni.getLabelsToNodes().size()); - NodeIDsInfo nodes = ltni.getLabelsToNodes().get( - new NodeLabelInfo("b", false)); - assertTrue(nodes.getNodeIDs().contains("nid2:0")); - assertTrue(nodes.getNodeIDs().contains("nid1:0")); - nodes = ltni.getLabelsToNodes().get(new NodeLabelInfo("a")); - assertTrue(nodes.getNodeIDs().contains("nid:0")); + response = replaceLabelsOnNode(NODE_2, LABEL_B); + assertHttp200(response); + + // Verify all, using get-labels-to-Nodes + response = getNodeLabelMappings(); + assertApplicationJsonUtf8Response(response); + LabelsToNodesInfo labelsToNodesInfo = response.getEntity(LabelsToNodesInfo.class); + assertLabelsToNodesInfo(labelsToNodesInfo, 2, Lists.newArrayList( + Pair.of(Pair.of(LABEL_B, false), Lists.newArrayList(NODE_1, NODE_2)), + Pair.of(Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY), Lists.newArrayList(NODE_0)) + )); // Verify, using get-labels-to-Nodes for specified set of labels - params = new MultivaluedMapImpl(); - params.add("labels", "a"); - response = - r.path("ws").path("v1").path("cluster") - .path("label-mappings").queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - ltni = response.getEntity(LabelsToNodesInfo.class); - assertEquals(1, ltni.getLabelsToNodes().size()); - nodes = ltni.getLabelsToNodes().get(new NodeLabelInfo("a")); - assertTrue(nodes.getNodeIDs().contains("nid:0")); + response = getNodeLabelMappingsByLabels(LABEL_A); + assertApplicationJsonUtf8Response(response); + labelsToNodesInfo = response.getEntity(LabelsToNodesInfo.class); + assertLabelsToNodesInfo(labelsToNodesInfo, 1, Lists.newArrayList( + Pair.of(Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY), Lists.newArrayList(NODE_0)) + )); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("get-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertTrue(nlsifo.getNodeLabelsInfo().contains(new NodeLabelInfo("a"))); + response = getLabelsOfNode(NODE_0); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfoContains(response.getEntity(NodeLabelsInfo.class), + Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY)); - // Replace - params = new MultivaluedMapImpl(); - params.add("labels", "b"); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("replace-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); + response = replaceLabelsOnNode(NODE_0, LABEL_B); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("get-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertTrue(nlsifo.getNodeLabelsInfo().contains( - new NodeLabelInfo("b", false))); - + response = getLabelsOfNode(NODE_0); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfoContains(response.getEntity(NodeLabelsInfo.class), Pair.of(LABEL_B, false)); + // Replace labels using node-to-labels - NodeToLabelsEntryList ntli = new NodeToLabelsEntryList(); - ArrayList labels = new ArrayList(); - labels.add("a"); - NodeToLabelsEntry nli = new NodeToLabelsEntry("nid:0", labels); - ntli.getNodeToLabels().add(nli); - response = - r.path("ws").path("v1").path("cluster") - .path("replace-node-to-labels") - .queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity(toJson(ntli, NodeToLabelsEntryList.class), - MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - + response = replaceNodeToLabels(Lists.newArrayList(Pair.of(NODE_0, + Lists.newArrayList(LABEL_A)))); + assertHttp200(response); + // Verify, using node-to-labels - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-to-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - NodeToLabelsInfo ntlinfo = response.getEntity(NodeToLabelsInfo.class); - NodeLabelsInfo nlinfo = ntlinfo.getNodeToLabels().get("nid:0"); - assertEquals(1, nlinfo.getNodeLabels().size()); - assertTrue(nlinfo.getNodeLabelsInfo().contains(new NodeLabelInfo("a"))); - + response = getNodeToLabels(); + assertApplicationJsonUtf8Response(response); + NodeToLabelsInfo nodeToLabelsInfo = response.getEntity(NodeToLabelsInfo.class); + NodeLabelsInfo nodeLabelsInfo = nodeToLabelsInfo.getNodeToLabels().get(NODE_0); + assertNodeLabelsSize(nodeLabelsInfo, 1); + assertNodeLabelsInfoContains(nodeLabelsInfo, Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY)); + // Remove all - params = new MultivaluedMapImpl(); - params.add("labels", ""); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("replace-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); + response = replaceLabelsOnNode(NODE_0, ""); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("get-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertTrue(nlsifo.getNodeLabelsInfo().isEmpty()); - + response = getLabelsOfNode(NODE_0); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsSize(response.getEntity(NodeLabelsInfo.class), 0); + // Add a label back for auth tests - params = new MultivaluedMapImpl(); - params.add("labels", "a"); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("replace-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); + response = replaceLabelsOnNode(NODE_0, LABEL_A); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("get-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertTrue(nlsifo.getNodeLabelsInfo().contains(new NodeLabelInfo("a"))); - + response = getLabelsOfNode(NODE_0); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfoContains(response.getEntity(NodeLabelsInfo.class), + Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY)); + // Auth fail replace labels on node - params = new MultivaluedMapImpl(); - params.add("labels", "b"); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("replace-labels") - .queryParam("user.name", notUserName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = replaceLabelsOnNodeWithUserName(NODE_0, notUserName, LABEL_B); + assertHttp401(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nid:0") - .path("get-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertTrue(nlsifo.getNodeLabelsInfo().contains(new NodeLabelInfo("a"))); - - // Fail to add a label with post - response = - r.path("ws").path("v1").path("cluster") - .path("add-node-labels").queryParam("user.name", notUserName) - .accept(MediaType.APPLICATION_JSON) - .entity("{\"nodeLabels\":\"c\"}", MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = getLabelsOfNode(NODE_0); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfoContains(response.getEntity(NodeLabelsInfo.class), + Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY)); + + // Fail to add a label with wrong user + response = addNodeLabelsWithUser(Lists.newArrayList(Pair.of("c", DEFAULT_NL_EXCLUSIVITY)), + notUserName); + assertHttp401(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals(2, nlsifo.getNodeLabels().size()); - + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsSize(response.getEntity(NodeLabelsInfo.class), 2); + // Remove cluster label (succeed, we no longer need it) - params = new MultivaluedMapImpl(); - params.add("labels", "b"); - response = - r.path("ws").path("v1").path("cluster") - .path("remove-node-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = removeNodeLabel(LABEL_B); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals(1, nlsifo.getNodeLabels().size()); - for (NodeLabelInfo nl : nlsifo.getNodeLabelsInfo()) { - assertEquals("a", nl.getName()); - assertTrue(nl.getExclusivity()); - } - + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfo(response.getEntity(NodeLabelsInfo.class), + Lists.newArrayList(Pair.of(LABEL_A, true))); + // Remove cluster label with post - params = new MultivaluedMapImpl(); - params.add("labels", "a"); - response = - r.path("ws").path("v1").path("cluster") - .path("remove-node-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = removeNodeLabel(LABEL_A); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals(0, nlsifo.getNodeLabels().size()); + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + nodeLabelsInfo = response.getEntity(NodeLabelsInfo.class); + assertEquals(0, nodeLabelsInfo.getNodeLabels().size()); // Following test cases are to test replace when distributed node label // configuration is on // Reset for testing : add cluster labels - nlsifo = new NodeLabelsInfo(); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("x", false)); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("y", false)); - response = - r.path("ws") - .path("v1") - .path("cluster") - .path("add-node-labels") - .queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsifo, NodeLabelsInfo.class), - MediaType.APPLICATION_JSON).post(ClientResponse.class); + response = addNodeLabels(Lists.newArrayList( + Pair.of(LABEL_X, false), Pair.of(LABEL_Y, false))); + assertHttp200(response); // Reset for testing : Add labels to a node - params = new MultivaluedMapImpl(); - params.add("labels", "y"); - response = - r.path("ws").path("v1").path("cluster").path("nodes").path("nid:0") - .path("replace-labels").queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); - - //setting rmWebService for non Centralized NodeLabel Configuration + response = replaceLabelsOnNode(NODE_0, LABEL_Y); + assertHttp200(response); + + //setting rmWebService for non-centralized NodeLabel Configuration rmWebService.isCentralizedNodeLabelConfiguration = false; // Case1 : Replace labels using node-to-labels - ntli = new NodeToLabelsEntryList(); - labels = new ArrayList(); - labels.add("x"); - nli = new NodeToLabelsEntry("nid:0", labels); - ntli.getNodeToLabels().add(nli); - response = - r.path("ws") - .path("v1") - .path("cluster") - .path("replace-node-to-labels") - .queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity(toJson(ntli, NodeToLabelsEntryList.class), - MediaType.APPLICATION_JSON).post(ClientResponse.class); + response = replaceNodeToLabels(Lists.newArrayList(Pair.of(NODE_0, + Lists.newArrayList(LABEL_X)))); + assertHttp404(response); // Verify, using node-to-labels that previous operation has failed - response = - r.path("ws").path("v1").path("cluster").path("get-node-to-labels") - .queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - ntlinfo = response.getEntity(NodeToLabelsInfo.class); - nlinfo = ntlinfo.getNodeToLabels().get("nid:0"); - assertEquals(1, nlinfo.getNodeLabels().size()); - assertFalse(nlinfo.getNodeLabelsInfo().contains( - new NodeLabelInfo("x", false))); + response = getNodeToLabels(); + assertApplicationJsonUtf8Response(response); + nodeToLabelsInfo = response.getEntity(NodeToLabelsInfo.class); + nodeLabelsInfo = nodeToLabelsInfo.getNodeToLabels().get(NODE_0); + assertNodeLabelsSize(nodeLabelsInfo, 1); + assertNodeLabelsInfoDoesNotContain(nodeLabelsInfo, Pair.of(LABEL_X, false)); // Case2 : failure to Replace labels using replace-labels - response = - r.path("ws").path("v1").path("cluster").path("nodes").path("nid:0") - .path("replace-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity("{\"nodeLabelName\": [\"x\"]}", MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - LOG.info("posted node nodelabel"); + response = replaceLabelsOnNode(NODE_0, LABEL_X); + assertHttp404(response); // Verify, using node-to-labels that previous operation has failed - response = - r.path("ws").path("v1").path("cluster").path("get-node-to-labels") - .queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - ntlinfo = response.getEntity(NodeToLabelsInfo.class); - nlinfo = ntlinfo.getNodeToLabels().get("nid:0"); - assertEquals(1, nlinfo.getNodeLabels().size()); - assertFalse(nlinfo.getNodeLabelsInfo().contains( - new NodeLabelInfo("x", false))); + response = getNodeToLabels(); + assertApplicationJsonUtf8Response(response); + nodeToLabelsInfo = response.getEntity(NodeToLabelsInfo.class); + nodeLabelsInfo = nodeToLabelsInfo.getNodeToLabels().get(NODE_0); + assertNodeLabelsSize(nodeLabelsInfo, 1); + assertNodeLabelsInfoDoesNotContain(nodeLabelsInfo, Pair.of(LABEL_X, false)); // Case3 : Remove cluster label should be successful - params = new MultivaluedMapImpl(); - params.add("labels", "x"); - response = - r.path("ws").path("v1").path("cluster") - .path("remove-node-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = removeNodeLabel(LABEL_X); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals(new NodeLabelInfo("y", false), - nlsifo.getNodeLabelsInfo().get(0)); - assertEquals("y", nlsifo.getNodeLabelsInfo().get(0).getName()); - assertFalse(nlsifo.getNodeLabelsInfo().get(0).getExclusivity()); + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfoAtPosition(response.getEntity(NodeLabelsInfo.class), Pair.of(LABEL_Y, + false), 0); // Remove y - params = new MultivaluedMapImpl(); - params.add("labels", "y"); - response = - r.path("ws").path("v1").path("cluster") - .path("remove-node-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = removeNodeLabel(LABEL_Y); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertTrue(nlsifo.getNodeLabelsInfo().isEmpty()); - - // add a new nodelabel with exclusity - nlsifo = new NodeLabelsInfo(); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("z", false)); - response = - r.path("ws") - .path("v1") - .path("cluster") - .path("add-node-labels") - .queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsifo, NodeLabelsInfo.class), - MediaType.APPLICATION_JSON).post(ClientResponse.class); + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsSize(response.getEntity(NodeLabelsInfo.class), 0); + // add a new nodelabel with exclusivity=false + response = addNodeLabels(Lists.newArrayList(Pair.of(LABEL_Z, false))); + assertHttp200(response); // Verify - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + assertNodeLabelsInfoAtPosition(response.getEntity(NodeLabelsInfo.class), + Pair.of(LABEL_Z, false), 0); + assertNodeLabelsSize(nodeLabelsInfo, 1); + } + + private void assertLabelsToNodesInfo(LabelsToNodesInfo labelsToNodesInfo, int size, + List, List>> nodeLabelsToNodesList) { + Map labelsToNodes = labelsToNodesInfo.getLabelsToNodes(); + assertNotNull("Labels to nodes mapping should not be null.", labelsToNodes); + assertEquals("Size of label to nodes mapping is not the expected.", size, labelsToNodes.size()); + + for (Pair, List> nodeLabelToNodes : nodeLabelsToNodesList) { + Pair expectedNLData = nodeLabelToNodes.getLeft(); + List expectedNodes = nodeLabelToNodes.getRight(); + NodeLabelInfo expectedNLInfo = new NodeLabelInfo(expectedNLData.getLeft(), + expectedNLData.getRight()); + NodeIDsInfo actualNodes = labelsToNodes.get(expectedNLInfo); + assertNotNull(String.format("Node info not found. Expected NodeLabel data: %s", + expectedNLData), actualNodes); + for (String expectedNode : expectedNodes) { + assertTrue(String.format("Can't find node ID in actual Node IDs list: %s", + actualNodes.getNodeIDs()), actualNodes.getNodeIDs().contains(expectedNode)); + } + } + } + + private void assertNodeLabelsInfo(NodeLabelsInfo nodeLabelsInfo, + List> nlInfos) { + assertEquals(nlInfos.size(), nodeLabelsInfo.getNodeLabels().size()); + + for (int i = 0; i < nodeLabelsInfo.getNodeLabelsInfo().size(); i++) { + Pair expected = nlInfos.get(i); + NodeLabelInfo actual = nodeLabelsInfo.getNodeLabelsInfo().get(i); + LOG.debug("Checking NodeLabelInfo: {}", actual); + assertEquals(expected.getLeft(), actual.getName()); + assertEquals(expected.getRight(), actual.getExclusivity()); + } + } + + private void assertNodeLabelsInfoAtPosition(NodeLabelsInfo nodeLabelsInfo, Pair nlInfo, int pos) { + NodeLabelInfo actual = nodeLabelsInfo.getNodeLabelsInfo().get(pos); + LOG.debug("Checking NodeLabelInfo: {}", actual); + assertEquals(nlInfo.getLeft(), actual.getName()); + assertEquals(nlInfo.getRight(), actual.getExclusivity()); + } + + private void assertNodeLabelsInfoContains(NodeLabelsInfo nodeLabelsInfo, + Pair nlInfo) { + NodeLabelInfo nodeLabelInfo = new NodeLabelInfo(nlInfo.getLeft(), nlInfo.getRight()); + assertTrue(String.format("Cannot find nodeLabelInfo '%s' among items of node label info list:" + + " %s", nodeLabelInfo, nodeLabelsInfo.getNodeLabelsInfo()), + nodeLabelsInfo.getNodeLabelsInfo().contains(nodeLabelInfo)); + } + + private void assertNodeLabelsInfoDoesNotContain(NodeLabelsInfo nodeLabelsInfo, Pair nlInfo) { + NodeLabelInfo nodeLabelInfo = new NodeLabelInfo(nlInfo.getLeft(), nlInfo.getRight()); + assertFalse(String.format("Should have not found nodeLabelInfo '%s' among " + + "items of node label info list: %s", nodeLabelInfo, nodeLabelsInfo.getNodeLabelsInfo()), + nodeLabelsInfo.getNodeLabelsInfo().contains(nodeLabelInfo)); + } + + private void assertNodeLabelsSize(NodeLabelsInfo nodeLabelsInfo, int expectedSize) { + assertEquals(expectedSize, nodeLabelsInfo.getNodeLabelsInfo().size()); + } + + private ClientResponse replaceNodeToLabels(List>> nodeToLabelInfos) throws Exception { + NodeToLabelsEntryList nodeToLabelsEntries = new NodeToLabelsEntryList(); + + for (Pair> nodeToLabelInfo : nodeToLabelInfos) { + ArrayList labelList = new ArrayList<>(nodeToLabelInfo.getRight()); + String nodeId = nodeToLabelInfo.getLeft(); + NodeToLabelsEntry nli = new NodeToLabelsEntry(nodeId, labelList); + nodeToLabelsEntries.getNodeToLabels().add(nli); + } + return post(PATH_REPLACE_NODE_TO_LABELS, userName, nodeToLabelsEntries, NodeToLabelsEntryList.class); + } + + private ClientResponse getNodeLabelMappings() { + return get(PATH_LABEL_MAPPINGS); + } + + private ClientResponse getNodeLabelMappingsByLabels(String... labelNames) { + MultivaluedMapImpl params = createMultiValuedMap(labelNames); + return get(PATH_LABEL_MAPPINGS, params); + } + + private ClientResponse replaceLabelsOnNode(String node, String... labelNames) throws Exception { + return replaceLabelsOnNodeWithUserName(node, userName, labelNames); + } + + private ClientResponse replaceLabelsOnNodeWithUserName(String node, + String userName, String... labelNames) throws Exception { + LOG.info("Replacing labels on node '{}', label(s): {}", node, labelNames); + MultivaluedMapImpl params = createMultiValuedMap(labelNames); + String path = UriBuilder.fromPath(PATH_NODES).path(node) + .path(PATH_REPLACE_LABELS).build().toString(); + return post(path, userName, null, null, params); + } + + private static MultivaluedMapImpl createMultiValuedMap(String[] labelNames) { + MultivaluedMapImpl params = new MultivaluedMapImpl(); + for (String labelName : labelNames) { + params.add("labels", labelName); + } + return params; + } + + private ClientResponse removeNodeLabel(String... labelNames) throws Exception { + MultivaluedMapImpl params = createMultiValuedMap(labelNames); + return post(PATH_REMOVE_LABELS, userName, null, null, params); + } + + private ClientResponse getLabelsOfNode(String node) { + String path = UriBuilder.fromPath(PATH_NODES).path(node) + .path(PATH_GET_LABELS).build().toString(); + return get(path); + } + + private ClientResponse getNodeLabels() { + return get(PATH_GET_NODE_LABELS); + } + + private ClientResponse getNodeToLabels() { + return get(PATH_GET_NODE_TO_LABELS); + } + + private ClientResponse addNodeLabels(List> nlInfos) throws Exception { + return addNodeLabelsInternal(nlInfos, userName); + } + + private ClientResponse addNodeLabelsWithUser(List> nlInfos, + String userName) throws Exception { + return addNodeLabelsInternal(nlInfos, userName); + } + + private ClientResponse addNodeLabelsInternal(List> nlInfos, + String userName) throws Exception { + NodeLabelsInfo nodeLabelsInfo = new NodeLabelsInfo(); + for (Pair nlInfo : nlInfos) { + NodeLabelInfo nodeLabelInfo = new NodeLabelInfo(nlInfo.getLeft(), nlInfo.getRight()); + nodeLabelsInfo.getNodeLabelsInfo().add(nodeLabelInfo); + } + return post(PATH_ADD_NODE_LABELS, userName, nodeLabelsInfo, NodeLabelsInfo.class); + } + + private void assertApplicationJsonUtf8Response(ClientResponse response) { assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals("z", nlsifo.getNodeLabelsInfo().get(0).getName()); - assertFalse(nlsifo.getNodeLabelsInfo().get(0).getExclusivity()); - assertEquals(1, nlsifo.getNodeLabels().size()); + } + + private void assertHttp200(ClientResponse response) { + assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()); + } + + private void assertHttp401(ClientResponse response) { + assertEquals(Response.Status.UNAUTHORIZED.getStatusCode(), response.getStatus()); + } + + private void assertHttp404(ClientResponse response) { + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), response.getStatus()); } @Test public void testLabelInvalidAddition() - throws UniformInterfaceException, Exception { - WebResource r = resource(); - ClientResponse response; + throws Exception { // Add a invalid label - NodeLabelsInfo nlsifo = new NodeLabelsInfo(); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("a&")); - response = r.path("ws").path("v1").path("cluster").path("add-node-labels") - .queryParam("user.name", userName).accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsifo, NodeLabelsInfo.class), - MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - String expectedmessage = + ClientResponse response = addNodeLabels(Lists.newArrayList(Pair.of("a&", + DEFAULT_NL_EXCLUSIVITY))); + String expectedMessage = "java.io.IOException: label name should only contains" + " {0-9, a-z, A-Z, -, _} and should not started with" + " {-,_}, now it is= a&"; - validateJsonExceptionContent(response, expectedmessage); + validateJsonExceptionContent(response, expectedMessage); } @Test public void testLabelChangeExclusivity() - throws Exception, JSONException { - WebResource r = resource(); + throws Exception { ClientResponse response; - NodeLabelsInfo nlsifo = new NodeLabelsInfo(); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("newlabel", true)); - response = r.path("ws").path("v1").path("cluster").path("add-node-labels") - .queryParam("user.name", userName).accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsifo, NodeLabelsInfo.class), - MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = addNodeLabels(Lists.newArrayList(Pair.of("newLabel", DEFAULT_NL_EXCLUSIVITY))); + assertHttp200(response); // new info and change exclusivity - NodeLabelsInfo nlsinfo2 = new NodeLabelsInfo(); - nlsinfo2.getNodeLabelsInfo().add(new NodeLabelInfo("newlabel", false)); - response = r.path("ws").path("v1").path("cluster").path("add-node-labels") - .queryParam("user.name", userName).accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsinfo2, NodeLabelsInfo.class), - MediaType.APPLICATION_JSON) - .post(ClientResponse.class); - String expectedmessage = + response = addNodeLabels(Lists.newArrayList(Pair.of("newLabel", false))); + String expectedMessage = "java.io.IOException: Exclusivity cannot be modified for an existing" - + " label with : "; - validateJsonExceptionContent(response, expectedmessage); + + " label with : "; + validateJsonExceptionContent(response, expectedMessage); } private void validateJsonExceptionContent(ClientResponse response, - String expectedmessage) + String expectedMessage) throws JSONException { Assert.assertEquals(BAD_REQUEST_CODE, response.getStatus()); JSONObject msg = response.getEntity(JSONObject.class); @@ -653,112 +599,76 @@ private void validateJsonExceptionContent(ClientResponse response, WebServicesTestUtils.checkStringMatch("exception classname", "org.apache.hadoop.yarn.webapp.BadRequestException", classname); WebServicesTestUtils.checkStringContains("exception message", - expectedmessage, message); + expectedMessage, message); } @Test public void testLabelInvalidReplace() - throws UniformInterfaceException, Exception { - WebResource r = resource(); + throws Exception { ClientResponse response; - // replace label which doesnt exist - MultivaluedMapImpl params = new MultivaluedMapImpl(); - params.add("labels", "idontexist"); - response = r.path("ws").path("v1").path("cluster").path("nodes") - .path("nid:0").path("replace-labels").queryParam("user.name", userName) - .queryParams(params).accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + // replace label which doesn't exist + response = replaceLabelsOnNode(NODE_0, "idontexist"); - String expectedmessage = + String expectedMessage = "Not all labels being replaced contained by known label" + " collections, please check, new labels=[idontexist]"; - validateJsonExceptionContent(response, expectedmessage); + validateJsonExceptionContent(response, expectedMessage); } @Test public void testLabelInvalidRemove() - throws UniformInterfaceException, Exception { - WebResource r = resource(); + throws Exception { ClientResponse response; - MultivaluedMapImpl params = new MultivaluedMapImpl(); - params.add("labels", "irealldontexist"); - response = - r.path("ws").path("v1").path("cluster").path("remove-node-labels") - .queryParam("user.name", userName).queryParams(params) - .accept(MediaType.APPLICATION_JSON).post(ClientResponse.class); - String expectedmessage = - "java.io.IOException: Node label=irealldontexist to be" + response = removeNodeLabel("ireallydontexist"); + String expectedMessage = + "java.io.IOException: Node label=ireallydontexist to be" + " removed doesn't existed in cluster node labels" + " collection."; - validateJsonExceptionContent(response, expectedmessage); + validateJsonExceptionContent(response, expectedMessage); } @Test public void testNodeLabelPartitionInfo() throws Exception { - WebResource r = resource(); - ClientResponse response; // Add a node label - NodeLabelsInfo nlsifo = new NodeLabelsInfo(); - nlsifo.getNodeLabelsInfo().add(new NodeLabelInfo("a")); - response = - r.path("ws").path("v1").path("cluster") - .path("add-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON) - .entity(toJson(nlsifo, NodeLabelsInfo.class), MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = addNodeLabels(Lists.newArrayList(Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY))); + assertHttp200(response); // Verify partition info in get-node-labels - response = - r.path("ws").path("v1").path("cluster") - .path("get-node-labels").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - nlsifo = response.getEntity(NodeLabelsInfo.class); - assertEquals(1, nlsifo.getNodeLabels().size()); - for (NodeLabelInfo nl : nlsifo.getNodeLabelsInfo()) { - assertEquals("a", nl.getName()); + response = getNodeLabels(); + assertApplicationJsonUtf8Response(response); + NodeLabelsInfo nodeLabelsInfo = response.getEntity(NodeLabelsInfo.class); + assertNodeLabelsSize(nodeLabelsInfo, 1); + for (NodeLabelInfo nl : nodeLabelsInfo.getNodeLabelsInfo()) { + assertEquals(LABEL_A, nl.getName()); assertTrue(nl.getExclusivity()); assertNotNull(nl.getPartitionInfo()); assertNotNull(nl.getPartitionInfo().getResourceAvailable()); } // Add node label to a node - MultivaluedMapImpl params = new MultivaluedMapImpl(); - params.add("labels", "a"); - response = - r.path("ws").path("v1").path("cluster") - .path("nodes").path("nodeId:0") - .path("replace-labels") - .queryParam("user.name", userName) - .queryParams(params) - .accept(MediaType.APPLICATION_JSON) - .post(ClientResponse.class); + response = replaceLabelsOnNode("nodeId:0", LABEL_A); + assertHttp200(response); // Verify partition info in label-mappings - response = - r.path("ws").path("v1").path("cluster") - .path("label-mappings").queryParam("user.name", userName) - .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); - assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, - response.getType().toString()); - LabelsToNodesInfo ltni = response.getEntity(LabelsToNodesInfo.class); - assertEquals(1, ltni.getLabelsToNodes().size()); - NodeIDsInfo nodes = ltni.getLabelsToNodes().get( - new NodeLabelInfo("a")); - assertTrue(nodes.getNodeIDs().contains("nodeId:0")); + response = getNodeLabelMappings(); + assertApplicationJsonUtf8Response(response); + LabelsToNodesInfo labelsToNodesInfo = response.getEntity(LabelsToNodesInfo.class); + assertLabelsToNodesInfo(labelsToNodesInfo, 1, Lists.newArrayList( + Pair.of(Pair.of(LABEL_A, DEFAULT_NL_EXCLUSIVITY), Lists.newArrayList("nodeId:0")) + )); + NodeIDsInfo nodes = labelsToNodesInfo.getLabelsToNodes().get(new NodeLabelInfo(LABEL_A)); assertNotNull(nodes.getPartitionInfo()); assertNotNull(nodes.getPartitionInfo().getResourceAvailable()); } @SuppressWarnings("rawtypes") - private String toJson(Object nsli, Class klass) throws Exception { + private String toJson(Object obj, Class klass) throws Exception { StringWriter sw = new StringWriter(); JSONJAXBContext ctx = new JSONJAXBContext(klass); JSONMarshaller jm = ctx.createJSONMarshaller(); - jm.marshallToJSON(nsli, sw); + jm.marshallToJSON(obj, sw); return sw.toString(); } } From 383b73417df80028011d229dce9daf8e4ecbdb49 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 10 Mar 2022 13:11:19 +0100 Subject: [PATCH 029/145] YARN-11036. Do not inherit from TestRMWebServicesCapacitySched. Contributed by Tamas Domok --- .../TestRMWebServicesCapacitySched.java | 47 +++++++++----- ...WebServicesCapacitySchedDynamicConfig.java | 65 +++++-------------- .../TestRMWebServicesSchedulerActivities.java | 31 +++++++-- 3 files changed, 73 insertions(+), 70 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java index 2be9b54c86549..b9ce10aaeddcb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java @@ -77,42 +77,37 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase { - protected static MockRM rm; + private MockRM rm; + + public static class WebServletModule extends ServletModule { + private final MockRM rm; + + WebServletModule(MockRM rm) { + this.rm = rm; + } - private static class WebServletModule extends ServletModule { @Override protected void configureServlets() { bind(JAXBContextResolver.class); bind(RMWebServices.class); bind(GenericExceptionHandler.class); - CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( - new Configuration(false)); - setupQueueConfiguration(csConf); - YarnConfiguration conf = new YarnConfiguration(csConf); - conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, - ResourceScheduler.class); - conf.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER, - YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); - rm = new MockRM(conf); bind(ResourceManager.class).toInstance(rm); serve("/*").with(GuiceContainer.class); } } public TestRMWebServicesCapacitySched() { - super(new WebAppDescriptor.Builder( - "org.apache.hadoop.yarn.server.resourcemanager.webapp") - .contextListenerClass(GuiceServletConfig.class) - .filterClass(com.google.inject.servlet.GuiceFilter.class) - .contextPath("jersey-guice-filter").servletPath("/").build()); + super(createWebAppDescriptor()); } @Before @Override public void setUp() throws Exception { super.setUp(); + rm = createMockRM(new CapacitySchedulerConfiguration( + new Configuration(false))); GuiceServletConfig.setInjector( - Guice.createInjector(new WebServletModule())); + Guice.createInjector(new WebServletModule(rm))); } public static void setupQueueConfiguration( @@ -389,4 +384,22 @@ public static void updateTestDataAutomatically(String configFilename, String act Assert.fail("overwrite should not fail " + e.getMessage()); } } + + public static WebAppDescriptor createWebAppDescriptor() { + return new WebAppDescriptor.Builder( + TestRMWebServicesCapacitySched.class.getPackage().getName()) + .contextListenerClass(GuiceServletConfig.class) + .filterClass(com.google.inject.servlet.GuiceFilter.class) + .contextPath("jersey-guice-filter").servletPath("/").build(); + } + + public static MockRM createMockRM(CapacitySchedulerConfiguration csConf) { + setupQueueConfiguration(csConf); + YarnConfiguration conf = new YarnConfiguration(csConf); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + conf.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER, + YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); + return new MockRM(conf); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java index 0e2ecd0bfad38..4cc50c4d57c1b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java @@ -19,10 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; import com.google.inject.Guice; -import com.google.inject.servlet.ServletModule; import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; -import com.sun.jersey.test.framework.WebAppDescriptor; import java.io.IOException; import java.util.HashMap; @@ -34,67 +31,28 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; -import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; import org.apache.hadoop.yarn.webapp.GuiceServletConfig; import org.apache.hadoop.yarn.webapp.JerseyTestBase; import org.junit.Test; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.GB; import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.assertJsonResponse; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createMockRM; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createWebAppDescriptor; public class TestRMWebServicesCapacitySchedDynamicConfig extends JerseyTestBase { - private static final int GB = 1024; - private static MockRM rm; + private MockRM rm; private CapacitySchedulerQueueManager autoQueueHandler; - private static class WebServletModule extends ServletModule { - private final Configuration conf; - - WebServletModule(Configuration conf) { - this.conf = conf; - } - - @Override - protected void configureServlets() { - bind(JAXBContextResolver.class); - bind(RMWebServices.class); - bind(GenericExceptionHandler.class); - conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, - ResourceScheduler.class); - conf.set(YarnConfiguration.RM_PLACEMENT_CONSTRAINTS_HANDLER, - YarnConfiguration.SCHEDULER_RM_PLACEMENT_CONSTRAINTS_HANDLER); - rm = new MockRM(conf); - bind(ResourceManager.class).toInstance(rm); - serve("/*").with(GuiceContainer.class); - } - } - - private void initResourceManager(Configuration conf) throws IOException { - GuiceServletConfig.setInjector( - Guice.createInjector(new WebServletModule(conf))); - rm.start(); - //Need to call reinitialize as - //MutableCSConfigurationProvider with InMemoryConfigurationStore - //somehow does not load the queues properly and falls back to default config. - //Therefore CS will think there's only the default queue there. - ((CapacityScheduler) rm.getResourceScheduler()).reinitialize(conf, - rm.getRMContext(), true); - } - public TestRMWebServicesCapacitySchedDynamicConfig() { - super(new WebAppDescriptor.Builder( - "org.apache.hadoop.yarn.server.resourcemanager.webapp") - .contextListenerClass(GuiceServletConfig.class) - .filterClass(com.google.inject.servlet.GuiceFilter.class) - .contextPath("jersey-guice-filter").servletPath("/").build()); + super(createWebAppDescriptor()); } @Test @@ -327,4 +285,17 @@ public static Configuration createConfiguration( return config; } } + + private void initResourceManager(Configuration conf) throws IOException { + rm = createMockRM(new CapacitySchedulerConfiguration(conf)); + GuiceServletConfig.setInjector( + Guice.createInjector(new TestRMWebServicesCapacitySched.WebServletModule(rm))); + rm.start(); + //Need to call reinitialize as + //MutableCSConfigurationProvider with InMemoryConfigurationStore + //somehow does not load the queues properly and falls back to default config. + //Therefore CS will think there's only the default queue there. + ((CapacityScheduler) rm.getResourceScheduler()).reinitialize(conf, + rm.getRMContext(), true); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java index 3fc3c26c00af2..ee472dafcb940 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesSchedulerActivities.java @@ -18,15 +18,20 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp; +import com.google.inject.Guice; import com.sun.jersey.api.client.ClientResponse; import com.sun.jersey.api.client.WebResource; import com.sun.jersey.core.util.MultivaluedMapImpl; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.webapp.GuiceServletConfig; +import org.apache.hadoop.yarn.webapp.JerseyTestBase; +import org.junit.Before; import org.apache.hadoop.http.JettyUtils; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerState; @@ -83,6 +88,8 @@ import static org.apache.hadoop.yarn.server.resourcemanager.webapp.ActivitiesTestUtils.verifyNumberOfNodes; import static org.apache.hadoop.yarn.server.resourcemanager.webapp.ActivitiesTestUtils.verifyQueueOrder; import static org.apache.hadoop.yarn.server.resourcemanager.webapp.ActivitiesTestUtils.verifyStateOfAllocations; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createMockRM; +import static org.apache.hadoop.yarn.server.resourcemanager.webapp.TestRMWebServicesCapacitySched.createWebAppDescriptor; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -91,11 +98,23 @@ /** * Tests for scheduler/app activities. */ -public class TestRMWebServicesSchedulerActivities - extends TestRMWebServicesCapacitySched { +public class TestRMWebServicesSchedulerActivities extends JerseyTestBase { + + private MockRM rm; - private static final Logger LOG = LoggerFactory.getLogger( - TestRMWebServicesSchedulerActivities.class); + public TestRMWebServicesSchedulerActivities() { + super(createWebAppDescriptor()); + } + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + rm = createMockRM(new CapacitySchedulerConfiguration( + new Configuration(false))); + GuiceServletConfig.setInjector( + Guice.createInjector(new TestRMWebServicesCapacitySched.WebServletModule(rm))); + } @Test public void testAssignMultipleContainersPerNodeHeartbeat() From d0fa9b5775185bd83e4a767a7dfc13ef89c5154a Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Thu, 10 Mar 2022 22:02:38 +0530 Subject: [PATCH 030/145] HADOOP-18155. Refactor tests in TestFileUtil (#4053) --- .../java/org/apache/hadoop/fs/FileUtil.java | 36 +- .../org/apache/hadoop/fs/TestFileUtil.java | 402 +++++++++++------- 2 files changed, 275 insertions(+), 163 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index 96d8a40512f0a..b788c7ec6b664 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -39,6 +39,7 @@ import java.nio.file.FileSystems; import java.nio.file.Files; import java.nio.file.LinkOption; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Enumeration; import java.util.List; @@ -992,6 +993,14 @@ private static void unpackEntries(TarArchiveInputStream tis, + " would create entry outside of " + outputDir); } + if (entry.isSymbolicLink() || entry.isLink()) { + String canonicalTargetPath = getCanonicalPath(entry.getLinkName(), outputDir); + if (!canonicalTargetPath.startsWith(targetDirPath)) { + throw new IOException( + "expanding " + entry.getName() + " would create entry outside of " + outputDir); + } + } + if (entry.isDirectory()) { File subDir = new File(outputDir, entry.getName()); if (!subDir.mkdirs() && !subDir.isDirectory()) { @@ -1007,10 +1016,12 @@ private static void unpackEntries(TarArchiveInputStream tis, } if (entry.isSymbolicLink()) { - // Create symbolic link relative to tar parent dir - Files.createSymbolicLink(FileSystems.getDefault() - .getPath(outputDir.getPath(), entry.getName()), - FileSystems.getDefault().getPath(entry.getLinkName())); + // Create symlink with canonical target path to ensure that we don't extract + // outside targetDirPath + String canonicalTargetPath = getCanonicalPath(entry.getLinkName(), outputDir); + Files.createSymbolicLink( + FileSystems.getDefault().getPath(outputDir.getPath(), entry.getName()), + FileSystems.getDefault().getPath(canonicalTargetPath)); return; } @@ -1022,7 +1033,8 @@ private static void unpackEntries(TarArchiveInputStream tis, } if (entry.isLink()) { - File src = new File(outputDir, entry.getLinkName()); + String canonicalTargetPath = getCanonicalPath(entry.getLinkName(), outputDir); + File src = new File(canonicalTargetPath); HardLink.createHardLink(src, outputFile); return; } @@ -1030,6 +1042,20 @@ private static void unpackEntries(TarArchiveInputStream tis, org.apache.commons.io.FileUtils.copyToFile(tis, outputFile); } + /** + * Gets the canonical path for the given path. + * + * @param path The path for which the canonical path needs to be computed. + * @param parentDir The parent directory to use if the path is a relative path. + * @return The canonical path of the given path. + */ + private static String getCanonicalPath(String path, File parentDir) throws IOException { + java.nio.file.Path targetPath = Paths.get(path); + return (targetPath.isAbsolute() ? + new File(path) : + new File(parentDir, path)).getCanonicalPath(); + } + /** * Class for creating hardlinks. * Supports Unix, WindXP. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java index e19900dfeacdb..29eafb9e4dac0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java @@ -42,13 +42,14 @@ import java.net.URL; import java.net.UnknownHostException; import java.nio.charset.StandardCharsets; -import java.nio.file.FileSystems; import java.nio.file.Files; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.jar.Attributes; import java.util.jar.JarFile; import java.util.jar.Manifest; @@ -60,9 +61,12 @@ import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.StringUtils; import org.apache.tools.tar.TarEntry; import org.apache.tools.tar.TarOutputStream; + +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -158,13 +162,12 @@ public void setup() throws IOException { FileUtils.forceMkdir(dir1); FileUtils.forceMkdir(dir2); - new File(del, FILE).createNewFile(); - File tmpFile = new File(tmp, FILE); - tmpFile.createNewFile(); + Verify.createNewFile(new File(del, FILE)); + File tmpFile = Verify.createNewFile(new File(tmp, FILE)); // create files - new File(dir1, FILE).createNewFile(); - new File(dir2, FILE).createNewFile(); + Verify.createNewFile(new File(dir1, FILE)); + Verify.createNewFile(new File(dir2, FILE)); // create a symlink to file File link = new File(del, LINK); @@ -173,7 +176,7 @@ public void setup() throws IOException { // create a symlink to dir File linkDir = new File(del, "tmpDir"); FileUtil.symLink(tmp.toString(), linkDir.toString()); - Assert.assertEquals(5, del.listFiles().length); + Assert.assertEquals(5, Objects.requireNonNull(del.listFiles()).length); // create files in partitioned directories createFile(partitioned, "part-r-00000", "foo"); @@ -200,13 +203,9 @@ public void tearDown() throws IOException { private File createFile(File directory, String name, String contents) throws IOException { File newFile = new File(directory, name); - PrintWriter pw = new PrintWriter(newFile); - try { + try (PrintWriter pw = new PrintWriter(newFile)) { pw.println(contents); } - finally { - pw.close(); - } return newFile; } @@ -218,11 +217,11 @@ public void testListFiles() throws IOException { //Test existing directory with no files case File newDir = new File(tmp.getPath(),"test"); - newDir.mkdir(); + Verify.mkdir(newDir); Assert.assertTrue("Failed to create test dir", newDir.exists()); files = FileUtil.listFiles(newDir); Assert.assertEquals(0, files.length); - newDir.delete(); + assertTrue(newDir.delete()); Assert.assertFalse("Failed to delete test dir", newDir.exists()); //Test non-existing directory case, this throws @@ -244,11 +243,11 @@ public void testListAPI() throws IOException { //Test existing directory with no files case File newDir = new File(tmp.getPath(),"test"); - newDir.mkdir(); + Verify.mkdir(newDir); Assert.assertTrue("Failed to create test dir", newDir.exists()); files = FileUtil.list(newDir); Assert.assertEquals("New directory unexpectedly contains files", 0, files.length); - newDir.delete(); + assertTrue(newDir.delete()); Assert.assertFalse("Failed to delete test dir", newDir.exists()); //Test non-existing directory case, this throws @@ -266,7 +265,7 @@ public void testListAPI() throws IOException { public void testFullyDelete() throws IOException { boolean ret = FileUtil.fullyDelete(del); Assert.assertTrue(ret); - Assert.assertFalse(del.exists()); + Verify.notExists(del); validateTmpDir(); } @@ -279,13 +278,13 @@ public void testFullyDelete() throws IOException { @Test (timeout = 30000) public void testFullyDeleteSymlinks() throws IOException { File link = new File(del, LINK); - Assert.assertEquals(5, del.list().length); + assertDelListLength(5); // Since tmpDir is symlink to tmp, fullyDelete(tmpDir) should not // delete contents of tmp. See setupDirs for details. boolean ret = FileUtil.fullyDelete(link); Assert.assertTrue(ret); - Assert.assertFalse(link.exists()); - Assert.assertEquals(4, del.list().length); + Verify.notExists(link); + assertDelListLength(4); validateTmpDir(); File linkDir = new File(del, "tmpDir"); @@ -293,8 +292,8 @@ public void testFullyDeleteSymlinks() throws IOException { // delete contents of tmp. See setupDirs for details. ret = FileUtil.fullyDelete(linkDir); Assert.assertTrue(ret); - Assert.assertFalse(linkDir.exists()); - Assert.assertEquals(3, del.list().length); + Verify.notExists(linkDir); + assertDelListLength(3); validateTmpDir(); } @@ -310,16 +309,16 @@ public void testFullyDeleteDanglingSymlinks() throws IOException { // to make y as a dangling link to file tmp/x boolean ret = FileUtil.fullyDelete(tmp); Assert.assertTrue(ret); - Assert.assertFalse(tmp.exists()); + Verify.notExists(tmp); // dangling symlink to file File link = new File(del, LINK); - Assert.assertEquals(5, del.list().length); + assertDelListLength(5); // Even though 'y' is dangling symlink to file tmp/x, fullyDelete(y) // should delete 'y' properly. ret = FileUtil.fullyDelete(link); Assert.assertTrue(ret); - Assert.assertEquals(4, del.list().length); + assertDelListLength(4); // dangling symlink to directory File linkDir = new File(del, "tmpDir"); @@ -327,22 +326,22 @@ public void testFullyDeleteDanglingSymlinks() throws IOException { // delete tmpDir properly. ret = FileUtil.fullyDelete(linkDir); Assert.assertTrue(ret); - Assert.assertEquals(3, del.list().length); + assertDelListLength(3); } @Test (timeout = 30000) public void testFullyDeleteContents() throws IOException { boolean ret = FileUtil.fullyDeleteContents(del); Assert.assertTrue(ret); - Assert.assertTrue(del.exists()); - Assert.assertEquals(0, del.listFiles().length); + Verify.exists(del); + Assert.assertEquals(0, Objects.requireNonNull(del.listFiles()).length); validateTmpDir(); } private void validateTmpDir() { - Assert.assertTrue(tmp.exists()); - Assert.assertEquals(1, tmp.listFiles().length); - Assert.assertTrue(new File(tmp, FILE).exists()); + Verify.exists(tmp); + Assert.assertEquals(1, Objects.requireNonNull(tmp.listFiles()).length); + Verify.exists(new File(tmp, FILE)); } /** @@ -366,15 +365,15 @@ private void validateTmpDir() { * @throws IOException */ private void setupDirsAndNonWritablePermissions() throws IOException { - new MyFile(del, FILE_1_NAME).createNewFile(); + Verify.createNewFile(new MyFile(del, FILE_1_NAME)); // "file1" is non-deletable by default, see MyFile.delete(). - xSubDir.mkdirs(); - file2.createNewFile(); + Verify.mkdirs(xSubDir); + Verify.createNewFile(file2); - xSubSubDir.mkdirs(); - file22.createNewFile(); + Verify.mkdirs(xSubSubDir); + Verify.createNewFile(file22); revokePermissions(file22); revokePermissions(xSubSubDir); @@ -382,8 +381,8 @@ private void setupDirsAndNonWritablePermissions() throws IOException { revokePermissions(file2); revokePermissions(xSubDir); - ySubDir.mkdirs(); - file3.createNewFile(); + Verify.mkdirs(ySubDir); + Verify.createNewFile(file3); File tmpFile = new File(tmp, FILE); tmpFile.createNewFile(); @@ -462,8 +461,8 @@ public void testFailFullyDeleteDirSymlinks() throws IOException { boolean ret = FileUtil.fullyDelete(linkDir); // fail symlink deletion Assert.assertFalse(ret); - Assert.assertTrue(linkDir.exists()); - Assert.assertEquals(5, del.list().length); + Verify.exists(linkDir); + assertDelListLength(5); // tmp dir should exist validateTmpDir(); // simulate disk recovers and turns good @@ -471,12 +470,94 @@ public void testFailFullyDeleteDirSymlinks() throws IOException { ret = FileUtil.fullyDelete(linkDir); // success symlink deletion Assert.assertTrue(ret); - Assert.assertFalse(linkDir.exists()); - Assert.assertEquals(4, del.list().length); + Verify.notExists(linkDir); + assertDelListLength(4); // tmp dir should exist validateTmpDir(); } + /** + * Asserts if the {@link TestFileUtil#del} meets the given expected length. + * + * @param expectedLength The expected length of the {@link TestFileUtil#del}. + */ + private void assertDelListLength(int expectedLength) { + Assertions.assertThat(del.list()).describedAs("del list").isNotNull().hasSize(expectedLength); + } + + /** + * Helper class to perform {@link File} operation and also verify them. + */ + public static class Verify { + /** + * Invokes {@link File#createNewFile()} on the given {@link File} instance. + * + * @param file The file to call {@link File#createNewFile()} on. + * @return The result of {@link File#createNewFile()}. + * @throws IOException As per {@link File#createNewFile()}. + */ + public static File createNewFile(File file) throws IOException { + assertTrue("Unable to create new file " + file, file.createNewFile()); + return file; + } + + /** + * Invokes {@link File#mkdir()} on the given {@link File} instance. + * + * @param file The file to call {@link File#mkdir()} on. + * @return The result of {@link File#mkdir()}. + */ + public static File mkdir(File file) { + assertTrue("Unable to mkdir for " + file, file.mkdir()); + return file; + } + + /** + * Invokes {@link File#mkdirs()} on the given {@link File} instance. + * + * @param file The file to call {@link File#mkdirs()} on. + * @return The result of {@link File#mkdirs()}. + */ + public static File mkdirs(File file) { + assertTrue("Unable to mkdirs for " + file, file.mkdirs()); + return file; + } + + /** + * Invokes {@link File#delete()} on the given {@link File} instance. + * + * @param file The file to call {@link File#delete()} on. + * @return The result of {@link File#delete()}. + */ + public static File delete(File file) { + assertTrue("Unable to delete " + file, file.delete()); + return file; + } + + /** + * Invokes {@link File#exists()} on the given {@link File} instance. + * + * @param file The file to call {@link File#exists()} on. + * @return The result of {@link File#exists()}. + */ + public static File exists(File file) { + assertTrue("Expected file " + file + " doesn't exist", file.exists()); + return file; + } + + /** + * Invokes {@link File#exists()} on the given {@link File} instance to check if the + * {@link File} doesn't exists. + * + * @param file The file to call {@link File#exists()} on. + * @return The negation of the result of {@link File#exists()}. + */ + public static File notExists(File file) { + assertFalse("Expected file " + file + " must not exist", file.exists()); + return file; + } + } + /** * Extend {@link File}. Same as {@link File} except for two things: (1) This * treats file1Name as a very special file which is not delete-able @@ -609,14 +690,13 @@ public void testGetDU() throws Exception { FileUtil.chmod(partitioned.getAbsolutePath(), "0777", true/*recursive*/); } } - + @Test (timeout = 30000) - public void testUnTar() throws IOException { + public void testUnTar() throws Exception { // make a simple tar: final File simpleTar = new File(del, FILE); - OutputStream os = new FileOutputStream(simpleTar); - TarOutputStream tos = new TarOutputStream(os); - try { + OutputStream os = new FileOutputStream(simpleTar); + try (TarOutputStream tos = new TarOutputStream(os)) { TarEntry te = new TarEntry("/bar/foo"); byte[] data = "some-content".getBytes("UTF-8"); te.setSize(data.length); @@ -625,55 +705,42 @@ public void testUnTar() throws IOException { tos.closeEntry(); tos.flush(); tos.finish(); - } finally { - tos.close(); } // successfully untar it into an existing dir: FileUtil.unTar(simpleTar, tmp); // check result: - assertTrue(new File(tmp, "/bar/foo").exists()); + Verify.exists(new File(tmp, "/bar/foo")); assertEquals(12, new File(tmp, "/bar/foo").length()); - - final File regularFile = new File(tmp, "QuickBrownFoxJumpsOverTheLazyDog"); - regularFile.createNewFile(); - assertTrue(regularFile.exists()); - try { - FileUtil.unTar(simpleTar, regularFile); - assertTrue("An IOException expected.", false); - } catch (IOException ioe) { - // okay - } + + final File regularFile = + Verify.createNewFile(new File(tmp, "QuickBrownFoxJumpsOverTheLazyDog")); + LambdaTestUtils.intercept(IOException.class, () -> FileUtil.unTar(simpleTar, regularFile)); } @Test (timeout = 30000) public void testReplaceFile() throws IOException { - final File srcFile = new File(tmp, "src"); - // src exists, and target does not exist: - srcFile.createNewFile(); - assertTrue(srcFile.exists()); + final File srcFile = Verify.createNewFile(new File(tmp, "src")); final File targetFile = new File(tmp, "target"); - assertTrue(!targetFile.exists()); + Verify.notExists(targetFile); FileUtil.replaceFile(srcFile, targetFile); - assertTrue(!srcFile.exists()); - assertTrue(targetFile.exists()); + Verify.notExists(srcFile); + Verify.exists(targetFile); // src exists and target is a regular file: - srcFile.createNewFile(); - assertTrue(srcFile.exists()); + Verify.createNewFile(srcFile); + Verify.exists(srcFile); FileUtil.replaceFile(srcFile, targetFile); - assertTrue(!srcFile.exists()); - assertTrue(targetFile.exists()); + Verify.notExists(srcFile); + Verify.exists(targetFile); // src exists, and target is a non-empty directory: - srcFile.createNewFile(); - assertTrue(srcFile.exists()); - targetFile.delete(); - targetFile.mkdirs(); - File obstacle = new File(targetFile, "obstacle"); - obstacle.createNewFile(); - assertTrue(obstacle.exists()); + Verify.createNewFile(srcFile); + Verify.exists(srcFile); + Verify.delete(targetFile); + Verify.mkdirs(targetFile); + File obstacle = Verify.createNewFile(new File(targetFile, "obstacle")); assertTrue(targetFile.exists() && targetFile.isDirectory()); try { FileUtil.replaceFile(srcFile, targetFile); @@ -682,9 +749,9 @@ public void testReplaceFile() throws IOException { // okay } // check up the post-condition: nothing is deleted: - assertTrue(srcFile.exists()); + Verify.exists(srcFile); assertTrue(targetFile.exists() && targetFile.isDirectory()); - assertTrue(obstacle.exists()); + Verify.exists(obstacle); } @Test (timeout = 30000) @@ -697,13 +764,13 @@ public void testCreateLocalTempFile() throws IOException { assertTrue(tmp1.exists() && tmp2.exists()); assertTrue(tmp1.canWrite() && tmp2.canWrite()); assertTrue(tmp1.canRead() && tmp2.canRead()); - tmp1.delete(); - tmp2.delete(); + Verify.delete(tmp1); + Verify.delete(tmp2); assertTrue(!tmp1.exists() && !tmp2.exists()); } @Test (timeout = 30000) - public void testUnZip() throws IOException { + public void testUnZip() throws Exception { // make sa simple zip final File simpleZip = new File(del, FILE); OutputStream os = new FileOutputStream(simpleZip); @@ -724,18 +791,12 @@ public void testUnZip() throws IOException { // successfully unzip it into an existing dir: FileUtil.unZip(simpleZip, tmp); // check result: - assertTrue(new File(tmp, "foo").exists()); + Verify.exists(new File(tmp, "foo")); assertEquals(12, new File(tmp, "foo").length()); - - final File regularFile = new File(tmp, "QuickBrownFoxJumpsOverTheLazyDog"); - regularFile.createNewFile(); - assertTrue(regularFile.exists()); - try { - FileUtil.unZip(simpleZip, regularFile); - assertTrue("An IOException expected.", false); - } catch (IOException ioe) { - // okay - } + + final File regularFile = + Verify.createNewFile(new File(tmp, "QuickBrownFoxJumpsOverTheLazyDog")); + LambdaTestUtils.intercept(IOException.class, () -> FileUtil.unZip(simpleZip, regularFile)); } @Test (timeout = 30000) @@ -781,24 +842,24 @@ public void testCopy5() throws IOException { final File dest = new File(del, "dest"); boolean result = FileUtil.copy(fs, srcPath, dest, false, conf); assertTrue(result); - assertTrue(dest.exists()); + Verify.exists(dest); assertEquals(content.getBytes().length + System.getProperty("line.separator").getBytes().length, dest.length()); - assertTrue(srcFile.exists()); // should not be deleted + Verify.exists(srcFile); // should not be deleted // copy regular file, delete src: - dest.delete(); - assertTrue(!dest.exists()); + Verify.delete(dest); + Verify.notExists(dest); result = FileUtil.copy(fs, srcPath, dest, true, conf); assertTrue(result); - assertTrue(dest.exists()); + Verify.exists(dest); assertEquals(content.getBytes().length + System.getProperty("line.separator").getBytes().length, dest.length()); - assertTrue(!srcFile.exists()); // should be deleted + Verify.notExists(srcFile); // should be deleted // copy a dir: - dest.delete(); - assertTrue(!dest.exists()); + Verify.delete(dest); + Verify.notExists(dest); srcPath = new Path(partitioned.toURI()); result = FileUtil.copy(fs, srcPath, dest, true, conf); assertTrue(result); @@ -810,7 +871,7 @@ public void testCopy5() throws IOException { assertEquals(3 + System.getProperty("line.separator").getBytes().length, f.length()); } - assertTrue(!partitioned.exists()); // should be deleted + Verify.notExists(partitioned); // should be deleted } @Test (timeout = 30000) @@ -898,8 +959,8 @@ public void testSymlinkRenameTo() throws Exception { // create the symlink FileUtil.symLink(file.getAbsolutePath(), link.getAbsolutePath()); - Assert.assertTrue(file.exists()); - Assert.assertTrue(link.exists()); + Verify.exists(file); + Verify.exists(link); File link2 = new File(del, "_link2"); @@ -909,10 +970,10 @@ public void testSymlinkRenameTo() throws Exception { // Make sure the file still exists // (NOTE: this would fail on Java6 on Windows if we didn't // copy the file in FileUtil#symlink) - Assert.assertTrue(file.exists()); + Verify.exists(file); - Assert.assertTrue(link2.exists()); - Assert.assertFalse(link.exists()); + Verify.exists(link2); + Verify.notExists(link); } /** @@ -927,13 +988,13 @@ public void testSymlinkDelete() throws Exception { // create the symlink FileUtil.symLink(file.getAbsolutePath(), link.getAbsolutePath()); - Assert.assertTrue(file.exists()); - Assert.assertTrue(link.exists()); + Verify.exists(file); + Verify.exists(link); // make sure that deleting a symlink works properly - Assert.assertTrue(link.delete()); - Assert.assertFalse(link.exists()); - Assert.assertTrue(file.exists()); + Verify.delete(link); + Verify.notExists(link); + Verify.exists(file); } /** @@ -960,13 +1021,13 @@ public void testSymlinkLength() throws Exception { Assert.assertEquals(data.length, file.length()); Assert.assertEquals(data.length, link.length()); - file.delete(); - Assert.assertFalse(file.exists()); + Verify.delete(file); + Verify.notExists(file); Assert.assertEquals(0, link.length()); - link.delete(); - Assert.assertFalse(link.exists()); + Verify.delete(link); + Verify.notExists(link); } /** @@ -1032,7 +1093,7 @@ public void testSymlinkFileAlreadyExists() throws IOException { public void testSymlinkSameFile() throws IOException { File file = new File(del, FILE); - file.delete(); + Verify.delete(file); // Create a symbolic link // The operation should succeed @@ -1105,21 +1166,21 @@ private void doUntarAndVerify(File tarFile, File untarDir) String parentDir = untarDir.getCanonicalPath() + Path.SEPARATOR + "name"; File testFile = new File(parentDir + Path.SEPARATOR + "version"); - Assert.assertTrue(testFile.exists()); + Verify.exists(testFile); Assert.assertTrue(testFile.length() == 0); String imageDir = parentDir + Path.SEPARATOR + "image"; testFile = new File(imageDir + Path.SEPARATOR + "fsimage"); - Assert.assertTrue(testFile.exists()); + Verify.exists(testFile); Assert.assertTrue(testFile.length() == 157); String currentDir = parentDir + Path.SEPARATOR + "current"; testFile = new File(currentDir + Path.SEPARATOR + "fsimage"); - Assert.assertTrue(testFile.exists()); + Verify.exists(testFile); Assert.assertTrue(testFile.length() == 4331); testFile = new File(currentDir + Path.SEPARATOR + "edits"); - Assert.assertTrue(testFile.exists()); + Verify.exists(testFile); Assert.assertTrue(testFile.length() == 1033); testFile = new File(currentDir + Path.SEPARATOR + "fstime"); - Assert.assertTrue(testFile.exists()); + Verify.exists(testFile); Assert.assertTrue(testFile.length() == 8); } @@ -1180,9 +1241,9 @@ public void testCreateJarWithClassPath() throws Exception { } // create non-jar files, which we expect to not be included in the classpath - Assert.assertTrue(new File(tmp, "text.txt").createNewFile()); - Assert.assertTrue(new File(tmp, "executable.exe").createNewFile()); - Assert.assertTrue(new File(tmp, "README").createNewFile()); + Verify.createNewFile(new File(tmp, "text.txt")); + Verify.createNewFile(new File(tmp, "executable.exe")); + Verify.createNewFile(new File(tmp, "README")); // create classpath jar String wildcardPath = tmp.getCanonicalPath() + File.separator + "*"; @@ -1268,9 +1329,9 @@ public void testGetJarsInDirectory() throws Exception { } // create non-jar files, which we expect to not be included in the result - assertTrue(new File(tmp, "text.txt").createNewFile()); - assertTrue(new File(tmp, "executable.exe").createNewFile()); - assertTrue(new File(tmp, "README").createNewFile()); + Verify.createNewFile(new File(tmp, "text.txt")); + Verify.createNewFile(new File(tmp, "executable.exe")); + Verify.createNewFile(new File(tmp, "README")); // pass in the directory String directory = tmp.getCanonicalPath(); @@ -1304,7 +1365,7 @@ public void setupCompareFs() { uri4 = new URI(uris4); uri5 = new URI(uris5); uri6 = new URI(uris6); - } catch (URISyntaxException use) { + } catch (URISyntaxException ignored) { } // Set up InetAddress inet1 = mock(InetAddress.class); @@ -1327,7 +1388,7 @@ public void setupCompareFs() { when(InetAddress.getByName(uris3)).thenReturn(inet3); when(InetAddress.getByName(uris4)).thenReturn(inet4); when(InetAddress.getByName(uris5)).thenReturn(inet5); - } catch (UnknownHostException ue) { + } catch (UnknownHostException ignored) { } fs1 = mock(FileSystem.class); @@ -1347,62 +1408,87 @@ public void setupCompareFs() { @Test public void testCompareFsNull() throws Exception { setupCompareFs(); - assertEquals(FileUtil.compareFs(null,fs1),false); - assertEquals(FileUtil.compareFs(fs1,null),false); + assertFalse(FileUtil.compareFs(null, fs1)); + assertFalse(FileUtil.compareFs(fs1, null)); } @Test public void testCompareFsDirectories() throws Exception { setupCompareFs(); - assertEquals(FileUtil.compareFs(fs1,fs1),true); - assertEquals(FileUtil.compareFs(fs1,fs2),false); - assertEquals(FileUtil.compareFs(fs1,fs5),false); - assertEquals(FileUtil.compareFs(fs3,fs4),true); - assertEquals(FileUtil.compareFs(fs1,fs6),false); + assertTrue(FileUtil.compareFs(fs1, fs1)); + assertFalse(FileUtil.compareFs(fs1, fs2)); + assertFalse(FileUtil.compareFs(fs1, fs5)); + assertTrue(FileUtil.compareFs(fs3, fs4)); + assertFalse(FileUtil.compareFs(fs1, fs6)); } @Test(timeout = 8000) public void testCreateSymbolicLinkUsingJava() throws IOException { final File simpleTar = new File(del, FILE); OutputStream os = new FileOutputStream(simpleTar); - TarArchiveOutputStream tos = new TarArchiveOutputStream(os); - File untarFile = null; - try { + try (TarArchiveOutputStream tos = new TarArchiveOutputStream(os)) { // Files to tar final String tmpDir = "tmp/test"; File tmpDir1 = new File(tmpDir, "dir1/"); File tmpDir2 = new File(tmpDir, "dir2/"); - // Delete the directories if they already exist - tmpDir1.mkdirs(); - tmpDir2.mkdirs(); + Verify.mkdirs(tmpDir1); + Verify.mkdirs(tmpDir2); - java.nio.file.Path symLink = FileSystems - .getDefault().getPath(tmpDir1.getPath() + "/sl"); + java.nio.file.Path symLink = Paths.get(tmpDir1.getPath(), "sl"); // Create Symbolic Link - Files.createSymbolicLink(symLink, - FileSystems.getDefault().getPath(tmpDir2.getPath())).toString(); + Files.createSymbolicLink(symLink, Paths.get(tmpDir2.getPath())); assertTrue(Files.isSymbolicLink(symLink.toAbsolutePath())); - // put entries in tar file + // Put entries in tar file putEntriesInTar(tos, tmpDir1.getParentFile()); tos.close(); - untarFile = new File(tmpDir, "2"); - // Untar using java + File untarFile = new File(tmpDir, "2"); + // Untar using Java FileUtil.unTarUsingJava(simpleTar, untarFile, false); // Check symbolic link and other directories are there in untar file assertTrue(Files.exists(untarFile.toPath())); - assertTrue(Files.exists(FileSystems.getDefault().getPath(untarFile - .getPath(), tmpDir))); - assertTrue(Files.isSymbolicLink(FileSystems.getDefault().getPath(untarFile - .getPath().toString(), symLink.toString()))); - + assertTrue(Files.exists(Paths.get(untarFile.getPath(), tmpDir))); + assertTrue(Files.isSymbolicLink(Paths.get(untarFile.getPath(), symLink.toString()))); } finally { FileUtils.deleteDirectory(new File("tmp")); - tos.close(); } + } + + @Test(expected = IOException.class) + public void testCreateArbitrarySymlinkUsingJava() throws IOException { + final File simpleTar = new File(del, FILE); + OutputStream os = new FileOutputStream(simpleTar); + File rootDir = new File("tmp"); + try (TarArchiveOutputStream tos = new TarArchiveOutputStream(os)) { + tos.setLongFileMode(TarArchiveOutputStream.LONGFILE_GNU); + + // Create arbitrary dir + File arbitraryDir = new File(rootDir, "arbitrary-dir/"); + Verify.mkdirs(arbitraryDir); + + // We will tar from the tar-root lineage + File tarRoot = new File(rootDir, "tar-root/"); + File symlinkRoot = new File(tarRoot, "dir1/"); + Verify.mkdirs(symlinkRoot); + + // Create Symbolic Link to an arbitrary dir + java.nio.file.Path symLink = Paths.get(symlinkRoot.getPath(), "sl"); + Files.createSymbolicLink(symLink, arbitraryDir.toPath().toAbsolutePath()); + + // Put entries in tar file + putEntriesInTar(tos, tarRoot); + putEntriesInTar(tos, new File(symLink.toFile(), "dir-outside-tar-root/")); + tos.close(); + + // Untar using Java + File untarFile = new File(rootDir, "extracted"); + FileUtil.unTarUsingJava(simpleTar, untarFile, false); + } finally { + FileUtils.deleteDirectory(rootDir); + } } private void putEntriesInTar(TarArchiveOutputStream tos, File f) @@ -1496,7 +1582,7 @@ public void testReadSymlinkWithAFileAsInput() throws IOException { String result = FileUtil.readLink(file); Assert.assertEquals("", result); - file.delete(); + Verify.delete(file); } /** From 481da19494fe13ca42651305b519e0670cafabf0 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Thu, 10 Mar 2022 22:15:35 +0100 Subject: [PATCH 031/145] YARN-10049. FIFOOrderingPolicy Improvements. Contributed by Benjamin Teke --- .../scheduler/policy/FifoComparator.java | 5 + .../policy/TestFifoOrderingPolicy.java | 91 ++++++++++++++----- 2 files changed, 71 insertions(+), 25 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java index 112c50ff1cc52..c62b738df5679 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/FifoComparator.java @@ -29,6 +29,11 @@ public class FifoComparator @Override public int compare(SchedulableEntity r1, SchedulableEntity r2) { int res = r1.compareInputOrderTo(r2); + + if (res == 0) { + res = (int) Math.signum(r1.getStartTime() - r2.getStartTime()); + } + return res; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java index 7ec2c01ec2512..62bc7124c4b9d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/policy/TestFifoOrderingPolicy.java @@ -18,16 +18,19 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy; -import java.util.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; -import org.junit.Assert; -import org.junit.Test; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; import org.apache.hadoop.yarn.api.records.Priority; +import org.junit.Assert; +import org.junit.Test; -import static org.assertj.core.api.Assertions.assertThat; - -public class TestFifoOrderingPolicy { +public +class TestFifoOrderingPolicy { @Test public void testFifoOrderingPolicy() { @@ -36,13 +39,17 @@ public void testFifoOrderingPolicy() { MockSchedulableEntity r1 = new MockSchedulableEntity(); MockSchedulableEntity r2 = new MockSchedulableEntity(); - assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(0); + assertEquals("The comparator should return 0 because the entities are created with " + + "the same values.", 0, + policy.getComparator().compare(r1, r2)); r1.setSerial(1); - assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(1); + assertEquals("The lhs entity has a larger serial, the comparator return " + + "value should be 1.", 1, policy.getComparator().compare(r1, r2)); r2.setSerial(2); - assertThat(policy.getComparator().compare(r1, r2)).isEqualTo(-1); + Assert.assertEquals("The rhs entity has a larger serial, the comparator return " + + "value should be -1.", -1, policy.getComparator().compare(r1, r2)); } @Test @@ -63,46 +70,80 @@ public void testIterators() { schedOrder.addSchedulableEntity(msp3); //Assignment, oldest to youngest - checkSerials(schedOrder.getAssignmentIterator(IteratorSelector.EMPTY_ITERATOR_SELECTOR), new long[]{1, 2, 3}); + checkSerials(Arrays.asList(1L, 2L, 3L), schedOrder.getAssignmentIterator( + IteratorSelector.EMPTY_ITERATOR_SELECTOR)); //Preemption, youngest to oldest - checkSerials(schedOrder.getPreemptionIterator(), new long[]{3, 2, 1}); + checkSerials(Arrays.asList(3L, 2L, 1L), schedOrder.getPreemptionIterator()); } - public void checkSerials(Iterator si, - long[] serials) { - for (int i = 0;i < serials.length;i++) { - Assert.assertEquals(si.next().getSerial(), - serials[i]); + public void checkSerials(List expectedSerials, Iterator + actualSerialIterator) { + for (long expectedSerial : expectedSerials) { + assertEquals(expectedSerial, actualSerialIterator.next().getSerial()); } } @Test - public void testFifoOrderingPolicyAlongWithPriorty() { + public void testFifoOrderingPolicyAlongWithPriority() { FifoOrderingPolicy policy = new FifoOrderingPolicy(); MockSchedulableEntity r1 = new MockSchedulableEntity(); MockSchedulableEntity r2 = new MockSchedulableEntity(); - Priority p1 = Priority.newInstance(1); - Priority p2 = Priority.newInstance(0); + assertEquals("Both r1 and r2 priority is null, the comparator should return 0.", 0, + policy.getComparator().compare(r1, r2)); - // Both r1 and r1 priority is null - Assert.assertEquals(0, policy.getComparator().compare(r1, r2)); + Priority p2 = Priority.newInstance(0); // r1 is null and r2 is not null r2.setApplicationPriority(p2); - Assert.assertEquals(-1, policy.getComparator().compare(r1, r2)); + Assert.assertTrue("The priority of r1 is null, the priority of r2 is not null, " + + "the comparator should return a negative value.", + policy.getComparator().compare(r1, r2) < 0); + + Priority p1 = Priority.newInstance(1); // r1 is not null and r2 is null - r2.setApplicationPriority(null); r1.setApplicationPriority(p1); - Assert.assertEquals(1, policy.getComparator().compare(r1, r2)); + r2.setApplicationPriority(null); + assertTrue("The priority of r1 is not null, the priority of r2 is null," + + "the comparator should return a positive value.", + policy.getComparator().compare(r1, r2) > 0); // r1 is not null and r2 is not null r1.setApplicationPriority(p1); r2.setApplicationPriority(p2); - Assert.assertEquals(-1, policy.getComparator().compare(r1, r2)); + Assert.assertTrue("Both priorities are not null, the r1 has higher priority, " + + "the result should be a negative value.", + policy.getComparator().compare(r1, r2) < 0); } + @Test + public void testOrderingUsingAppSubmitTime() { + FifoOrderingPolicy policy = + new FifoOrderingPolicy(); + MockSchedulableEntity r1 = new MockSchedulableEntity(); + MockSchedulableEntity r2 = new MockSchedulableEntity(); + + // R1, R2 has been started at same time + assertEquals(r1.getStartTime(), r2.getStartTime()); + + // No changes, equal + assertEquals("The submit times are the same, the comparator should return 0.", 0, + policy.getComparator().compare(r1, r2)); + + // R2 has been started after R1 + r1.setStartTime(5); + r2.setStartTime(10); + Assert.assertTrue("r2 was started after r1, " + + "the comparator should return a negative value.", + policy.getComparator().compare(r1, r2) < 0); + + // R1 has been started after R2 + r1.setStartTime(10); + r2.setStartTime(5); + Assert.assertTrue("r2 was started before r1, the comparator should return a positive value.", + policy.getComparator().compare(r1, r2) > 0); + } } From ed65aa23240b3dd6b56e86e5f0e9d38069fb3b01 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Thu, 10 Mar 2022 22:22:58 +0100 Subject: [PATCH 032/145] YARN-11067. Resource overcommitment due to incorrect resource normalisation logical order. Contributed by Andras Gyori --- .../scheduler/capacity/ParentQueue.java | 37 +++++++-------- .../TestAbsoluteResourceConfiguration.java | 47 +++++++++++++++++++ 2 files changed, 64 insertions(+), 20 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index c624aab71cc90..87ebc0b40f3b1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -1294,17 +1294,24 @@ public boolean hasChildQueues() { private void calculateEffectiveResourcesAndCapacity(String label, Resource clusterResource) { + // Update effective resources for my self; + if (rootQueue) { + Resource resourceByLabel = labelManager.getResourceByLabel(label, clusterResource); + usageTracker.getQueueResourceQuotas().setEffectiveMinResource(label, resourceByLabel); + usageTracker.getQueueResourceQuotas().setEffectiveMaxResource(label, resourceByLabel); + } else { + super.updateEffectiveResources(clusterResource); + } + + recalculateEffectiveMinRatio(label, clusterResource); + } + + private void recalculateEffectiveMinRatio(String label, Resource clusterResource) { // For root queue, ensure that max/min resource is updated to latest // cluster resource. - Resource resourceByLabel = labelManager.getResourceByLabel(label, - clusterResource); - - /* - * == Below logic are added to calculate effectiveMinRatioPerResource == - */ + Resource resourceByLabel = labelManager.getResourceByLabel(label, clusterResource); - // Total configured min resources of direct children of this given parent - // queue + // Total configured min resources of direct children of this given parent queue Resource configuredMinResources = Resource.newInstance(0L, 0); for (CSQueue childQueue : getChildQueues()) { Resources.addTo(configuredMinResources, @@ -1312,8 +1319,7 @@ private void calculateEffectiveResourcesAndCapacity(String label, } // Factor to scale down effective resource: When cluster has sufficient - // resources, effective_min_resources will be same as configured - // min_resources. + // resources, effective_min_resources will be same as configured min_resources. Resource numeratorForMinRatio = null; if (getQueuePath().equals("root")) { if (!resourceByLabel.equals(Resources.none()) && Resources.lessThan(resourceCalculator, @@ -1324,21 +1330,12 @@ private void calculateEffectiveResourcesAndCapacity(String label, if (Resources.lessThan(resourceCalculator, clusterResource, usageTracker.getQueueResourceQuotas().getEffectiveMinResource(label), configuredMinResources)) { - numeratorForMinRatio = usageTracker.getQueueResourceQuotas() - .getEffectiveMinResource(label); + numeratorForMinRatio = usageTracker.getQueueResourceQuotas().getEffectiveMinResource(label); } } effectiveMinResourceRatio.put(label, getEffectiveMinRatio( configuredMinResources, numeratorForMinRatio)); - - // Update effective resources for my self; - if (rootQueue) { - usageTracker.getQueueResourceQuotas().setEffectiveMinResource(label, resourceByLabel); - usageTracker.getQueueResourceQuotas().setEffectiveMaxResource(label, resourceByLabel); - } else{ - super.updateEffectiveResources(clusterResource); - } } private Map getEffectiveMinRatio( diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.java index d7c80b5dda117..8d68cbf7932eb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAbsoluteResourceConfiguration.java @@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent; import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Assert; import org.junit.Test; @@ -100,6 +101,21 @@ public class TestAbsoluteResourceConfiguration { private static Set resourceTypes = new HashSet<>( Arrays.asList("memory", "vcores")); + private CapacitySchedulerConfiguration setupNormalizationConfiguration() { + CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); + csConf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[]{QUEUEA, QUEUEB}); + csConf.setQueues(QUEUEA_FULL.getFullPath(), new String[]{QUEUEA1, QUEUEA2}); + +// 60, 28 + csConf.setMinimumResourceRequirement("", QUEUEA_FULL, Resource.newInstance(50 * GB, 20)); + csConf.setMinimumResourceRequirement("", QUEUEA1_FULL, Resource.newInstance(30 * GB, 15)); + csConf.setMinimumResourceRequirement("", QUEUEA2_FULL, Resource.newInstance(20 * GB, 5)); + csConf.setMinimumResourceRequirement("", QUEUEB_FULL, Resource.newInstance(10 * GB, 8)); + + return csConf; + } + private CapacitySchedulerConfiguration setupSimpleQueueConfiguration( boolean isCapacityNeeded) { CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(); @@ -292,6 +308,37 @@ public void testSimpleMinMaxResourceConfigurartionPerQueue() rm.close(); } + @Test + public void testNormalizationAfterNodeRemoval() throws Exception { + CapacitySchedulerConfiguration csConf = setupNormalizationConfiguration(); + csConf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + + MockRM rm = new MockRM(csConf); + + rm.start(); + rm.registerNode("h1:1234", 8 * GB, 4); + rm.registerNode("h2:1234", 8 * GB, 4); + rm.registerNode("h3:1234", 8 * GB, 4); + MockNM nm = rm.registerNode("h4:1234", 8 * GB, 4); + rm.registerNode("h5:1234", 28 * GB, 12); + + // Send a removal event to CS. MockRM#unregisterNode does not reflect the real world scenario, + // therefore we manually need to invoke this removal event. + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + cs.handle(new NodeRemovedSchedulerEvent(rm.getRMContext().getRMNodes().get(nm.getNodeId()))); + + Resource res = Resources.add( + cs.getQueue(QUEUEA1_FULL.getFullPath()).getEffectiveCapacity(""), + cs.getQueue(QUEUEA2_FULL.getFullPath()).getEffectiveCapacity("")); + Resource resParent = cs.getQueue(QUEUEA_FULL.getFullPath()).getEffectiveCapacity(""); + + // Check if there is no overcommitment on behalf of the child queues + Assert.assertTrue(String.format("Summarized resource %s of all children is greater than " + + "their parent's %s", res, resParent), + Resources.lessThan(cs.getResourceCalculator(), cs.getClusterResource(), res, resParent)); + } + @Test public void testEffectiveMinMaxResourceConfigurartionPerQueue() throws Exception { From 672e380c4f6ffcb0a6fee6d8263166e16b4323c2 Mon Sep 17 00:00:00 2001 From: Mukund Thakur Date: Fri, 11 Mar 2022 13:05:45 +0530 Subject: [PATCH 033/145] HADOOP-18112: Implement paging during multi object delete. (#4045) Multi object delete of size more than 1000 is not supported by S3 and fails with MalformedXML error. So implementing paging of requests to reduce the number of keys in a single request. Page size can be configured using "fs.s3a.bulk.delete.page.size" Contributed By: Mukund Thakur --- .../java/org/apache/hadoop/util/Lists.java | 24 ++++ .../org/apache/hadoop/util/TestLists.java | 44 +++++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 78 +++++-------- .../hadoop/fs/s3a/api/RequestFactory.java | 5 +- .../hadoop/fs/s3a/impl/DeleteOperation.java | 45 ++------ .../fs/s3a/impl/OperationCallbacks.java | 12 +- .../hadoop/fs/s3a/impl/RenameOperation.java | 8 +- .../fs/s3a/impl/RequestFactoryImpl.java | 5 +- .../hadoop/fs/s3a/tools/MarkerTool.java | 2 +- .../fs/s3a/tools/MarkerToolOperations.java | 9 +- .../s3a/tools/MarkerToolOperationsImpl.java | 10 +- .../fs/s3a/ITestS3AFailureHandling.java | 38 ++++++- .../apache/hadoop/fs/s3a/S3ATestUtils.java | 107 +++++++++++++++++- .../s3a/impl/ITestPartialRenamesDeletes.java | 105 ----------------- .../fs/s3a/impl/TestRequestFactory.java | 2 +- .../fs/s3a/scale/ITestS3ADeleteManyFiles.java | 2 +- .../s3a/test/MinimalOperationCallbacks.java | 9 +- 17 files changed, 273 insertions(+), 232 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java index b6d74ee679281..5d9cc0502afaa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java @@ -232,4 +232,28 @@ private static boolean addAll(Collection addTo, return addAll(addTo, elementsToAdd.iterator()); } + /** + * Returns consecutive sub-lists of a list, each of the same size + * (the final list may be smaller). + * @param originalList original big list. + * @param pageSize desired size of each sublist ( last one + * may be smaller) + * @return a list of sub lists. + */ + public static List> partition(List originalList, int pageSize) { + + Preconditions.checkArgument(originalList != null && originalList.size() > 0, + "Invalid original list"); + Preconditions.checkArgument(pageSize > 0, "Page size should " + + "be greater than 0 for performing partition"); + + List> result = new ArrayList<>(); + int i=0; + while (i < originalList.size()) { + result.add(originalList.subList(i, + Math.min(i + pageSize, originalList.size()))); + i = i + pageSize; + } + return result; + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLists.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLists.java index 537e3781edc0e..53241da695c63 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLists.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLists.java @@ -18,9 +18,11 @@ package org.apache.hadoop.util; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; +import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -79,6 +81,48 @@ public void testItrLinkedLists() { Assert.assertEquals(4, list.size()); } + @Test + public void testListsPartition() { + List list = new ArrayList<>(); + list.add("a"); + list.add("b"); + list.add("c"); + list.add("d"); + list.add("e"); + List> res = Lists. + partition(list, 2); + Assertions.assertThat(res) + .describedAs("Number of partitions post partition") + .hasSize(3); + Assertions.assertThat(res.get(0)) + .describedAs("Number of elements in first partition") + .hasSize(2); + Assertions.assertThat(res.get(2)) + .describedAs("Number of elements in last partition") + .hasSize(1); + + List> res2 = Lists. + partition(list, 1); + Assertions.assertThat(res2) + .describedAs("Number of partitions post partition") + .hasSize(5); + Assertions.assertThat(res2.get(0)) + .describedAs("Number of elements in first partition") + .hasSize(1); + Assertions.assertThat(res2.get(4)) + .describedAs("Number of elements in last partition") + .hasSize(1); + + List> res3 = Lists. + partition(list, 6); + Assertions.assertThat(res3) + .describedAs("Number of partitions post partition") + .hasSize(1); + Assertions.assertThat(res3.get(0)) + .describedAs("Number of elements in first partition") + .hasSize(5); + } + @Test public void testArrayListWithSize() { List list = Lists.newArrayListWithCapacity(3); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index c8a73d956d844..86da70ecdd316 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -135,6 +135,7 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; +import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileStatus; @@ -225,6 +226,7 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfOperation; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfSupplier; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; +import static org.apache.hadoop.util.Preconditions.checkArgument; import static org.apache.hadoop.util.functional.RemoteIterators.typeCastingRemoteIterator; /** @@ -550,6 +552,8 @@ public void initialize(URI name, Configuration originalConf) pageSize = intOption(getConf(), BULK_DELETE_PAGE_SIZE, BULK_DELETE_PAGE_SIZE_DEFAULT, 0); + checkArgument(pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE, + "page size out of range: %s", pageSize); listing = new Listing(listingOperationCallbacks, createStoreContext()); } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation @@ -2026,14 +2030,12 @@ public CopyResult copyFile(final String srcKey, } @Override - public DeleteObjectsResult removeKeys( - final List keysToDelete, - final boolean deleteFakeDir, - final boolean quiet) + public void removeKeys( + final List keysToDelete, + final boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { auditSpan.activate(); - return S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir, - quiet); + S3AFileSystem.this.removeKeys(keysToDelete, deleteFakeDir); } @Override @@ -2818,10 +2820,6 @@ public void incrementPutProgressStatistics(String key, long bytes) { * @param keysToDelete collection of keys to delete on the s3-backend. * if empty, no request is made of the object store. * @param deleteFakeDir indicates whether this is for deleting fake dirs - * @param quiet should a bulk query be quiet, or should its result list - * all deleted keys? - * @return the deletion result if a multi object delete was invoked - * and it returned without a failure. * @throws InvalidRequestException if the request was rejected due to * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not @@ -2831,10 +2829,9 @@ public void incrementPutProgressStatistics(String key, long bytes) { * @throws AmazonClientException other amazon-layer failure. */ @Retries.RetryRaw - private DeleteObjectsResult removeKeysS3( - List keysToDelete, - boolean deleteFakeDir, - boolean quiet) + private void removeKeysS3( + List keysToDelete, + boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { if (LOG.isDebugEnabled()) { @@ -2847,16 +2844,28 @@ private DeleteObjectsResult removeKeysS3( } if (keysToDelete.isEmpty()) { // exit fast if there are no keys to delete - return null; + return; } for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { blockRootDelete(keyVersion.getKey()); } - DeleteObjectsResult result = null; try { if (enableMultiObjectsDelete) { - result = deleteObjects( - getRequestFactory().newBulkDeleteRequest(keysToDelete, quiet)); + if (keysToDelete.size() <= pageSize) { + deleteObjects(getRequestFactory() + .newBulkDeleteRequest(keysToDelete)); + } else { + // Multi object deletion of more than 1000 keys is not supported + // by s3. So we are paging the keys by page size. + LOG.debug("Partitioning the keys to delete as it is more than " + + "page size. Number of keys: {}, Page size: {}", + keysToDelete.size(), pageSize); + for (List batchOfKeysToDelete : + Lists.partition(keysToDelete, pageSize)) { + deleteObjects(getRequestFactory() + .newBulkDeleteRequest(batchOfKeysToDelete)); + } + } } else { for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) { deleteObject(keyVersion.getKey()); @@ -2872,7 +2881,6 @@ private DeleteObjectsResult removeKeysS3( throw ex; } noteDeleted(keysToDelete.size(), deleteFakeDir); - return result; } /** @@ -2889,7 +2897,7 @@ private void noteDeleted(final int count, final boolean deleteFakeDir) { } /** - * Invoke {@link #removeKeysS3(List, boolean, boolean)}. + * Invoke {@link #removeKeysS3(List, boolean)}. * If a {@code MultiObjectDeleteException} is raised, the * relevant statistics are updated. * @@ -2910,35 +2918,9 @@ public void removeKeys( final boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { - removeKeys(keysToDelete, deleteFakeDir, - true); - } - - /** - * Invoke {@link #removeKeysS3(List, boolean, boolean)}. - * @param keysToDelete collection of keys to delete on the s3-backend. - * if empty, no request is made of the object store. - * @param deleteFakeDir indicates whether this is for deleting fake dirs. - * @param quiet should a bulk query be quiet, or should its result list - * all deleted keys - * @return the deletion result if a multi object delete was invoked - * and it returned without a failure, else null. - * @throws InvalidRequestException if the request was rejected due to - * a mistaken attempt to delete the root directory. - * @throws MultiObjectDeleteException one or more of the keys could not - * be deleted in a multiple object delete operation. - * @throws AmazonClientException amazon-layer failure. - * @throws IOException other IO Exception. - */ - @Retries.RetryRaw - private DeleteObjectsResult removeKeys( - final List keysToDelete, - final boolean deleteFakeDir, - final boolean quiet) - throws MultiObjectDeleteException, AmazonClientException, IOException { try (DurationInfo ignored = new DurationInfo(LOG, false, - "Deleting %d keys", keysToDelete.size())) { - return removeKeysS3(keysToDelete, deleteFakeDir, quiet); + "Deleting %d keys", keysToDelete.size())) { + removeKeysS3(keysToDelete, deleteFakeDir); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java index ee5728688b74e..97a15d95132f4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java @@ -291,12 +291,9 @@ ListObjectsV2Request newListObjectsV2Request(String key, /** * Bulk delete request. * @param keysToDelete list of keys to delete. - * @param quiet should a bulk query be quiet, or should its result list - * all deleted keys? * @return the request */ DeleteObjectsRequest newBulkDeleteRequest( - List keysToDelete, - boolean quiet); + List keysToDelete); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java index 3d2ab22b44dc4..a45bfe46f169f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/DeleteOperation.java @@ -25,7 +25,6 @@ import java.util.stream.Collectors; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.DeleteObjectsResult; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; import org.slf4j.Logger; @@ -365,8 +364,7 @@ private CompletableFuture submitDelete( callableWithinAuditSpan( getAuditSpan(), () -> { asyncDeleteAction( - keyList, - LOG.isDebugEnabled()); + keyList); return null; })); } @@ -376,20 +374,16 @@ private CompletableFuture submitDelete( * the keys from S3 and paths from S3Guard. * * @param keyList keys to delete. - * @param auditDeletedKeys should the results be audited and undeleted * entries logged? * @throws IOException failure */ @Retries.RetryTranslated private void asyncDeleteAction( - final List keyList, - final boolean auditDeletedKeys) + final List keyList) throws IOException { - List deletedObjects = new ArrayList<>(); try (DurationInfo ignored = new DurationInfo(LOG, false, "Delete page of %d keys", keyList.size())) { - DeleteObjectsResult result; if (!keyList.isEmpty()) { // first delete the files. List files = keyList.stream() @@ -397,15 +391,12 @@ private void asyncDeleteAction( .map(e -> e.keyVersion) .collect(Collectors.toList()); LOG.debug("Deleting of {} file objects", files.size()); - result = Invoker.once("Remove S3 Files", + Invoker.once("Remove S3 Files", status.getPath().toString(), () -> callbacks.removeKeys( files, - false, - !auditDeletedKeys)); - if (result != null) { - deletedObjects.addAll(result.getDeletedObjects()); - } + false + )); // now the dirs List dirs = keyList.stream() .filter(e -> e.isDirMarker) @@ -413,32 +404,12 @@ private void asyncDeleteAction( .collect(Collectors.toList()); LOG.debug("Deleting of {} directory markers", dirs.size()); // This is invoked with deleteFakeDir. - result = Invoker.once("Remove S3 Dir Markers", + Invoker.once("Remove S3 Dir Markers", status.getPath().toString(), () -> callbacks.removeKeys( dirs, - true, - !auditDeletedKeys)); - if (result != null) { - deletedObjects.addAll(result.getDeletedObjects()); - } - } - if (auditDeletedKeys) { - // audit the deleted keys - if (deletedObjects.size() != keyList.size()) { - // size mismatch - LOG.warn("Size mismatch in deletion operation. " - + "Expected count of deleted files: {}; " - + "actual: {}", - keyList.size(), deletedObjects.size()); - // strip out the deleted keys - for (DeleteObjectsResult.DeletedObject del : deletedObjects) { - keyList.removeIf(kv -> kv.getKey().equals(del.getKey())); - } - for (DeleteEntry kv : keyList) { - LOG.debug("{}", kv.getKey()); - } - } + true + )); } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java index a72dc7e10b33e..ecfe2c0ba0a24 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java @@ -24,7 +24,6 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.DeleteObjectsResult; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.transfer.model.CopyResult; @@ -138,10 +137,6 @@ CopyResult copyFile(String srcKey, * @param keysToDelete collection of keys to delete on the s3-backend. * if empty, no request is made of the object store. * @param deleteFakeDir indicates whether this is for deleting fake dirs. - * @param quiet should a bulk query be quiet, or should its result list - * all deleted keys - * @return the deletion result if a multi object delete was invoked - * and it returned without a failure, else null. * @throws InvalidRequestException if the request was rejected due to * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not @@ -150,10 +145,9 @@ CopyResult copyFile(String srcKey, * @throws IOException other IO Exception. */ @Retries.RetryRaw - DeleteObjectsResult removeKeys( - List keysToDelete, - boolean deleteFakeDir, - boolean quiet) + void removeKeys( + List keysToDelete, + boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java index c1700ef389cc3..bc9ad669b56f5 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java @@ -49,6 +49,7 @@ import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.RENAME_PARALLEL_LIMIT; +import static org.apache.hadoop.util.Preconditions.checkArgument; /** * A parallelized rename operation. @@ -155,6 +156,9 @@ public RenameOperation( this.destKey = destKey; this.destStatus = destStatus; this.callbacks = callbacks; + checkArgument(pageSize > 0 + && pageSize <= InternalConstants.MAX_ENTRIES_TO_DELETE, + "page size out of range: %s", pageSize); this.pageSize = pageSize; } @@ -586,8 +590,8 @@ private void removeSourceObjects( sourcePath.toString(), () -> callbacks.removeKeys( keys, - false, - true)); + false + )); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index 04cff49be3d80..fa58323decd03 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -575,12 +575,11 @@ public DeleteObjectRequest newDeleteObjectRequest(String key) { @Override public DeleteObjectsRequest newBulkDeleteRequest( - List keysToDelete, - boolean quiet) { + List keysToDelete) { return prepareRequest( new DeleteObjectsRequest(bucket) .withKeys(keysToDelete) - .withQuiet(quiet)); + .withQuiet(true)); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java index bd09ca652a69d..230f07793d9ea 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerTool.java @@ -817,7 +817,7 @@ pages, suffix(pages), end); once("Remove S3 Keys", tracker.getBasePath().toString(), () -> - operations.removeKeys(page, true, false)); + operations.removeKeys(page, true)); summary.deleteRequests++; // and move to the start of the next page start = end; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java index 7d7627dfc03b4..a701f86f7b0c3 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperations.java @@ -23,7 +23,6 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.DeleteObjectsResult; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import org.apache.hadoop.fs.InvalidRequestException; @@ -58,10 +57,7 @@ RemoteIterator listObjects( * @param keysToDelete collection of keys to delete on the s3-backend. * if empty, no request is made of the object store. * @param deleteFakeDir indicates whether this is for deleting fake dirs. - * @param quiet should a bulk query be quiet, or should its result list * all deleted keys - * @return the deletion result if a multi object delete was invoked - * and it returned without a failure, else null. * @throws InvalidRequestException if the request was rejected due to * a mistaken attempt to delete the root directory. * @throws MultiObjectDeleteException one or more of the keys could not @@ -70,10 +66,9 @@ RemoteIterator listObjects( * @throws IOException other IO Exception. */ @Retries.RetryMixed - DeleteObjectsResult removeKeys( + void removeKeys( List keysToDelete, - boolean deleteFakeDir, - boolean quiet) + boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java index 7ccbc41bbea45..ccf80e1dde00e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/tools/MarkerToolOperationsImpl.java @@ -23,7 +23,6 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.DeleteObjectsResult; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import org.apache.hadoop.fs.Path; @@ -55,13 +54,12 @@ public RemoteIterator listObjects(final Path path, } @Override - public DeleteObjectsResult removeKeys( + public void removeKeys( final List keysToDelete, - final boolean deleteFakeDir, - final boolean quiet) + final boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { - return operationCallbacks.removeKeys(keysToDelete, deleteFakeDir, - quiet); + operationCallbacks.removeKeys(keysToDelete, deleteFakeDir + ); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java index c34ba22b10513..c0f6a4b23226b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFailureHandling.java @@ -20,10 +20,13 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest; +import org.assertj.core.api.Assertions; import org.junit.Assume; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.statistics.StoreStatisticNames; import org.apache.hadoop.fs.store.audit.AuditSpan; @@ -37,9 +40,11 @@ import java.nio.file.AccessDeniedException; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.failIf; -import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.*; import static org.apache.hadoop.test.LambdaTestUtils.*; +import static org.apache.hadoop.util.functional.RemoteIterators.mappingRemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; /** * ITest for failure handling, primarily multipart deletion. @@ -72,6 +77,37 @@ public void testMultiObjectDeleteNoFile() throws Throwable { removeKeys(getFileSystem(), "ITestS3AFailureHandling/missingFile"); } + /** + * See HADOOP-18112. + */ + @Test + public void testMultiObjectDeleteLargeNumKeys() throws Exception { + S3AFileSystem fs = getFileSystem(); + Path path = path("largeDir"); + mkdirs(path); + createFiles(fs, path, 1, 1005, 0); + RemoteIterator locatedFileStatusRemoteIterator = + fs.listFiles(path, false); + List keys = toList(mappingRemoteIterator(locatedFileStatusRemoteIterator, + locatedFileStatus -> fs.pathToKey(locatedFileStatus.getPath()))); + // After implementation of paging during multi object deletion, + // no exception is encountered. + Long bulkDeleteReqBefore = getNumberOfBulkDeleteRequestsMadeTillNow(fs); + try (AuditSpan span = span()) { + fs.removeKeys(buildDeleteRequest(keys.toArray(new String[0])), false); + } + Long bulkDeleteReqAfter = getNumberOfBulkDeleteRequestsMadeTillNow(fs); + // number of delete requests is 5 as we have default page size of 250. + Assertions.assertThat(bulkDeleteReqAfter - bulkDeleteReqBefore) + .describedAs("Number of batched bulk delete requests") + .isEqualTo(5); + } + + private Long getNumberOfBulkDeleteRequestsMadeTillNow(S3AFileSystem fs) { + return fs.getIOStatistics().counters() + .get(StoreStatisticNames.OBJECT_BULK_DELETE_REQUEST); + } + private void removeKeys(S3AFileSystem fileSystem, String... keys) throws IOException { try (AuditSpan span = span()) { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index 55ddba9bbd1e0..d965e6e57a28f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -52,7 +52,11 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.Service; import org.apache.hadoop.service.ServiceOperations; +import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; import org.apache.hadoop.util.BlockingThreadPoolExecutorService; +import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -70,6 +74,7 @@ import java.net.URISyntaxException; import java.text.DateFormat; import java.text.SimpleDateFormat; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.TreeSet; @@ -78,6 +83,10 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; +import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; +import static org.apache.hadoop.test.GenericTestUtils.buildPaths; import static org.apache.hadoop.util.Preconditions.checkNotNull; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH; import static org.apache.commons.lang3.StringUtils.isNotEmpty; @@ -95,8 +104,23 @@ @InterfaceAudience.Private @InterfaceStability.Unstable public final class S3ATestUtils { + private static final Logger LOG = LoggerFactory.getLogger( - S3ATestUtils.class); + S3ATestUtils.class); + + /** Many threads for scale performance: {@value}. */ + public static final int EXECUTOR_THREAD_COUNT = 64; + /** + * For submitting work. + */ + private static final ListeningExecutorService EXECUTOR = + MoreExecutors.listeningDecorator( + BlockingThreadPoolExecutorService.newInstance( + EXECUTOR_THREAD_COUNT, + EXECUTOR_THREAD_COUNT * 2, + 30, TimeUnit.SECONDS, + "test-operations")); + /** * Value to set a system property to (in maven) to declare that @@ -821,6 +845,87 @@ public static StoreContext createMockStoreContext( .build(); } + /** + * Write the text to a file asynchronously. Logs the operation duration. + * @param fs filesystem + * @param path path + * @return future to the patch created. + */ + private static CompletableFuture put(FileSystem fs, + Path path, String text) { + return submit(EXECUTOR, () -> { + try (DurationInfo ignore = + new DurationInfo(LOG, false, "Creating %s", path)) { + createFile(fs, path, true, text.getBytes(Charsets.UTF_8)); + return path; + } + }); + } + + /** + * Build a set of files in a directory tree. + * @param fs filesystem + * @param destDir destination + * @param depth file depth + * @param fileCount number of files to create. + * @param dirCount number of dirs to create at each level + * @return the list of files created. + */ + public static List createFiles(final FileSystem fs, + final Path destDir, + final int depth, + final int fileCount, + final int dirCount) throws IOException { + return createDirsAndFiles(fs, destDir, depth, fileCount, dirCount, + new ArrayList<>(fileCount), + new ArrayList<>(dirCount)); + } + + /** + * Build a set of files in a directory tree. + * @param fs filesystem + * @param destDir destination + * @param depth file depth + * @param fileCount number of files to create. + * @param dirCount number of dirs to create at each level + * @param paths [out] list of file paths created + * @param dirs [out] list of directory paths created. + * @return the list of files created. + */ + public static List createDirsAndFiles(final FileSystem fs, + final Path destDir, + final int depth, + final int fileCount, + final int dirCount, + final List paths, + final List dirs) throws IOException { + buildPaths(paths, dirs, destDir, depth, fileCount, dirCount); + List> futures = new ArrayList<>(paths.size() + + dirs.size()); + + // create directories. With dir marker retention, that adds more entries + // to cause deletion issues + try (DurationInfo ignore = + new DurationInfo(LOG, "Creating %d directories", dirs.size())) { + for (Path path : dirs) { + futures.add(submit(EXECUTOR, () ->{ + fs.mkdirs(path); + return path; + })); + } + waitForCompletion(futures); + } + + try (DurationInfo ignore = + new DurationInfo(LOG, "Creating %d files", paths.size())) { + for (Path path : paths) { + futures.add(put(fs, path, path.getName())); + } + waitForCompletion(futures); + return paths; + } + } + /** * Helper class to do diffs of metrics. */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java index 068b7b2dda5b7..378f4a70433d7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java @@ -26,14 +26,9 @@ import java.util.List; import java.util.Set; import java.util.TreeSet; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import com.amazonaws.services.s3.model.MultiObjectDeleteException; -import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; -import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; -import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; @@ -42,13 +37,11 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.util.BlockingThreadPoolExecutorService; import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; @@ -69,13 +62,10 @@ import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden; import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig; import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.DELEGATION_TOKEN_BINDING; -import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; -import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount; import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; -import static org.apache.hadoop.test.GenericTestUtils.buildPaths; import static org.apache.hadoop.test.LambdaTestUtils.eval; /** @@ -112,20 +102,6 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase { private static final Statement STATEMENT_ALL_BUCKET_READ_ACCESS = statement(true, S3_ALL_BUCKETS, S3_BUCKET_READ_OPERATIONS); - /** Many threads for scale performance: {@value}. */ - public static final int EXECUTOR_THREAD_COUNT = 64; - - /** - * For submitting work. - */ - private static final ListeningExecutorService EXECUTOR = - MoreExecutors.listeningDecorator( - BlockingThreadPoolExecutorService.newInstance( - EXECUTOR_THREAD_COUNT, - EXECUTOR_THREAD_COUNT * 2, - 30, TimeUnit.SECONDS, - "test-operations")); - /** * The number of files in a non-scaled test. @@ -742,87 +718,6 @@ private Set listFilesUnderPath(Path path, boolean recursive) return files; } - /** - * Write the text to a file asynchronously. Logs the operation duration. - * @param fs filesystem - * @param path path - * @return future to the patch created. - */ - private static CompletableFuture put(FileSystem fs, - Path path, String text) { - return submit(EXECUTOR, () -> { - try (DurationInfo ignore = - new DurationInfo(LOG, false, "Creating %s", path)) { - createFile(fs, path, true, text.getBytes(Charsets.UTF_8)); - return path; - } - }); - } - - /** - * Build a set of files in a directory tree. - * @param fs filesystem - * @param destDir destination - * @param depth file depth - * @param fileCount number of files to create. - * @param dirCount number of dirs to create at each level - * @return the list of files created. - */ - public static List createFiles(final FileSystem fs, - final Path destDir, - final int depth, - final int fileCount, - final int dirCount) throws IOException { - return createDirsAndFiles(fs, destDir, depth, fileCount, dirCount, - new ArrayList(fileCount), - new ArrayList(dirCount)); - } - - /** - * Build a set of files in a directory tree. - * @param fs filesystem - * @param destDir destination - * @param depth file depth - * @param fileCount number of files to create. - * @param dirCount number of dirs to create at each level - * @param paths [out] list of file paths created - * @param dirs [out] list of directory paths created. - * @return the list of files created. - */ - public static List createDirsAndFiles(final FileSystem fs, - final Path destDir, - final int depth, - final int fileCount, - final int dirCount, - final List paths, - final List dirs) throws IOException { - buildPaths(paths, dirs, destDir, depth, fileCount, dirCount); - List> futures = new ArrayList<>(paths.size() - + dirs.size()); - - // create directories. With dir marker retention, that adds more entries - // to cause deletion issues - try (DurationInfo ignore = - new DurationInfo(LOG, "Creating %d directories", dirs.size())) { - for (Path path : dirs) { - futures.add(submit(EXECUTOR, () ->{ - fs.mkdirs(path); - return path; - })); - } - waitForCompletion(futures); - } - - try (DurationInfo ignore = - new DurationInfo(LOG, "Creating %d files", paths.size())) { - for (Path path : paths) { - futures.add(put(fs, path, path.getName())); - } - waitForCompletion(futures); - return paths; - } - } - /** * Verifies that s3:DeleteObjectVersion is not required for rename. *

diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java index dbd89b960f96d..9bc3aef83aacb 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java @@ -164,7 +164,7 @@ private void createFactoryObjects(RequestFactory factory) { new ArrayList<>())); a(factory.newCopyObjectRequest(path, path2, md)); a(factory.newDeleteObjectRequest(path)); - a(factory.newBulkDeleteRequest(new ArrayList<>(), true)); + a(factory.newBulkDeleteRequest(new ArrayList<>())); a(factory.newDirectoryMarkerRequest(path)); a(factory.newGetObjectRequest(path)); a(factory.newGetObjectMetadataRequest(path)); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java index d5862bcb33516..dbdd8b5da6a3c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADeleteManyFiles.java @@ -38,7 +38,7 @@ import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; import static org.apache.hadoop.fs.s3a.Constants.USER_AGENT_PREFIX; import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR; -import static org.apache.hadoop.fs.s3a.impl.ITestPartialRenamesDeletes.createFiles; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.createFiles; import static org.apache.hadoop.test.GenericTestUtils.filenameOfIndex; /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java index a2aebc82720d1..fa1ad2db62af7 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalOperationCallbacks.java @@ -23,7 +23,6 @@ import com.amazonaws.AmazonClientException; import com.amazonaws.services.s3.model.DeleteObjectsRequest; -import com.amazonaws.services.s3.model.DeleteObjectsResult; import com.amazonaws.services.s3.model.MultiObjectDeleteException; import com.amazonaws.services.s3.transfer.model.CopyResult; @@ -99,13 +98,11 @@ public CopyResult copyFile( } @Override - public DeleteObjectsResult removeKeys( - List keysToDelete, - boolean deleteFakeDir, - boolean quiet) + public void removeKeys( + List keysToDelete, + boolean deleteFakeDir) throws MultiObjectDeleteException, AmazonClientException, IOException { - return null; } @Override From a32cfc21693244d70ebf07ff3c81c5d6b4ab04a5 Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Sat, 12 Mar 2022 18:40:09 +0800 Subject: [PATCH 034/145] HDFS-15382. Split one FsDatasetImpl lock to block pool grain locks. (#3941). Contributed by limingxiang. Signed-off-by: He Xiaoqiao Signed-off-by: litao --- .../hdfs/server/datanode/BPServiceActor.java | 5 + .../hdfs/server/datanode/BlockSender.java | 4 +- .../hadoop/hdfs/server/datanode/DataNode.java | 12 +- .../hdfs/server/datanode/DiskBalancer.java | 18 +- .../datanode/fsdataset/FsDatasetSpi.java | 20 +- .../fsdataset/impl/BlockPoolSlice.java | 3 +- .../fsdataset/impl/FsDatasetImpl.java | 402 +++++++++--------- .../datanode/fsdataset/impl/FsVolumeList.java | 22 + .../fsdataset/impl/ProvidedVolumeImpl.java | 5 +- .../datanode/fsdataset/impl/ReplicaMap.java | 124 +++--- .../server/datanode/SimulatedFSDataset.java | 20 +- .../server/datanode/TestBPOfferService.java | 3 + .../server/datanode/TestBlockRecovery2.java | 7 +- .../server/datanode/TestDirectoryScanner.java | 13 +- .../extdataset/ExternalDatasetImpl.java | 10 +- .../impl/FsDatasetImplTestUtils.java | 2 +- .../fsdataset/impl/TestFsDatasetImpl.java | 121 +----- .../fsdataset/impl/TestFsVolumeList.java | 3 +- .../impl/TestInterDatanodeProtocol.java | 17 +- .../fsdataset/impl/TestProvidedImpl.java | 10 +- .../fsdataset/impl/TestReplicaMap.java | 7 +- .../fsdataset/impl/TestWriteToReplica.java | 3 +- 22 files changed, 384 insertions(+), 447 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 6199626fd2ef6..838259d7f6a0b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeLifelineProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.server.common.IncorrectVersionException; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; @@ -309,6 +310,10 @@ private void connectToNNAndHandshake() throws IOException { // info. NamespaceInfo nsInfo = retrieveNamespaceInfo(); + // init block pool lock when init. + dn.getDataSetLockManager().addLock(LockLevel.BLOCK_POOl, + nsInfo.getBlockPoolID()); + // Verify that this matches the other NN in this HA pair. // This also initializes our block pool in the DN if we are // the first NN connection for this BP. 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 172a245b3525a..5c4212fea537f 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 @@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference; import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream; import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams; @@ -256,7 +257,8 @@ class BlockSender implements java.io.Closeable { // the append write. ChunkChecksum chunkChecksum = null; final long replicaVisibleLength; - try(AutoCloseableLock lock = datanode.data.acquireDatasetReadLock()) { + try (AutoCloseableLock lock = datanode.getDataSetLockManager().readLock( + LockLevel.BLOCK_POOl, block.getBlockPoolId())) { replica = getReplica(block, datanode); replicaVisibleLength = replica.getVisibleLength(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 84382275bd965..15e8a9e359799 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -146,6 +146,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.HDFSPolicyProvider; import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker; import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker; import org.apache.hadoop.hdfs.util.DataTransferThrottler; @@ -433,6 +434,7 @@ public static InetSocketAddress createSocketAddr(String target) { .availableProcessors(); private static final double CONGESTION_RATIO = 1.5; private DiskBalancer diskBalancer; + private DataSetLockManager dataSetLockManager; private final ExecutorService xferService; @@ -474,6 +476,7 @@ private static Tracer createTracer(Configuration conf) { this.pipelineSupportSlownode = false; this.socketFactory = NetUtils.getDefaultSocketFactory(conf); this.dnConf = new DNConf(this); + this.dataSetLockManager = new DataSetLockManager(conf); initOOBTimeout(); storageLocationChecker = null; volumeChecker = new DatasetVolumeChecker(conf, new Timer()); @@ -492,6 +495,7 @@ private static Tracer createTracer(Configuration conf) { super(conf); this.tracer = createTracer(conf); this.fileIoProvider = new FileIoProvider(conf, this); + this.dataSetLockManager = new DataSetLockManager(conf); this.blockScanner = new BlockScanner(this); this.lastDiskErrorCheck = 0; this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY, @@ -2461,6 +2465,7 @@ public void shutdown() { notifyAll(); } tracer.close(); + dataSetLockManager.lockLeakCheck(); } /** @@ -3367,7 +3372,8 @@ void transferReplicaForPipelineRecovery(final ExtendedBlock b, final BlockConstructionStage stage; //get replica information - try(AutoCloseableLock lock = data.acquireDatasetReadLock()) { + try (AutoCloseableLock lock = dataSetLockManager.readLock( + LockLevel.BLOCK_POOl, b.getBlockPoolId())) { Block storedBlock = data.getStoredBlock(b.getBlockPoolId(), b.getBlockId()); if (null == storedBlock) { @@ -4084,6 +4090,10 @@ private static boolean isWrite(BlockConstructionStage stage) { || stage == PIPELINE_SETUP_APPEND_RECOVERY); } + public DataSetLockManager getDataSetLockManager() { + return dataSetLockManager; + } + boolean isSlownodeByNameserviceId(String nsId) { return blockPoolManager.isSlownodeByNameserviceId(nsId); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java index 0f710a143ad87..4126140678759 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java @@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi .FsVolumeReferences; -import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.server.datanode.DiskBalancerWorkStatus @@ -502,16 +501,13 @@ private void createWorkPlan(NodePlan plan) throws DiskBalancerException { private Map getStorageIDToVolumeBasePathMap() throws DiskBalancerException { Map storageIDToVolBasePathMap = new HashMap<>(); - FsDatasetSpi.FsVolumeReferences references; - try { - try(AutoCloseableLock lock = this.dataset.acquireDatasetReadLock()) { - references = this.dataset.getFsVolumeReferences(); - for (int ndx = 0; ndx < references.size(); ndx++) { - FsVolumeSpi vol = references.get(ndx); - storageIDToVolBasePathMap.put(vol.getStorageID(), - vol.getBaseURI().getPath()); - } - references.close(); + // Get volumes snapshot so no need to acquire dataset lock. + try (FsDatasetSpi.FsVolumeReferences references = dataset. + getFsVolumeReferences()) { + for (int ndx = 0; ndx < references.size(); ndx++) { + FsVolumeSpi vol = references.get(ndx); + storageIDToVolBasePathMap.put(vol.getStorageID(), + vol.getBaseURI().getPath()); } } catch (IOException ex) { LOG.error("Disk Balancer - Internal Error.", ex); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java index f162ea1b3ae15..e39ef817b6f29 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java @@ -35,8 +35,9 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MountVolumeMap; -import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; @@ -241,7 +242,7 @@ StorageReport[] getStorageReports(String bpid) * Gets a list of references to the finalized blocks for the given block pool. *

* Callers of this function should call - * {@link FsDatasetSpi#acquireDatasetLock} to avoid blocks' status being + * {@link FsDatasetSpi#acquireDatasetLockManager} to avoid blocks' status being * changed during list iteration. *

* @return a list of references to the finalized blocks for the given block @@ -657,21 +658,12 @@ ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block, ReplicaInfo moveBlockAcrossVolumes(final ExtendedBlock block, FsVolumeSpi destination) throws IOException; - /** - * Acquire the lock of the data set. This prevents other threads from - * modifying the volume map structure inside the datanode, but other changes - * are still possible. For example modifying the genStamp of a block instance. - */ - AutoCloseableLock acquireDatasetLock(); - /*** - * Acquire the read lock of the data set. This prevents other threads from - * modifying the volume map structure inside the datanode, but other changes - * are still possible. For example modifying the genStamp of a block instance. + * Acquire lock Manager for the data set. This prevents other threads from + * modifying the volume map structure inside the datanode. * @return The AutoClosable read lock instance. */ - AutoCloseableLock acquireDatasetReadLock(); - + DataNodeLockManager acquireDatasetLockManager(); /** * Deep copy the replica info belonging to given block pool. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java index 0ef1d56de34d6..eff079a353da6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java @@ -43,7 +43,6 @@ import java.util.concurrent.ForkJoinTask; import java.util.concurrent.RecursiveAction; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hdfs.server.datanode.FSCachingGetSpaceUsed; @@ -914,7 +913,7 @@ void shutdown(BlockListAsLongs blocksListToPersist) { private boolean readReplicasFromCache(ReplicaMap volumeMap, final RamDiskReplicaTracker lazyWriteReplicaMap) { - ReplicaMap tmpReplicaMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap tmpReplicaMap = new ReplicaMap(); File replicaFile = new File(replicaCacheDir, REPLICA_CACHE_FILE); // Check whether the file exists or not. if (!replicaFile.exists()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index 441a6f96e4c23..002d99abc5ba7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -32,7 +32,6 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -41,9 +40,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; -import java.util.concurrent.locks.Condition; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.ReentrantReadWriteLock; import javax.management.NotCompliantMBeanException; import javax.management.ObjectName; @@ -64,6 +60,10 @@ import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.ExtendedBlockId; +import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; +import org.apache.hadoop.hdfs.server.datanode.DataSetLockManager; import org.apache.hadoop.hdfs.server.datanode.FileIoProvider; import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica; import org.apache.hadoop.hdfs.server.datanode.LocalReplica; @@ -119,7 +119,6 @@ import org.apache.hadoop.util.DataChecksum; import org.apache.hadoop.util.DiskChecker.DiskErrorException; import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException; -import org.apache.hadoop.util.InstrumentedReadWriteLock; import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.Sets; @@ -188,7 +187,8 @@ public StorageReport[] getStorageReports(String bpid) @Override public FsVolumeImpl getVolume(final ExtendedBlock b) { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { final ReplicaInfo r = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock()); return r != null ? (FsVolumeImpl) r.getVolume() : null; @@ -198,7 +198,8 @@ public FsVolumeImpl getVolume(final ExtendedBlock b) { @Override // FsDatasetSpi public Block getStoredBlock(String bpid, long blkid) throws IOException { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + bpid)) { ReplicaInfo r = volumeMap.get(bpid, blkid); if (r == null) { return null; @@ -210,12 +211,16 @@ public Block getStoredBlock(String bpid, long blkid) @Override public Set deepCopyReplica(String bpid) throws IOException { - Set replicas = null; - try (AutoCloseableLock lock = datasetReadLock.acquire()) { - replicas = new HashSet<>(volumeMap.replicas(bpid) == null ? Collections. - EMPTY_SET : volumeMap.replicas(bpid)); + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { + Set replicas = new HashSet<>(); + volumeMap.replicas(bpid, (iterator) -> { + while (iterator.hasNext()) { + ReplicaInfo b = iterator.next(); + replicas.add(b); + } + }); + return replicas; } - return Collections.unmodifiableSet(replicas); } /** @@ -275,13 +280,7 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b) private boolean blockPinningEnabled; private final int maxDataLength; - @VisibleForTesting - final AutoCloseableLock datasetWriteLock; - @VisibleForTesting - final AutoCloseableLock datasetReadLock; - @VisibleForTesting - final InstrumentedReadWriteLock datasetRWLock; - private final Condition datasetWriteLockCondition; + private final DataSetLockManager lockManager; private static String blockPoolId = ""; // Make limited notify times from DirectoryScanner to NameNode. @@ -300,33 +299,7 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b) this.dataStorage = storage; this.conf = conf; this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf); - this.datasetRWLock = new InstrumentedReadWriteLock( - conf.getBoolean(DFSConfigKeys.DFS_DATANODE_LOCK_FAIR_KEY, - DFSConfigKeys.DFS_DATANODE_LOCK_FAIR_DEFAULT), - "FsDatasetRWLock", LOG, conf.getTimeDuration( - DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY, - DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_DEFAULT, - TimeUnit.MILLISECONDS), - conf.getTimeDuration( - DFSConfigKeys.DFS_DATANODE_LOCK_REPORTING_THRESHOLD_MS_KEY, - DFSConfigKeys.DFS_DATANODE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT, - TimeUnit.MILLISECONDS)); - this.datasetWriteLock = new AutoCloseableLock(datasetRWLock.writeLock()); - boolean enableRL = conf.getBoolean( - DFSConfigKeys.DFS_DATANODE_LOCK_READ_WRITE_ENABLED_KEY, - DFSConfigKeys.DFS_DATANODE_LOCK_READ_WRITE_ENABLED_DEFAULT); - // The read lock can be disabled by the above config key. If it is disabled - // then we simply make the both the read and write lock variables hold - // the write lock. All accesses to the lock are via these variables, so that - // effectively disables the read lock. - if (enableRL) { - LOG.info("The datanode lock is a read write lock"); - this.datasetReadLock = new AutoCloseableLock(datasetRWLock.readLock()); - } else { - LOG.info("The datanode lock is an exclusive write lock"); - this.datasetReadLock = this.datasetWriteLock; - } - this.datasetWriteLockCondition = datasetWriteLock.newCondition(); + this.lockManager = datanode.getDataSetLockManager(); // The number of volumes required for operation is the total number // of volumes minus the number of failed volumes we can tolerate. @@ -365,7 +338,7 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b) } storageMap = new ConcurrentHashMap(); - volumeMap = new ReplicaMap(datasetReadLock, datasetWriteLock); + volumeMap = new ReplicaMap(lockManager); ramDiskReplicaTracker = RamDiskReplicaTracker.getInstance(conf, this); @SuppressWarnings("unchecked") @@ -421,16 +394,6 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b) lastDirScannerNotifyTime = System.currentTimeMillis(); } - @Override - public AutoCloseableLock acquireDatasetLock() { - return datasetWriteLock.acquire(); - } - - @Override - public AutoCloseableLock acquireDatasetReadLock() { - return datasetReadLock.acquire(); - } - /** * Gets initial volume failure information for all volumes that failed * immediately at startup. The method works by determining the set difference @@ -465,42 +428,40 @@ private static List getInitialVolumeFailureInfos( * Activate a volume to serve requests. * @throws IOException if the storage UUID already exists. */ - private void activateVolume( + private synchronized void activateVolume( ReplicaMap replicaMap, Storage.StorageDirectory sd, StorageType storageType, FsVolumeReference ref) throws IOException { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { - DatanodeStorage dnStorage = storageMap.get(sd.getStorageUuid()); - if (dnStorage != null) { - final String errorMsg = String.format( - "Found duplicated storage UUID: %s in %s.", - sd.getStorageUuid(), sd.getVersionFile()); - LOG.error(errorMsg); - throw new IOException(errorMsg); - } - // Check if there is same storage type on the mount. - // Only useful when same disk tiering is turned on. - FsVolumeImpl volumeImpl = (FsVolumeImpl) ref.getVolume(); - FsVolumeReference checkRef = volumes - .getMountVolumeMap() - .getVolumeRefByMountAndStorageType( - volumeImpl.getMount(), volumeImpl.getStorageType()); - if (checkRef != null) { - final String errorMsg = String.format( - "Storage type %s already exists on same mount: %s.", - volumeImpl.getStorageType(), volumeImpl.getMount()); - checkRef.close(); - LOG.error(errorMsg); - throw new IOException(errorMsg); - } - volumeMap.mergeAll(replicaMap); - storageMap.put(sd.getStorageUuid(), - new DatanodeStorage(sd.getStorageUuid(), - DatanodeStorage.State.NORMAL, - storageType)); - asyncDiskService.addVolume(volumeImpl); - volumes.addVolume(ref); - } + DatanodeStorage dnStorage = storageMap.get(sd.getStorageUuid()); + if (dnStorage != null) { + final String errorMsg = String.format( + "Found duplicated storage UUID: %s in %s.", + sd.getStorageUuid(), sd.getVersionFile()); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + // Check if there is same storage type on the mount. + // Only useful when same disk tiering is turned on. + FsVolumeImpl volumeImpl = (FsVolumeImpl) ref.getVolume(); + FsVolumeReference checkRef = volumes + .getMountVolumeMap() + .getVolumeRefByMountAndStorageType( + volumeImpl.getMount(), volumeImpl.getStorageType()); + if (checkRef != null) { + final String errorMsg = String.format( + "Storage type %s already exists on same mount: %s.", + volumeImpl.getStorageType(), volumeImpl.getMount()); + checkRef.close(); + LOG.error(errorMsg); + throw new IOException(errorMsg); + } + volumeMap.mergeAll(replicaMap); + storageMap.put(sd.getStorageUuid(), + new DatanodeStorage(sd.getStorageUuid(), + DatanodeStorage.State.NORMAL, + storageType)); + asyncDiskService.addVolume(volumeImpl); + volumes.addVolume(ref); } private void addVolume(Storage.StorageDirectory sd) throws IOException { @@ -517,8 +478,8 @@ private void addVolume(Storage.StorageDirectory sd) throws IOException { .setConf(this.conf) .build(); FsVolumeReference ref = fsVolume.obtainReference(); - ReplicaMap tempVolumeMap = - new ReplicaMap(datasetReadLock, datasetWriteLock); + // no need to acquire lock. + ReplicaMap tempVolumeMap = new ReplicaMap(); fsVolume.getVolumeMap(tempVolumeMap, ramDiskReplicaTracker); activateVolume(tempVolumeMap, sd, storageLocation.getStorageType(), ref); @@ -557,13 +518,13 @@ public void addVolume(final StorageLocation location, StorageType storageType = location.getStorageType(); final FsVolumeImpl fsVolume = createFsVolume(sd.getStorageUuid(), sd, location); - final ReplicaMap tempVolumeMap = - new ReplicaMap(new ReentrantReadWriteLock()); + // no need to add lock + final ReplicaMap tempVolumeMap = new ReplicaMap(); ArrayList exceptions = Lists.newArrayList(); for (final NamespaceInfo nsInfo : nsInfos) { String bpid = nsInfo.getBlockPoolID(); - try { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { fsVolume.addBlockPool(bpid, this.conf, this.timer); fsVolume.getVolumeMap(bpid, tempVolumeMap, ramDiskReplicaTracker); } catch (IOException e) { @@ -603,7 +564,9 @@ public void removeVolumes( new ArrayList<>(storageLocsToRemove); Map> blkToInvalidate = new HashMap<>(); List storageToRemove = new ArrayList<>(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + // This object lock is protect data structure related volumes like add and + // remove.This will obtain volumeMap lock again if access replicaInfo. + synchronized (this) { for (int idx = 0; idx < dataStorage.getNumStorageDirs(); idx++) { Storage.StorageDirectory sd = dataStorage.getStorageDir(idx); final StorageLocation sdLocation = sd.getStorageLocation(); @@ -615,7 +578,7 @@ public void removeVolumes( // Disable the volume from the service. asyncDiskService.removeVolume(sd.getStorageUuid()); volumes.removeVolume(sdLocation, clearFailure); - volumes.waitVolumeRemoved(5000, datasetWriteLockCondition); + volumes.waitVolumeRemoved(5000, this); // Removed all replica information for the blocks on the volume. // Unlike updating the volumeMap in addVolume(), this operation does @@ -623,18 +586,19 @@ public void removeVolumes( for (String bpid : volumeMap.getBlockPoolList()) { List blocks = blkToInvalidate .computeIfAbsent(bpid, (k) -> new ArrayList<>()); - for (Iterator it = - volumeMap.replicas(bpid).iterator(); it.hasNext();) { - ReplicaInfo block = it.next(); - final StorageLocation blockStorageLocation = - block.getVolume().getStorageLocation(); - LOG.trace("checking for block " + block.getBlockId() + - " with storageLocation " + blockStorageLocation); - if (blockStorageLocation.equals(sdLocation)) { - blocks.add(block); - it.remove(); + volumeMap.replicas(bpid, (iterator) -> { + while (iterator.hasNext()) { + ReplicaInfo block = iterator.next(); + final StorageLocation blockStorageLocation = + block.getVolume().getStorageLocation(); + LOG.trace("checking for block " + block.getBlockId() + + " with storageLocation " + blockStorageLocation); + if (blockStorageLocation.equals(sdLocation)) { + blocks.add(block); + iterator.remove(); + } } - } + }); } storageToRemove.add(sd.getStorageUuid()); storageLocationsToRemove.remove(sdLocation); @@ -662,8 +626,8 @@ public void removeVolumes( } } - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { - for(String storageUuid : storageToRemove) { + synchronized (this) { + for (String storageUuid : storageToRemove) { storageMap.remove(storageUuid); } } @@ -853,7 +817,8 @@ public InputStream getBlockInputStream(ExtendedBlock b, long seekOffset) throws IOException { ReplicaInfo info; - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { info = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock()); } @@ -941,7 +906,8 @@ ReplicaInfo getReplicaInfo(String bpid, long blkid) @Override // FsDatasetSpi public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkOffset, long metaOffset) throws IOException { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { ReplicaInfo info = getReplicaInfo(b); FsVolumeReference ref = info.getVolume().obtainReference(); try { @@ -1117,7 +1083,8 @@ public ReplicaInfo moveBlockAcrossStorage(ExtendedBlock block, targetStorageType, targetStorageId); boolean useVolumeOnSameMount = false; - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + block.getBlockPoolId())) { if (shouldConsiderSameMountVolume) { volumeRef = volumes.getVolumeByMount(targetStorageType, ((FsVolumeImpl) replicaInfo.getVolume()).getMount(), @@ -1311,7 +1278,8 @@ public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block, FsVolumeSpi FsVolumeReference volumeRef = null; - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + block.getBlockPoolId())) { volumeRef = destination.obtainReference(); } @@ -1325,6 +1293,11 @@ public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block, FsVolumeSpi return replicaInfo; } + @Override + public DataNodeLockManager acquireDatasetLockManager() { + return lockManager; + } + /** * Compute and store the checksum for a block file that does not already have * its checksum computed. @@ -1399,7 +1372,8 @@ static void computeChecksum(ReplicaInfo srcReplica, File dstMeta, @Override // FsDatasetSpi public ReplicaHandler append(ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { // If the block was successfully finalized because all packets // were successfully processed at the Datanode but the ack for // some of the packets were not received by the client. The client @@ -1451,7 +1425,7 @@ public ReplicaHandler append(ExtendedBlock b, private ReplicaInPipeline append(String bpid, ReplicaInfo replicaInfo, long newGS, long estimateBlockLen) throws IOException { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { // If the block is cached, start uncaching it. if (replicaInfo.getState() != ReplicaState.FINALIZED) { throw new IOException("Only a Finalized replica can be appended to; " @@ -1547,7 +1521,8 @@ public ReplicaHandler recoverAppend( while (true) { try { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen); FsVolumeReference ref = replicaInfo.getVolume().obtainReference(); ReplicaInPipeline replica; @@ -1579,7 +1554,8 @@ public Replica recoverClose(ExtendedBlock b, long newGS, LOG.info("Recover failed close " + b); while (true) { try { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { // check replica's state ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen); // bump the replica's GS @@ -1602,7 +1578,8 @@ public ReplicaHandler createRbw( StorageType storageType, String storageId, ExtendedBlock b, boolean allowLazyPersist) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getBlockId()); if (replicaInfo != null) { @@ -1680,7 +1657,8 @@ public ReplicaHandler recoverRbw( try { while (true) { try { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId()); // check the replica's state @@ -1711,7 +1689,8 @@ public ReplicaHandler recoverRbw( private ReplicaHandler recoverRbwImpl(ReplicaInPipeline rbw, ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { // check generation stamp long replicaGenerationStamp = rbw.getGenerationStamp(); if (replicaGenerationStamp < b.getGenerationStamp() || @@ -1772,7 +1751,8 @@ private ReplicaHandler recoverRbwImpl(ReplicaInPipeline rbw, public ReplicaInPipeline convertTemporaryToRbw( final ExtendedBlock b) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { final long blockId = b.getBlockId(); final long expectedGs = b.getGenerationStamp(); final long visible = b.getNumBytes(); @@ -1851,7 +1831,8 @@ public ReplicaHandler createTemporary(StorageType storageType, ReplicaInfo lastFoundReplicaInfo = null; boolean isInPipeline = false; do { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { ReplicaInfo currentReplicaInfo = volumeMap.get(b.getBlockPoolId(), b.getBlockId()); if (currentReplicaInfo == lastFoundReplicaInfo) { @@ -1906,7 +1887,8 @@ public ReplicaHandler createTemporary(StorageType storageType, false); } long startHoldLockTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { FsVolumeReference ref = volumes.getNextVolume(storageType, storageId, b .getNumBytes()); FsVolumeImpl v = (FsVolumeImpl) ref.getVolume(); @@ -1967,7 +1949,8 @@ public void finalizeBlock(ExtendedBlock b, boolean fsyncDir) ReplicaInfo replicaInfo = null; ReplicaInfo finalizedReplicaInfo = null; long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { if (Thread.interrupted()) { // Don't allow data modifications from interrupted threads throw new IOException("Cannot finalize block from Interrupted Thread"); @@ -2003,7 +1986,7 @@ public void finalizeBlock(ExtendedBlock b, boolean fsyncDir) private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo) throws IOException { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { // Compare generation stamp of old and new replica before finalizing if (volumeMap.get(bpid, replicaInfo.getBlockId()).getGenerationStamp() > replicaInfo.getGenerationStamp()) { @@ -2049,7 +2032,8 @@ private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo) @Override // FsDatasetSpi public void unfinalizeBlock(ExtendedBlock b) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + b.getBlockPoolId())) { ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock()); if (replicaInfo != null && @@ -2107,47 +2091,50 @@ public Map getBlockReports(String bpid) { new HashMap(); List curVolumes = null; - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { curVolumes = volumes.getVolumes(); for (FsVolumeSpi v : curVolumes) { builders.put(v.getStorageID(), BlockListAsLongs.builder(maxDataLength)); } Set missingVolumesReported = new HashSet<>(); - for (ReplicaInfo b : volumeMap.replicas(bpid)) { - // skip PROVIDED replicas. - if (b.getVolume().getStorageType() == StorageType.PROVIDED) { - continue; - } - String volStorageID = b.getVolume().getStorageID(); - switch(b.getState()) { - case FINALIZED: - case RBW: - case RWR: - break; - case RUR: - // use the original replica. - b = b.getOriginalReplica(); - break; - case TEMPORARY: - continue; - default: - assert false : "Illegal ReplicaInfo state."; - continue; - } - BlockListAsLongs.Builder storageBuilder = builders.get(volStorageID); - // a storage in the process of failing will not be in the volumes list - // but will be in the replica map. - if (storageBuilder != null) { - storageBuilder.add(b); - } else { - if (!missingVolumesReported.contains(volStorageID)) { - LOG.warn("Storage volume: " + volStorageID + " missing for the" - + " replica block: " + b + ". Probably being removed!"); - missingVolumesReported.add(volStorageID); + volumeMap.replicas(bpid, (iterator) -> { + while (iterator.hasNext()) { + ReplicaInfo b = iterator.next(); + // skip PROVIDED replicas. + if (b.getVolume().getStorageType() == StorageType.PROVIDED) { + continue; + } + String volStorageID = b.getVolume().getStorageID(); + switch(b.getState()) { + case FINALIZED: + case RBW: + case RWR: + break; + case RUR: + // use the original replica. + b = b.getOriginalReplica(); + break; + case TEMPORARY: + continue; + default: + assert false : "Illegal ReplicaInfo state."; + continue; + } + BlockListAsLongs.Builder storageBuilder = builders.get(volStorageID); + // a storage in the process of failing will not be in the volumes list + // but will be in the replica map. + if (storageBuilder != null) { + storageBuilder.add(b); + } else { + if (!missingVolumesReported.contains(volStorageID)) { + LOG.warn("Storage volume: " + volStorageID + " missing for the" + + " replica block: " + b + ". Probably being removed!"); + missingVolumesReported.add(volStorageID); + } } } - } + }); } for (FsVolumeImpl v : curVolumes) { @@ -2162,7 +2149,7 @@ public Map getBlockReports(String bpid) { * Gets a list of references to the finalized blocks for the given block pool. *

* Callers of this function should call - * {@link FsDatasetSpi#acquireDatasetLock()} to avoid blocks' status being + * {@link FsDatasetSpi#acquireDatasetLockManager()} ()} to avoid blocks' status being * changed during list iteration. *

* @return a list of references to the finalized blocks for the given block @@ -2170,14 +2157,17 @@ public Map getBlockReports(String bpid) { */ @Override public List getFinalizedBlocks(String bpid) { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { - final List finalized = new ArrayList( - volumeMap.size(bpid)); - for (ReplicaInfo b : volumeMap.replicas(bpid)) { - if (b.getState() == ReplicaState.FINALIZED) { - finalized.add(b); + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { + ArrayList finalized = + new ArrayList<>(volumeMap.size(bpid)); + volumeMap.replicas(bpid, (iterator) -> { + while (iterator.hasNext()) { + ReplicaInfo b = iterator.next(); + if (b.getState() == ReplicaState.FINALIZED) { + finalized.add(new FinalizedReplica((FinalizedReplica)b)); + } } - } + }); return finalized; } } @@ -2310,7 +2300,7 @@ private void invalidate(String bpid, Block[] invalidBlks, boolean async) for (int i = 0; i < invalidBlks.length; i++) { final ReplicaInfo removing; final FsVolumeImpl v; - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { final ReplicaInfo info = volumeMap.get(bpid, invalidBlks[i]); if (info == null) { ReplicaInfo infoByBlockId = @@ -2433,7 +2423,7 @@ private void cacheBlock(String bpid, long blockId) { long length, genstamp; Executor volumeExecutor; - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { ReplicaInfo info = volumeMap.get(bpid, blockId); boolean success = false; try { @@ -2501,7 +2491,8 @@ public boolean isCached(String bpid, long blockId) { @Override // FsDatasetSpi public boolean contains(final ExtendedBlock block) { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + block.getBlockPoolId())) { final long blockId = block.getLocalBlock().getBlockId(); final String bpid = block.getBlockPoolId(); final ReplicaInfo r = volumeMap.get(bpid, blockId); @@ -2628,7 +2619,7 @@ public void checkAndUpdate(String bpid, ScanInfo scanInfo) curDirScannerNotifyCount = 0; lastDirScannerNotifyTime = startTimeMs; } - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { memBlockInfo = volumeMap.get(bpid, blockId); if (memBlockInfo != null && memBlockInfo.getState() != ReplicaState.FINALIZED) { @@ -2851,7 +2842,7 @@ public ReplicaInfo getReplica(String bpid, long blockId) { @Override public String getReplicaString(String bpid, long blockId) { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { final Replica r = volumeMap.get(bpid, blockId); return r == null ? "null" : r.toString(); } @@ -2865,12 +2856,26 @@ public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock) datanode.getDnConf().getXceiverStopTimeout()); } + ReplicaRecoveryInfo initReplicaRecovery(String bpid, ReplicaMap map, + Block block, long recoveryId, long xceiverStopTimeout) throws IOException { + while (true) { + try { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + return initReplicaRecoveryImpl(bpid, map, block, recoveryId); + } + } catch (MustStopExistingWriter e) { + e.getReplicaInPipeline().stopWriter(xceiverStopTimeout); + } + } + } + /** static version of {@link #initReplicaRecovery(RecoveringBlock)}. */ static ReplicaRecoveryInfo initReplicaRecovery(String bpid, ReplicaMap map, - Block block, long recoveryId, long xceiverStopTimeout) throws IOException { + Block block, long recoveryId, long xceiverStopTimeout, DataSetLockManager + lockManager) throws IOException { while (true) { try { - try (AutoCloseableLock lock = map.getLock().acquire()) { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { return initReplicaRecoveryImpl(bpid, map, block, recoveryId); } } catch (MustStopExistingWriter e) { @@ -2959,7 +2964,8 @@ public Replica updateReplicaUnderRecovery( final long newBlockId, final long newlength) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + oldBlock.getBlockPoolId())) { //get replica final String bpid = oldBlock.getBlockPoolId(); final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId()); @@ -3078,7 +3084,8 @@ private ReplicaInfo updateReplicaUnderRecovery( @Override // FsDatasetSpi public long getReplicaVisibleLength(final ExtendedBlock block) throws IOException { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + block.getBlockPoolId())) { final Replica replica = getReplicaInfo(block.getBlockPoolId(), block.getBlockId()); if (replica.getGenerationStamp() < block.getGenerationStamp()) { @@ -3095,7 +3102,7 @@ public void addBlockPool(String bpid, Configuration conf) throws IOException { LOG.info("Adding block pool " + bpid); AddBlockPoolException volumeExceptions = new AddBlockPoolException(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { try { volumes.addBlockPool(bpid, conf); } catch (AddBlockPoolException e) { @@ -3125,7 +3132,7 @@ public static void setBlockPoolId(String bpid) { @Override public void shutdownBlockPool(String bpid) { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { LOG.info("Removing block pool " + bpid); Map blocksPerVolume = getBlockReports(bpid); @@ -3199,7 +3206,7 @@ public Map getVolumeInfoMap() { @Override //FsDatasetSpi public void deleteBlockPool(String bpid, boolean force) throws IOException { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { List curVolumes = volumes.getVolumes(); if (!force) { for (FsVolumeImpl volume : curVolumes) { @@ -3228,7 +3235,8 @@ public void deleteBlockPool(String bpid, boolean force) @Override // FsDatasetSpi public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block) throws IOException { - try (AutoCloseableLock lock = datasetReadLock.acquire()) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, + block.getBlockPoolId())) { final Replica replica = volumeMap.get(block.getBlockPoolId(), block.getBlockId()); if (replica == null) { @@ -3282,7 +3290,7 @@ public void clearRollingUpgradeMarker(String bpid) throws IOException { @Override public void onCompleteLazyPersist(String bpId, long blockId, long creationTime, File[] savedFiles, FsVolumeImpl targetVolume) { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpId)) { ramDiskReplicaTracker.recordEndLazyPersist(bpId, blockId, savedFiles); targetVolume.incDfsUsedAndNumBlocks(bpId, savedFiles[0].length() @@ -3416,7 +3424,8 @@ private boolean saveNextReplica() { try { block = ramDiskReplicaTracker.dequeueNextReplicaToPersist(); if (block != null) { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + block.getBlockPoolId())) { replicaInfo = volumeMap.get(block.getBlockPoolId(), block.getBlockId()); // If replicaInfo is null, the block was either deleted before @@ -3483,7 +3492,7 @@ public void evictBlocks(long bytesNeeded) throws IOException { ReplicaInfo replicaInfo, newReplicaInfo; final String bpid = replicaState.getBlockPoolId(); - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { replicaInfo = getReplicaInfo(replicaState.getBlockPoolId(), replicaState.getBlockId()); Preconditions.checkState(replicaInfo.getVolume().isTransientStorage()); @@ -3661,18 +3670,21 @@ public int getVolumeCount() { } void stopAllDataxceiverThreads(FsVolumeImpl volume) { - try (AutoCloseableLock lock = datasetWriteLock.acquire()) { - for (String bpid : volumeMap.getBlockPoolList()) { - Collection replicas = volumeMap.replicas(bpid); - for (ReplicaInfo replicaInfo : replicas) { - if ((replicaInfo.getState() == ReplicaState.TEMPORARY - || replicaInfo.getState() == ReplicaState.RBW) - && replicaInfo.getVolume().equals(volume)) { - ReplicaInPipeline replicaInPipeline = - (ReplicaInPipeline) replicaInfo; - replicaInPipeline.interruptThread(); + for (String bpid : volumeMap.getBlockPoolList()) { + try (AutoCloseDataSetLock lock = lockManager + .writeLock(LockLevel.BLOCK_POOl, bpid)) { + volumeMap.replicas(bpid, (iterator) -> { + while (iterator.hasNext()) { + ReplicaInfo replicaInfo = iterator.next(); + if ((replicaInfo.getState() == ReplicaState.TEMPORARY + || replicaInfo.getState() == ReplicaState.RBW) + && replicaInfo.getVolume().equals(volume)) { + ReplicaInPipeline replicaInPipeline = + (ReplicaInPipeline) replicaInfo; + replicaInPipeline.interruptThread(); + } } - } + }); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java index 95470bb8ffe26..262a24bd3aa45 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeList.java @@ -345,6 +345,28 @@ void waitVolumeRemoved(int sleepMillis, Condition condition) { FsDatasetImpl.LOG.info("Volume reference is released."); } + /** + * Wait for the reference of the volume removed from a previous + * {@link #removeVolume(FsVolumeImpl)} call to be released. + * + * @param sleepMillis interval to recheck. + */ + void waitVolumeRemoved(int sleepMillis, Object condition) { + while (!checkVolumesRemoved()) { + if (FsDatasetImpl.LOG.isDebugEnabled()) { + FsDatasetImpl.LOG.debug("Waiting for volume reference to be released."); + } + try { + condition.wait(sleepMillis); + } catch (InterruptedException e) { + FsDatasetImpl.LOG.info("Thread interrupted when waiting for " + + "volume reference to be released."); + Thread.currentThread().interrupt(); + } + } + FsDatasetImpl.LOG.info("Volume reference is released."); + } + @Override public String toString() { return volumes.toString(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java index eae119712f7c4..69a46257317bf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ProvidedVolumeImpl.java @@ -30,7 +30,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReentrantReadWriteLock; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; @@ -134,7 +133,7 @@ static class ProvidedBlockPoolSlice { ProvidedBlockPoolSlice(String bpid, ProvidedVolumeImpl volume, Configuration conf) { this.providedVolume = volume; - bpVolumeMap = new ReplicaMap(new ReentrantReadWriteLock()); + bpVolumeMap = new ReplicaMap(); Class fmt = conf.getClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS, TextFileRegionAliasMap.class, BlockAliasMap.class); @@ -219,7 +218,7 @@ private void incrNumBlocks() { } public boolean isEmpty() { - return bpVolumeMap.replicas(bpid).size() == 0; + return bpVolumeMap.size(bpid) == 0; } public void shutdown(BlockListAsLongs blocksListsAsLongs) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java index c1d103ed50dba..25d302b86c20a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java @@ -18,46 +18,49 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; import java.util.Collection; -import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; import java.util.Map; -import java.util.concurrent.locks.ReadWriteLock; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; +import org.apache.hadoop.hdfs.server.common.NoLockManager; import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo; import org.apache.hadoop.util.LightWeightResizableGSet; -import org.apache.hadoop.util.AutoCloseableLock; /** * Maintains the replica map. */ class ReplicaMap { // Lock object to synchronize this instance. - private final AutoCloseableLock readLock; - private final AutoCloseableLock writeLock; - + private DataNodeLockManager lockManager; + // Map of block pool Id to another map of block Id to ReplicaInfo. private final Map> map = - new HashMap<>(); + new ConcurrentHashMap<>(); - ReplicaMap(AutoCloseableLock readLock, AutoCloseableLock writeLock) { - if (readLock == null || writeLock == null) { + ReplicaMap(DataNodeLockManager manager) { + if (manager == null) { throw new HadoopIllegalArgumentException( - "Lock to synchronize on cannot be null"); + "Object to synchronize on cannot be null"); } - this.readLock = readLock; - this.writeLock = writeLock; + this.lockManager = manager; } - ReplicaMap(ReadWriteLock lock) { - this(new AutoCloseableLock(lock.readLock()), - new AutoCloseableLock(lock.writeLock())); + // Used for ut or temp replicaMap that no need to protected by lock. + ReplicaMap() { + this.lockManager = new NoLockManager(); } String[] getBlockPoolList() { - try (AutoCloseableLock l = readLock.acquire()) { - return map.keySet().toArray(new String[map.keySet().size()]); - } + Set bpset = map.keySet(); + return bpset.toArray(new String[bpset.size()]); } private void checkBlockPool(String bpid) { @@ -100,7 +103,7 @@ ReplicaInfo get(String bpid, Block block) { */ ReplicaInfo get(String bpid, long blockId) { checkBlockPool(bpid); - try (AutoCloseableLock l = readLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); return m != null ? m.get(new Block(blockId)) : null; } @@ -117,7 +120,7 @@ ReplicaInfo get(String bpid, long blockId) { ReplicaInfo add(String bpid, ReplicaInfo replicaInfo) { checkBlockPool(bpid); checkBlock(replicaInfo); - try (AutoCloseableLock l = writeLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m == null) { // Add an entry for block pool if it does not exist already @@ -135,7 +138,7 @@ ReplicaInfo add(String bpid, ReplicaInfo replicaInfo) { ReplicaInfo addAndGet(String bpid, ReplicaInfo replicaInfo) { checkBlockPool(bpid); checkBlock(replicaInfo); - try (AutoCloseableLock l = writeLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m == null) { // Add an entry for block pool if it does not exist already @@ -164,13 +167,28 @@ void addAll(ReplicaMap other) { * Merge all entries from the given replica map into the local replica map. */ void mergeAll(ReplicaMap other) { - other.map.forEach( - (bp, replicaInfos) -> { - replicaInfos.forEach( - replicaInfo -> add(bp, replicaInfo) - ); + Set bplist = other.map.keySet(); + for (String bp : bplist) { + checkBlockPool(bp); + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bp)) { + LightWeightResizableGSet replicaInfos = other.map.get(bp); + LightWeightResizableGSet curSet = map.get(bp); + HashSet replicaSet = new HashSet<>(); + //Can't add to GSet while in another GSet iterator may cause endlessLoop + for (ReplicaInfo replicaInfo : replicaInfos) { + replicaSet.add(replicaInfo); } - ); + for (ReplicaInfo replicaInfo : replicaSet) { + checkBlock(replicaInfo); + if (curSet == null) { + // Add an entry for block pool if it does not exist already + curSet = new LightWeightResizableGSet<>(); + map.put(bp, curSet); + } + curSet.put(replicaInfo); + } + } + } } /** @@ -184,7 +202,7 @@ void mergeAll(ReplicaMap other) { ReplicaInfo remove(String bpid, Block block) { checkBlockPool(bpid); checkBlock(block); - try (AutoCloseableLock l = writeLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m != null) { ReplicaInfo replicaInfo = m.get(block); @@ -206,7 +224,7 @@ ReplicaInfo remove(String bpid, Block block) { */ ReplicaInfo remove(String bpid, long blockId) { checkBlockPool(bpid); - try (AutoCloseableLock l = writeLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m != null) { return m.remove(new Block(blockId)); @@ -221,7 +239,7 @@ ReplicaInfo remove(String bpid, long blockId) { * @return the number of replicas in the map */ int size(String bpid) { - try (AutoCloseableLock l = readLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); return m != null ? m.size() : 0; } @@ -229,11 +247,9 @@ int size(String bpid) { /** * Get a collection of the replicas for given block pool - * This method is not synchronized. It needs to be synchronized - * externally using the lock, both for getting the replicas - * values from the map and iterating over it. Mutex can be accessed using - * {@link #getLock()} method. - * + * This method is not synchronized. If you want to keep thread safe + * Use method {@link #replicas(String, Consumer>)}. + * * @param bpid block pool id * @return a collection of the replicas belonging to the block pool */ @@ -243,9 +259,25 @@ Collection replicas(String bpid) { return m != null ? m.values() : null; } + /** + * execute function for one block pool and protect by LockManager. + * This method is synchronized. + * + * @param bpid block pool id + */ + void replicas(String bpid, Consumer> consumer) { + LightWeightResizableGSet m = null; + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { + m = map.get(bpid); + if (m !=null) { + m.getIterator(consumer); + } + } + } + void initBlockPool(String bpid) { checkBlockPool(bpid); - try (AutoCloseableLock l = writeLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m == null) { // Add an entry for block pool if it does not exist already @@ -257,26 +289,8 @@ void initBlockPool(String bpid) { void cleanUpBlockPool(String bpid) { checkBlockPool(bpid); - try (AutoCloseableLock l = writeLock.acquire()) { + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { map.remove(bpid); } } - - /** - * Get the lock object used for synchronizing ReplicasMap - * @return lock object - */ - AutoCloseableLock getLock() { - return writeLock; - } - - /** - * Get the lock object used for synchronizing the ReplicasMap for read only - * operations. - * @return The read lock object - */ - AutoCloseableLock getReadLock() { - return readLock; - } - } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java index 73e8bf7cb618e..3313c7c7a0360 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java @@ -40,6 +40,8 @@ import javax.management.ObjectName; import javax.management.StandardMBean; +import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MountVolumeMap; import org.apache.hadoop.thirdparty.com.google.common.math.LongMath; import org.apache.commons.lang3.ArrayUtils; @@ -48,7 +50,6 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetImplTestUtils; -import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; @@ -162,7 +163,7 @@ public static byte simulatedByte(Block b, long offsetInBlk) { static final byte[] nullCrcFileData; - private final AutoCloseableLock datasetLock; + private final DataNodeLockManager datasetLockManager; private final FileIoProvider fileIoProvider; static { @@ -707,6 +708,8 @@ public SimulatedFSDataset(DataStorage storage, Configuration conf) { public SimulatedFSDataset(DataNode datanode, DataStorage storage, Configuration conf) { this.datanode = datanode; + this.datasetLockManager = datanode == null ? new DataSetLockManager() : + datanode.getDataSetLockManager(); int storageCount; if (storage != null && storage.getNumStorageDirs() > 0) { storageCount = storage.getNumStorageDirs(); @@ -721,9 +724,6 @@ public SimulatedFSDataset(DataNode datanode, DataStorage storage, Configuration registerMBean(datanodeUuid); this.fileIoProvider = new FileIoProvider(conf, datanode); - - this.datasetLock = new AutoCloseableLock(); - this.storages = new ArrayList<>(); for (int i = 0; i < storageCount; i++) { this.storages.add(new SimulatedStorage( @@ -1587,14 +1587,8 @@ public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block, } @Override - public AutoCloseableLock acquireDatasetLock() { - return datasetLock.acquire(); - } - - @Override - public AutoCloseableLock acquireDatasetReadLock() { - // No RW lock implementation in simulated dataset currently. - return datasetLock.acquire(); + public DataNodeLockManager acquireDatasetLockManager() { + return null; } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java index 057dd6459fdc2..0c0fe618174c4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java @@ -128,6 +128,7 @@ public class TestBPOfferService { private final int[] heartbeatCounts = new int[3]; private DataNode mockDn; private FsDatasetSpi mockFSDataset; + private DataSetLockManager dataSetLockManager = new DataSetLockManager(); private boolean isSlownode; @Before @@ -153,6 +154,7 @@ public void setupMocks() throws Exception { // Wire the dataset to the DN. Mockito.doReturn(mockFSDataset).when(mockDn).getFSDataset(); + Mockito.doReturn(dataSetLockManager).when(mockDn).getDataSetLockManager(); } /** @@ -508,6 +510,7 @@ public void testPickActiveNameNode() throws Exception { public void testBPInitErrorHandling() throws Exception { final DataNode mockDn = Mockito.mock(DataNode.class); Mockito.doReturn(true).when(mockDn).shouldRun(); + Mockito.doReturn(dataSetLockManager).when(mockDn).getDataSetLockManager(); Configuration conf = new Configuration(); File dnDataDir = new File( new File(TEST_BUILD_DATA, "testBPInitErrorHandling"), "data"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java index 8d2df18711256..9d7c820249c65 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery2.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.util.AutoCloseableLock; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -252,10 +251,8 @@ public void testRaceBetweenReplicaRecoveryAndFinalizeBlock() final BlockRecoveryCommand.RecoveringBlock recoveringBlock = new BlockRecoveryCommand.RecoveringBlock(block.getBlock(), locations, block.getBlock().getGenerationStamp() + 1); - try(AutoCloseableLock lock = dataNode.data.acquireDatasetLock()) { - Thread.sleep(2000); - dataNode.initReplicaRecovery(recoveringBlock); - } + Thread.sleep(2000); + dataNode.initReplicaRecovery(recoveringBlock); } catch (Exception e) { LOG.error("Something went wrong.", e); recoveryInitResult.set(false); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java index b6beb9a4878e3..74c70cec76967 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java @@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler; import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult; import org.apache.hadoop.hdfs.server.datanode.fsdataset.DataNodeVolumeMetrics; @@ -133,7 +134,8 @@ private List createFile(String fileNamePrefix, long fileLen, /** Truncate a block file. */ private long truncateBlockFile() throws IOException { - try (AutoCloseableLock lock = fds.acquireDatasetLock()) { + try (AutoCloseableLock lock = fds.acquireDatasetLockManager().writeLock( + LockLevel.BLOCK_POOl, bpid)) { for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) { File f = new File(b.getBlockURI()); File mf = new File(b.getMetadataURI()); @@ -158,7 +160,8 @@ private long truncateBlockFile() throws IOException { /** Delete a block file */ private long deleteBlockFile() { - try (AutoCloseableLock lock = fds.acquireDatasetLock()) { + try (AutoCloseableLock lock = fds.acquireDatasetLockManager(). + writeLock(LockLevel.BLOCK_POOl, bpid)) { for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) { File f = new File(b.getBlockURI()); File mf = new File(b.getMetadataURI()); @@ -174,7 +177,8 @@ private long deleteBlockFile() { /** Delete block meta file */ private long deleteMetaFile() { - try (AutoCloseableLock lock = fds.acquireDatasetLock()) { + try (AutoCloseableLock lock = fds.acquireDatasetLockManager(). + writeLock(LockLevel.BLOCK_POOl, bpid)) { for (ReplicaInfo b : FsDatasetTestUtil.getReplicas(fds, bpid)) { // Delete a metadata file if (b.metadataExists() && b.deleteMetadata()) { @@ -193,7 +197,8 @@ private long deleteMetaFile() { * @throws IOException */ private void duplicateBlock(long blockId) throws IOException { - try (AutoCloseableLock lock = fds.acquireDatasetLock()) { + try (AutoCloseableLock lock = fds.acquireDatasetLockManager(). + writeLock(LockLevel.BLOCK_POOl, bpid)) { ReplicaInfo b = FsDatasetTestUtil.fetchReplicaInfo(fds, bpid, blockId); try (FsDatasetSpi.FsVolumeReferences volumes = fds.getFsVolumeReferences()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java index 3fbd4de721260..1c6597eb4541a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java @@ -23,8 +23,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; +import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MountVolumeMap; -import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo; @@ -452,12 +453,7 @@ public ReplicaInfo moveBlockAcrossVolumes(ExtendedBlock block, } @Override - public AutoCloseableLock acquireDatasetLock() { - return null; - } - - @Override - public AutoCloseableLock acquireDatasetReadLock() { + public DataNodeLockManager acquireDatasetLockManager() { return null; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java index 67176d8a1a3e6..7e8497e1ebdbc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImplTestUtils.java @@ -434,7 +434,7 @@ public void changeStoredGenerationStamp( @Override public Iterator getStoredReplicas(String bpid) throws IOException { // Reload replicas from the disk. - ReplicaMap replicaMap = new ReplicaMap(dataset.datasetRWLock); + ReplicaMap replicaMap = new ReplicaMap(dataset.acquireDatasetLockManager()); try (FsVolumeReferences refs = dataset.getFsVolumeReferences()) { for (FsVolumeSpi vol : refs) { FsVolumeImpl volume = (FsVolumeImpl) vol; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java index f11f4b9d1cdb1..23a72f9acfa4e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java @@ -21,11 +21,11 @@ import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Paths; -import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import org.apache.hadoop.fs.DF; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.datanode.DataSetLockManager; import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner; import org.apache.hadoop.hdfs.server.datanode.LocalReplica; @@ -70,7 +70,6 @@ import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; -import org.apache.hadoop.util.AutoCloseableLock; import org.apache.hadoop.util.DiskChecker; import org.apache.hadoop.util.FakeTimer; import org.apache.hadoop.util.Lists; @@ -94,7 +93,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DN_CACHED_DFSUSED_CHECK_INTERVAL_MS; @@ -140,6 +138,7 @@ public class TestFsDatasetImpl { private DataNode datanode; private DataStorage storage; private FsDatasetImpl dataset; + private DataSetLockManager manager = new DataSetLockManager(); private final static String BLOCKPOOL = "BP-TEST"; @@ -213,6 +212,7 @@ public void setUp() throws IOException { this.conf.set(DFSConfigKeys.DFS_DATANODE_REPLICA_CACHE_ROOT_DIR_KEY, replicaCacheRootDir); + when(datanode.getDataSetLockManager()).thenReturn(manager); when(datanode.getConf()).thenReturn(conf); final DNConf dnConf = new DNConf(datanode); when(datanode.getDnConf()).thenReturn(dnConf); @@ -232,118 +232,6 @@ public void setUp() throws IOException { assertEquals(0, dataset.getNumFailedVolumes()); } - @Test(timeout=10000) - public void testReadLockEnabledByDefault() - throws Exception { - final FsDatasetSpi ds = dataset; - AtomicBoolean accessed = new AtomicBoolean(false); - CountDownLatch latch = new CountDownLatch(1); - CountDownLatch waiterLatch = new CountDownLatch(1); - - Thread holder = new Thread() { - public void run() { - try (AutoCloseableLock l = ds.acquireDatasetReadLock()) { - latch.countDown(); - // wait for the waiter thread to access the lock. - waiterLatch.await(); - } catch (Exception e) { - } - } - }; - - Thread waiter = new Thread() { - public void run() { - try { - latch.await(); - } catch (InterruptedException e) { - waiterLatch.countDown(); - return; - } - try (AutoCloseableLock l = ds.acquireDatasetReadLock()) { - accessed.getAndSet(true); - // signal the holder thread. - waiterLatch.countDown(); - } catch (Exception e) { - } - } - }; - waiter.start(); - holder.start(); - holder.join(); - waiter.join(); - // The holder thread is still holding the lock, but the waiter can still - // run as the lock is a shared read lock. - // Otherwise test will timeout with deadlock. - assertEquals(true, accessed.get()); - holder.interrupt(); - } - - @Test(timeout=20000) - public void testReadLockCanBeDisabledByConfig() - throws Exception { - HdfsConfiguration conf = new HdfsConfiguration(); - conf.setBoolean( - DFSConfigKeys.DFS_DATANODE_LOCK_READ_WRITE_ENABLED_KEY, false); - MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf) - .numDataNodes(1).build(); - try { - AtomicBoolean accessed = new AtomicBoolean(false); - cluster.waitActive(); - DataNode dn = cluster.getDataNodes().get(0); - final FsDatasetSpi ds = DataNodeTestUtils.getFSDataset(dn); - - CountDownLatch latch = new CountDownLatch(1); - CountDownLatch waiterLatch = new CountDownLatch(1); - Thread holder = new Thread() { - public void run() { - try (AutoCloseableLock l = ds.acquireDatasetReadLock()) { - latch.countDown(); - // wait for the waiter thread to access the lock. - waiterLatch.await(); - } catch (Exception e) { - } - } - }; - - Thread waiter = new Thread() { - public void run() { - try { - // Wait for holder to get ds read lock. - latch.await(); - } catch (InterruptedException e) { - waiterLatch.countDown(); - return; - } - try (AutoCloseableLock l = ds.acquireDatasetReadLock()) { - accessed.getAndSet(true); - // signal the holder thread. - waiterLatch.countDown(); - } catch (Exception e) { - } - } - }; - waiter.start(); - holder.start(); - // Wait for sometime to make sure we are in deadlock, - try { - GenericTestUtils.waitFor(() -> - accessed.get(), - 100, 10000); - fail("Waiter thread should not execute."); - } catch (TimeoutException e) { - } - // Release waiterLatch to exit deadlock. - waiterLatch.countDown(); - holder.join(); - waiter.join(); - // After releasing waiterLatch water - // thread will be able to execute. - assertTrue(accessed.get()); - } finally { - cluster.shutdown(); - } - } - @Test public void testAddVolumes() throws IOException { final int numNewVolumes = 3; @@ -413,6 +301,7 @@ public void testAddVolumeWithSameDiskTiering() throws IOException { final ShortCircuitRegistry shortCircuitRegistry = new ShortCircuitRegistry(conf); when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry); + when(datanode.getDataSetLockManager()).thenReturn(manager); createStorageDirs(storage, conf, 1); dataset = new FsDatasetImpl(datanode, storage, conf); @@ -480,6 +369,8 @@ public void testAddVolumeWithCustomizedCapacityRatio() final ShortCircuitRegistry shortCircuitRegistry = new ShortCircuitRegistry(conf); when(datanode.getShortCircuitRegistry()).thenReturn(shortCircuitRegistry); + when(datanode.getDataSetLockManager()).thenReturn(manager); + createStorageDirs(storage, conf, 0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java index 4d8e0c99980d2..1b638e28fa7de 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsVolumeList.java @@ -57,7 +57,6 @@ import java.util.concurrent.ForkJoinPool; import java.util.concurrent.Future; import java.util.concurrent.TimeoutException; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DU_RESERVED_PERCENTAGE_KEY; @@ -407,7 +406,7 @@ public void run() { fs.close(); FsDatasetImpl fsDataset = (FsDatasetImpl) cluster.getDataNodes().get(0) .getFSDataset(); - ReplicaMap volumeMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap volumeMap = new ReplicaMap(fsDataset.acquireDatasetLockManager()); RamDiskReplicaTracker ramDiskReplicaMap = RamDiskReplicaTracker .getInstance(conf, fsDataset); FsVolumeImpl vol = (FsVolumeImpl) fsDataset.getFsVolumeReferences().get(0); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java index b72b1cd1bb8bd..5290337e2f270 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java @@ -25,7 +25,6 @@ import java.net.InetSocketAddress; import java.net.SocketTimeoutException; import java.util.List; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -47,6 +46,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; +import org.apache.hadoop.hdfs.server.datanode.DataSetLockManager; import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica; import org.apache.hadoop.hdfs.server.datanode.Replica; import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo; @@ -236,7 +236,8 @@ public void testInitReplicaRecovery() throws IOException { final long firstblockid = 10000L; final long gs = 7777L; final long length = 22L; - final ReplicaMap map = new ReplicaMap(new ReentrantReadWriteLock()); + DataSetLockManager manager = new DataSetLockManager(); + final ReplicaMap map = new ReplicaMap(manager); String bpid = "BP-TEST"; final Block[] blocks = new Block[5]; for(int i = 0; i < blocks.length; i++) { @@ -252,7 +253,7 @@ public void testInitReplicaRecovery() throws IOException { final long recoveryid = gs + 1; final ReplicaRecoveryInfo recoveryInfo = FsDatasetImpl .initReplicaRecovery(bpid, map, blocks[0], recoveryid, - DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT); + DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT, manager); assertEquals(originalInfo, recoveryInfo); final ReplicaUnderRecovery updatedInfo = (ReplicaUnderRecovery)map.get(bpid, b); @@ -263,7 +264,7 @@ public void testInitReplicaRecovery() throws IOException { final long recoveryid2 = gs + 2; final ReplicaRecoveryInfo recoveryInfo2 = FsDatasetImpl .initReplicaRecovery(bpid, map, blocks[0], recoveryid2, - DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT); + DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT, manager); assertEquals(originalInfo, recoveryInfo2); final ReplicaUnderRecovery updatedInfo2 = (ReplicaUnderRecovery)map.get(bpid, b); @@ -273,7 +274,7 @@ public void testInitReplicaRecovery() throws IOException { //case RecoveryInProgressException try { FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid, - DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT); + DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT, manager); Assert.fail(); } catch(RecoveryInProgressException ripe) { @@ -286,7 +287,7 @@ public void testInitReplicaRecovery() throws IOException { final Block b = new Block(firstblockid - 1, length, gs); ReplicaRecoveryInfo r = FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid, - DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT); + DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT, manager); Assert.assertNull("Data-node should not have this replica.", r); } @@ -295,7 +296,7 @@ public void testInitReplicaRecovery() throws IOException { final Block b = new Block(firstblockid + 1, length, gs); try { FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid, - DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT); + DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT, manager); Assert.fail(); } catch(IOException ioe) { @@ -309,7 +310,7 @@ public void testInitReplicaRecovery() throws IOException { final Block b = new Block(firstblockid, length, gs+1); try { FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid, - DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT); + DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT, manager); fail("InitReplicaRecovery should fail because replica's " + "gs is less than the block's gs"); } catch (IOException e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java index afd816864a1d7..f8d66c2f2cccf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestProvidedImpl.java @@ -44,7 +44,6 @@ import java.util.Optional; import java.util.Random; import java.util.Set; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; @@ -66,6 +65,7 @@ import org.apache.hadoop.hdfs.server.datanode.BlockScanner; import org.apache.hadoop.hdfs.server.datanode.DNConf; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataSetLockManager; import org.apache.hadoop.hdfs.server.datanode.DataStorage; import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner; import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ScanInfoVolumeReport; @@ -109,6 +109,7 @@ public class TestProvidedImpl { private DataNode datanode; private DataStorage storage; private FsDatasetImpl dataset; + private DataSetLockManager manager = new DataSetLockManager(); private static Map blkToPathMap; private static List providedVolumes; private static long spaceUsed = 0; @@ -319,6 +320,7 @@ public void setUp() throws IOException { conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 0); when(datanode.getConf()).thenReturn(conf); + when(datanode.getDataSetLockManager()).thenReturn(manager); final DNConf dnConf = new DNConf(datanode); when(datanode.getDnConf()).thenReturn(dnConf); // reset the space used @@ -400,7 +402,7 @@ public void testProvidedVolumeImpl() throws IOException { public void testBlockLoad() throws IOException { for (int i = 0; i < providedVolumes.size(); i++) { FsVolumeImpl vol = providedVolumes.get(i); - ReplicaMap volumeMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap volumeMap = new ReplicaMap(); vol.getVolumeMap(volumeMap, null); assertEquals(vol.getBlockPoolList().length, BLOCK_POOL_IDS.length); @@ -476,7 +478,7 @@ private int getBlocksInProvidedVolumes(String basePath, int numBlocks, vol.setFileRegionProvider(BLOCK_POOL_IDS[CHOSEN_BP_ID], new TestFileRegionBlockAliasMap(fileRegionIterator, minBlockId, numBlocks)); - ReplicaMap volumeMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap volumeMap = new ReplicaMap(); vol.getVolumeMap(BLOCK_POOL_IDS[CHOSEN_BP_ID], volumeMap, null); totalBlocks += volumeMap.size(BLOCK_POOL_IDS[CHOSEN_BP_ID]); } @@ -586,7 +588,7 @@ public void testProvidedReplicaSuffixExtraction() { public void testProvidedReplicaPrefix() throws Exception { for (int i = 0; i < providedVolumes.size(); i++) { FsVolumeImpl vol = providedVolumes.get(i); - ReplicaMap volumeMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap volumeMap = new ReplicaMap(); vol.getVolumeMap(volumeMap, null); Path expectedPrefix = new Path( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java index 59203bb7d3468..3058598bb6c6e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestReplicaMap.java @@ -26,13 +26,12 @@ import org.junit.Before; import org.junit.Test; -import java.util.concurrent.locks.ReentrantReadWriteLock; /** * Unit test for ReplicasMap class */ public class TestReplicaMap { - private final ReplicaMap map = new ReplicaMap(new ReentrantReadWriteLock()); + private final ReplicaMap map = new ReplicaMap(); private final String bpid = "BP-TEST"; private final Block block = new Block(1234, 1234, 1234); @@ -112,7 +111,7 @@ public void testRemove() { @Test public void testMergeAll() { - ReplicaMap temReplicaMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap temReplicaMap = new ReplicaMap(); Block tmpBlock = new Block(5678, 5678, 5678); temReplicaMap.add(bpid, new FinalizedReplica(tmpBlock, null, null)); @@ -123,7 +122,7 @@ public void testMergeAll() { @Test public void testAddAll() { - ReplicaMap temReplicaMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap temReplicaMap = new ReplicaMap(); Block tmpBlock = new Block(5678, 5678, 5678); temReplicaMap.add(bpid, new FinalizedReplica(tmpBlock, null, null)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java index e939389624130..fe78a0f2a41a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java @@ -27,7 +27,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.StorageType; @@ -550,7 +549,7 @@ public void testReplicaMapAfterDatanodeRestart() throws Exception { bpList.size() == 2); createReplicas(bpList, volumes, cluster.getFsDatasetTestUtils(dn)); - ReplicaMap oldReplicaMap = new ReplicaMap(new ReentrantReadWriteLock()); + ReplicaMap oldReplicaMap = new ReplicaMap(); oldReplicaMap.addAll(dataSet.volumeMap); cluster.restartDataNode(0); From 7b5eac27ff380a31428a71f5fd162a9cdee05dd8 Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Fri, 4 Mar 2022 16:17:46 -0800 Subject: [PATCH 035/145] HDFS-16495: RBF should prepend the client ip rather than append it. Fixes #4054 Signed-off-by: Owen O'Malley --- .../org/apache/hadoop/ipc/CallerContext.java | 2 +- .../federation/router/RouterRpcClient.java | 31 +++++++++++++------ .../federation/router/TestRouterRpc.java | 13 ++++---- .../router/TestRouterRpcMultiDestination.java | 5 ++- 4 files changed, 32 insertions(+), 19 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java index ed8dcf2242f2c..c8b4135d088ed 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java @@ -116,7 +116,7 @@ public String toString() { /** The caller context builder. */ public static final class Builder { - private static final String KEY_VALUE_SEPARATOR = ":"; + public static final String KEY_VALUE_SEPARATOR = ":"; /** * The illegal separators include '\t', '\n', '='. * User should not set illegal separator. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index 5683f416b3809..436418da00348 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java @@ -466,7 +466,7 @@ private Object invokeMethod( + router.getRouterId()); } - appendClientIpPortToCallerContextIfAbsent(); + addClientIpToCallerContext(); Object ret = null; if (rpcMonitor != null) { @@ -588,19 +588,32 @@ private Object invokeMethod( /** * For tracking which is the actual client address. * It adds trace info "clientIp:ip" and "clientPort:port" - * to caller context if they are absent. + * in the caller context, removing the old values if they were + * already present. */ - private void appendClientIpPortToCallerContextIfAbsent() { + private void addClientIpToCallerContext() { CallerContext ctx = CallerContext.getCurrent(); String origContext = ctx == null ? null : ctx.getContext(); byte[] origSignature = ctx == null ? null : ctx.getSignature(); - CallerContext.setCurrent( - new CallerContext.Builder(origContext, contextFieldSeparator) - .appendIfAbsent(CLIENT_IP_STR, Server.getRemoteAddress()) - .appendIfAbsent(CLIENT_PORT_STR, + CallerContext.Builder builder = + new CallerContext.Builder("", contextFieldSeparator) + .append(CLIENT_IP_STR, Server.getRemoteAddress()) + .append(CLIENT_PORT_STR, Integer.toString(Server.getRemotePort())) - .setSignature(origSignature) - .build()); + .setSignature(origSignature); + // Append the original caller context + if (origContext != null) { + for (String part : origContext.split(contextFieldSeparator)) { + String[] keyValue = + part.split(CallerContext.Builder.KEY_VALUE_SEPARATOR, 2); + if (keyValue.length == 2) { + builder.appendIfAbsent(keyValue[0], keyValue[1]); + } else if (keyValue.length == 1) { + builder.append(keyValue[0]); + } + } + } + CallerContext.setCurrent(builder.build()); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java index 2ab40b031495b..ae58bf8b6126f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java @@ -1950,9 +1950,10 @@ public void testMkdirsWithCallerContext() throws IOException { FsPermission permission = new FsPermission("755"); routerProtocol.mkdirs(dirPath, permission, false); - // The audit log should contains "callerContext=clientContext,clientIp:" - assertTrue(auditlog.getOutput() - .contains("callerContext=clientContext,clientIp:")); + // The audit log should contains "callerContext=clientIp:...,clientContext" + final String logOutput = auditlog.getOutput(); + assertTrue(logOutput.contains("callerContext=clientIp:")); + assertTrue(logOutput.contains(",clientContext")); assertTrue(verifyFileExists(routerFS, dirPath)); } @@ -1997,9 +1998,9 @@ public void testAddClientIpPortToCallerContext() throws IOException { // Create a directory via the router. routerProtocol.getFileInfo(dirPath); - // The audit log should contains the original clientIp and clientPort + // The audit log should not contain the original clientIp and clientPort // set by client. - assertTrue(auditLog.getOutput().contains("clientIp:1.1.1.1")); - assertTrue(auditLog.getOutput().contains("clientPort:1234")); + assertFalse(auditLog.getOutput().contains("clientIp:1.1.1.1")); + assertFalse(auditLog.getOutput().contains("clientPort:1234")); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java index e50464c0be7b5..370a1250a7c11 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java @@ -464,9 +464,8 @@ public void testCallerContextWithMultiDestinations() throws IOException { for (String line : auditLog.getOutput().split("\n")) { if (line.contains(auditFlag)) { // assert origin caller context exist in audit log - assertTrue(line.contains("callerContext=clientContext")); - String callerContext = line.substring( - line.indexOf("callerContext=clientContext")); + String callerContext = line.substring(line.indexOf("callerContext=")); + assertTrue(callerContext.contains("clientContext")); // assert client ip info exist in caller context assertTrue(callerContext.contains(clientIpInfo)); // assert client ip info appears only once in caller context From 8b8158f02df18424b2406fd66b34c1bdf3d7ab55 Mon Sep 17 00:00:00 2001 From: Xing Lin Date: Tue, 22 Feb 2022 15:52:03 -0800 Subject: [PATCH 036/145] HADOOP-18144: getTrashRoot in ViewFileSystem should return a path in ViewFS. To get the new behavior, define fs.viewfs.trash.force-inside-mount-point to be true. If the trash root for path p is in the same mount point as path p, and one of: * The mount point isn't at the top of the target fs. * The resolved path of path is root (eg it is the fallback FS). * The trash root isn't in user's target fs home directory. get the corresponding viewFS path for the trash root and return it. Otherwise, use /.Trash/. Signed-off-by: Owen O'Malley --- .../apache/hadoop/fs/TrashPolicyDefault.java | 4 +- .../apache/hadoop/fs/viewfs/Constants.java | 7 +- .../hadoop/fs/viewfs/ViewFileSystem.java | 192 +++++++++++------- .../TestViewFileSystemLocalFileSystem.java | 9 + ...ileSystemWithAuthorityLocalFileSystem.java | 10 + .../fs/viewfs/ViewFileSystemBaseTest.java | 102 ++++++---- .../fs/viewfs/TestViewFileSystemHdfs.java | 8 + 7 files changed, 206 insertions(+), 126 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java index 99467f5633625..f4228dea69f49 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java @@ -191,8 +191,8 @@ public boolean moveToTrash(Path path) throws IOException { cause = e; } } - throw (IOException) - new IOException("Failed to move to trash: " + path).initCause(cause); + throw new IOException("Failed to move " + path + " to trash " + trashPath, + cause); } @SuppressWarnings("deprecation") diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index 94b07973ac824..21f4d99f891c2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -134,8 +134,9 @@ public interface Constants { HCFSMountTableConfigLoader.class; /** - * Enable ViewFileSystem to return a trashRoot which is local to mount point. + * Force ViewFileSystem to return a trashRoot that is inside a mount point. */ - String CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH = "fs.viewfs.mount.point.local.trash"; - boolean CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH_DEFAULT = false; + String CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT = + "fs.viewfs.trash.force-inside-mount-point"; + boolean CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT_DEFAULT = false; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index 8c3cdb82d7447..fde0faa59b45e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -25,8 +25,8 @@ import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT; import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_555; -import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH; -import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH_DEFAULT; +import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT; +import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT_DEFAULT; import java.util.function.Function; import java.io.FileNotFoundException; @@ -1132,47 +1132,77 @@ public Collection getAllStoragePolicies() * Get the trash root directory for current user when the path * specified is deleted. * - * If CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH is not set, return - * the default trash root from targetFS. + * If FORCE_INSIDE_MOUNT_POINT flag is not set, return the default trash root + * from targetFS. * - * When CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH is set to true, - * 1) If path p is in fallback FS or from the same mount point as the default - * trash root for targetFS, return the default trash root for targetFS. - * 2) else, return a trash root in the mounted targetFS - * (/mntpoint/.Trash/{user}) + * When FORCE_INSIDE_MOUNT_POINT is set to true, + *
    + *
  1. + * If the trash root for path p is in the same mount point as path p, + * and one of: + *
      + *
    1. The mount point isn't at the top of the target fs.
    2. + *
    3. The resolved path of path is root (in fallback FS).
    4. + *
    5. The trash isn't in user's target fs home directory + * get the corresponding viewFS path for the trash root and return + * it. + *
    6. + *
    + *
  2. + *
  3. + * else, return the trash root under the root of the mount point + * (/{mntpoint}/.Trash/{user}). + *
  4. + *
+ * + * These conditions handle several different important cases: + *
    + *
  • File systems may need to have more local trash roots, such as + * encryption zones or snapshot roots.
  • + *
  • The fallback mount should use the user's home directory.
  • + *
  • Cloud storage systems should not use trash in an implicity defined + * home directory, per a container, unless it is the fallback fs.
  • + *
* * @param path the trash root of the path to be determined. * @return the trash root path. */ @Override public Path getTrashRoot(Path path) { - boolean useMountPointLocalTrash = - config.getBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, - CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH_DEFAULT); try { InodeTree.ResolveResult res = fsState.resolve(getUriPath(path), true); + Path targetFSTrashRoot = + res.targetFileSystem.getTrashRoot(res.remainingPath); - Path trashRoot = res.targetFileSystem.getTrashRoot(res.remainingPath); - if (!useMountPointLocalTrash) { - return trashRoot; - } else { - // Path p is either in a mount point or in the fallback FS + // Allow clients to use old behavior of delegating to target fs. + if (!config.getBoolean(CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT, + CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT_DEFAULT)) { + return targetFSTrashRoot; + } - if (ROOT_PATH.equals(new Path(res.resolvedPath)) - || trashRoot.toUri().getPath().startsWith(res.resolvedPath)) { - // Path p is in the fallback FS or targetFileSystem.trashRoot is in - // the same mount point as Path p - return trashRoot; - } else { - // targetFileSystem.trashRoot is in a different mount point from - // Path p. Return the trash root for the mount point. - Path mountPointRoot = - res.targetFileSystem.getFileStatus(new Path("/")).getPath(); - return new Path(mountPointRoot, - TRASH_PREFIX + "/" + ugi.getShortUserName()); - } + // The trash root path from the target fs + String targetFSTrashRootPath = targetFSTrashRoot.toUri().getPath(); + // The mount point path in the target fs + String mountTargetPath = res.targetFileSystem.getUri().getPath(); + if (!mountTargetPath.endsWith("/")) { + mountTargetPath = mountTargetPath + "/"; + } + + Path targetFsUserHome = res.targetFileSystem.getHomeDirectory(); + if (targetFSTrashRootPath.startsWith(mountTargetPath) && + !(mountTargetPath.equals(ROOT_PATH.toString()) && + !res.resolvedPath.equals(ROOT_PATH.toString()) && + (targetFsUserHome != null && targetFSTrashRootPath.startsWith( + targetFsUserHome.toUri().getPath())))) { + String relativeTrashRoot = + targetFSTrashRootPath.substring(mountTargetPath.length()); + return makeQualified(new Path(res.resolvedPath, relativeTrashRoot)); + } else { + // Return the trash root for the mount point. + return makeQualified(new Path(res.resolvedPath, + TRASH_PREFIX + "/" + ugi.getShortUserName())); } } catch (IOException | IllegalArgumentException e) { throw new NotInMountpointException(path, "getTrashRoot"); @@ -1182,72 +1212,78 @@ public Path getTrashRoot(Path path) { /** * Get all the trash roots for current user or all users. * + * When FORCE_INSIDE_MOUNT_POINT is set to true, we also return trash roots + * under the root of each mount point, with their viewFS paths. + * * @param allUsers return trash roots for all users if true. * @return all Trash root directories. */ @Override public Collection getTrashRoots(boolean allUsers) { - List trashRoots = new ArrayList<>(); + // A map from targetFSPath -> FileStatus. + // FileStatus can be from targetFS or viewFS. + HashMap trashRoots = new HashMap<>(); for (FileSystem fs : getChildFileSystems()) { - trashRoots.addAll(fs.getTrashRoots(allUsers)); + for (FileStatus trash : fs.getTrashRoots(allUsers)) { + trashRoots.put(trash.getPath(), trash); + } } - // Add trash dirs for each mount point - boolean useMountPointLocalTrash = - config.getBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, - CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH_DEFAULT); - if (useMountPointLocalTrash) { + // Return trashRoots if FORCE_INSIDE_MOUNT_POINT is disabled. + if (!config.getBoolean(CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT, + CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT_DEFAULT)) { + return trashRoots.values(); + } - Set currentTrashPaths = new HashSet<>(); - for (FileStatus file : trashRoots) { - currentTrashPaths.add(file.getPath()); - } + // Get trash roots in TRASH_PREFIX dir inside mount points and fallback FS. + List> mountPoints = + fsState.getMountPoints(); + // If we have a fallback FS, add a mount point for it as <"", fallback FS>. + // The source path of a mount point shall not end with '/', thus for + // fallback fs, we set its mount point src as "". + if (fsState.getRootFallbackLink() != null) { + mountPoints.add(new InodeTree.MountPoint<>("", + fsState.getRootFallbackLink())); + } - MountPoint[] mountPoints = getMountPoints(); - try { - for (int i = 0; i < mountPoints.length; i++) { - Path trashRoot = makeQualified( - new Path(mountPoints[i].mountedOnPath + "/" + TRASH_PREFIX)); + try { + for (InodeTree.MountPoint mountPoint : mountPoints) { - // Continue if trashRoot does not exist for this filesystem - if (!exists(trashRoot)) { - continue; - } + Path trashRoot = + makeQualified(new Path(mountPoint.src + "/" + TRASH_PREFIX)); - InodeTree.ResolveResult res = - fsState.resolve(getUriPath(trashRoot), true); - - if (!allUsers) { - Path userTrash = - new Path("/" + TRASH_PREFIX + "/" + ugi.getShortUserName()); - try { - FileStatus file = res.targetFileSystem.getFileStatus(userTrash); - if (!currentTrashPaths.contains(file.getPath())) { - trashRoots.add(file); - currentTrashPaths.add(file.getPath()); - } - } catch (FileNotFoundException ignored) { - } - } else { - FileStatus[] targetFsTrashRoots = - res.targetFileSystem.listStatus(new Path("/" + TRASH_PREFIX)); - for (FileStatus file : targetFsTrashRoots) { - // skip if we already include it in currentTrashPaths - if (currentTrashPaths.contains(file.getPath())) { - continue; - } + // Continue if trashRoot does not exist for this mount point + if (!exists(trashRoot)) { + continue; + } - trashRoots.add(file); - currentTrashPaths.add(file.getPath()); - } + FileSystem targetFS = mountPoint.target.getTargetFileSystem(); + if (!allUsers) { + Path userTrashRoot = new Path(trashRoot, ugi.getShortUserName()); + if (exists(userTrashRoot)) { + Path targetFSUserTrashRoot = targetFS.makeQualified( + new Path(targetFS.getUri().getPath(), + TRASH_PREFIX + "/" + ugi.getShortUserName())); + trashRoots.put(targetFSUserTrashRoot, getFileStatus(userTrashRoot)); + } + } else { + FileStatus[] mountPointTrashRoots = listStatus(trashRoot); + for (FileStatus trash : mountPointTrashRoots) { + // Remove the mountPoint and the leading '/' to get the + // relative targetFsTrash path + String targetFsTrash = trash.getPath().toUri().getPath() + .substring(mountPoint.src.length() + 1); + Path targetFsTrashPath = targetFS.makeQualified( + new Path(targetFS.getUri().getPath(), targetFsTrash)); + trashRoots.put(targetFsTrashPath, trash); } } - } catch (IOException e) { - LOG.warn("Exception in get all trash roots", e); } + } catch (IOException e) { + LOG.warn("Exception in get all trash roots for mount points", e); } - return trashRoots; + return trashRoots.values(); } @Override diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java index 808d8b06c35ba..adc5db87e7725 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLocalFileSystem.java @@ -33,6 +33,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX; +import org.apache.hadoop.security.UserGroupInformation; import org.junit.After; import org.junit.Before; @@ -61,6 +63,13 @@ public void setUp() throws Exception { } + @Override + Path getTrashRootInFallBackFS() throws IOException { + return new Path( + "/" + TRASH_PREFIX + "/" + UserGroupInformation.getCurrentUser() + .getShortUserName()); + } + @Test public void testNflyWriteSimple() throws IOException { LOG.info("Starting testNflyWriteSimple"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java index 877c2228c1eea..9223338f34bf5 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java @@ -25,6 +25,9 @@ import org.apache.hadoop.fs.FileSystemTestHelper; import org.apache.hadoop.fs.FsConstants; import org.apache.hadoop.fs.Path; +import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX; +import org.apache.hadoop.security.UserGroupInformation; +import java.io.IOException; import org.junit.After; import org.junit.Assert; @@ -63,6 +66,13 @@ public void tearDown() throws Exception { super.tearDown(); } + @Override + Path getTrashRootInFallBackFS() throws IOException { + return new Path( + "/" + TRASH_PREFIX + "/" + UserGroupInformation.getCurrentUser() + .getShortUserName()); + } + @Override @Test public void testBasicPaths() { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index 91a90751e873e..90ffa06bfce30 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -69,7 +69,7 @@ import static org.apache.hadoop.fs.FileSystemTestHelper.*; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE; import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_555; -import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH; +import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT; import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX; import org.junit.After; @@ -1104,35 +1104,46 @@ public void testTrashRoot() throws IOException { Assert.assertTrue("", fsView.getTrashRoots(true).size() > 0); } + // Default implementation of getTrashRoot for a fallback FS mounted at root: + // e.g., fallbackFS.uri.getPath = '/' + Path getTrashRootInFallBackFS() throws IOException { + return new Path(fsTarget.getHomeDirectory().toUri().getPath(), + TRASH_PREFIX); + } + /** - * Test the localized trash root for getTrashRoot. + * Test TRASH_FORCE_INSIDE_MOUNT_POINT feature for getTrashRoot. */ @Test - public void testTrashRootLocalizedTrash() throws IOException { + public void testTrashRootForceInsideMountPoint() throws IOException { UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); Configuration conf2 = new Configuration(conf); - conf2.setBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, true); + conf2.setBoolean(CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT, true); ConfigUtil.addLinkFallback(conf2, targetTestRoot.toUri()); FileSystem fsView2 = FileSystem.get(FsConstants.VIEWFS_URI, conf2); - // Case 1: path p not in the default FS. - // Return a trash root within the mount point. + // Case 1: path p in the /data mount point. + // Return a trash root within the /data mount point. Path dataTestPath = new Path("/data/dir/file"); - Path dataTrashRoot = new Path(targetTestRoot, - "data/" + TRASH_PREFIX + "/" + ugi.getShortUserName()); + Path dataTrashRoot = fsView2.makeQualified( + new Path("/data/" + TRASH_PREFIX + "/" + ugi.getShortUserName())); Assert.assertEquals(dataTrashRoot, fsView2.getTrashRoot(dataTestPath)); - // Case 2: path p not found in mount table, fall back to the default FS - // Return a trash root in user home dir + // Case 2: path p not found in mount table. + // Return a trash root in fallback FS. Path nonExistentPath = new Path("/nonExistentDir/nonExistentFile"); - Path userTrashRoot = new Path(fsTarget.getHomeDirectory(), TRASH_PREFIX); - Assert.assertEquals(userTrashRoot, fsView2.getTrashRoot(nonExistentPath)); + Path expectedTrash = + fsView2.makeQualified(getTrashRootInFallBackFS()); + Assert.assertEquals(expectedTrash, fsView2.getTrashRoot(nonExistentPath)); - // Case 3: turn off the CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH flag. + // Case 3: turn off the CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT flag. // Return a trash root in user home dir. - conf2.setBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, false); + conf2.setBoolean(CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT, false); fsView2 = FileSystem.get(FsConstants.VIEWFS_URI, conf2); - Assert.assertEquals(userTrashRoot, fsView2.getTrashRoot(dataTestPath)); + Path targetFSUserHomeTrashRoot = fsTarget.makeQualified( + new Path(fsTarget.getHomeDirectory(), TRASH_PREFIX)); + Assert.assertEquals(targetFSUserHomeTrashRoot, + fsView2.getTrashRoot(dataTestPath)); // Case 4: viewFS without fallback. Expect exception for a nonExistent path conf2 = new Configuration(conf); @@ -1141,30 +1152,14 @@ public void testTrashRootLocalizedTrash() throws IOException { fsView2.getTrashRoot(nonExistentPath); } catch (NotInMountpointException ignored) { } - - // Case 5: path p is in the same mount point as targetFS.getTrashRoot(). - // Return targetFS.getTrashRoot() - // Use a new Configuration object, so that we can start with an empty - // mount table. This would avoid a conflict between the /user link in - // setupMountPoints() and homeDir we will need to setup for this test. - // default homeDir for hdfs is /user/. - Configuration conf3 = ViewFileSystemTestSetup.createConfig(); - Path homeDir = fsTarget.getHomeDirectory(); - String homeParentDir = homeDir.getParent().toUri().getPath(); - conf3.setBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, true); - ConfigUtil.addLink(conf3, homeParentDir, - new Path(targetTestRoot, homeParentDir).toUri()); - Path homeTestPath = new Path(homeDir.toUri().getPath(), "testuser/file"); - FileSystem fsView3 = FileSystem.get(FsConstants.VIEWFS_URI, conf3); - Assert.assertEquals(userTrashRoot, fsView3.getTrashRoot(homeTestPath)); } /** * A mocked FileSystem which returns a deep trash dir. */ - static class MockTrashRootFS extends MockFileSystem { + static class DeepTrashRootMockFS extends MockFileSystem { public static final Path TRASH = - new Path("/mnt/very/deep/deep/trash/dir/.Trash"); + new Path("/vol/very/deep/deep/trash/dir/.Trash"); @Override public Path getTrashRoot(Path path) { @@ -1173,28 +1168,31 @@ public Path getTrashRoot(Path path) { } /** - * Test a trash root that is inside a mount point for getTrashRoot + * Test getTrashRoot that is very deep inside a mount point. */ @Test public void testTrashRootDeepTrashDir() throws IOException { Configuration conf2 = ViewFileSystemTestSetup.createConfig(); - conf2.setBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, true); - conf2.setClass("fs.mocktrashfs.impl", MockTrashRootFS.class, + conf2.setBoolean(CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT, true); + conf2.setClass("fs.mocktrashfs.impl", DeepTrashRootMockFS.class, FileSystem.class); - ConfigUtil.addLink(conf2, "/mnt", URI.create("mocktrashfs://mnt/path")); - Path testPath = new Path(MockTrashRootFS.TRASH, "projs/proj"); + ConfigUtil.addLink(conf2, "/mnt/datavol1", + URI.create("mocktrashfs://localhost/vol")); + Path testPath = new Path("/mnt/datavol1/projs/proj"); FileSystem fsView2 = FileSystem.get(FsConstants.VIEWFS_URI, conf2); - Assert.assertEquals(MockTrashRootFS.TRASH, fsView2.getTrashRoot(testPath)); + Path expectedTrash = fsView2.makeQualified( + new Path("/mnt/datavol1/very/deep/deep/trash/dir/.Trash")); + Assert.assertEquals(expectedTrash, fsView2.getTrashRoot(testPath)); } /** - * Test localized trash roots in getTrashRoots() for all users. + * Test getTrashRoots() for all users. */ @Test public void testTrashRootsAllUsers() throws IOException { Configuration conf2 = new Configuration(conf); - conf2.setBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, true); + conf2.setBoolean(CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT, true); FileSystem fsView2 = FileSystem.get(FsConstants.VIEWFS_URI, conf2); // Case 1: verify correct trash roots from fsView and fsView2 @@ -1230,17 +1228,27 @@ public void testTrashRootsAllUsers() throws IOException { FileSystem fsView4 = FileSystem.get(FsConstants.VIEWFS_URI, conf4); int trashRootsNum4 = fsView4.getTrashRoots(true).size(); Assert.assertEquals(afterTrashRootsNum2 + 2, trashRootsNum4); + + // Case 4: test trash roots in fallback FS + fsTarget.mkdirs(new Path(targetTestRoot, ".Trash/user10")); + fsTarget.mkdirs(new Path(targetTestRoot, ".Trash/user11")); + fsTarget.mkdirs(new Path(targetTestRoot, ".Trash/user12")); + Configuration conf5 = new Configuration(conf2); + ConfigUtil.addLinkFallback(conf5, targetTestRoot.toUri()); + FileSystem fsView5 = FileSystem.get(FsConstants.VIEWFS_URI, conf5); + int trashRootsNum5 = fsView5.getTrashRoots(true).size(); + Assert.assertEquals(afterTrashRootsNum2 + 3, trashRootsNum5); } /** - * Test localized trash roots in getTrashRoots() for current user. + * Test getTrashRoots() for current user. */ @Test public void testTrashRootsCurrentUser() throws IOException { String currentUser = UserGroupInformation.getCurrentUser().getShortUserName(); Configuration conf2 = new Configuration(conf); - conf2.setBoolean(CONFIG_VIEWFS_MOUNT_POINT_LOCAL_TRASH, true); + conf2.setBoolean(CONFIG_VIEWFS_TRASH_FORCE_INSIDE_MOUNT_POINT, true); FileSystem fsView2 = FileSystem.get(FsConstants.VIEWFS_URI, conf2); int beforeTrashRootNum = fsView.getTrashRoots(false).size(); @@ -1256,6 +1264,14 @@ public void testTrashRootsCurrentUser() throws IOException { int afterTrashRootsNum2 = fsView2.getTrashRoots(false).size(); Assert.assertEquals(beforeTrashRootNum, afterTrashRootsNum); Assert.assertEquals(beforeTrashRootNum2 + 2, afterTrashRootsNum2); + + // Test trash roots in fallback FS + Configuration conf3 = new Configuration(conf2); + fsTarget.mkdirs(new Path(targetTestRoot, TRASH_PREFIX + "/" + currentUser)); + ConfigUtil.addLinkFallback(conf3, targetTestRoot.toUri()); + FileSystem fsView3 = FileSystem.get(FsConstants.VIEWFS_URI, conf3); + int trashRootsNum3 = fsView3.getTrashRoots(false).size(); + Assert.assertEquals(afterTrashRootsNum2 + 1, trashRootsNum3); } @Test(expected = NotInMountpointException.class) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java index fdc746464f4e5..62dc3076d00a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java @@ -55,6 +55,7 @@ import org.apache.hadoop.test.GenericTestUtils; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY; +import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX; import org.junit.After; import org.junit.AfterClass; @@ -181,6 +182,13 @@ int getExpectedDelegationTokenCountWithCredentials() { return 2; } + @Override + Path getTrashRootInFallBackFS() throws IOException { + return new Path( + "/" + TRASH_PREFIX + "/" + UserGroupInformation.getCurrentUser() + .getShortUserName()); + } + @Test public void testTrashRootsAfterEncryptionZoneDeletion() throws Exception { final Path zone = new Path("/EZ"); From 1c0bc35305aea6ab8037241fab10862615f3e296 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 16 Mar 2022 07:02:29 +0530 Subject: [PATCH 037/145] HDFS-16502. Reconfigure Block Invalidate limit (#4064) Signed-off-by: Wei-Chiu Chuang --- .../blockmanagement/DatanodeManager.java | 33 ++++++++++++------- .../hadoop/hdfs/server/namenode/NameNode.java | 27 ++++++++++++++- .../namenode/TestNameNodeReconfigure.java | 31 +++++++++++++++++ .../hadoop/hdfs/tools/TestDFSAdmin.java | 21 +++++++----- 4 files changed, 91 insertions(+), 21 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index cfb1d83ec5bc6..cb601e94f822c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -314,18 +314,12 @@ public class DatanodeManager { DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT); // 5 minutes this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval + 10 * 1000 * heartbeatIntervalSeconds; - // Effected block invalidate limit is the bigger value between - // value configured in hdfs-site.xml, and 20 * HB interval. final int configuredBlockInvalidateLimit = conf.getInt( DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY, DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT); - final int countedBlockInvalidateLimit = 20*(int)(heartbeatIntervalSeconds); - this.blockInvalidateLimit = Math.max(countedBlockInvalidateLimit, - configuredBlockInvalidateLimit); - LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY - + ": configured=" + configuredBlockInvalidateLimit - + ", counted=" + countedBlockInvalidateLimit - + ", effected=" + blockInvalidateLimit); + // Block invalidate limit also has some dependency on heartbeat interval. + // Check setBlockInvalidateLimit(). + setBlockInvalidateLimit(configuredBlockInvalidateLimit); this.checkIpHostnameInRegistration = conf.getBoolean( DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY, DFSConfigKeys.DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT); @@ -2088,8 +2082,25 @@ private void setHeartbeatInterval(long intervalSeconds, this.heartbeatRecheckInterval = recheckInterval; this.heartbeatExpireInterval = 2L * recheckInterval + 10 * 1000 * intervalSeconds; - this.blockInvalidateLimit = Math.max(20 * (int) (intervalSeconds), - blockInvalidateLimit); + this.blockInvalidateLimit = getBlockInvalidateLimit(blockInvalidateLimit); + } + + private int getBlockInvalidateLimitFromHBInterval() { + return 20 * (int) heartbeatIntervalSeconds; + } + + private int getBlockInvalidateLimit(int configuredBlockInvalidateLimit) { + return Math.max(getBlockInvalidateLimitFromHBInterval(), configuredBlockInvalidateLimit); + } + + public void setBlockInvalidateLimit(int configuredBlockInvalidateLimit) { + final int countedBlockInvalidateLimit = getBlockInvalidateLimitFromHBInterval(); + // Effected block invalidate limit is the bigger value between + // value configured in hdfs-site.xml, and 20 * HB interval. + this.blockInvalidateLimit = getBlockInvalidateLimit(configuredBlockInvalidateLimit); + LOG.info("{} : configured={}, counted={}, effected={}", + DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY, configuredBlockInvalidateLimit, + countedBlockInvalidateLimit, this.blockInvalidateLimit); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index 8cd5d25247343..ef0eef8510cd2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -121,6 +121,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NN_NOT_BECOME_ACTIVE_IN_SAFEMODE; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NN_NOT_BECOME_ACTIVE_IN_SAFEMODE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_DEFAULT; @@ -337,7 +338,8 @@ public enum OperationCategory { DFS_IMAGE_PARALLEL_LOAD_KEY, DFS_NAMENODE_AVOID_SLOW_DATANODE_FOR_READ_KEY, DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY, - DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY)); + DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, + DFS_BLOCK_INVALIDATE_LIMIT_KEY)); private static final String USAGE = "Usage: hdfs namenode [" + StartupOption.BACKUP.getName() + "] | \n\t[" @@ -2210,6 +2212,8 @@ protected String reconfigurePropertyImpl(String property, String newVal) || (property.equals(DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY)) || (property.equals(DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY))) { return reconfigureSlowNodesParameters(datanodeManager, property, newVal); + } else if (property.equals(DFS_BLOCK_INVALIDATE_LIMIT_KEY)) { + return reconfigureBlockInvalidateLimit(datanodeManager, property, newVal); } else { throw new ReconfigurationException(property, newVal, getConf().get( property)); @@ -2434,6 +2438,27 @@ String reconfigureSlowNodesParameters(final DatanodeManager datanodeManager, } } + private String reconfigureBlockInvalidateLimit(final DatanodeManager datanodeManager, + final String property, final String newVal) throws ReconfigurationException { + namesystem.writeLock(); + try { + if (newVal == null) { + datanodeManager.setBlockInvalidateLimit(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT); + } else { + datanodeManager.setBlockInvalidateLimit(Integer.parseInt(newVal)); + } + final String updatedBlockInvalidateLimit = + String.valueOf(datanodeManager.getBlockInvalidateLimit()); + LOG.info("RECONFIGURE* changed blockInvalidateLimit to {}", updatedBlockInvalidateLimit); + return updatedBlockInvalidateLimit; + } catch (NumberFormatException e) { + throw new ReconfigurationException(property, newVal, getConf().get(property), e); + } finally { + namesystem.writeUnlock(); + } + } + + @Override // ReconfigurableBase protected Configuration getNewConf() { return new HdfsConfiguration(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java index 33debdb14923f..2f06918c3e167 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java @@ -453,6 +453,37 @@ public void testReconfigureMaxSlowpeerCollectNodes() assertEquals(10, datanodeManager.getMaxSlowpeerCollectNodes()); } + @Test + public void testBlockInvalidateLimit() throws ReconfigurationException { + final NameNode nameNode = cluster.getNameNode(); + final DatanodeManager datanodeManager = nameNode.namesystem + .getBlockManager().getDatanodeManager(); + + assertEquals(DFS_BLOCK_INVALIDATE_LIMIT_KEY + " is not correctly set", + customizedBlockInvalidateLimit, datanodeManager.getBlockInvalidateLimit()); + + try { + nameNode.reconfigureProperty(DFS_BLOCK_INVALIDATE_LIMIT_KEY, "non-numeric"); + fail("Should not reach here"); + } catch (ReconfigurationException e) { + assertEquals( + "Could not change property dfs.block.invalidate.limit from '500' to 'non-numeric'", + e.getMessage()); + } + + nameNode.reconfigureProperty(DFS_BLOCK_INVALIDATE_LIMIT_KEY, "2500"); + + assertEquals(DFS_BLOCK_INVALIDATE_LIMIT_KEY + " is not honored after reconfiguration", 2500, + datanodeManager.getBlockInvalidateLimit()); + + nameNode.reconfigureProperty(DFS_HEARTBEAT_INTERVAL_KEY, "500"); + + // 20 * 500 (10000) > 2500 + // Hence, invalid block limit should be reset to 10000 + assertEquals(DFS_BLOCK_INVALIDATE_LIMIT_KEY + " is not reconfigured correctly", 10000, + datanodeManager.getBlockInvalidateLimit()); + } + @After public void shutDown() throws IOException { if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java index 78598760df42c..0f8c4cdc8a4ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java @@ -31,6 +31,7 @@ import java.util.function.Supplier; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY; @@ -433,15 +434,17 @@ public void testNameNodeGetReconfigurableProperties() throws IOException { final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("namenode", address, outs, errs); - assertEquals(16, outs.size()); - assertEquals(DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY, outs.get(1)); - assertEquals(DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, outs.get(2)); - assertEquals(DFS_HEARTBEAT_INTERVAL_KEY, outs.get(3)); - assertEquals(DFS_IMAGE_PARALLEL_LOAD_KEY, outs.get(4)); - assertEquals(DFS_NAMENODE_AVOID_SLOW_DATANODE_FOR_READ_KEY, outs.get(5)); - assertEquals(DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY, outs.get(6)); - assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(7)); - assertEquals(DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, outs.get(8)); + assertEquals(17, outs.size()); + assertTrue(outs.get(0).contains("Reconfigurable properties:")); + assertEquals(DFS_BLOCK_INVALIDATE_LIMIT_KEY, outs.get(1)); + assertEquals(DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY, outs.get(2)); + assertEquals(DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, outs.get(3)); + assertEquals(DFS_HEARTBEAT_INTERVAL_KEY, outs.get(4)); + assertEquals(DFS_IMAGE_PARALLEL_LOAD_KEY, outs.get(5)); + assertEquals(DFS_NAMENODE_AVOID_SLOW_DATANODE_FOR_READ_KEY, outs.get(6)); + assertEquals(DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY, outs.get(7)); + assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(8)); + assertEquals(DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, outs.get(9)); assertEquals(errs.size(), 0); } From a237526988b8d69beccd837a3e9c238cfeb6f5db Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Wed, 16 Mar 2022 20:43:38 +0800 Subject: [PATCH 038/145] HDFS-16494.Removed reuse of AvailableSpaceVolumeChoosingPolicy#initLocks(). (#4048). Contributed by JiangHua Zhu. --- .../datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java index 72ed47c67fa93..5d12fa72bb165 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java @@ -63,7 +63,6 @@ public class AvailableSpaceVolumeChoosingPolicy public AvailableSpaceVolumeChoosingPolicy() { this(new Random()); - initLocks(); } private void initLocks() { From 7f6a891f0313528b566de3c97b95e1bf2e4e4421 Mon Sep 17 00:00:00 2001 From: litao Date: Thu, 17 Mar 2022 12:04:48 +0800 Subject: [PATCH 039/145] HDFS-16499. [SPS]: Should not start indefinitely while another SPS process is running (#4058) --- .../sps/ExternalStoragePolicySatisfier.java | 12 ++++-- .../TestExternalStoragePolicySatisfier.java | 43 ++++++++++++++++++- 2 files changed, 51 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java index 15cdc6eb47bcf..efbee7c030f98 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalStoragePolicySatisfier.java @@ -38,6 +38,7 @@ import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,20 +97,25 @@ private static void secureLogin(Configuration conf) socAddr.getHostName()); } - private static NameNodeConnector getNameNodeConnector(Configuration conf) - throws IOException, InterruptedException { + public static NameNodeConnector getNameNodeConnector(Configuration conf) + throws InterruptedException { final Collection namenodes = DFSUtil.getInternalNsRpcUris(conf); final Path externalSPSPathId = HdfsServerConstants.MOVER_ID_PATH; + String serverName = ExternalStoragePolicySatisfier.class.getSimpleName(); while (true) { try { final List nncs = NameNodeConnector .newNameNodeConnectors(namenodes, - ExternalStoragePolicySatisfier.class.getSimpleName(), + serverName, externalSPSPathId, conf, NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS); return nncs.get(0); } catch (IOException e) { LOG.warn("Failed to connect with namenode", e); + if (e.getMessage().equals("Another " + serverName + " is running.")) { + ExitUtil.terminate(-1, + "Exit immediately because another " + serverName + " is running"); + } Thread.sleep(3000); // retry the connection after few secs } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java index 77922a09912e2..361d61d54e258 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java @@ -90,6 +90,8 @@ import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils.LogCapturer; +import org.apache.hadoop.test.LambdaTestUtils; +import org.apache.hadoop.util.ExitUtil; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -197,9 +199,24 @@ private void createCluster() throws IOException { writeContent(FILE); } + private void createCluster(boolean createMoverPath) throws IOException { + getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE); + setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES, + STORAGES_PER_DATANODE, CAPACITY, createMoverPath)); + getFS(); + writeContent(FILE); + } + private MiniDFSCluster startCluster(final Configuration conf, StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn, long nodeCapacity) throws IOException { + return startCluster(conf, storageTypes, numberOfDatanodes, storagesPerDn, + nodeCapacity, false); + } + + private MiniDFSCluster startCluster(final Configuration conf, + StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn, + long nodeCapacity, boolean createMoverPath) throws IOException { long[][] capacities = new long[numberOfDatanodes][storagesPerDn]; for (int i = 0; i < numberOfDatanodes; i++) { for (int j = 0; j < storagesPerDn; j++) { @@ -212,7 +229,7 @@ private MiniDFSCluster startCluster(final Configuration conf, cluster.waitActive(); nnc = DFSTestUtil.getNameNodeConnector(getConf(), - HdfsServerConstants.MOVER_ID_PATH, 1, false); + HdfsServerConstants.MOVER_ID_PATH, 1, createMoverPath); externalSps = new StoragePolicySatisfier(getConf()); externalCtxt = new ExternalSPSContext(externalSps, nnc); @@ -428,6 +445,30 @@ public void testWhenStoragePolicySetToCOLD() } } + @Test(timeout = 300000) + public void testInfiniteStartWhenAnotherSPSRunning() + throws Exception { + + try { + // Create cluster and create mover path when get NameNodeConnector. + createCluster(true); + + // Disable system exit for assert. + ExitUtil.disableSystemExit(); + + // Get NameNodeConnector one more time to simulate starting other sps process. + // It should exit immediately when another sps is running. + LambdaTestUtils.intercept(ExitUtil.ExitException.class, + "Exit immediately because another ExternalStoragePolicySatisfier is running", + () -> ExternalStoragePolicySatisfier.getNameNodeConnector(config)); + } finally { + // Reset first exit exception to avoid AssertionError in MiniDFSCluster#shutdown. + // This has no effect on functionality. + ExitUtil.resetFirstExitException(); + shutdownCluster(); + } + } + private void doTestWhenStoragePolicySetToCOLD() throws Exception { // Change policy to COLD dfs.setStoragePolicy(new Path(FILE), COLD); From 9037f9a334978aea9ff288547637e0038c63894f Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 16 Mar 2022 15:12:03 +0000 Subject: [PATCH 040/145] HADOOP-18162. hadoop-common support for MAPREDUCE-7341 Manifest Committer * New statistic names in StoreStatisticNames (for joint use with s3a committers) * Improvements to IOStatistics implementation classes * RateLimiting wrapper to guava RateLimiter * S3A committer Tasks moved over as TaskPool and added support for RemoteIterator * JsonSerialization.load() to fail fast if source does not exist + tests. This commit is a prerequisite for the main MAPREDUCE-7341 Manifest Committer patch. Contributed by Steve Loughran Change-Id: Ia92e2ab5083ac3d8d3d713a4d9cb3e9e0278f654 --- .../fs/statistics/StoreStatisticNames.java | 6 + .../statistics/impl/IOStatisticsBinding.java | 29 +- .../fs/statistics/impl/IOStatisticsStore.java | 11 + .../impl/PairedDurationTrackerFactory.java | 5 + .../impl/StatisticDurationTracker.java | 7 + .../apache/hadoop/util/JsonSerialization.java | 5 +- .../org/apache/hadoop/util/RateLimiting.java | 52 ++ .../hadoop/util/RateLimitingFactory.java | 102 +++ .../CloseableTaskPoolSubmitter.java | 74 +++ .../hadoop/util/functional/TaskPool.java | 613 ++++++++++++++++++ .../hadoop/util/functional/TestTaskPool.java | 585 +++++++++++++++++ 11 files changed, 1487 insertions(+), 2 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimiting.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimitingFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/TaskPool.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestTaskPool.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index 9ec8dcdb3dc9b..166007f5c9a42 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -112,6 +112,9 @@ public final class StoreStatisticNames { /** {@value}. */ public static final String OP_MODIFY_ACL_ENTRIES = "op_modify_acl_entries"; + /** {@value}. */ + public static final String OP_MSYNC = "op_msync"; + /** {@value}. */ public static final String OP_OPEN = "op_open"; @@ -172,6 +175,9 @@ public final class StoreStatisticNames { public static final String STORE_IO_THROTTLED = "store_io_throttled"; + /** Rate limiting was reported {@value}. */ + public static final String STORE_IO_RATE_LIMITED = "store_io_rate_limited"; + /** Requests made of a store: {@value}. */ public static final String STORE_IO_REQUEST = "store_io_request"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java index c2eab9d772a66..70366fc7a3057 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -21,6 +21,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.io.Serializable; +import java.time.Duration; import java.util.Iterator; import java.util.Map; import java.util.concurrent.Callable; @@ -450,12 +451,37 @@ public static B trackDuration( * @param factory factory of duration trackers * @param statistic statistic key * @param input input callable. + * @throws IOException IO failure. */ public static void trackDurationOfInvocation( DurationTrackerFactory factory, String statistic, InvocationRaisingIOE input) throws IOException { + measureDurationOfInvocation(factory, statistic, input); + } + + /** + * Given an IOException raising callable/lambda expression, + * execute it and update the relevant statistic, + * returning the measured duration. + * + * {@link #trackDurationOfInvocation(DurationTrackerFactory, String, InvocationRaisingIOE)} + * with the duration returned for logging etc.; added as a new + * method to avoid linking problems with any code calling the existing + * method. + * + * @param factory factory of duration trackers + * @param statistic statistic key + * @param input input callable. + * @return the duration of the operation, as measured by the duration tracker. + * @throws IOException IO failure. + */ + public static Duration measureDurationOfInvocation( + DurationTrackerFactory factory, + String statistic, + InvocationRaisingIOE input) throws IOException { + // create the tracker outside try-with-resources so // that failures can be set in the catcher. DurationTracker tracker = createTracker(factory, statistic); @@ -473,6 +499,7 @@ public static void trackDurationOfInvocation( // set the failed flag. tracker.close(); } + return tracker.asDuration(); } /** @@ -622,7 +649,7 @@ public static B trackDurationOfSupplier( * @param statistic statistic to track * @return a duration tracker. */ - private static DurationTracker createTracker( + public static DurationTracker createTracker( @Nullable final DurationTrackerFactory factory, final String statistic) { return factory != null diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java index 1b4139e463a9e..c083ad8c3c2ed 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsStore.java @@ -255,4 +255,15 @@ default long incrementCounter(String key) { */ void addTimedOperation(String prefix, Duration duration); + /** + * Add a statistics sample as a min, max and mean and count. + * @param key key to add. + * @param count count. + */ + default void addSample(String key, long count) { + incrementCounter(key, count); + addMeanStatisticSample(key, count); + addMaximumSample(key, count); + addMinimumSample(key, count); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java index 33b13f78418a9..9bc01338a1497 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/PairedDurationTrackerFactory.java @@ -88,6 +88,11 @@ public void close() { public Duration asDuration() { return firstDuration.asDuration(); } + + @Override + public String toString() { + return firstDuration.toString(); + } } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java index ef9e7cb107a0d..04d30135f6bd3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StatisticDurationTracker.java @@ -103,4 +103,11 @@ public void close() { } iostats.addTimedOperation(name, asDuration()); } + + @Override + public String toString() { + return " Duration of " + + (failed? (key + StoreStatisticNames.SUFFIX_FAILURES) : key) + + ": " + super.toString(); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java index 2b794f9c38245..d2bc8cd2960ca 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java @@ -172,6 +172,9 @@ public synchronized T fromJsonStream(InputStream stream) throws IOException { @SuppressWarnings("unchecked") public synchronized T load(File jsonFile) throws IOException, JsonParseException, JsonMappingException { + if (!jsonFile.exists()) { + throw new FileNotFoundException("No such file: " + jsonFile); + } if (!jsonFile.isFile()) { throw new FileNotFoundException("Not a file: " + jsonFile); } @@ -181,7 +184,7 @@ public synchronized T load(File jsonFile) try { return mapper.readValue(jsonFile, classType); } catch (IOException e) { - LOG.error("Exception while parsing json file {}", jsonFile, e); + LOG.warn("Exception while parsing json file {}", jsonFile, e); throw e; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimiting.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimiting.java new file mode 100644 index 0000000000000..ae119c0e630f4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimiting.java @@ -0,0 +1,52 @@ +/* + * 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.hadoop.util; + +import java.time.Duration; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Minimal subset of google rate limiter class. + * Can be used to throttle use of object stores where excess load + * will trigger cluster-wide throttling, backoff etc. and so collapse + * performance. + * The time waited is returned as a Duration type. + * The google rate limiter implements this by allowing a caller to ask for + * more capacity than is available. This will be granted + * but the subsequent request will be blocked if the bucket of + * capacity hasn't let refilled to the point where there is + * capacity again. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface RateLimiting { + + /** + * Acquire rate limiter capacity. + * If there is not enough space, the permits will be acquired, + * but the subsequent call will block until the capacity has been + * refilled. + * @param requestedCapacity capacity to acquire. + * @return time spent waiting for output. + */ + Duration acquire(int requestedCapacity); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimitingFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimitingFactory.java new file mode 100644 index 0000000000000..621415456e125 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RateLimitingFactory.java @@ -0,0 +1,102 @@ +/* + * 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.hadoop.util; + +import java.time.Duration; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.RateLimiter; + +/** + * Factory for Rate Limiting. + * This should be only place in the code where the guava RateLimiter is imported. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class RateLimitingFactory { + + private static final RateLimiting UNLIMITED = new NoRateLimiting(); + + /** + * No waiting took place. + */ + private static final Duration INSTANTLY = Duration.ofMillis(0); + + private RateLimitingFactory() { + } + + /** + * No Rate Limiting. + */ + private static class NoRateLimiting implements RateLimiting { + + + @Override + public Duration acquire(int requestedCapacity) { + return INSTANTLY; + } + } + + /** + * Rate limiting restricted to that of a google rate limiter. + */ + private static final class RestrictedRateLimiting implements RateLimiting { + private final RateLimiter limiter; + + /** + * Constructor. + * @param capacityPerSecond capacity in permits/second. + */ + private RestrictedRateLimiting(int capacityPerSecond) { + this.limiter = RateLimiter.create(capacityPerSecond); + } + + @Override + public Duration acquire(int requestedCapacity) { + final double delayMillis = limiter.acquire(requestedCapacity); + return delayMillis == 0 + ? INSTANTLY + : Duration.ofMillis((long) (delayMillis * 1000)); + } + + } + + /** + * Get the unlimited rate. + * @return a rate limiter which always has capacity. + */ + public static RateLimiting unlimitedRate() { + return UNLIMITED; + } + + /** + * Create an instance. + * If the rate is 0; return the unlimited rate. + * @param capacity capacity in permits/second. + * @return limiter restricted to the given capacity. + */ + public static RateLimiting create(int capacity) { + + return capacity == 0 + ? unlimitedRate() + : new RestrictedRateLimiting(capacity); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java new file mode 100644 index 0000000000000..26b687a3c5610 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java @@ -0,0 +1,74 @@ +/* + * 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.hadoop.util.functional; + +import java.io.Closeable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import static java.util.Objects.requireNonNull; + +/** + * A task submitter which is closeable, and whose close() call + * shuts down the pool. This can help manage + * thread pool lifecycles. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class CloseableTaskPoolSubmitter implements TaskPool.Submitter, + Closeable { + + /** Executors. */ + private ExecutorService pool; + + /** + * Constructor. + * @param pool non-null executor. + */ + public CloseableTaskPoolSubmitter(final ExecutorService pool) { + this.pool = requireNonNull(pool); + } + + /** + * Get the pool. + * @return the pool. + */ + public ExecutorService getPool() { + return pool; + } + + /** + * Shut down the pool. + */ + @Override + public void close() { + if (pool != null) { + pool.shutdown(); + pool = null; + } + } + + @Override + public Future submit(final Runnable task) { + return pool.submit(task); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/TaskPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/TaskPool.java new file mode 100644 index 0000000000000..0abaab211de04 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/TaskPool.java @@ -0,0 +1,613 @@ +/* + * 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.hadoop.util.functional; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.RemoteIterator; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.util.functional.RemoteIterators.remoteIteratorFromIterable; + +/** + * Utility class for parallel execution, takes closures for the various + * actions. + * There is no retry logic: it is expected to be handled by the closures. + * From {@code org.apache.hadoop.fs.s3a.commit.Tasks} which came from + * the Netflix committer patch. + * Apache Iceberg has its own version of this, with a common ancestor + * at some point in its history. + * A key difference with this class is that the iterator is always, + * internally, an {@link RemoteIterator}. + * This is to allow tasks to be scheduled while incremental operations + * such as paged directory listings are still collecting in results. + * + * While awaiting completion, this thread spins and sleeps a time of + * {@link #SLEEP_INTERVAL_AWAITING_COMPLETION}, which, being a + * busy-wait, is inefficient. + * There's an implicit assumption that remote IO is being performed, and + * so this is not impacting throughput/performance. + * + * History: + * This class came with the Netflix contributions to the S3A committers + * in HADOOP-13786. + * It was moved into hadoop-common for use in the manifest committer and + * anywhere else it is needed, and renamed in the process as + * "Tasks" has too many meanings in the hadoop source. + * The iterator was then changed from a normal java iterable + * to a hadoop {@link org.apache.hadoop.fs.RemoteIterator}. + * This allows a task pool to be supplied with incremental listings + * from object stores, scheduling work as pages of listing + * results come in, rather than blocking until the entire + * directory/directory tree etc has been enumerated. + * + * There is a variant of this in Apache Iceberg in + * {@code org.apache.iceberg.util.Tasks} + * That is not derived from any version in the hadoop codebase, it + * just shares a common ancestor somewhere in the Netflix codebase. + * It is the more sophisticated version. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class TaskPool { + private static final Logger LOG = + LoggerFactory.getLogger(TaskPool.class); + + /** + * Interval in milliseconds to await completion. + */ + private static final int SLEEP_INTERVAL_AWAITING_COMPLETION = 10; + + private TaskPool() { + } + + /** + * Callback invoked to process an item. + * @param item type being processed + * @param exception class which may be raised + */ + @FunctionalInterface + public interface Task { + void run(I item) throws E; + } + + /** + * Callback invoked on a failure. + * @param item type being processed + * @param exception class which may be raised + */ + @FunctionalInterface + public interface FailureTask { + + /** + * process a failure. + * @param item item the task is processing + * @param exception the exception which was raised. + * @throws E Exception of type E + */ + void run(I item, Exception exception) throws E; + } + + /** + * Builder for task execution. + * @param item type + */ + public static class Builder { + private final RemoteIterator items; + private Submitter service = null; + private FailureTask onFailure = null; + private boolean stopOnFailure = false; + private boolean suppressExceptions = false; + private Task revertTask = null; + private boolean stopRevertsOnFailure = false; + private Task abortTask = null; + private boolean stopAbortsOnFailure = false; + private int sleepInterval = SLEEP_INTERVAL_AWAITING_COMPLETION; + + /** + * Create the builder. + * @param items items to process + */ + Builder(RemoteIterator items) { + this.items = requireNonNull(items, "items"); + } + + /** + * Create the builder. + * @param items items to process + */ + Builder(Iterable items) { + this(remoteIteratorFromIterable(items)); + } + + /** + * Declare executor service: if null, the tasks are executed in a single + * thread. + * @param submitter service to schedule tasks with. + * @return this builder. + */ + public Builder executeWith(@Nullable Submitter submitter) { + + this.service = submitter; + return this; + } + + /** + * Task to invoke on failure. + * @param task task + * @return the builder + */ + public Builder onFailure(FailureTask task) { + this.onFailure = task; + return this; + } + + public Builder stopOnFailure() { + this.stopOnFailure = true; + return this; + } + + /** + * Suppress exceptions from tasks. + * RemoteIterator exceptions are not suppressable. + * @return the builder. + */ + public Builder suppressExceptions() { + return suppressExceptions(true); + } + + /** + * Suppress exceptions from tasks. + * RemoteIterator exceptions are not suppressable. + * @param suppress new value + * @return the builder. + */ + public Builder suppressExceptions(boolean suppress) { + this.suppressExceptions = suppress; + return this; + } + + /** + * Task to revert with after another task failed. + * @param task task to execute + * @return the builder + */ + public Builder revertWith(Task task) { + this.revertTask = task; + return this; + } + + /** + * Stop trying to revert if one operation fails. + * @return the builder + */ + public Builder stopRevertsOnFailure() { + this.stopRevertsOnFailure = true; + return this; + } + + /** + * Task to abort with after another task failed. + * @param task task to execute + * @return the builder + */ + public Builder abortWith(Task task) { + this.abortTask = task; + return this; + } + + /** + * Stop trying to abort if one operation fails. + * @return the builder + */ + public Builder stopAbortsOnFailure() { + this.stopAbortsOnFailure = true; + return this; + } + + /** + * Set the sleep interval. + * @param value new value + * @return the builder + */ + public Builder sleepInterval(final int value) { + sleepInterval = value; + return this; + } + + /** + * Execute the task across the data. + * @param task task to execute + * @param exception which may be raised in execution. + * @return true if the operation executed successfully + * @throws E any exception raised. + * @throws IOException IOExceptions raised by remote iterator or in execution. + */ + public boolean run(Task task) throws E, IOException { + requireNonNull(items, "items"); + if (!items.hasNext()) { + // if there are no items, return without worrying about + // execution pools, errors etc. + return true; + } + if (service != null) { + // thread pool, so run in parallel + return runParallel(task); + } else { + // single threaded execution. + return runSingleThreaded(task); + } + } + + /** + * Single threaded execution. + * @param task task to execute + * @param exception which may be raised in execution. + * @return true if the operation executed successfully + * @throws E any exception raised. + * @throws IOException IOExceptions raised by remote iterator or in execution. + */ + private boolean runSingleThreaded(Task task) + throws E, IOException { + List succeeded = new ArrayList<>(); + List exceptions = new ArrayList<>(); + + RemoteIterator iterator = items; + boolean threw = true; + try { + while (iterator.hasNext()) { + I item = iterator.next(); + try { + task.run(item); + succeeded.add(item); + + } catch (Exception e) { + exceptions.add(e); + + if (onFailure != null) { + try { + onFailure.run(item, e); + } catch (Exception failException) { + LOG.error("Failed to clean up on failure", e); + // keep going + } + } + + if (stopOnFailure) { + break; + } + } + } + + threw = false; + } catch (IOException iteratorIOE) { + // an IOE is reaised here during iteration + LOG.debug("IOException when iterating through {}", iterator, iteratorIOE); + throw iteratorIOE; + } finally { + // threw handles exceptions that were *not* caught by the catch block, + // and exceptions that were caught and possibly handled by onFailure + // are kept in exceptions. + if (threw || !exceptions.isEmpty()) { + if (revertTask != null) { + boolean failed = false; + for (I item : succeeded) { + try { + revertTask.run(item); + } catch (Exception e) { + LOG.error("Failed to revert task", e); + failed = true; + // keep going + } + if (stopRevertsOnFailure && failed) { + break; + } + } + } + + if (abortTask != null) { + boolean failed = false; + while (iterator.hasNext()) { + try { + abortTask.run(iterator.next()); + } catch (Exception e) { + failed = true; + LOG.error("Failed to abort task", e); + // keep going + } + if (stopAbortsOnFailure && failed) { + break; + } + } + } + } + } + + if (!suppressExceptions && !exceptions.isEmpty()) { + TaskPool.throwOne(exceptions); + } + + return exceptions.isEmpty(); + } + + /** + * Parallel execution. + * @param task task to execute + * @param exception which may be raised in execution. + * @return true if the operation executed successfully + * @throws E any exception raised. + * @throws IOException IOExceptions raised by remote iterator or in execution. + */ + private boolean runParallel(final Task task) + throws E, IOException { + final Queue succeeded = new ConcurrentLinkedQueue<>(); + final Queue exceptions = new ConcurrentLinkedQueue<>(); + final AtomicBoolean taskFailed = new AtomicBoolean(false); + final AtomicBoolean abortFailed = new AtomicBoolean(false); + final AtomicBoolean revertFailed = new AtomicBoolean(false); + + List> futures = new ArrayList<>(); + + IOException iteratorIOE = null; + final RemoteIterator iterator = this.items; + try { + while(iterator.hasNext()) { + final I item = iterator.next(); + // submit a task for each item that will either run or abort the task + futures.add(service.submit(() -> { + if (!(stopOnFailure && taskFailed.get())) { + // run the task + boolean threw = true; + try { + LOG.debug("Executing task"); + task.run(item); + succeeded.add(item); + LOG.debug("Task succeeded"); + + threw = false; + + } catch (Exception e) { + taskFailed.set(true); + exceptions.add(e); + LOG.info("Task failed {}", e.toString()); + LOG.debug("Task failed", e); + + if (onFailure != null) { + try { + onFailure.run(item, e); + } catch (Exception failException) { + LOG.warn("Failed to clean up on failure", e); + // swallow the exception + } + } + } finally { + if (threw) { + taskFailed.set(true); + } + } + + } else if (abortTask != null) { + // abort the task instead of running it + if (stopAbortsOnFailure && abortFailed.get()) { + return; + } + + boolean failed = true; + try { + LOG.info("Aborting task"); + abortTask.run(item); + failed = false; + } catch (Exception e) { + LOG.error("Failed to abort task", e); + // swallow the exception + } finally { + if (failed) { + abortFailed.set(true); + } + } + } + })); + } + } catch (IOException e) { + // iterator failure. + LOG.debug("IOException when iterating through {}", iterator, e); + iteratorIOE = e; + // mark as a task failure so all submitted tasks will halt/abort + taskFailed.set(true); + } + + // let the above tasks complete (or abort) + waitFor(futures, sleepInterval); + int futureCount = futures.size(); + futures.clear(); + + if (taskFailed.get() && revertTask != null) { + // at least one task failed, revert any that succeeded + LOG.info("Reverting all {} succeeded tasks from {} futures", + succeeded.size(), futureCount); + for (final I item : succeeded) { + futures.add(service.submit(() -> { + if (stopRevertsOnFailure && revertFailed.get()) { + return; + } + + boolean failed = true; + try { + revertTask.run(item); + failed = false; + } catch (Exception e) { + LOG.error("Failed to revert task", e); + // swallow the exception + } finally { + if (failed) { + revertFailed.set(true); + } + } + })); + } + + // let the revert tasks complete + waitFor(futures, sleepInterval); + } + + // give priority to execution exceptions over + // iterator exceptions. + if (!suppressExceptions && !exceptions.isEmpty()) { + // there's an exception list to build up, cast and throw. + TaskPool.throwOne(exceptions); + } + + // raise any iterator exception. + // this can not be suppressed. + if (iteratorIOE != null) { + throw iteratorIOE; + } + + // return true if all tasks succeeded. + return !taskFailed.get(); + } + } + + /** + * Wait for all the futures to complete; there's a small sleep between + * each iteration; enough to yield the CPU. + * @param futures futures. + * @param sleepInterval Interval in milliseconds to await completion. + */ + private static void waitFor(Collection> futures, int sleepInterval) { + int size = futures.size(); + LOG.debug("Waiting for {} tasks to complete", size); + int oldNumFinished = 0; + while (true) { + int numFinished = (int) futures.stream().filter(Future::isDone).count(); + + if (oldNumFinished != numFinished) { + LOG.debug("Finished count -> {}/{}", numFinished, size); + oldNumFinished = numFinished; + } + + if (numFinished == size) { + // all of the futures are done, stop looping + break; + } else { + try { + Thread.sleep(sleepInterval); + } catch (InterruptedException e) { + futures.forEach(future -> future.cancel(true)); + Thread.currentThread().interrupt(); + break; + } + } + } + } + + /** + * Create a task builder for the iterable. + * @param items item source. + * @param type of result. + * @return builder. + */ + public static Builder foreach(Iterable items) { + return new Builder<>(requireNonNull(items, "items")); + } + + /** + * Create a task builder for the remote iterator. + * @param items item source. + * @param type of result. + * @return builder. + */ + public static Builder foreach(RemoteIterator items) { + return new Builder<>(items); + } + + public static Builder foreach(I[] items) { + return new Builder<>(Arrays.asList(requireNonNull(items, "items"))); + } + + /** + * Throw one exception, adding the others as suppressed + * exceptions attached to the one thrown. + * This method never completes normally. + * @param exceptions collection of exceptions + * @param class of exceptions + * @throws E an extracted exception. + */ + private static void throwOne( + Collection exceptions) + throws E { + Iterator iter = exceptions.iterator(); + Exception e = iter.next(); + Class exceptionClass = e.getClass(); + + while (iter.hasNext()) { + Exception other = iter.next(); + if (!exceptionClass.isInstance(other)) { + e.addSuppressed(other); + } + } + + TaskPool.castAndThrow(e); + } + + /** + * Raise an exception of the declared type. + * This method never completes normally. + * @param e exception + * @param class of exceptions + * @throws E a recast exception. + */ + @SuppressWarnings("unchecked") + private static void castAndThrow(Exception e) throws E { + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } + throw (E) e; + } + + /** + * Interface to whatever lets us submit tasks. + */ + public interface Submitter { + + /** + * Submit work. + * @param task task to execute + * @return the future of the submitted task. + */ + Future submit(Runnable task); + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestTaskPool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestTaskPool.java new file mode 100644 index 0000000000000..dfee6fc75dcb3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/functional/TestTaskPool.java @@ -0,0 +1,585 @@ +/* + * 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.hadoop.util.functional; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.test.HadoopTestBase; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test Task Pool class. + * This is pulled straight out of the S3A version. + */ +@RunWith(Parameterized.class) +public class TestTaskPool extends HadoopTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestTaskPool.class); + + public static final int ITEM_COUNT = 16; + + private static final int FAILPOINT = 8; + + private final int numThreads; + + /** + * Thread pool for task execution. + */ + private ExecutorService threadPool; + + /** + * Task submitter bonded to the thread pool, or + * null for the 0-thread case. + */ + private TaskPool.Submitter submitter; + + private final CounterTask failingTask + = new CounterTask("failing committer", FAILPOINT, Item::commit); + + private final FailureCounter failures + = new FailureCounter("failures", 0, null); + + private final CounterTask reverter + = new CounterTask("reverter", 0, Item::revert); + + private final CounterTask aborter + = new CounterTask("aborter", 0, Item::abort); + + /** + * Test array for parameterized test runs: how many threads and + * to use. Threading makes some of the assertions brittle; there are + * more checks on single thread than parallel ops. + * @return a list of parameter tuples. + */ + @Parameterized.Parameters(name = "threads={0}") + public static Collection params() { + return Arrays.asList(new Object[][]{ + {0}, + {1}, + {3}, + {8}, + {16}, + }); + } + + private List items; + + /** + * Construct the parameterized test. + * @param numThreads number of threads + */ + public TestTaskPool(int numThreads) { + this.numThreads = numThreads; + } + + /** + * In a parallel test run there is more than one thread doing the execution. + * @return true if the threadpool size is >1 + */ + public boolean isParallel() { + return numThreads > 1; + } + + @Before + public void setup() { + items = IntStream.rangeClosed(1, ITEM_COUNT) + .mapToObj(i -> new Item(i, + String.format("With %d threads", numThreads))) + .collect(Collectors.toList()); + + if (numThreads > 0) { + threadPool = Executors.newFixedThreadPool(numThreads, + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(getMethodName() + "-pool-%d") + .build()); + submitter = new PoolSubmitter(); + } else { + submitter = null; + } + + } + + @After + public void teardown() { + if (threadPool != null) { + threadPool.shutdown(); + threadPool = null; + } + } + + private class PoolSubmitter implements TaskPool.Submitter { + + @Override + public Future submit(final Runnable task) { + return threadPool.submit(task); + } + + } + + /** + * create the builder. + * @return pre-inited builder + */ + private TaskPool.Builder builder() { + return TaskPool.foreach(items).executeWith(submitter); + } + + private void assertRun(TaskPool.Builder builder, + CounterTask task) throws IOException { + boolean b = builder.run(task); + assertTrue("Run of " + task + " failed", b); + } + + private void assertFailed(TaskPool.Builder builder, + CounterTask task) throws IOException { + boolean b = builder.run(task); + assertFalse("Run of " + task + " unexpectedly succeeded", b); + } + + private String itemsToString() { + return "[" + items.stream().map(Item::toString) + .collect(Collectors.joining("\n")) + "]"; + } + + @Test + public void testSimpleInvocation() throws Throwable { + CounterTask t = new CounterTask("simple", 0, Item::commit); + assertRun(builder(), t); + t.assertInvoked("", ITEM_COUNT); + } + + @Test + public void testFailNoStoppingSuppressed() throws Throwable { + assertFailed(builder().suppressExceptions(), failingTask); + failingTask.assertInvoked("Continued through operations", ITEM_COUNT); + items.forEach(Item::assertCommittedOrFailed); + } + + @Test + public void testFailFastSuppressed() throws Throwable { + assertFailed(builder() + .suppressExceptions() + .stopOnFailure(), + failingTask); + if (isParallel()) { + failingTask.assertInvokedAtLeast("stop fast", FAILPOINT); + } else { + failingTask.assertInvoked("stop fast", FAILPOINT); + } + } + + @Test + public void testFailedCallAbortSuppressed() throws Throwable { + assertFailed(builder() + .stopOnFailure() + .suppressExceptions() + .abortWith(aborter), + failingTask); + failingTask.assertInvokedAtLeast("success", FAILPOINT); + if (!isParallel()) { + aborter.assertInvokedAtLeast("abort", 1); + // all uncommitted items were aborted + items.stream().filter(i -> !i.committed) + .map(Item::assertAborted); + items.stream().filter(i -> i.committed) + .forEach(i -> assertFalse(i.toString(), i.aborted)); + } + } + + @Test + public void testFailedCalledWhenNotStoppingSuppressed() throws Throwable { + assertFailed(builder() + .suppressExceptions() + .onFailure(failures), + failingTask); + failingTask.assertInvokedAtLeast("success", FAILPOINT); + // only one failure was triggered + failures.assertInvoked("failure event", 1); + } + + @Test + public void testFailFastCallRevertSuppressed() throws Throwable { + assertFailed(builder() + .stopOnFailure() + .revertWith(reverter) + .abortWith(aborter) + .suppressExceptions() + .onFailure(failures), + failingTask); + failingTask.assertInvokedAtLeast("success", FAILPOINT); + if (!isParallel()) { + aborter.assertInvokedAtLeast("abort", 1); + // all uncommitted items were aborted + items.stream().filter(i -> !i.committed) + .filter(i -> !i.failed) + .forEach(Item::assertAborted); + } + // all committed were reverted + items.stream().filter(i -> i.committed && !i.failed) + .forEach(Item::assertReverted); + // all reverted items are committed + items.stream().filter(i -> i.reverted) + .forEach(Item::assertCommitted); + + // only one failure was triggered + failures.assertInvoked("failure event", 1); + } + + @Test + public void testFailSlowCallRevertSuppressed() throws Throwable { + assertFailed(builder() + .suppressExceptions() + .revertWith(reverter) + .onFailure(failures), + failingTask); + failingTask.assertInvokedAtLeast("success", FAILPOINT); + // all committed were reverted + // identify which task failed from the set + int failing = failures.getItem().id; + items.stream() + .filter(i -> i.id != failing) + .filter(i -> i.committed) + .forEach(Item::assertReverted); + // all reverted items are committed + items.stream().filter(i -> i.reverted) + .forEach(Item::assertCommitted); + + // only one failure was triggered + failures.assertInvoked("failure event", 1); + } + + @Test + public void testFailFastExceptions() throws Throwable { + intercept(IOException.class, + () -> builder() + .stopOnFailure() + .run(failingTask)); + if (isParallel()) { + failingTask.assertInvokedAtLeast("stop fast", FAILPOINT); + } else { + failingTask.assertInvoked("stop fast", FAILPOINT); + } + } + + @Test + public void testFailSlowExceptions() throws Throwable { + intercept(IOException.class, + () -> builder() + .run(failingTask)); + failingTask.assertInvoked("continued through operations", ITEM_COUNT); + items.forEach(Item::assertCommittedOrFailed); + } + + @Test + public void testFailFastExceptionsWithAbortFailure() throws Throwable { + CounterTask failFirst = new CounterTask("task", 1, Item::commit); + CounterTask a = new CounterTask("aborter", 1, Item::abort); + intercept(IOException.class, + () -> builder() + .stopOnFailure() + .abortWith(a) + .run(failFirst)); + if (!isParallel()) { + // expect the other tasks to be aborted + a.assertInvokedAtLeast("abort", ITEM_COUNT - 1); + } + } + + @Test + public void testFailFastExceptionsWithAbortFailureStopped() throws Throwable { + CounterTask failFirst = new CounterTask("task", 1, Item::commit); + CounterTask a = new CounterTask("aborter", 1, Item::abort); + intercept(IOException.class, + () -> builder() + .stopOnFailure() + .stopAbortsOnFailure() + .abortWith(a) + .run(failFirst)); + if (!isParallel()) { + // expect the other tasks to be aborted + a.assertInvoked("abort", 1); + } + } + + /** + * Fail the last one committed, all the rest will be reverted. + * The actual ID of the last task has to be picke dup from the + * failure callback, as in the pool it may be one of any. + */ + @Test + public void testRevertAllSuppressed() throws Throwable { + CounterTask failLast = new CounterTask("task", ITEM_COUNT, Item::commit); + + assertFailed(builder() + .suppressExceptions() + .stopOnFailure() + .revertWith(reverter) + .abortWith(aborter) + .onFailure(failures), + failLast); + failLast.assertInvoked("success", ITEM_COUNT); + int abCount = aborter.getCount(); + int revCount = reverter.getCount(); + assertEquals(ITEM_COUNT, 1 + abCount + revCount); + // identify which task failed from the set + int failing = failures.getItem().id; + // all committed were reverted + items.stream() + .filter(i -> i.id != failing) + .filter(i -> i.committed) + .forEach(Item::assertReverted); + items.stream() + .filter(i -> i.id != failing) + .filter(i -> !i.committed) + .forEach(Item::assertAborted); + // all reverted items are committed + items.stream().filter(i -> i.reverted) + .forEach(Item::assertCommitted); + + // only one failure was triggered + failures.assertInvoked("failure event", 1); + } + + + /** + * The Item which tasks process. + */ + private final class Item { + + private final int id; + + private final String text; + + private volatile boolean committed, aborted, reverted, failed; + + private Item(int item, String text) { + this.id = item; + this.text = text; + } + + boolean commit() { + committed = true; + return true; + } + + boolean abort() { + aborted = true; + return true; + } + + boolean revert() { + reverted = true; + return true; + } + + boolean fail() { + failed = true; + return true; + } + + public Item assertCommitted() { + assertTrue(toString() + " was not committed in\n" + + itemsToString(), + committed); + return this; + } + + public Item assertCommittedOrFailed() { + assertTrue(toString() + " was not committed nor failed in\n" + + itemsToString(), + committed || failed); + return this; + } + + public Item assertAborted() { + assertTrue(toString() + " was not aborted in\n" + + itemsToString(), + aborted); + return this; + } + + public Item assertReverted() { + assertTrue(toString() + " was not reverted in\n" + + itemsToString(), + reverted); + return this; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("Item{"); + sb.append(String.format("[%02d]", id)); + sb.append(", committed=").append(committed); + sb.append(", aborted=").append(aborted); + sb.append(", reverted=").append(reverted); + sb.append(", failed=").append(failed); + sb.append(", text=").append(text); + sb.append('}'); + return sb.toString(); + } + } + + /** + * Class which can count invocations and, if limit > 0, will raise + * an exception on the specific invocation of {@link #note(Object)} + * whose count == limit. + */ + private class BaseCounter { + + private final AtomicInteger counter = new AtomicInteger(0); + + private final int limit; + + private final String name; + + private Item item; + + private final Optional> action; + + /** + * Base counter, tracks items. + * @param name name for string/exception/logs. + * @param limit limit at which an exception is raised, 0 == never + * @param action optional action to invoke after the increment, + * before limit check + */ + BaseCounter(String name, + int limit, + Function action) { + this.name = name; + this.limit = limit; + this.action = Optional.ofNullable(action); + } + + /** + * Apply the action to an item; log at info afterwards with both the + * before and after string values of the item. + * @param i item to process. + * @throws IOException failure in the action + */ + void process(Item i) throws IOException { + this.item = i; + int count = counter.incrementAndGet(); + if (limit == count) { + i.fail(); + LOG.info("{}: Failed {}", this, i); + throw new IOException(String.format("%s: Limit %d reached for %s", + this, limit, i)); + } + String before = i.toString(); + action.map(a -> a.apply(i)); + LOG.info("{}: {} -> {}", this, before, i); + } + + int getCount() { + return counter.get(); + } + + Item getItem() { + return item; + } + + void assertInvoked(String text, int expected) { + assertEquals(toString() + ": " + text, expected, getCount()); + } + + void assertInvokedAtLeast(String text, int expected) { + int actual = getCount(); + assertTrue(toString() + ": " + text + + "-expected " + expected + + " invocations, but got " + actual + + " in " + itemsToString(), + expected <= actual); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "BaseCounter{"); + sb.append("name='").append(name).append('\''); + sb.append(", count=").append(counter.get()); + sb.append(", limit=").append(limit); + sb.append(", item=").append(item); + sb.append('}'); + return sb.toString(); + } + } + + private final class CounterTask + extends BaseCounter implements TaskPool.Task { + + private CounterTask(String name, int limit, + Function action) { + super(name, limit, action); + } + + @Override + public void run(Item item) throws IOException { + process(item); + } + + } + + private final class FailureCounter + extends BaseCounter + implements TaskPool.FailureTask { + + private Exception exception; + + private FailureCounter(String name, int limit, + Function action) { + super(name, limit, action); + } + + @Override + public void run(Item item, Exception ex) throws IOException { + process(item); + this.exception = ex; + } + + private Exception getException() { + return exception; + } + + } + +} From 7328c34ba5049441dc8d1c30b2fe15784a82acf9 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 16 Mar 2022 15:21:54 +0000 Subject: [PATCH 041/145] MAPREDUCE-7341. Add an intermediate manifest committer for Azure and GCS This is a mapreduce/spark output committer optimized for performance and correctness on Azure ADLS Gen 2 storage (via the abfs connector) and Google Cloud Storage (via the external gcs connector library). * It is safe to use with HDFS, however it has not been optimized for that use. * It is *not* safe for use with S3, and will fail if an attempt is made to do so. Contributed by Steve Loughran Change-Id: I6f3502e79c578b9fd1a8c1485f826784b5421fca --- .../hadoop-mapreduce-client-core/pom.xml | 7 +- .../output/PathOutputCommitterFactory.java | 10 +- .../committer/manifest/ManifestCommitter.java | 761 +++++++ .../manifest/ManifestCommitterConfig.java | 375 ++++ .../manifest/ManifestCommitterConstants.java | 240 +++ .../manifest/ManifestCommitterFactory.java | 57 + .../ManifestCommitterStatisticNames.java | 264 +++ .../manifest/files/AbstractManifestData.java | 137 ++ .../manifest/files/DiagnosticKeys.java | 44 + .../committer/manifest/files/DirEntry.java | 202 ++ .../committer/manifest/files/EntryStatus.java | 87 + .../committer/manifest/files/FileEntry.java | 189 ++ .../manifest/files/ManifestPrinter.java | 144 ++ .../manifest/files/ManifestSuccessData.java | 493 +++++ .../manifest/files/TaskManifest.java | 365 ++++ .../manifest/files/package-info.java | 41 + .../manifest/impl/AuditingIntegration.java | 94 + .../manifest/impl/InternalConstants.java | 130 ++ .../impl/ManifestCommitterSupport.java | 374 ++++ .../impl/ManifestStoreOperations.java | 291 +++ ...ifestStoreOperationsThroughFileSystem.java | 187 ++ .../impl/OutputValidationException.java | 40 + .../committer/manifest/impl/package-info.java | 30 + .../committer/manifest/package-info.java | 34 + .../manifest/stages/AbortTaskStage.java | 63 + .../stages/AbstractJobOrTaskStage.java | 942 +++++++++ .../manifest/stages/CleanupJobStage.java | 511 +++++ .../manifest/stages/CommitJobStage.java | 245 +++ .../manifest/stages/CommitTaskStage.java | 115 ++ .../stages/CreateOutputDirectoriesStage.java | 423 ++++ .../manifest/stages/JobOrTaskStage.java | 33 + .../manifest/stages/LoadManifestsStage.java | 293 +++ .../manifest/stages/RenameFilesStage.java | 173 ++ .../manifest/stages/SaveSuccessFileStage.java | 79 + .../stages/SaveTaskManifestStage.java | 81 + .../manifest/stages/SetupJobStage.java | 63 + .../manifest/stages/SetupTaskStage.java | 54 + .../manifest/stages/StageConfig.java | 556 +++++ .../manifest/stages/StageEventCallbacks.java | 37 + .../stages/TaskAttemptScanDirectoryStage.java | 214 ++ .../stages/ValidateRenamedFilesStage.java | 200 ++ .../manifest/stages/package-info.java | 30 + .../src/main/resources/mapred-default.xml | 19 + .../src/site/markdown/manifest_committer.md | 605 ++++++ .../manifest_committer_architecture.md | 335 +++ .../markdown/manifest_committer_protocol.md | 617 ++++++ .../AbstractManifestCommitterTest.java | 1085 ++++++++++ .../manifest/CommitterTestBinding.java | 152 ++ .../ManifestCommitterTestSupport.java | 420 ++++ .../manifest/StubStoreOperations.java | 109 + .../committer/manifest/TaggedFileStatus.java | 54 + .../committer/manifest/TestCleanupStage.java | 142 ++ .../manifest/TestCommitTaskStage.java | 126 ++ .../TestCreateOutputDirectoriesStage.java | 307 +++ .../TestJobThroughManifestCommitter.java | 601 ++++++ .../manifest/TestLoadManifestsStage.java | 141 ++ .../manifest/TestManifestCommitProtocol.java | 1801 +++++++++++++++++ .../manifest/TestRenameStageFailure.java | 379 ++++ .../manifest/TestTaskManifestFileIO.java | 185 ++ .../manifest/TextOutputForTests.java | 136 ++ .../committer/manifest/ThreadLeakTracker.java | 83 + .../UnreliableManifestStoreOperations.java | 380 ++++ .../committer/manifest/package-info.java | 29 + .../src/test/resources/contract/localfs.xml | 138 ++ .../src/test/resources/core-site.xml | 33 + 65 files changed, 16579 insertions(+), 6 deletions(-) create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterFactory.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterStatisticNames.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/EntryStatus.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestPrinter.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestSuccessData.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/TaskManifest.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/package-info.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/AuditingIntegration.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperations.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperationsThroughFileSystem.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/OutputValidationException.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/package-info.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbortTaskStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitTaskStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/JobOrTaskStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveSuccessFileStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveTaskManifestStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupJobStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupTaskStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageEventCallbacks.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/TaskAttemptScanDirectoryStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/package-info.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_architecture.md create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_protocol.md create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/CommitterTestBinding.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/StubStoreOperations.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TaggedFileStatus.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCleanupStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCommitTaskStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestManifestCommitProtocol.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestTaskManifestFileIO.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TextOutputForTests.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ThreadLeakTracker.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/UnreliableManifestStoreOperations.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/contract/localfs.xml create mode 100644 hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/core-site.xml diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml index 957202a7dd0ba..7530428d7525d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/pom.xml @@ -103,9 +103,14 @@ org.apache.maven.plugins maven-surefire-plugin - + file:///${project.basedir}/src/test/resources/log4j.properties + + + ${project.build.directory} + + ${project.build.directory} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java index f3f0dabd6551f..cf83bf2d7f4b5 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java @@ -52,8 +52,8 @@ * attempt context and a possibly null path. * */ -@InterfaceAudience.Private -@InterfaceStability.Unstable +@InterfaceAudience.Public +@InterfaceStability.Evolving public class PathOutputCommitterFactory extends Configured { private static final Logger LOG = LoggerFactory.getLogger(PathOutputCommitterFactory.class); @@ -158,7 +158,7 @@ public static PathOutputCommitterFactory getCommitterFactory( scheme); if (StringUtils.isNotEmpty(conf.getTrimmed(schemeKey))) { // it does, so use that key in the classname lookup - LOG.debug("Using schema-specific factory for {}", outputPath); + LOG.info("Using schema-specific factory for {}", outputPath); key = schemeKey; } else { LOG.debug("No scheme-specific factory defined in {}", schemeKey); @@ -171,7 +171,7 @@ public static PathOutputCommitterFactory getCommitterFactory( String trimmedValue = conf.getTrimmed(key, ""); if (StringUtils.isEmpty(trimmedValue)) { // empty/null value, use default - LOG.debug("No output committer factory defined," + LOG.info("No output committer factory defined," + " defaulting to FileOutputCommitterFactory"); factory = FileOutputCommitterFactory.class; } else { @@ -179,7 +179,7 @@ public static PathOutputCommitterFactory getCommitterFactory( factory = conf.getClass(key, FileOutputCommitterFactory.class, PathOutputCommitterFactory.class); - LOG.debug("Using OutputCommitter factory class {} from key {}", + LOG.info("Using OutputCommitter factory class {} from key {}", factory, key); } return ReflectionUtils.newInstance(factory, conf); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java new file mode 100644 index 0000000000000..99625e8242896 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java @@ -0,0 +1,761 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbortTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageEventCallbacks; +import org.apache.hadoop.util.functional.CloseableTaskPoolSubmitter; + +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_DIAGNOSTICS_MANIFEST_DIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_SUMMARY_REPORT_DIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_COMPLETED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_FAILED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME_RECOVERED; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_ABORT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CLEANUP; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration.updateCommonContextOnCommitterExit; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration.updateCommonContextOnCommitterEntry; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createIOStatisticsStore; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createJobSummaryFilename; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createManifestOutcome; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestPathForTask; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage.cleanupStageOptionsFromConfig; + +/** + * This is the Intermediate-Manifest committer. + * At every entry point it updates the thread's audit context with + * the current stage info; this is a placeholder for + * adding audit information to stores other than S3A. + * + * This is tagged as public/stable. This is mandatory + * for the classname and PathOutputCommitter implementation + * classes. + */ +@InterfaceAudience.Public +@InterfaceStability.Stable +public class ManifestCommitter extends PathOutputCommitter implements + IOStatisticsSource, StageEventCallbacks { + + public static final Logger LOG = LoggerFactory.getLogger( + ManifestCommitter.class); + + /** + * Role: task committer. + */ + public static final String TASK_COMMITTER = "task committer"; + + /** + * Role: job committer. + */ + public static final String JOB_COMMITTER = "job committer"; + + /** + * Committer Configuration as extracted from + * the job/task context and set in the constructor. + */ + private final ManifestCommitterConfig baseConfig; + + /** + * Destination of the job. + */ + private final Path destinationDir; + + /** + * For tasks, the attempt directory. + * Null for jobs. + */ + private final Path taskAttemptDir; + + /** + * IOStatistics to update. + */ + private final IOStatisticsStore iostatistics; + + /** + * The job Manifest Success data; only valid after a job successfully + * commits. + */ + private ManifestSuccessData successReport; + + /** + * The active stage; is updated by a callback from within the stages. + */ + private String activeStage; + + /** + * The task manifest of the task commit. + * Null unless this is a task attempt and the + * task has successfully been committed. + */ + private TaskManifest taskAttemptCommittedManifest; + + /** + * Create a committer. + * @param outputPath output path + * @param context job/task context + * @throws IOException failure. + */ + public ManifestCommitter(final Path outputPath, + final TaskAttemptContext context) throws IOException { + super(outputPath, context); + this.destinationDir = resolveDestinationDirectory(outputPath, + context.getConfiguration()); + this.iostatistics = createIOStatisticsStore().build(); + this.baseConfig = enterCommitter( + context.getTaskAttemptID() != null, + context); + + this.taskAttemptDir = baseConfig.getTaskAttemptDir(); + LOG.info("Created ManifestCommitter with JobID {}," + + " Task Attempt {} and destination {}", + context.getJobID(), context.getTaskAttemptID(), outputPath); + } + + /** + * Committer method invoked; generates a config for it. + * Calls {@code #updateCommonContextOnCommitterEntry()} + * to update the audit context. + * @param isTask is this a task entry point? + * @param context context + * @return committer config + */ + private ManifestCommitterConfig enterCommitter(boolean isTask, + JobContext context) { + ManifestCommitterConfig committerConfig = + new ManifestCommitterConfig( + getOutputPath(), + isTask ? TASK_COMMITTER : JOB_COMMITTER, + context, + iostatistics, + this); + updateCommonContextOnCommitterEntry(committerConfig); + return committerConfig; + } + + /** + * Set up a job through a {@link SetupJobStage}. + * @param jobContext Context of the job whose output is being written. + * @throws IOException IO Failure. + */ + @Override + public void setupJob(final JobContext jobContext) throws IOException { + ManifestCommitterConfig committerConfig = enterCommitter(false, + jobContext); + StageConfig stageConfig = + committerConfig + .createStageConfig() + .withOperations(createManifestStoreOperations()) + .build(); + // set up the job. + new SetupJobStage(stageConfig) + .apply(committerConfig.getCreateJobMarker()); + logCommitterStatisticsAtDebug(); + } + + /** + * Set up a task through a {@link SetupTaskStage}. + * Classic FileOutputCommitter is a no-op here, relying + * on RecordWriters to create the dir implicitly on file + * create(). + * FileOutputCommitter also uses the existence of that + * file as a flag to indicate task commit is needed. + * @param context task context. + * @throws IOException IO Failure. + */ + @Override + public void setupTask(final TaskAttemptContext context) + throws IOException { + ManifestCommitterConfig committerConfig = + enterCommitter(true, context); + StageConfig stageConfig = + committerConfig + .createStageConfig() + .withOperations(createManifestStoreOperations()) + .build(); + // create task attempt dir; delete if present. Or fail? + new SetupTaskStage(stageConfig).apply(""); + logCommitterStatisticsAtDebug(); + } + + /** + * Always return true. + * This way, even if there is no output, stats are collected. + * @param context task context. + * @return true + * @throws IOException IO Failure. + */ + @Override + public boolean needsTaskCommit(final TaskAttemptContext context) + throws IOException { + LOG.info("Probe for needsTaskCommit({})", + context.getTaskAttemptID()); + return true; + } + + /** + * Failure during Job Commit is not recoverable from. + * + * @param jobContext + * Context of the job whose output is being written. + * @return false, always + * @throws IOException never + */ + @Override + public boolean isCommitJobRepeatable(final JobContext jobContext) + throws IOException { + LOG.info("Probe for isCommitJobRepeatable({}): returning false", + jobContext.getJobID()); + return false; + } + + /** + * Declare that task recovery is not supported. + * It would be, if someone added the code *and tests*. + * @param jobContext + * Context of the job whose output is being written. + * @return false, always + * @throws IOException never + */ + @Override + public boolean isRecoverySupported(final JobContext jobContext) + throws IOException { + LOG.info("Probe for isRecoverySupported({}): returning false", + jobContext.getJobID()); + return false; + } + + /** + * + * @param taskContext Context of the task whose output is being recovered + * @throws IOException always + */ + @Override + public void recoverTask(final TaskAttemptContext taskContext) + throws IOException { + LOG.warn("Rejecting recoverTask({}) call", taskContext.getTaskAttemptID()); + throw new IOException("Cannot recover task " + + taskContext.getTaskAttemptID()); + } + + /** + * Commit the task. + * This is where the task attempt tree list takes place. + * @param context task context. + * @throws IOException IO Failure. + */ + @Override + public void commitTask(final TaskAttemptContext context) + throws IOException { + ManifestCommitterConfig committerConfig = enterCommitter(true, + context); + try { + StageConfig stageConfig = committerConfig.createStageConfig() + .withOperations(createManifestStoreOperations()) + .build(); + taskAttemptCommittedManifest = new CommitTaskStage(stageConfig) + .apply(null).getTaskManifest(); + iostatistics.incrementCounter(COMMITTER_TASKS_COMPLETED_COUNT, 1); + } catch (IOException e) { + iostatistics.incrementCounter(COMMITTER_TASKS_FAILED_COUNT, 1); + throw e; + } finally { + logCommitterStatisticsAtDebug(); + updateCommonContextOnCommitterExit(); + } + + } + + /** + * Abort a task. + * @param context task context + * @throws IOException failure during the delete + */ + @Override + public void abortTask(final TaskAttemptContext context) + throws IOException { + ManifestCommitterConfig committerConfig = enterCommitter(true, + context); + try { + new AbortTaskStage( + committerConfig.createStageConfig() + .withOperations(createManifestStoreOperations()) + .build()) + .apply(false); + } finally { + logCommitterStatisticsAtDebug(); + updateCommonContextOnCommitterExit(); + } + } + + /** + * Get the manifest success data for this job; creating on demand if needed. + * @param committerConfig source config. + * @return the current {@link #successReport} value; never null. + */ + private ManifestSuccessData getOrCreateSuccessData( + ManifestCommitterConfig committerConfig) { + if (successReport == null) { + successReport = createManifestOutcome( + committerConfig.createStageConfig(), activeStage); + } + return successReport; + } + + /** + * This is the big job commit stage. + * Load the manifests, prepare the destination, rename + * the files then cleanup the job directory. + * @param jobContext Context of the job whose output is being written. + * @throws IOException failure. + */ + @Override + public void commitJob(final JobContext jobContext) throws IOException { + + ManifestCommitterConfig committerConfig = enterCommitter(false, jobContext); + + // create the initial success data. + // this is overwritten by that created during the operation sequence, + // but if the sequence fails before that happens, it + // will be saved to the report directory. + ManifestSuccessData marker = getOrCreateSuccessData(committerConfig); + IOException failure = null; + try (CloseableTaskPoolSubmitter ioProcs = + committerConfig.createSubmitter(); + ManifestStoreOperations storeOperations = createManifestStoreOperations()) { + // the stage config will be shared across all stages. + StageConfig stageConfig = committerConfig.createStageConfig() + .withOperations(storeOperations) + .withIOProcessors(ioProcs) + .build(); + + // commit the job, including any cleanup and validation. + final Configuration conf = jobContext.getConfiguration(); + CommitJobStage.Result result = new CommitJobStage(stageConfig).apply( + new CommitJobStage.Arguments( + committerConfig.getCreateJobMarker(), + committerConfig.getValidateOutput(), + conf.getTrimmed(OPT_DIAGNOSTICS_MANIFEST_DIR, ""), + cleanupStageOptionsFromConfig( + OP_STAGE_JOB_CLEANUP, conf) + )); + marker = result.getJobSuccessData(); + // update the cached success with the new report. + setSuccessReport(marker); + + } catch (IOException e) { + // failure. record it for the summary + failure = e; + // rethrow + throw e; + } finally { + // save the report summary, even on failure + maybeSaveSummary(activeStage, + committerConfig, + marker, + failure, + true, + true); + // print job commit stats + LOG.info("{}: Job Commit statistics {}", + committerConfig.getName(), + ioStatisticsToPrettyString(iostatistics)); + // and warn of rename problems + final Long recoveries = iostatistics.counters().get(OP_COMMIT_FILE_RENAME_RECOVERED); + if (recoveries != null && recoveries > 0) { + LOG.warn("{}: rename failures were recovered from. Number of recoveries: {}", + committerConfig.getName(), recoveries); + } + updateCommonContextOnCommitterExit(); + } + } + + /** + * Abort the job. + * Invokes + * {@link #executeCleanup(String, JobContext, ManifestCommitterConfig)} + * then saves the (ongoing) job report data if reporting is enabled. + * @param jobContext Context of the job whose output is being written. + * @param state final runstate of the job + * @throws IOException failure during cleanup; report failure are swallowed + */ + @Override + public void abortJob(final JobContext jobContext, + final JobStatus.State state) + throws IOException { + LOG.info("Aborting Job {} in state {}", jobContext.getJobID(), state); + ManifestCommitterConfig committerConfig = enterCommitter(false, + jobContext); + ManifestSuccessData report = getOrCreateSuccessData( + committerConfig); + IOException failure = null; + + try { + executeCleanup(OP_STAGE_JOB_ABORT, jobContext, committerConfig); + } catch (IOException e) { + // failure. + failure = e; + } + report.setSuccess(false); + // job abort does not overwrite any existing report, so a job commit + // failure cause will be preserved. + maybeSaveSummary(activeStage, committerConfig, report, failure, + true, false); + // print job stats + LOG.info("Job Abort statistics {}", + ioStatisticsToPrettyString(iostatistics)); + updateCommonContextOnCommitterExit(); + } + + /** + * Execute the {@code CleanupJobStage} to remove the job attempt dir. + * This does + * @param jobContext Context of the job whose output is being written. + * @throws IOException failure during cleanup + */ + @SuppressWarnings("deprecation") + @Override + public void cleanupJob(final JobContext jobContext) throws IOException { + ManifestCommitterConfig committerConfig = enterCommitter(false, + jobContext); + try { + executeCleanup(OP_STAGE_JOB_CLEANUP, jobContext, committerConfig); + } finally { + logCommitterStatisticsAtDebug(); + updateCommonContextOnCommitterExit(); + } + } + + /** + * Perform the cleanup operation for job cleanup or abort. + * @param statisticName statistic/stage name + * @param jobContext job context + * @param committerConfig committer config + * @throws IOException failure + * @return the outcome + */ + private CleanupJobStage.Result executeCleanup( + final String statisticName, + final JobContext jobContext, + final ManifestCommitterConfig committerConfig) throws IOException { + try (CloseableTaskPoolSubmitter ioProcs = + committerConfig.createSubmitter()) { + + return new CleanupJobStage( + committerConfig.createStageConfig() + .withOperations(createManifestStoreOperations()) + .withIOProcessors(ioProcs) + .build()) + .apply(cleanupStageOptionsFromConfig( + statisticName, + jobContext.getConfiguration())); + } + } + + /** + * Output path: destination directory of the job. + * @return the overall job destination directory. + */ + @Override + public Path getOutputPath() { + return getDestinationDir(); + } + + /** + * Work path of the current task attempt. + * This is null if the task does not have one. + * @return a path. + */ + @Override + public Path getWorkPath() { + return getTaskAttemptDir(); + } + + /** + * Get the job destination dir. + * @return dest dir. + */ + private Path getDestinationDir() { + return destinationDir; + } + + /** + * Get the task attempt dir. + * May be null. + * @return a path or null. + */ + private Path getTaskAttemptDir() { + return taskAttemptDir; + } + + /** + * Callback on stage entry. + * Sets {@link #activeStage} and updates the + * common context. + * @param stage new stage + */ + @Override + public void enterStage(String stage) { + activeStage = stage; + AuditingIntegration.enterStage(stage); + } + + /** + * Remove stage from common audit context. + * @param stage stage exited. + */ + @Override + public void exitStage(String stage) { + AuditingIntegration.exitStage(); + } + + /** + * Get the unique ID of this job. + * @return job ID (yarn, spark) + */ + public String getJobUniqueId() { + return baseConfig.getJobUniqueId(); + } + + /** + * Get the config of the task attempt this instance was constructed + * with. + * @return a configuration. + */ + public Configuration getConf() { + return baseConfig.getConf(); + } + + /** + * Get the manifest Success data; only valid after a job. + * @return the job _SUCCESS data, or null. + */ + public ManifestSuccessData getSuccessReport() { + return successReport; + } + + private void setSuccessReport(ManifestSuccessData successReport) { + this.successReport = successReport; + } + + /** + * Get the manifest of the last committed task. + * @return a task manifest or null. + */ + @VisibleForTesting + TaskManifest getTaskAttemptCommittedManifest() { + return taskAttemptCommittedManifest; + } + + /** + * Compute the path where the output of a task attempt is stored until + * that task is committed. + * @param context the context of the task attempt. + * @return the path where a task attempt should be stored. + */ + @VisibleForTesting + public Path getTaskAttemptPath(TaskAttemptContext context) { + return enterCommitter(false, context).getTaskAttemptDir(); + } + + /** + * The path to where the manifest file of a task attempt will be + * saved when the task is committed. + * This path will be the same for all attempts of the same task. + * @param context the context of the task attempt. + * @return the path where a task attempt should be stored. + */ + @VisibleForTesting + public Path getTaskManifestPath(TaskAttemptContext context) { + final Path dir = enterCommitter(false, context).getTaskManifestDir(); + + return manifestPathForTask(dir, + context.getTaskAttemptID().getTaskID().toString()); + } + + /** + * Compute the path where the output of a task attempt is stored until + * that task is committed. + * @param context the context of the task attempt. + * @return the path where a task attempt should be stored. + */ + @VisibleForTesting + public Path getJobAttemptPath(JobContext context) { + + return enterCommitter(false, context).getJobAttemptDir(); + } + + /** + * Get the final output path, including resolving any relative path. + * @param outputPath output path + * @param conf configuration to create any FS with + * @return a resolved path. + * @throws IOException failure. + */ + private Path resolveDestinationDirectory(Path outputPath, + Configuration conf) throws IOException { + return FileSystem.get(outputPath.toUri(), conf).makeQualified(outputPath); + } + + /** + * Create manifest store operations for the destination store. + * This MUST NOT be used for the success report operations, as + * they may be to a different filesystem. + * This is a point which can be overridden during testing. + * @return a new store operations instance bonded to the destination fs. + * @throws IOException failure to instantiate. + */ + protected ManifestStoreOperations createManifestStoreOperations() throws IOException { + return ManifestCommitterSupport.createManifestStoreOperations( + baseConfig.getConf(), + baseConfig.getDestinationFileSystem(), + baseConfig.getDestinationDir()); + } + + /** + * Log IO Statistics at debug. + */ + private void logCommitterStatisticsAtDebug() { + logIOStatisticsAtDebug(LOG, "Committer Statistics", this); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ManifestCommitter{"); + sb.append(baseConfig); + sb.append(", iostatistics=").append(ioStatisticsToPrettyString(iostatistics)); + sb.append('}'); + return sb.toString(); + } + + /** + * Save a summary to the report dir if the config option + * is set. + * The IOStatistics of the summary will be updated to the latest + * snapshot of the committer's statistics, so the report is up + * to date. + * The report will updated with the current active stage, + * and if {@code thrown} is non-null, it will be added to the + * diagnistics (and the job tagged as a failure). + * Static for testability. + * @param activeStage active stage + * @param config configuration to use. + * @param report summary file. + * @param thrown any exception indicting failure. + * @param quiet should exceptions be swallowed. + * @param overwrite should the existing file be overwritten + * @return the path of a file, if successfully saved + * @throws IOException if a failure occured and quiet==false + */ + private static Path maybeSaveSummary( + String activeStage, + ManifestCommitterConfig config, + ManifestSuccessData report, + Throwable thrown, + boolean quiet, + boolean overwrite) throws IOException { + Configuration conf = config.getConf(); + String reportDir = conf.getTrimmed(OPT_SUMMARY_REPORT_DIR, ""); + if (reportDir.isEmpty()) { + LOG.debug("No summary directory set in " + OPT_SUMMARY_REPORT_DIR); + return null; + } + LOG.debug("Summary directory set in to {}" + OPT_SUMMARY_REPORT_DIR, + reportDir); + + // update to the latest statistics + report.snapshotIOStatistics(config.getIOStatistics()); + + Path reportDirPath = new Path(reportDir); + Path path = new Path(reportDirPath, + createJobSummaryFilename(config.getJobUniqueId())); + + if (thrown != null) { + report.recordJobFailure(thrown); + } + report.putDiagnostic(STAGE, activeStage); + // the store operations here is explicitly created for the FS where + // the reports go, which may not be the target FS of the job. + + final FileSystem fs = path.getFileSystem(conf); + try (ManifestStoreOperations operations = new ManifestStoreOperationsThroughFileSystem(fs)) { + if (!overwrite) { + // check for file existence so there is no need to worry about + // precisely what exception is raised when overwrite=false and dest file + // exists + try { + FileStatus st = operations.getFileStatus(path); + // get here and the file exists + LOG.debug("Report already exists: {}", st); + return null; + } catch (FileNotFoundException ignored) { + } + } + operations.save(report, path, overwrite); + LOG.info("Job summary saved to {}", path); + return path; + } catch (IOException e) { + LOG.debug("Failed to save summary to {}", path, e); + if (quiet) { + return null; + } else { + throw e; + } + } + } + + @Override + public IOStatisticsStore getIOStatistics() { + return iostatistics; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java new file mode 100644 index 0000000000000..40dffee203b8d --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java @@ -0,0 +1,375 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.IOException; +import java.util.Objects; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageEventCallbacks; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.functional.CloseableTaskPoolSubmitter; + +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.*; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.buildJobUUID; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.getAppAttemptId; + +/** + * The configuration for the committer as built up from the job configuration + * and data passed down from the committer factory. + * Isolated for ease of dev/test + */ +public final class ManifestCommitterConfig implements IOStatisticsSource { + + /** + * Final destination of work. + * This is unqualified. + */ + private final Path destinationDir; + + /** + * Role: used in log/text messages. + */ + private final String role; + + /** + * This is the directory for all intermediate work: where the output + * format will write data. + * Will be null if built from a job context. + */ + private final Path taskAttemptDir; + + /** Configuration of the job. */ + private final Configuration conf; + + /** The job context. For a task, this can be cast to a TaskContext. */ + private final JobContext jobContext; + + /** Should a job marker be created? */ + private final boolean createJobMarker; + + /** + * Job ID Or UUID -without any attempt suffix. + * This is expected/required to be unique, though + * Spark has had "issues" there until recently + * with lack of uniqueness of generated MR Job IDs. + */ + private final String jobUniqueId; + + /** + * Where did the job Unique ID come from? + */ + private final String jobUniqueIdSource; + + /** + * Number of this attempt; starts at zero. + */ + private final int jobAttemptNumber; + + /** + * Job ID + AttemptID. + */ + private final String jobAttemptId; + + /** + * Task ID: used as the filename of the manifest. + * Will be "" if built from a job context. + */ + private final String taskId; + + /** + * Task attempt ID. Determines the working + * directory for task attempts to write data into, + * and for the task committer to scan. + * Will be "" if built from a job context. + */ + private final String taskAttemptId; + + /** Any progressable for progress callbacks. */ + private final Progressable progressable; + + /** + * IOStatistics to update. + */ + private final IOStatisticsStore iostatistics; + + + /** Should the output be validated after the commit? */ + private final boolean validateOutput; + + /** + * Attempt directory management. + */ + private final ManifestCommitterSupport.AttemptDirectories dirs; + + /** + * Callback when a stage is entered. + */ + private final StageEventCallbacks stageEventCallbacks; + + /** + * Name for logging. + */ + private final String name; + + /** + * Delete target paths on commit? Stricter, but + * higher IO cost. + */ + private final boolean deleteTargetPaths; + + /** + * Constructor. + * @param outputPath destination path of the job. + * @param role role for log messages. + * @param context job/task context + * @param iostatistics IO Statistics + * @param stageEventCallbacks stage event callbacks. + */ + + ManifestCommitterConfig( + final Path outputPath, + final String role, + final JobContext context, + final IOStatisticsStore iostatistics, + final StageEventCallbacks stageEventCallbacks) { + this.role = role; + this.jobContext = context; + this.conf = context.getConfiguration(); + this.destinationDir = outputPath; + this.iostatistics = iostatistics; + this.stageEventCallbacks = stageEventCallbacks; + + Pair pair = buildJobUUID(conf, context.getJobID()); + this.jobUniqueId = pair.getLeft(); + this.jobUniqueIdSource = pair.getRight(); + this.jobAttemptNumber = getAppAttemptId(context); + this.jobAttemptId = this.jobUniqueId + "_" + jobAttemptNumber; + + // build directories + this.dirs = new ManifestCommitterSupport.AttemptDirectories(outputPath, + this.jobUniqueId, jobAttemptNumber); + + // read in configuration options + this.createJobMarker = conf.getBoolean( + SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, + DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER); + this.validateOutput = conf.getBoolean( + OPT_VALIDATE_OUTPUT, + OPT_VALIDATE_OUTPUT_DEFAULT); + this.deleteTargetPaths = conf.getBoolean( + OPT_DELETE_TARGET_FILES, + OPT_DELETE_TARGET_FILES_DEFAULT); + + // if constructed with a task attempt, build the task ID and path. + if (context instanceof TaskAttemptContext) { + // it's a task + final TaskAttemptContext tac = (TaskAttemptContext) context; + TaskAttemptID taskAttempt = Objects.requireNonNull( + tac.getTaskAttemptID()); + taskAttemptId = taskAttempt.toString(); + taskId = taskAttempt.getTaskID().toString(); + // Task attempt dir; must be different across instances + taskAttemptDir = dirs.getTaskAttemptPath(taskAttemptId); + // the context is also the progress callback. + progressable = tac; + name = String.format(InternalConstants.NAME_FORMAT_TASK_ATTEMPT, taskAttemptId); + + } else { + // it's a job + taskId = ""; + taskAttemptId = ""; + taskAttemptDir = null; + progressable = null; + name = String.format(InternalConstants.NAME_FORMAT_JOB_ATTEMPT, jobAttemptId); + } + } + + @Override + public String toString() { + return "ManifestCommitterConfig{" + + "name=" + name + + ", destinationDir=" + destinationDir + + ", role='" + role + '\'' + + ", taskAttemptDir=" + taskAttemptDir + + ", createJobMarker=" + createJobMarker + + ", jobUniqueId='" + jobUniqueId + '\'' + + ", jobUniqueIdSource='" + jobUniqueIdSource + '\'' + + ", jobAttemptNumber=" + jobAttemptNumber + + ", jobAttemptId='" + jobAttemptId + '\'' + + ", taskId='" + taskId + '\'' + + ", taskAttemptId='" + taskAttemptId + '\'' + + '}'; + } + + /** + * Get the destination filesystem. + * @return destination FS. + * @throws IOException Problems binding to the destination FS. + */ + FileSystem getDestinationFileSystem() throws IOException { + return FileSystem.get(destinationDir.toUri(), conf); + } + + /** + * Create the stage config from the committer + * configuration. + * This does not bind the store operations + * or processors. + * @return a stage config with configuration options passed in. + */ + StageConfig createStageConfig() { + StageConfig stageConfig = new StageConfig(); + stageConfig + .withIOStatistics(iostatistics) + .withJobAttemptNumber(jobAttemptNumber) + .withJobDirectories(dirs) + .withJobId(jobUniqueId) + .withJobIdSource(jobUniqueIdSource) + .withName(name) + .withProgressable(progressable) + .withStageEventCallbacks(stageEventCallbacks) + .withTaskAttemptDir(taskAttemptDir) + .withTaskAttemptId(taskAttemptId) + .withTaskId(taskId) + .withDeleteTargetPaths(deleteTargetPaths); + + return stageConfig; + } + + public Path getDestinationDir() { + return destinationDir; + } + + public String getRole() { + return role; + } + + public Path getTaskAttemptDir() { + return taskAttemptDir; + } + + public Path getJobAttemptDir() { + return dirs.getJobAttemptDir(); + } + + public Path getTaskManifestDir() { + return dirs.getTaskManifestDir(); + } + + public Configuration getConf() { + return conf; + } + + public JobContext getJobContext() { + return jobContext; + } + + public boolean getCreateJobMarker() { + return createJobMarker; + } + + public String getJobAttemptId() { + return jobAttemptId; + } + + public String getTaskAttemptId() { + return taskAttemptId; + } + + public String getTaskId() { + return taskId; + } + + public String getJobUniqueId() { + return jobUniqueId; + } + + public boolean getValidateOutput() { + return validateOutput; + } + + public String getName() { + return name; + } + + @Override + public IOStatisticsStore getIOStatistics() { + return iostatistics; + } + + /** + * Create a new submitter task pool from the + * {@link ManifestCommitterConstants#OPT_IO_PROCESSORS} + * settings. + * @return a new thread pool. + */ + public CloseableTaskPoolSubmitter createSubmitter() { + return createSubmitter( + OPT_IO_PROCESSORS, OPT_IO_PROCESSORS_DEFAULT); + } + + /** + * Create a new submitter task pool. + * @param key config key with pool size. + * @param defVal default value. + * @return a new task pool. + */ + public CloseableTaskPoolSubmitter createSubmitter(String key, int defVal) { + int numThreads = conf.getInt(key, defVal); + if (numThreads <= 0) { + // ignore the setting if it is too invalid. + numThreads = defVal; + } + return createCloseableTaskSubmitter(numThreads, getJobAttemptId()); + } + + /** + * Create a new submitter task pool. + * + * @param numThreads thread count. + * @param jobAttemptId job ID + * @return a new task pool. + */ + public static CloseableTaskPoolSubmitter createCloseableTaskSubmitter( + final int numThreads, + final String jobAttemptId) { + return new CloseableTaskPoolSubmitter( + HadoopExecutors.newFixedThreadPool(numThreads, + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("manifest-committer-" + jobAttemptId + "-%d") + .build())); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java new file mode 100644 index 0000000000000..eb344e8a27e90 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConstants.java @@ -0,0 +1,240 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem; + +/** + * Public constants for the manifest committer. + * This includes all configuration options and their default values. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class ManifestCommitterConstants { + + /** + * Suffix to use in manifest files in the manifest subdir. + * Value: {@value}. + */ + public static final String MANIFEST_SUFFIX = "-manifest.json"; + + /** + * Prefix for summary files in the report dir. Call + */ + public static final String SUMMARY_FILENAME_PREFIX = "summary-"; + + /** + * Format string used to build a summary file from a Job ID. + */ + public static final String SUMMARY_FILENAME_FORMAT = + SUMMARY_FILENAME_PREFIX + "%s.json"; + + /** + * Suffix to use for temp files before renaming them. + * Value: {@value}. + */ + public static final String TMP_SUFFIX = ".tmp"; + + /** + * Initial number of all app attempts. + * This is fixed in YARN; for Spark jobs the + * same number "0" is used. + */ + public static final int INITIAL_APP_ATTEMPT_ID = 0; + + /** + * Format string for building a job dir. + * Value: {@value}. + */ + public static final String JOB_DIR_FORMAT_STR = "%s"; + + /** + * Format string for building a job attempt dir. + * This uses the job attempt number so previous versions + * can be found trivially. + * Value: {@value}. + */ + public static final String JOB_ATTEMPT_DIR_FORMAT_STR = "%02d"; + + /** + * Name of directory under job attempt dir for manifests. + */ + public static final String JOB_TASK_MANIFEST_SUBDIR = "manifests"; + + /** + * Name of directory under job attempt dir for task attempts. + */ + public static final String JOB_TASK_ATTEMPT_SUBDIR = "tasks"; + + + /** + * Committer classname as recorded in the committer _SUCCESS file. + */ + public static final String MANIFEST_COMMITTER_CLASSNAME = + ManifestCommitter.class.getName(); + + /** + * Marker file to create on success: {@value}. + */ + public static final String SUCCESS_MARKER = "_SUCCESS"; + + /** Default job marker option: {@value}. */ + public static final boolean DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER = true; + + /** + * The limit to the number of committed objects tracked during + * job commits and saved to the _SUCCESS file. + * Value: {@value}. + */ + public static final int SUCCESS_MARKER_FILE_LIMIT = 100; + + /** + * The UUID for jobs: {@value}. + * This was historically created in Spark 1.x's SQL queries, + * but "went away". + * It has been restored in recent spark releases. + * If found: it is used instead of the MR job attempt ID. + */ + public static final String SPARK_WRITE_UUID = "spark.sql.sources.writeJobUUID"; + + /** + * String to use as source of the job ID. + * This SHOULD be kept in sync with that of + * {@code AbstractS3ACommitter.JobUUIDSource}. + * Value: {@value}. + */ + public static final String JOB_ID_SOURCE_MAPREDUCE = "JobID"; + + /** + * Prefix to use for config options: {@value}. + */ + public static final String OPT_PREFIX = "mapreduce.manifest.committer."; + + /** + * Should dir cleanup do parallel deletion of task attempt dirs + * before trying to delete the toplevel dirs. + * For GCS this may deliver speedup, while on ABFS it may avoid + * timeouts in certain deployments. + * Value: {@value}. + */ + public static final String OPT_CLEANUP_PARALLEL_DELETE = + OPT_PREFIX + "cleanup.parallel.delete"; + + /** + * Default value: {@value}. + */ + public static final boolean OPT_CLEANUP_PARALLEL_DELETE_DIRS_DEFAULT = true; + + /** + * Threads to use for IO. + */ + public static final String OPT_IO_PROCESSORS = OPT_PREFIX + "io.threads"; + + /** + * Default value: {@value}. + */ + public static final int OPT_IO_PROCESSORS_DEFAULT = 64; + + /** + * Directory for saving job summary reports. + * These are the _SUCCESS files, but are saved even on + * job failures. + * Value: {@value}. + */ + public static final String OPT_SUMMARY_REPORT_DIR = + OPT_PREFIX + "summary.report.directory"; + + /** + * Directory for moving manifests under for diagnostics. + * Value: {@value}. + */ + public static final String OPT_DIAGNOSTICS_MANIFEST_DIR = + OPT_PREFIX + "diagnostics.manifest.directory"; + + /** + * Should the output be validated? + * This will check expected vs actual file lengths, and, + * if etags can be obtained, etags. + * Value: {@value}. + */ + public static final String OPT_VALIDATE_OUTPUT = OPT_PREFIX + "validate.output"; + + /** + * Default value: {@value}. + */ + public static final boolean OPT_VALIDATE_OUTPUT_DEFAULT = false; + + /** + * Should job commit delete for files/directories at the targets + * of renames, and, if found, deleting them? + * + * This is part of the effective behavior of the FileOutputCommitter, + * however it adds an extra delete call per file being committed. + * + * If a job is writing to a directory which has only just been created + * or were unique filenames are being used, there is no need to perform + * this preparation. + * The recognition of newly created dirs is automatic. + * + * Value: {@value}. + */ + public static final String OPT_DELETE_TARGET_FILES = + OPT_PREFIX + "delete.target.files"; + + /** + * Default value: {@value}. + */ + public static final boolean OPT_DELETE_TARGET_FILES_DEFAULT = false; + + /** + * Name of the factory: {@value}. + */ + public static final String MANIFEST_COMMITTER_FACTORY = + ManifestCommitterFactory.class.getName(); + + /** + * Classname of the store operations; filesystems and tests + * may override. + * Value: {@value}. + */ + public static final String OPT_STORE_OPERATIONS_CLASS = OPT_PREFIX + "store.operations.classname"; + + /** + * Default classname of the store operations. + * Value: {@value}. + */ + public static final String STORE_OPERATIONS_CLASS_DEFAULT = + ManifestStoreOperationsThroughFileSystem.class.getName(); + + /** + * Stage attribute in audit context: {@value}. + */ + public static final String CONTEXT_ATTR_STAGE = "st"; + + /** + * Task ID attribute in audit context: {@value}. + */ + public static final String CONTEXT_ATTR_TASK_ATTEMPT_ID = "ta"; + + private ManifestCommitterConstants() { + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterFactory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterFactory.java new file mode 100644 index 0000000000000..b07182cf999ed --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterFactory.java @@ -0,0 +1,57 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.UNSUPPORTED_FS_SCHEMAS; + +/** + * This is the committer factory to register as the source of committers + * for the job/filesystem schema. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class ManifestCommitterFactory extends PathOutputCommitterFactory { + + /** + * Name of this factory. + */ + public static final String NAME = ManifestCommitterFactory.class.getName(); + + @Override + public ManifestCommitter createOutputCommitter(final Path outputPath, + final TaskAttemptContext context) throws IOException { + // safety check. S3A does not support this, so fail fast. + final String scheme = outputPath.toUri().getScheme(); + if (UNSUPPORTED_FS_SCHEMAS.contains(scheme)) { + throw new PathIOException(outputPath.toString(), + "This committer does not work with the filesystem of type " + scheme); + } + return new ManifestCommitter(outputPath, context); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterStatisticNames.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterStatisticNames.java new file mode 100644 index 0000000000000..243fd6087328d --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterStatisticNames.java @@ -0,0 +1,264 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_RENAME; + +/** + * Statistic names for committers. + * Please keep in sync with org.apache.hadoop.fs.s3a.Statistic + * so that S3A and manifest committers are in sync. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class ManifestCommitterStatisticNames { + + + /** Amount of data committed: {@value}. */ + public static final String COMMITTER_BYTES_COMMITTED_COUNT = + "committer_bytes_committed"; + + /** Duration Tracking of time to commit an entire job: {@value}. */ + public static final String COMMITTER_COMMIT_JOB = + "committer_commit_job"; + + /** Number of files committed: {@value}. */ + public static final String COMMITTER_FILES_COMMITTED_COUNT = + "committer_files_committed"; + + /** "Count of successful tasks:: {@value}. */ + public static final String COMMITTER_TASKS_COMPLETED_COUNT = + "committer_tasks_completed"; + + /** Count of failed tasks: {@value}. */ + public static final String COMMITTER_TASKS_FAILED_COUNT = + "committer_tasks_failed"; + + /** Count of commits aborted: {@value}. */ + public static final String COMMITTER_COMMITS_ABORTED_COUNT = + "committer_commits_aborted"; + + /** Count of commits reverted: {@value}. */ + public static final String COMMITTER_COMMITS_REVERTED_COUNT = + "committer_commits_reverted"; + + /** Count of commits failed: {@value}. */ + public static final String COMMITTER_COMMITS_FAILED = + "committer_commits" + StoreStatisticNames.SUFFIX_FAILURES; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_FILE_COUNT_MEAN = + "committer_task_file_count"; + + /** + * File Size. + */ + public static final String COMMITTER_FILE_SIZE_MEAN = + "committer_task_file_size"; + + /** + * What is a task attempt's directory count. + */ + public static final String COMMITTER_TASK_DIRECTORY_COUNT_MEAN = + "committer_task_directory_count"; + + /** + * What is the depth of a task attempt's directory tree. + */ + public static final String COMMITTER_TASK_DIRECTORY_DEPTH_MEAN = + "committer_task_directory_depth"; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_TASK_FILE_COUNT_MEAN = + "committer_task_file_count"; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_TASK_FILE_SIZE_MEAN = + "committer_task_file_size"; + + /** + * The size of manifest files. This will be a MeanStatistic. + * Useful to highlight whether this size is too large and + * effort to optimise for file IO and memory consumption + * justified. + */ + public static final String COMMITTER_TASK_MANIFEST_FILE_SIZE = + "committer_task_manifest_file_size"; + + /** + * A file renamed during the commit operation {@value}. + */ + public static final String OP_COMMIT_FILE_RENAME = + "commit_file_rename"; + + /** + * A file rename during the commit operation was recovered from a failure {@value}. + */ + public static final String OP_COMMIT_FILE_RENAME_RECOVERED = + "commit_file_rename_recovered"; + + /** Directory creation {@value}. */ + public static final String OP_CREATE_DIRECTORIES = "op_create_directories"; + + /** Creating a single directory {@value}. */ + public static final String OP_CREATE_ONE_DIRECTORY = + "op_create_one_directory"; + + /** + * A file in the destination directory tree has been deleted. + * {@value}. + */ + public static final String OP_DELETE_FILE_UNDER_DESTINATION = + "op_delete_file_under_destination"; + + /** Directory scan {@value}. */ + public static final String OP_DIRECTORY_SCAN = "op_directory_scan"; + + /** + * Overall job commit {@value}. + */ + public static final String OP_STAGE_JOB_COMMIT = COMMITTER_COMMIT_JOB; + + /** {@value}. */ + public static final String OP_LOAD_ALL_MANIFESTS = "op_load_all_manifests"; + + /** + * Load a task manifest: {@value}. + */ + public static final String OP_LOAD_MANIFEST = "op_load_manifest"; + + /** + * mkdir failure statistic: {@value}. + * This is incremented whenever mkdir() returns false, e.g because + * there was a file at the end of the path. + */ + public static final String OP_MKDIRS_RETURNED_FALSE = "op_mkdir_returned_false"; + + /** + * msync statistic: {@value}. + * This should be the same as StoreStatisticNames.OP_MSYNC; it's + * duplicated just to isolate this committer into a single JAR + * for ease of testing. + */ + public static final String OP_MSYNC = "op_msync"; + + /** + * Preparing ancestor dirs: {@value}. + * Probing for paths being files, and if true: deleting them. + */ + public static final String OP_PREPARE_DIR_ANCESTORS = "op_prepare_dir_ancestors"; + + /** Rename a dir: {@value}. */ + public static final String OP_RENAME_DIR = OP_RENAME; + + + /** Rename a file: {@value}. */ + public static final String OP_RENAME_FILE = OP_RENAME; + + /** + * Save a task manifest: {@value}. + */ + public static final String OP_SAVE_TASK_MANIFEST = + "task_stage_save_task_manifest"; + + /** + * Task abort: {@value}. + */ + public static final String OP_STAGE_TASK_ABORT_TASK + = "task_stage_abort_task"; + + /** + * Job abort: {@value}. + */ + public static final String OP_STAGE_JOB_ABORT = "job_stage_abort"; + + /** + * Job cleanup: {@value}. + */ + public static final String OP_STAGE_JOB_CLEANUP = "job_stage_cleanup"; + + /** + * Prepare Directories Stage: {@value}. + */ + public static final String OP_STAGE_JOB_CREATE_TARGET_DIRS = + "job_stage_create_target_dirs"; + + /** + * Load Manifest Stage: {@value}. + */ + public static final String OP_STAGE_JOB_LOAD_MANIFESTS = + "job_stage_load_manifests"; + + /** + * Rename files stage duration: {@value}. + */ + public static final String OP_STAGE_JOB_RENAME_FILES = + "job_stage_rename_files"; + + + /** + * Job Setup Stage: {@value}. + */ + public static final String OP_STAGE_JOB_SETUP = "job_stage_setup"; + + /** + * Job saving _SUCCESS marker Stage: {@value}. + */ + public static final String OP_STAGE_JOB_SAVE_SUCCESS = + "job_stage_save_success_marker"; + + /** + * Output Validation (within job commit) Stage: {@value}. + */ + public static final String OP_STAGE_JOB_VALIDATE_OUTPUT = + "job_stage_optional_validate_output"; + + /** + * Task saving manifest file Stage: {@value}. + */ + public static final String OP_STAGE_TASK_SAVE_MANIFEST = + "task_stage_save_manifest"; + + /** + * Task Setup Stage: {@value}. + */ + public static final String OP_STAGE_TASK_SETUP = "task_stage_setup"; + + /** + * Task Commit Stage: {@value}. + */ + public static final String OP_STAGE_TASK_COMMIT = "task_stage_commit"; + + /** Task Scan directory Stage: {@value}. */ + public static final String OP_STAGE_TASK_SCAN_DIRECTORY + = "task_stage_scan_directory"; + + private ManifestCommitterStatisticNames() { + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java new file mode 100644 index 0000000000000..7020d5ca2d337 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java @@ -0,0 +1,137 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; + +import com.fasterxml.jackson.annotation.JsonInclude; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.util.JsonSerialization; + +import static java.util.Objects.requireNonNull; + +/** + * Class for single/multiple commit data structures. + */ +@SuppressWarnings("serial") +@InterfaceAudience.Private +@InterfaceStability.Unstable +@JsonInclude(JsonInclude.Include.NON_NULL) +public abstract class AbstractManifestData + implements Serializable, IOStatisticsSource { + + + /** + * Convert a path to a string which can be included in the JSON. + * @param path path + * @return a string value, or, if path==null, null. + */ + public static String marshallPath(@Nullable Path path) { + return path != null + ? path.toUri().toString() + : null; + } + + /** + * Convert a string path to Path type, by way of a URI. + * @param path path as a string + * @return path value + * @throws RuntimeException marshalling failure. + */ + public static Path unmarshallPath(String path) { + try { + return new Path(new URI(requireNonNull(path, "No path"))); + } catch (URISyntaxException e) { + throw new RuntimeException( + "Failed to parse \"" + path + "\" : " + e, + e); + + } + } + + /** + * Validate the data: those fields which must be non empty, must be set. + * @return the validated instance. + * @throws IOException if the data is invalid + */ + public abstract T validate() throws IOException; + + /** + * Serialize to JSON and then to a byte array, after performing a + * preflight validation of the data to be saved. + * @return the data in a persistable form. + * @throws IOException serialization problem or validation failure. + */ + public abstract byte[] toBytes() throws IOException; + + /** + * Save to a hadoop filesystem. + * @param fs filesystem + * @param path path + * @param overwrite should any existing file be overwritten + * @throws IOException IO exception + */ + public abstract void save(FileSystem fs, Path path, boolean overwrite) + throws IOException; + + /** + * Get a (usually shared) JSON serializer. + * @return a serializer. Call + */ + public abstract JsonSerialization createSerializer(); + + /** + * Verify that all instances in a collection are of the given class. + * @param it iterator + * @param classname classname to require + * @throws IOException on a failure + */ + void validateCollectionClass(Iterable it, Class classname) + throws IOException { + for (Object o : it) { + verify(o.getClass().equals(classname), + "Collection element is not a %s: %s", classname, o.getClass()); + } + } + + /** + * Verify that a condition holds. + * @param expression expression which must be true + * @param message message to raise on a failure + * @param args arguments for the message formatting + * @throws IOException on a failure + */ + + static void verify(boolean expression, + String message, + Object... args) throws IOException { + if (!expression) { + throw new IOException(String.format(message, args)); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java new file mode 100644 index 0000000000000..cb673383c58be --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DiagnosticKeys.java @@ -0,0 +1,44 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Diagnostic keys in the manifests. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class DiagnosticKeys { + /** + * Attribute added to diagnostics in _SUCCESS file. + */ + public static final String PRINCIPAL = "principal"; + public static final String STAGE = "stage"; + public static final String EXCEPTION = "exception"; + public static final String STACKTRACE = "stacktrace"; + + + /** Directory where manifests were renamed: {@value}. */ + public static final String MANIFESTS = "manifests"; + + private DiagnosticKeys() { + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java new file mode 100644 index 0000000000000..15e8cac779592 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/DirEntry.java @@ -0,0 +1,202 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.verify; + +/** + * A directory entry in the task manifest. + * Uses shorter field names for smaller files. + * Hash and equals are on dir name only; there's no real expectation + * that those operations are needed. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class DirEntry implements Serializable { + + private static final long serialVersionUID = 5658520530209859765L; + + /** + * Destination directory. + */ + @JsonProperty("d") + private String dir; + + /** + * Type of dest entry as found when probed for in task commit. + */ + @JsonProperty("t") + private int type; + + /** + * Level in the treewalk. + */ + @JsonProperty("l") + private int level; + + /** + * Constructor only for use by jackson. + * Do Not Delete. + */ + private DirEntry() { + } + + /** + * Construct an entry. + * + * @param dir destination path. + * @param type type of dest entry + * @param level Level in the treewalk. + * + */ + public DirEntry( + final String dir, + final int type, + final int level) { + this.dir = requireNonNull(dir); + this.type = type; + this.level = level; + } + + /** + * Construct an entry. + * + * @param dir destination path. + * @param type type of dest entry + * @param level Level in the treewalk. + * + */ + public DirEntry( + final Path dir, + final int type, + final int level) { + this(marshallPath(dir), type, level); + } + + public void setDir(final String dir) { + this.dir = dir; + } + + public String getDir() { + return dir; + } + + @JsonIgnore + public Path getDestPath() { + return unmarshallPath(dir); + } + + public int getType() { + return type; + } + + public void setType(final int type) { + this.type = type; + } + + public void setLevel(final int level) { + this.level = level; + } + + public int getLevel() { + return level; + } + + @JsonIgnore + public EntryStatus getStatus() { + return EntryStatus.toEntryStatus(type); + } + + @JsonIgnore + public void setStatus(EntryStatus status) { + setType(status.ordinal()); + } + public void validate() throws IOException { + final String s = toString(); + verify(dir != null && dir.length() > 0, + "destination path is missing from " + s); + verify(type >= 0, + "Invalid type in " + s); + verify(level >= 0, + "Invalid level in " + s); + } + + @Override + public String toString() { + return "DirEntry{" + + "dir='" + dir + '\'' + + ", type=" + type + + ", level=" + level + + '}'; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DirEntry dirEntry = (DirEntry) o; + return dir.equals(dirEntry.dir); + } + + @Override + public int hashCode() { + return Objects.hash(dir); + } + + /** + * A directory entry. + * @param dest destination path. + * @param type type + * @param level Level in the treewalk. + * @return an entry + */ + public static DirEntry dirEntry(Path dest, int type, int level) { + return new DirEntry(dest, type, level); + } + + /** + * A directory entry. + * @param dest destination path. + * @param type type + * @param level Level in the treewalk. + * @return an entry + */ + public static DirEntry dirEntry(Path dest, EntryStatus type, int level) { + return dirEntry(dest, type.ordinal(), level); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/EntryStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/EntryStatus.java new file mode 100644 index 0000000000000..73ac5d7f1b179 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/EntryStatus.java @@ -0,0 +1,87 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import javax.annotation.Nullable; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileStatus; + +/** + * Status of a file or dir entry, designed to be marshalled as + * an integer -the ordinal value of the enum is the + * wire value. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public enum EntryStatus { + + unknown, + not_found, + file, + dir, + created_dir; + + /** + * Go from a marshalled type to a status value. + * Any out of range value is converted to unknown. + * @param type type + * @return the status value. + */ + public static EntryStatus toEntryStatus(int type) { + switch (type) { + case 1: + return not_found; + case 2: + return file; + case 3: + return dir; + case 4: + return created_dir; + case 0: + default: + return unknown; + } + } + + + /** + * Go from the result of a getFileStatus call or + * listing entry to a status. + * A null argument is mapped to {@link #not_found} + * @param st file status + * @return the status enum. + */ + public static EntryStatus toEntryStatus(@Nullable FileStatus st) { + + if (st == null) { + return not_found; + } + if (st.isDirectory()) { + return dir; + } + if (st.isFile()) { + return file; + } + return unknown; + } + + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java new file mode 100644 index 0000000000000..bc6cdd94b3f26 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/FileEntry.java @@ -0,0 +1,189 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.verify; + +/** + * A File entry in the task manifest. + * Uses shorter field names for smaller files. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +@JsonInclude(JsonInclude.Include.NON_NULL) +public final class FileEntry implements Serializable { + + private static final long serialVersionUID = -550288489009777867L; + + @JsonProperty("s") + private String source; + + @JsonProperty("d") + private String dest; + + @JsonProperty("z") + private long size; + + /** + * Etag value if we can extract this. + */ + @JsonProperty("e") + private String etag; + + /** + * Constructor only for use by jackson. + * Do Not Delete. + */ + private FileEntry() { + } + + /** + * Construct an entry. + * @param source source path. + * @param dest destination path. + * @param size file size. + * @param etag optional etag + */ + public FileEntry( + final String source, + final String dest, + final long size, + final String etag) { + this.source = source; + this.dest = dest; + this.size = size; + this.etag = etag; + } + + + /** + * Construct an entry. + * @param source source path. + * @param dest destination path. + * @param size file size. + * @param etag optional etag + */ + public FileEntry( + final Path source, + final Path dest, + final long size, + final String etag) { + this(marshallPath(source), marshallPath(dest), size, etag); + } + + + public void setSource(final String source) { + this.source = source; + } + + public String getSource() { + return source; + } + + @JsonIgnore + public Path getSourcePath() { + return unmarshallPath(source); + } + + public void setDest(final String dest) { + this.dest = dest; + } + + public String getDest() { + return dest; + } + + @JsonIgnore + public Path getDestPath() { + return unmarshallPath(dest); + } + + public long getSize() { + return size; + } + + public void setSize(final long size) { + this.size = size; + } + + public String getEtag() { + return etag; + } + + public void setEtag(final String etag) { + this.etag = etag; + } + + public void validate() throws IOException { + final String s = toString(); + verify(source != null && source.length() > 0, + "Source is missing from " + s); + verify(dest != null && dest.length() > 0, + "Source is missing from " + s); + verify(size >= 0, + "Invalid size in " + s); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "FileOrDirEntry{"); + sb.append("source='").append(source).append('\''); + sb.append(", dest='").append(dest).append('\''); + sb.append(", size=").append(size); + sb.append(", etag='").append(etag).append('\''); + sb.append('}'); + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FileEntry that = (FileEntry) o; + return size == that.size && source.equals(that.source) && dest.equals( + that.dest) && + Objects.equals(etag, that.etag); + } + + @Override + public int hashCode() { + return Objects.hash(source, dest); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestPrinter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestPrinter.java new file mode 100644 index 0000000000000..c95ec7b11be05 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestPrinter.java @@ -0,0 +1,144 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import java.io.IOException; +import java.io.PrintStream; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.ExitUtil; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; + +/** + * Tool to print a manifest. + */ +public class ManifestPrinter extends Configured implements Tool { + + private static final String USAGE = "ManifestPrinter "; + + /** + * Output for printing. + */ + private final PrintStream out; + + /** + * Print to System.out. + */ + public ManifestPrinter() { + this(null, System.out); + } + + /** + * Print to the supplied stream. + * @param conf configuration + * @param out output + */ + public ManifestPrinter(Configuration conf, PrintStream out) { + super(conf); + this.out = out; + } + + @Override + public int run(String[] args) throws Exception { + if (args.length != 1) { + printUsage(); + return -1; + } + Path path = new Path(args[0]); + loadAndPrintManifest(path.getFileSystem(getConf()), path); + return 0; + } + + /** + * Load and print a manifest. + * @param fs filesystem. + * @param path path + * @throws IOException failure to load + * @return the manifest + */ + public ManifestSuccessData loadAndPrintManifest(FileSystem fs, Path path) + throws IOException { + // load the manifest + println("Manifest file: %s", path); + final ManifestSuccessData success = ManifestSuccessData.load(fs, path); + + printManifest(success); + return success; + } + + private void printManifest(ManifestSuccessData success) { + field("succeeded", success.getSuccess()); + field("created", success.getDate()); + field("committer", success.getCommitter()); + field("hostname", success.getHostname()); + field("description", success.getDescription()); + field("jobId", success.getJobId()); + field("jobIdSource", success.getJobIdSource()); + field("stage", success.getStage()); + println("Diagnostics\n%s", + success.dumpDiagnostics(" ", " = ", "\n")); + println("Statistics:\n%s", + ioStatisticsToPrettyString(success.getIOStatistics())); + out.flush(); + } + + private void printUsage() { + println(USAGE); + } + + /** + * Print a line to the output stream. + * @param format format string + * @param args arguments. + */ + private void println(String format, Object... args) { + out.format(format, args); + out.println(); + } + + /** + * Print a field, if non-null. + * @param name field name. + * @param value value. + */ + private void field(String name, Object value) { + if (value != null) { + println("%s: %s", name, value); + } + } + + /** + * Entry point. + */ + public static void main(String[] argv) throws Exception { + + try { + int res = ToolRunner.run(new ManifestPrinter(), argv); + System.exit(res); + } catch (ExitUtil.ExitException e) { + ExitUtil.terminate(e); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestSuccessData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestSuccessData.java new file mode 100644 index 0000000000000..73e73075ab3dc --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/ManifestSuccessData.java @@ -0,0 +1,493 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSupport; +import org.apache.hadoop.util.JsonSerialization; + +/** + * Summary data saved into a {@code _SUCCESS} marker file. + * + * This is a copy of the S3A committer success data format, with + * a goal of being/remaining compatible. + * This makes it easier for tests in downstream modules to + * be able to parse the success files from any of the committers. + * + * This should be considered public; it is based on the S3A + * format, which has proven stable over time. + * + * The JSON format SHOULD be considered public and evolving + * with compatibility across versions. + * + * All the Java serialization data is different and may change + * across versions with no stability guarantees other than + * "manifest summaries MAY be serialized between processes with + * the exact same version of this binary on their classpaths." + * That is sufficient for testing in Spark. + * + * To aid with Java serialization, the maps and lists are + * exclusively those which serialize well. + * IOStatisticsSnapshot has a lot of complexity in marshalling + * there; this class doesn't worry about concurrent access + * so is simpler. + * + */ +@SuppressWarnings({"unused", "CollectionDeclaredAsConcreteClass"}) +@InterfaceAudience.Public +@InterfaceStability.Unstable +@JsonInclude(JsonInclude.Include.NON_NULL) +public class ManifestSuccessData + extends AbstractManifestData { + + private static final Logger LOG = + LoggerFactory.getLogger(ManifestSuccessData.class); + + /** + * Supported version value: {@value}. + * If this is changed the value of {@link #serialVersionUID} will change, + * to avoid deserialization problems. + */ + public static final int VERSION = 1; + + /** + * Serialization ID: {@value}. + */ + private static final long serialVersionUID = 4755993198698104084L + VERSION; + + /** + * Name to include in persisted data, so as to differentiate from + * any other manifests: {@value}. + */ + public static final String NAME + = "org.apache.hadoop.fs.s3a.commit.files.SuccessData/" + VERSION; + + /** + * Name of file; includes version marker. + */ + private String name; + + /** Timestamp of creation. */ + private long timestamp; + + /** + * Did this succeed? + * It is implicitly true in a _SUCCESS file, but if the file + * is also saved to a log dir, then it depends on the outcome + */ + private boolean success = true; + + /** Timestamp as date string; no expectation of parseability. */ + private String date; + + /** + * Host which created the file (implicitly: committed the work). + */ + private String hostname; + + /** + * Committer name. + */ + private String committer; + + /** + * Description text. + */ + private String description; + + /** Job ID, if known. */ + private String jobId = ""; + + /** + * Source of the job ID. + */ + private String jobIdSource = ""; + + /** + * Metrics. + * Uses a treemap for serialization. + */ + private TreeMap metrics = new TreeMap<>(); + + /** + * Diagnostics information. + * Uses a treemap for serialization. + */ + private TreeMap diagnostics = new TreeMap<>(); + + /** + * Filenames in the commit. + */ + private ArrayList filenames = new ArrayList<>(0); + + /** + * IOStatistics. + */ + @JsonProperty("iostatistics") + private IOStatisticsSnapshot iostatistics = new IOStatisticsSnapshot(); + + /** + * State (committed, aborted). + */ + private String state; + + /** + * Stage: last stage executed. + */ + private String stage; + + @Override + public ManifestSuccessData validate() throws IOException { + verify(name != null, + "Incompatible file format: no 'name' field"); + verify(NAME.equals(name), + "Incompatible file format: " + name); + return this; + } + + @Override + public JsonSerialization createSerializer() { + return serializer(); + } + + @Override + public byte[] toBytes() throws IOException { + return serializer().toBytes(this); + } + + /** + * To JSON. + * @return json string value. + * @throws IOException failure + */ + public String toJson() throws IOException { + return serializer().toJson(this); + } + + @Override + public void save(FileSystem fs, Path path, boolean overwrite) + throws IOException { + // always set the name field before being saved. + name = NAME; + serializer().save(fs, path, this, overwrite); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ManifestSuccessData{"); + sb.append("committer='").append(committer).append('\''); + sb.append(", hostname='").append(hostname).append('\''); + sb.append(", description='").append(description).append('\''); + sb.append(", date='").append(date).append('\''); + sb.append(", filenames=[").append( + StringUtils.join(filenames, ", ")) + .append("]"); + sb.append('}'); + return sb.toString(); + } + + /** + * Dump the metrics (if any) to a string. + * The metrics are sorted for ease of viewing. + * @param prefix prefix before every entry + * @param middle string between key and value + * @param suffix suffix to each entry + * @return the dumped string + */ + public String dumpMetrics(String prefix, String middle, String suffix) { + return joinMap(metrics, prefix, middle, suffix); + } + + /** + * Dump the diagnostics (if any) to a string. + * @param prefix prefix before every entry + * @param middle string between key and value + * @param suffix suffix to each entry + * @return the dumped string + */ + public String dumpDiagnostics(String prefix, String middle, String suffix) { + return joinMap(diagnostics, prefix, middle, suffix); + } + + /** + * Join any map of string to value into a string, sorting the keys first. + * @param map map to join + * @param prefix prefix before every entry + * @param middle string between key and value + * @param suffix suffix to each entry + * @return a string for reporting. + */ + protected static String joinMap(Map map, + String prefix, + String middle, String suffix) { + if (map == null) { + return ""; + } + List list = new ArrayList<>(map.keySet()); + Collections.sort(list); + StringBuilder sb = new StringBuilder(list.size() * 32); + for (String k : list) { + sb.append(prefix) + .append(k) + .append(middle) + .append(map.get(k)) + .append(suffix); + } + return sb.toString(); + } + + /** + * Load an instance from a file, then validate it. + * @param fs filesystem + * @param path path + * @return the loaded instance + * @throws IOException IO failure + */ + public static ManifestSuccessData load(FileSystem fs, Path path) + throws IOException { + LOG.debug("Reading success data from {}", path); + ManifestSuccessData instance = serializer().load(fs, path); + instance.validate(); + return instance; + } + + /** + * Get a JSON serializer for this class. + * @return a serializer. + */ + public static JsonSerialization serializer() { + return new JsonSerialization<>(ManifestSuccessData.class, false, true); + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** @return timestamp of creation. */ + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + /** @return timestamp as date; no expectation of parseability. */ + public String getDate() { + return date; + } + + public void setDate(String date) { + this.date = date; + } + + /** + * @return host which created the file (implicitly: committed the work). + */ + public String getHostname() { + return hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + /** + * @return committer name. + */ + public String getCommitter() { + return committer; + } + + public void setCommitter(String committer) { + this.committer = committer; + } + + /** + * @return any description text. + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * @return any metrics. + */ + public Map getMetrics() { + return metrics; + } + + public void setMetrics(TreeMap metrics) { + this.metrics = metrics; + } + + /** + * @return a list of filenames in the commit. + */ + public List getFilenames() { + return filenames; + } + + /** + * Get the list of filenames as paths. + * @return the paths. + */ + @JsonIgnore + public List getFilenamePaths() { + return getFilenames().stream() + .map(AbstractManifestData::unmarshallPath) + .collect(Collectors.toList()); + } + + /** + * Set the list of filename paths. + */ + @JsonIgnore + public void setFilenamePaths(List paths) { + setFilenames(new ArrayList<>( + paths.stream() + .map(AbstractManifestData::marshallPath) + .collect(Collectors.toList()))); + } + + public void setFilenames(ArrayList filenames) { + this.filenames = filenames; + } + + public Map getDiagnostics() { + return diagnostics; + } + + public void setDiagnostics(TreeMap diagnostics) { + this.diagnostics = diagnostics; + } + + /** + * Add a diagnostics entry. + * @param key name + * @param value value + */ + public void putDiagnostic(String key, String value) { + diagnostics.put(key, value); + } + + /** @return Job ID, if known. */ + public String getJobId() { + return jobId; + } + + public void setJobId(String jobId) { + this.jobId = jobId; + } + + public String getJobIdSource() { + return jobIdSource; + } + + public void setJobIdSource(final String jobIdSource) { + this.jobIdSource = jobIdSource; + } + + @Override + public IOStatisticsSnapshot getIOStatistics() { + return iostatistics; + } + + public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { + this.iostatistics = ioStatistics; + } + + /** + * Set the IOStatistics to a snapshot of the source. + * @param iostats. Statistics; may be null. + */ + public void snapshotIOStatistics(IOStatistics iostats) { + setIOStatistics(IOStatisticsSupport.snapshotIOStatistics(iostats)); + } + + /** + * Set the success flag. + * @param success did the job succeed? + */ + public void setSuccess(boolean success) { + this.success = success; + } + + /** + * Get the success flag. + * @return did the job succeed? + */ + public boolean getSuccess() { + return success; + } + + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + public String getStage() { + return stage; + } + + /** + * Note a failure by setting success flag to false, + * then add the exception to the diagnostics. + * @param thrown throwable + */ + public void recordJobFailure(Throwable thrown) { + setSuccess(false); + String stacktrace = ExceptionUtils.getStackTrace(thrown); + diagnostics.put(DiagnosticKeys.EXCEPTION, thrown.toString()); + diagnostics.put(DiagnosticKeys.STACKTRACE, stacktrace); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/TaskManifest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/TaskManifest.java new file mode 100644 index 0000000000000..a06b837aba5dc --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/TaskManifest.java @@ -0,0 +1,365 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.files; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.util.JsonSerialization; + +/** + * This is the manifest of files which were created by + * this task attempt. + * + * Versioning: + * In a rolling cluster update, MR or Spark workers deployed on a newer + * node (and/or with a newer version of artifacts in a cluster-FS hosted + * tar.gz file) may be a later version of this class than that of + * job committer. + * If any changes are made to the manifest which are backwards compatible, + * this new manifest can still be loaded from JSON and processed. + * + * If the manifest is no longer compatible, the job output may + * be invalid. + * + * It is CRITICAL that the {@link #VERSION} constant is updated whenever + * such an incompatible change is made. + */ +@SuppressWarnings("unused") +@InterfaceAudience.Private +@InterfaceStability.Unstable +@JsonInclude(JsonInclude.Include.NON_NULL) +public class TaskManifest extends AbstractManifestData { + + /** + * Supported version value: {@value}. + * If this is changed the value of {@code serialVersionUID} will change, + * to avoid deserialization problems. + */ + public static final int VERSION = 1; + + /** + * Manifest type. + */ + public static final String TYPE = + "org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest/" + + VERSION; + + private static final Logger LOG = + LoggerFactory.getLogger(TaskManifest.class); + + /** + * Serialization version. + */ + private static final long serialVersionUID = 7090285511966046094L + VERSION; + + /** + * Manifest type. + */ + @JsonProperty("type") + private String type = TYPE; + + /** Version marker. */ + @JsonProperty("version") + private int version = VERSION; + + /** + * Job ID; constant over multiple attempts. + */ + @JsonProperty("jobId") + private String jobId; + + /** + * Number of the job attempt; starts at zero. + */ + @JsonProperty("jobAttemptNumber") + private int jobAttemptNumber; + + /** + * Task Attempt ID. + */ + @JsonProperty("taskID") + private String taskID; + + /** + * Task Attempt ID. + */ + @JsonProperty("taskAttemptID") + private String taskAttemptID; + + /** + * The task attempt directory. + */ + @JsonProperty("taskAttemptDir") + private String taskAttemptDir; + + /** + * The list of files to commit from this task attempt, including + * precalculated destination and size. + */ + @JsonProperty("files") + private final List filesToCommit = new ArrayList<>(); + + /** + * The list of directories needed by this task attempt, both + * source and destination. + * All these directories must exist in the destination before any of + * the files can be renamed there. + */ + @JsonProperty("directories") + private final List destDirectories = new ArrayList<>(); + + /** + * Any custom extra data committers may choose to add. + */ + private final Map extraData = new HashMap<>(0); + + /** + * IOStatistics. + */ + @JsonProperty("iostatistics") + private IOStatisticsSnapshot iostatistics = new IOStatisticsSnapshot(); + + /** + * Empty constructor; will be used by jackson as well as in application + * code. + */ + public TaskManifest() { + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public int getVersion() { + return version; + } + + public void setVersion(int version) { + this.version = version; + } + + @Override + public IOStatisticsSnapshot getIOStatistics() { + return iostatistics; + } + + public void setIOStatistics( + @Nullable final IOStatisticsSnapshot ioStatistics) { + this.iostatistics = ioStatistics; + } + + public String getJobId() { + return jobId; + } + + public void setJobId(final String jobId) { + this.jobId = jobId; + } + + public int getJobAttemptNumber() { + return jobAttemptNumber; + } + + public void setJobAttemptNumber(final int jobAttemptNumber) { + this.jobAttemptNumber = jobAttemptNumber; + } + + public String getTaskID() { + return taskID; + } + + public void setTaskID(final String taskID) { + this.taskID = taskID; + } + + public String getTaskAttemptID() { + return taskAttemptID; + } + + public void setTaskAttemptID(final String taskAttemptID) { + this.taskAttemptID = taskAttemptID; + } + + public String getTaskAttemptDir() { + return taskAttemptDir; + } + + public void setTaskAttemptDir(final String taskAttemptDir) { + this.taskAttemptDir = taskAttemptDir; + } + + /** + * Add a file to the list of files to commit. + * @param entry entry to add + */ + public void addFileToCommit(FileEntry entry) { + filesToCommit.add(entry); + } + + public List getFilesToCommit() { + return filesToCommit; + } + + /** + * Calculate the total amount of data which will be committed. + * @return the sum of sizes of all files to commit. + */ + @JsonIgnore + public long getTotalFileSize() { + return filesToCommit.stream().mapToLong(FileEntry::getSize).sum(); + } + + /** + * All the destination directories. + * @return directory list. + */ + public List getDestDirectories() { + return destDirectories; + } + + /** + * Add a directory to the list of directories to create. + * @param entry entry to add + */ + public void addDirectory(DirEntry entry) { + destDirectories.add(entry); + } + + public Map getExtraData() { + return extraData; + } + + @Override + public byte[] toBytes() throws IOException { + return serializer().toBytes(this); + } + + /** + * To JSON. + * @return json string value. + * @throws IOException failure + */ + public String toJson() throws IOException { + return serializer().toJson(this); + } + + @Override + public void save(FileSystem fs, Path path, boolean overwrite) + throws IOException { + serializer().save(fs, path, this, overwrite); + } + + /** + * Validate the data: those fields which must be non empty, must be set. + * @throws IOException if the data is invalid + * @return + */ + public TaskManifest validate() throws IOException { + verify(TYPE.equals(type), "Wrong type: %s", type); + verify(version == VERSION, "Wrong version: %s", version); + validateCollectionClass(extraData.keySet(), String.class); + validateCollectionClass(extraData.values(), String.class); + Set destinations = new HashSet<>(filesToCommit.size()); + validateCollectionClass(filesToCommit, FileEntry.class); + for (FileEntry c : filesToCommit) { + c.validate(); + verify(!destinations.contains(c.getDest()), + "Destination %s is written to by more than one pending commit", + c.getDest()); + destinations.add(c.getDest()); + } + return this; + } + + /** + * Get a JSON serializer for this class. + * @return a serializer. + */ + @Override + public JsonSerialization createSerializer() { + return serializer(); + } + + /** + * Create a JSON serializer for this class. + * @return a serializer. + */ + public static JsonSerialization serializer() { + return new JsonSerialization<>(TaskManifest.class, false, true); + } + + /** + * Load an instance from a file, then validate it. + * @param fs filesystem + * @param path path + * @return the loaded instance + * @throws IOException IO failure/the data is invalid + */ + public static TaskManifest load(FileSystem fs, Path path) + throws IOException { + LOG.debug("Reading Manifest in file {}", path); + return serializer().load(fs, path).validate(); + } + + /** + * Load an instance from a file, then validate it. + * If loading through a listing; use this API so that filestatus + * hints can be used. + * @param serializer serializer. + * @param fs filesystem + * @param path path to load from + * @param status status of file to load + * @return the loaded instance + * @throws IOException IO failure/the data is invalid + */ + public static TaskManifest load( + JsonSerialization serializer, + FileSystem fs, + Path path, + FileStatus status) + throws IOException { + LOG.debug("Reading Manifest in file {}", path); + return serializer.load(fs, path, status) + .validate(); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/package-info.java new file mode 100644 index 0000000000000..e1fbb4ac3325d --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/package-info.java @@ -0,0 +1,41 @@ +/* + * 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. + */ + +/** + * Persistence formats. + * These are the persistence formats used for passing data from tasks + * to the job committer + * {@link org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest}, + * and for a {@code _SUCCESS} file, which is in + * {@link org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData}. + * The {@code _SUCCESS} file is a copy of the S3A Committer + * {@code org.apache.hadoop.fs.s3a.commit.files.ManifestSuccessData}, + * the intent being that at the JSON-level they are compatible. + * This is to aid testing/validation and support calls, with one single + * format to load. + * + * Consult the individual formats for their declarations of access; + * the _SUCCESS file is one which tests may use. + * + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +package org.apache.hadoop.mapreduce.lib.output.committer.manifest.files; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/AuditingIntegration.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/AuditingIntegration.java new file mode 100644 index 0000000000000..de67a2a88c7d3 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/AuditingIntegration.java @@ -0,0 +1,94 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.audit.CommonAuditContext; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConfig; + +import static org.apache.hadoop.fs.audit.AuditConstants.PARAM_JOB_ID; +import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.CONTEXT_ATTR_STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.CONTEXT_ATTR_TASK_ATTEMPT_ID; + +/** + * Helper class to support integration with Hadoop 3.3.2+ Auditing. + * This MUST BE the sole place where fs.audit methods are used, so can be replaced + * by a stub class on any backport. + */ +@InterfaceAudience.Private +public final class AuditingIntegration { + private AuditingIntegration() { + } + + /** + * Add jobID to current context; also + * task attempt ID if set. + */ + public static void updateCommonContextOnCommitterEntry( + ManifestCommitterConfig committerConfig) { + CommonAuditContext context = currentAuditContext(); + context.put(PARAM_JOB_ID, + committerConfig.getJobUniqueId()); + // maybe the task attempt ID. + if (!committerConfig.getTaskAttemptId().isEmpty()) { + context.put(CONTEXT_ATTR_TASK_ATTEMPT_ID, + committerConfig.getTaskAttemptId()); + } + } + + /** + * Callback on stage entry. + * Sets the activeStage and updates the + * common context. + * @param stage new stage + */ + public static void enterStage(String stage) { + currentAuditContext().put(CONTEXT_ATTR_STAGE, stage); + } + + /** + * Remove stage from common audit context. + */ + public static void exitStage() { + currentAuditContext().remove(CONTEXT_ATTR_STAGE); + } + + /** + * Remove commit info at the end of the task or job. + */ + public static void updateCommonContextOnCommitterExit() { + currentAuditContext().remove(PARAM_JOB_ID); + currentAuditContext().remove(CONTEXT_ATTR_TASK_ATTEMPT_ID); + } + + /** + * Update the thread context with the stage name and + * job ID. + * This MUST be invoked at the start of methods invoked in helper threads, + * to ensure that they are all annotated with job and stage. + * @param jobId job ID. + * @param stage stage name. + */ + public static void enterStageWorker(String jobId, String stage) { + CommonAuditContext context = currentAuditContext(); + context.put(PARAM_JOB_ID, jobId); + context.put(CONTEXT_ATTR_STAGE, stage); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java new file mode 100644 index 0000000000000..15f9899f3551e --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/InternalConstants.java @@ -0,0 +1,130 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import java.util.Set; + +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; + +import org.apache.hadoop.classification.InterfaceAudience; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_CONTINUE_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_GET_FILE_STATUS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_IS_DIRECTORY; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_IS_FILE; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_LIST_STATUS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_MKDIRS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.*; + +/** + * Constants internal to the manifest committer. + */ +@InterfaceAudience.Private +public final class InternalConstants { + private InternalConstants() { + } + + /** + * Durations. + */ + public static final String[] DURATION_STATISTICS = { + + /* Job stages. */ + OP_STAGE_JOB_ABORT, + OP_STAGE_JOB_CLEANUP, + OP_STAGE_JOB_COMMIT, + OP_STAGE_JOB_CREATE_TARGET_DIRS, + OP_STAGE_JOB_LOAD_MANIFESTS, + OP_STAGE_JOB_RENAME_FILES, + OP_STAGE_JOB_SAVE_SUCCESS, + OP_STAGE_JOB_SETUP, + OP_STAGE_JOB_VALIDATE_OUTPUT, + + /* Task stages. */ + + OP_STAGE_TASK_ABORT_TASK, + OP_STAGE_TASK_COMMIT, + OP_STAGE_TASK_SAVE_MANIFEST, + OP_STAGE_TASK_SCAN_DIRECTORY, + OP_STAGE_TASK_SETUP, + + /* Lower level store/fs operations. */ + OP_COMMIT_FILE_RENAME, + OP_CREATE_DIRECTORIES, + OP_CREATE_ONE_DIRECTORY, + OP_DIRECTORY_SCAN, + OP_DELETE, + OP_DELETE_FILE_UNDER_DESTINATION, + OP_GET_FILE_STATUS, + OP_IS_DIRECTORY, + OP_IS_FILE, + OP_LIST_STATUS, + OP_LOAD_MANIFEST, + OP_LOAD_ALL_MANIFESTS, + OP_MKDIRS, + OP_MKDIRS_RETURNED_FALSE, + OP_MSYNC, + OP_PREPARE_DIR_ANCESTORS, + OP_RENAME_FILE, + OP_SAVE_TASK_MANIFEST, + + OBJECT_LIST_REQUEST, + OBJECT_CONTINUE_LIST_REQUEST, + + STORE_IO_RATE_LIMITED + }; + + /** + * Counters. + */ + public static final String[] COUNTER_STATISTICS = { + COMMITTER_BYTES_COMMITTED_COUNT, + COMMITTER_FILES_COMMITTED_COUNT, + COMMITTER_TASKS_COMPLETED_COUNT, + COMMITTER_TASKS_FAILED_COUNT, + COMMITTER_TASK_DIRECTORY_COUNT_MEAN, + COMMITTER_TASK_DIRECTORY_DEPTH_MEAN, + COMMITTER_TASK_FILE_COUNT_MEAN, + COMMITTER_TASK_FILE_SIZE_MEAN, + COMMITTER_TASK_MANIFEST_FILE_SIZE, + OP_COMMIT_FILE_RENAME_RECOVERED, + }; + + /** + * Error string from ABFS connector on timeout. + */ + public static final String OPERATION_TIMED_OUT = "OperationTimedOut"; + + /** + * Format string for task attempt names. + */ + public static final String NAME_FORMAT_TASK_ATTEMPT = "[Task-Attempt %s]"; + + /** + * Format string for job attempt names. + */ + public static final String NAME_FORMAT_JOB_ATTEMPT = "[Job-Attempt %s]"; + + /** Schemas of filesystems we know to not work with this committer. */ + public static final Set UNSUPPORTED_FS_SCHEMAS = + ImmutableSet.of("s3a", "wasb"); +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java new file mode 100644 index 0000000000000..f6edde5f460b4 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java @@ -0,0 +1,374 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import java.io.IOException; +import java.time.ZonedDateTime; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.EtagSource; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.statistics.IOStatisticsAggregator; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStoreBuilder; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.UserGroupInformation; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore; +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.PENDING_DIR_NAME; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.INITIAL_APP_ATTEMPT_ID; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ATTEMPT_DIR_FORMAT_STR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_DIR_FORMAT_STR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ID_SOURCE_MAPREDUCE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_ATTEMPT_SUBDIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_MANIFEST_SUBDIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SPARK_WRITE_UUID; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUMMARY_FILENAME_FORMAT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.TMP_SUFFIX; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.PRINCIPAL; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.COUNTER_STATISTICS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.DURATION_STATISTICS; + +/** + * Class for manifest committer support util methods. + */ + +@InterfaceAudience.Private +public final class ManifestCommitterSupport { + + private ManifestCommitterSupport() { + } + + /** + * Create an IOStatistics Store with the standard statistics + * set up. + * @return a store builder preconfigured with the standard stats. + */ + public static IOStatisticsStoreBuilder createIOStatisticsStore() { + + final IOStatisticsStoreBuilder store + = iostatisticsStore(); + + store.withCounters(COUNTER_STATISTICS); + store.withMaximums(COUNTER_STATISTICS); + store.withMinimums(COUNTER_STATISTICS); + store.withMeanStatistics(COUNTER_STATISTICS); + store.withDurationTracking(DURATION_STATISTICS); + return store; + } + + /** + * If the object is an IOStatisticsSource, get and add + * its IOStatistics. + * @param o source object. + */ + public static void maybeAddIOStatistics(IOStatisticsAggregator ios, + Object o) { + if (o instanceof IOStatisticsSource) { + ios.aggregate(((IOStatisticsSource) o).getIOStatistics()); + } + } + + /** + * Build a Job UUID from the job conf (if it is + * {@link ManifestCommitterConstants#SPARK_WRITE_UUID} + * or the MR job ID. + * @param conf job/task configuration + * @param jobId job ID from YARN or spark. + * @return (a job ID, source) + */ + public static Pair buildJobUUID(Configuration conf, + JobID jobId) { + String jobUUID = conf.getTrimmed(SPARK_WRITE_UUID, ""); + if (jobUUID.isEmpty()) { + jobUUID = jobId.toString(); + return Pair.of(jobUUID, JOB_ID_SOURCE_MAPREDUCE); + } else { + return Pair.of(jobUUID, SPARK_WRITE_UUID); + } + } + + /** + * Get the location of pending job attempts. + * @param out the base output directory. + * @return the location of pending job attempts. + */ + public static Path getPendingJobAttemptsPath(Path out) { + return new Path(out, PENDING_DIR_NAME); + } + + /** + * Get the Application Attempt Id for this job. + * @param context the context to look in + * @return the Application Attempt Id for a given job. + */ + public static int getAppAttemptId(JobContext context) { + return getAppAttemptId(context.getConfiguration()); + } + + /** + * Get the Application Attempt Id for this job + * by looking for {@link MRJobConfig#APPLICATION_ATTEMPT_ID} + * in the configuration, falling back to 0 if unset. + * For spark it will always be 0, for MR it will be set in the AM + * to the {@code ApplicationAttemptId} the AM is launched with. + * @param conf job configuration. + * @return the Application Attempt Id for the job. + */ + public static int getAppAttemptId(Configuration conf) { + return conf.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, + INITIAL_APP_ATTEMPT_ID); + } + + /** + * Get the path in the job attempt dir for a manifest for a task. + * @param manifestDir manifest directory + * @param taskId taskID. + * @return the final path to rename the manifest file to + */ + public static Path manifestPathForTask(Path manifestDir, String taskId) { + + return new Path(manifestDir, taskId + MANIFEST_SUFFIX); + } + + /** + * Get the path in the manifest subdir for the temp path to save a + * task attempt's manifest before renaming it to the + * path defined by {@link #manifestPathForTask(Path, String)}. + * @param manifestDir manifest directory + * @param taskAttemptId task attempt ID. + * @return the path to save/load the manifest. + */ + public static Path manifestTempPathForTaskAttempt(Path manifestDir, + String taskAttemptId) { + return new Path(manifestDir, + taskAttemptId + MANIFEST_SUFFIX + TMP_SUFFIX); + } + + /** + * Create a task attempt dir; stage config must be for a task attempt. + * @param stageConfig state config. + * @return a manifest with job and task attempt info set up. + */ + public static TaskManifest createTaskManifest(StageConfig stageConfig) { + final TaskManifest manifest = new TaskManifest(); + manifest.setTaskAttemptID(stageConfig.getTaskAttemptId()); + manifest.setTaskID(stageConfig.getTaskId()); + manifest.setJobId(stageConfig.getJobId()); + manifest.setJobAttemptNumber(stageConfig.getJobAttemptNumber()); + manifest.setTaskAttemptDir( + stageConfig.getTaskAttemptDir().toUri().toString()); + return manifest; + } + + /** + * Create success/outcome data. + * @param stageConfig configuration. + * @param stage + * @return a _SUCCESS object with some diagnostics. + */ + public static ManifestSuccessData createManifestOutcome( + StageConfig stageConfig, String stage) { + final ManifestSuccessData outcome = new ManifestSuccessData(); + outcome.setJobId(stageConfig.getJobId()); + outcome.setJobIdSource(stageConfig.getJobIdSource()); + outcome.setCommitter(MANIFEST_COMMITTER_CLASSNAME); + // real timestamp + outcome.setTimestamp(System.currentTimeMillis()); + final ZonedDateTime now = ZonedDateTime.now(); + outcome.setDate(now.toString()); + outcome.setHostname(NetUtils.getLocalHostname()); + // add some extra diagnostics which can still be parsed by older + // builds of test applications. + // Audit Span information can go in here too, in future. + try { + outcome.putDiagnostic(PRINCIPAL, + UserGroupInformation.getCurrentUser().getShortUserName()); + } catch (IOException ignored) { + // don't know who we are? exclude from the diagnostics. + } + outcome.putDiagnostic(STAGE, stage); + return outcome; + } + + /** + * Create the filename for a report from the jobID. + * @param jobId jobId + * @return filename for a report. + */ + public static String createJobSummaryFilename(String jobId) { + return String.format(SUMMARY_FILENAME_FORMAT, jobId); + } + + /** + * Get an etag from a FileStatus which MUST BE + * an implementation of EtagSource and + * whose etag MUST NOT BE null/empty. + * @param status the status; may be null. + * @return the etag or null if not provided + */ + public static String getEtag(FileStatus status) { + if (status instanceof EtagSource) { + return ((EtagSource) status).getEtag(); + } else { + return null; + } + } + + /** + * Create the manifest store operations for the given FS. + * This supports binding to custom filesystem handlers. + * @param conf configuration. + * @param filesystem fs. + * @param path path under FS. + * @return a bonded store operations. + * @throws IOException on binding/init problems. + */ + public static ManifestStoreOperations createManifestStoreOperations( + final Configuration conf, + final FileSystem filesystem, + final Path path) throws IOException { + try { + final Class storeClass = conf.getClass( + OPT_STORE_OPERATIONS_CLASS, + ManifestStoreOperationsThroughFileSystem.class, + ManifestStoreOperations.class); + final ManifestStoreOperations operations = storeClass. + getDeclaredConstructor().newInstance(); + operations.bindToFileSystem(filesystem, path); + return operations; + } catch (Exception e) { + throw new PathIOException(path.toString(), + "Failed to create Store Operations from configuration option " + + OPT_STORE_OPERATIONS_CLASS + + ":" + e, e); + } + } + + /** + * Logic to create directory names from job and attempt. + * This is self-contained it so it can be used in tests + * as well as in the committer. + */ + public static class AttemptDirectories { + + /** + * Job output path. + */ + private final Path outputPath; + + /** + * Path for the job attempt. + */ + private final Path jobAttemptDir; + + /** + * Path for the job. + */ + private final Path jobPath; + + /** + * Subdir under the job attempt dir where task + * attempts will have subdirectories. + */ + private final Path jobAttemptTaskSubDir; + + /** + * temp directory under job dest dir. + */ + private final Path outputTempSubDir; + + /** + * Directory to save manifests into. + */ + private final Path taskManifestDir; + + /** + * Build the attempt directories. + * @param outputPath output path + * @param jobUniqueId job ID/UUID + * @param jobAttemptNumber job attempt number + */ + public AttemptDirectories( + Path outputPath, + String jobUniqueId, + int jobAttemptNumber) { + this.outputPath = requireNonNull(outputPath, "Output path"); + + this.outputTempSubDir = new Path(outputPath, PENDING_DIR_NAME); + // build the path for the job + this.jobPath = new Path(outputTempSubDir, + String.format(JOB_DIR_FORMAT_STR, jobUniqueId)); + + // then the specific path underneath that for the attempt. + this.jobAttemptDir = new Path(jobPath, + String.format(JOB_ATTEMPT_DIR_FORMAT_STR, jobAttemptNumber)); + + // subdir for task attempts. + this.jobAttemptTaskSubDir = new Path(jobAttemptDir, JOB_TASK_ATTEMPT_SUBDIR); + + this.taskManifestDir = new Path(jobAttemptDir, JOB_TASK_MANIFEST_SUBDIR); + } + + public Path getOutputPath() { + return outputPath; + } + + public Path getJobAttemptDir() { + return jobAttemptDir; + } + + public Path getJobPath() { + return jobPath; + } + + public Path getJobAttemptTaskSubDir() { + return jobAttemptTaskSubDir; + } + + public Path getTaskAttemptPath(String taskAttemptId) { + return new Path(jobAttemptTaskSubDir, taskAttemptId); + } + + public Path getOutputTempSubDir() { + return outputTempSubDir; + } + + public Path getTaskManifestDir() { + return taskManifestDir; + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperations.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperations.java new file mode 100644 index 0000000000000..b81fa9dd32add --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperations.java @@ -0,0 +1,291 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.time.Duration; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.JsonSerialization; + +/** + * FileSystem operations which are needed to generate the task manifest. + * The specific choice of which implementation to use is configurable. + * Object store implementations MAY subclass if they + * need to implement resilient commit operations. + * However, the actual API MUST NOT be used outside + * the manifest committer and its tests. + */ +@InterfaceAudience.LimitedPrivate("mapreduce, object-stores") +@InterfaceStability.Unstable +public abstract class ManifestStoreOperations implements Closeable { + + /** + * Bind to the filesystem. + * This is called by the manifest committer after the operations + * have been instantiated. + * @param fileSystem target FS + * @param path actual path under FS. + * @throws IOException if there are binding problems. + */ + public void bindToFileSystem(FileSystem fileSystem, Path path) throws IOException { + + } + + /** + * Forward to {@link FileSystem#getFileStatus(Path)}. + * @param path path + * @return status + * @throws IOException failure. + */ + public abstract FileStatus getFileStatus(Path path) throws IOException; + + /** + * Is a path a file? Used during directory creation. + * The is a copy & paste of FileSystem.isFile(); + * {@code StoreOperationsThroughFileSystem} calls into + * the FS direct so that stores which optimize their probes + * can save on IO. + * @param path path to probe + * @return true if the path exists and resolves to a file + * @throws IOException failure other than FileNotFoundException + */ + public boolean isFile(Path path) throws IOException { + try { + return getFileStatus(path).isFile(); + } catch (FileNotFoundException e) { + return false; + } + } + + /** + * Forward to {@link FileSystem#delete(Path, boolean)}. + * If it returns without an error: there is nothing at + * the end of the path. + * @param path path + * @param recursive recursive delete. + * @return true if the path was deleted. + * @throws IOException failure. + */ + public abstract boolean delete(Path path, boolean recursive) + throws IOException; + + /** + * Forward to {@link FileSystem#mkdirs(Path)}. + * Usual "what does 'false' mean" ambiguity. + * @param path path + * @return true if the directory was created. + * @throws IOException failure. + */ + public abstract boolean mkdirs(Path path) throws IOException; + + /** + * Forward to {@link FileSystem#rename(Path, Path)}. + * Usual "what does 'false' mean" ambiguity. + * @param source source file + * @param dest destination path -which must not exist. + * @return the return value of the rename + * @throws IOException failure. + */ + public abstract boolean renameFile(Path source, Path dest) + throws IOException; + + /** + * Rename a dir; defaults to invoking + * Forward to {@link #renameFile(Path, Path)}. + * Usual "what does 'false' mean?" ambiguity. + * @param source source file + * @param dest destination path -which must not exist. + * @return true if the directory was created. + * @throws IOException failure. + */ + public boolean renameDir(Path source, Path dest) + throws IOException { + return renameFile(source, dest); + } + + /** + * List the directory. + * @param path path to list. + * @return an iterator over the results. + * @throws IOException any immediate failure. + */ + public abstract RemoteIterator listStatusIterator(Path path) + throws IOException; + + /** + * Load a task manifest from the store. + * with a real FS, this is done with + * {@link TaskManifest#load(JsonSerialization, FileSystem, Path, FileStatus)} + * + * @param serializer serializer. + * @param st status with the path and other data. + * @return the manifest + * @throws IOException failure to load/parse + */ + public abstract TaskManifest loadTaskManifest( + JsonSerialization serializer, + FileStatus st) throws IOException; + + /** + * Save a task manifest by {@code FileSystem.create(path)}. + * there's no attempt at renaming anything here. + * @param manifestData the manifest/success file + * @param path temp path for the initial save + * @param overwrite should create(overwrite=true) be used? + * @throws IOException failure to load/parse + */ + public abstract > void save( + T manifestData, + Path path, + boolean overwrite) throws IOException; + + /** + * Make an msync() call; swallow when unsupported. + * @param path path + * @throws IOException IO failure + */ + public void msync(Path path) throws IOException { + + } + + + /** + * Extract an etag from a status if the conditions are met. + * If the conditions are not met, return null or ""; they will + * both be treated as "no etags available" + *
+   *   1. The status is of a type which the implementation recognizes
+   *   as containing an etag.
+   *   2. After casting the etag field can be retrieved
+   *   3. and that value is non-null/non-empty.
+   * 
+ * @param status status, which may be null of any subclass of FileStatus. + * @return either a valid etag, or null or "". + */ + public String getEtag(FileStatus status) { + return ManifestCommitterSupport.getEtag(status); + } + + /** + * Does the store preserve etags through renames. + * If true, and if the source listing entry has an etag, + * it will be used to attempt to validate a failed rename. + * @param path path to probe. + * @return true if etag comparison is a valid strategy. + */ + public boolean storePreservesEtagsThroughRenames(Path path) { + return false; + } + + /** + * Does the store provide rename resilience through an + * implementation of {@link #commitFile(FileEntry)}? + * If true then that method will be invoked to commit work + * @return true if resilient commit support is available. + */ + public boolean storeSupportsResilientCommit() { + return false; + } + + /** + * Commit one file through any resilient API. + * This operation MUST rename source to destination, + * else raise an exception. + * The result indicates whether or not some + * form of recovery took place. + * + * If etags were collected during task commit, these will be + * in the entries passed in here. + * + * The base implementation always raises + * {@code UnsupportedOperationException} + * @param entry entry to commit + * @return the result of the commit + * @throws IOException failure. + * @throws UnsupportedOperationException if not available. + * + */ + public CommitFileResult commitFile(FileEntry entry) throws IOException { + throw new UnsupportedOperationException("Resilient commit not supported"); + } + + /** + * Outcome from the operation {@link #commitFile(FileEntry)}. + * As a rename failure MUST raise an exception, this result + * only declares whether or not some form of recovery took place. + */ + public static final class CommitFileResult { + + /** Did recovery take place? */ + private final boolean recovered; + + /** Time waiting for IO capacity, may be null. */ + @Nullable + private final Duration waitTime; + + /** + * Full commit result. + * @param recovered Did recovery take place? + * @param waitTime any time spent waiting for IO capacity. + */ + public static CommitFileResult fromResilientCommit( + final boolean recovered, + final Duration waitTime) { + return new CommitFileResult(recovered, waitTime); + } + + /** + * Full commit result. + * @param recovered Did recovery take place? + * @param waitTime any time spent waiting for IO capacity. + */ + public CommitFileResult(final boolean recovered, + @Nullable final Duration waitTime) { + + this.recovered = recovered; + this.waitTime = waitTime; + } + + /** + * Did some form of recovery take place? + * @return true if the commit succeeded through some form of (etag-based) recovery + */ + public boolean recovered() { + return recovered; + } + + @Nullable + public Duration getWaitTime() { + return waitTime; + } + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperationsThroughFileSystem.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperationsThroughFileSystem.java new file mode 100644 index 0000000000000..9a0b972bc735b --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestStoreOperationsThroughFileSystem.java @@ -0,0 +1,187 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.CommonPathCapabilities; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.JsonSerialization; + +/** + * Implementation of manifest store operations through the filesystem API. + * This class is subclassed in the ABFS module, which does add the resilient + * commit method. + */ +@InterfaceAudience.LimitedPrivate("mapreduce, object-stores") +@InterfaceStability.Unstable +public class ManifestStoreOperationsThroughFileSystem extends ManifestStoreOperations { + + /** + * Filesystem; set in {@link #bindToFileSystem(FileSystem, Path)}. + */ + private FileSystem fileSystem; + + /** + * Has a call to FileSystem.msync() failed as unsupported? + * If so, no new attempts will be made when + * (@link {@link #msync(Path)} is invoked. + */ + private boolean msyncUnsupported = false; + + /** + * Direct Constructor. + * @param fileSystem filesystem to write through. + */ + public ManifestStoreOperationsThroughFileSystem(final FileSystem fileSystem) { + this.fileSystem = fileSystem; + } + + /** + * Constructor used for introspection-based binding. + */ + public ManifestStoreOperationsThroughFileSystem() { + } + + @Override + public void close() throws IOException { + /* no-op; FS is assumed to be shared. */ + + } + + /** + * Get the filesystem. + * @return the filesystem; null until bound. + */ + public FileSystem getFileSystem() { + return fileSystem; + } + + @Override + public void bindToFileSystem(FileSystem filesystem, Path path) throws IOException { + fileSystem = filesystem; + } + + @Override + public FileStatus getFileStatus(Path path) throws IOException { + return fileSystem.getFileStatus(path); + } + + /** + * Using FileSystem.isFile to offer stores the option to optimize their probes. + * @param path path to probe + * @return true if the path resolves to a file. + * @throws IOException IO failure. + */ + @SuppressWarnings("deprecation") + @Override + public boolean isFile(Path path) throws IOException { + return fileSystem.isFile(path); + } + + @Override + public boolean delete(Path path, boolean recursive) + throws IOException { + return fileSystem.delete(path, recursive); + } + + @Override + public boolean mkdirs(Path path) + throws IOException { + return fileSystem.mkdirs(path); + } + + @Override + public boolean renameFile(Path source, Path dest) + throws IOException { + return fileSystem.rename(source, dest); + } + + @Override + public RemoteIterator listStatusIterator(Path path) + throws IOException { + return fileSystem.listStatusIterator(path); + } + + @Override + public TaskManifest loadTaskManifest( + JsonSerialization serializer, + FileStatus st) throws IOException { + return TaskManifest.load(serializer, fileSystem, st.getPath(), st); + } + + @Override + public > void save( + final T manifestData, + final Path path, + final boolean overwrite) throws IOException { + manifestData.save(fileSystem, path, overwrite); + } + + /** + * Probe filesystem capabilities. + * @param path path to probe. + * @return true if the FS declares its renames work. + */ + @Override + public boolean storePreservesEtagsThroughRenames(Path path) { + try { + return fileSystem.hasPathCapability(path, + CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME); + } catch (IOException ignored) { + return false; + } + } + + /** + * Invokes FileSystem msync(); swallows UnsupportedOperationExceptions. + * This ensures client metadata caches are in sync in an HDFS-HA deployment. + * No other filesystems support this; in the absence of a hasPathCapability() + * probe, after the operation is rejected, an atomic boolean is set + * to stop further attempts from even trying. + * @param path path + * @throws IOException failure to synchronize. + */ + @Override + public void msync(Path path) throws IOException { + // there's need for atomicity here, as the sole cost of + // multiple failures + if (msyncUnsupported) { + return; + } + // qualify so we can be confident that the FS being synced + // is the one we expect. + fileSystem.makeQualified(path); + try { + fileSystem.msync(); + } catch (UnsupportedOperationException ignored) { + // this exception is the default. + // set the unsupported flag so no future attempts are made. + msyncUnsupported = true; + } + } + +} \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/OutputValidationException.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/OutputValidationException.java new file mode 100644 index 0000000000000..f1dacc25fbcef --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/OutputValidationException.java @@ -0,0 +1,40 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; + +/** + * Exception raised during validation. + * This can be treated differently from other outcomes. + */ +@InterfaceAudience.Private +public class OutputValidationException extends PathIOException { + public OutputValidationException(Path path, String error) { + super(path.toUri().toString(), error); + } + + public OutputValidationException(Path path, + String error, + Throwable cause) { + super(path.toUri().toString(), error, cause); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/package-info.java new file mode 100644 index 0000000000000..eb7dda6cc77fe --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/package-info.java @@ -0,0 +1,30 @@ +/* + * 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. + */ + +/** + * Implementation classes for the manifest committer. + * Nothing outside this module should be using these classes, + * except where explicitly stated. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java new file mode 100644 index 0000000000000..c00ae5ad2249f --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java @@ -0,0 +1,34 @@ +/* + * 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. + */ + +/** + * Intermediate manifest committer. + * + * Optimized for object stores where listing is slow, directory renames may not + * be atomic, and the output is a deep tree of files intermixed with + * the output of (many) other task attempts. + * + * All classes in this module are private/unstable, except where stated. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.mapreduce.lib.output.committer.manifest; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbortTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbortTaskStage.java new file mode 100644 index 0000000000000..c2b44c2a924fd --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbortTaskStage.java @@ -0,0 +1,63 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_ABORT_TASK; + +/** + * Abort a task. + * + * This is done by deleting the task directory. + * Exceptions may/may not be suppressed. + */ +public class AbortTaskStage extends + AbstractJobOrTaskStage { + + private static final Logger LOG = LoggerFactory.getLogger( + AbortTaskStage.class); + + public AbortTaskStage(final StageConfig stageConfig) { + super(true, stageConfig, OP_STAGE_TASK_ABORT_TASK, false); + } + + /** + * Delete the task attempt directory. + * @param suppressExceptions should exceptions be ignored? + * @return the directory + * @throws IOException failure when exceptions were not suppressed + */ + @Override + protected Path executeStage(final Boolean suppressExceptions) + throws IOException { + final Path dir = getTaskAttemptDir(); + if (dir != null) { + LOG.info("{}: Deleting task attempt directory {}", getName(), dir); + deleteDir(dir, suppressExceptions); + } + return dir; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java new file mode 100644 index 0000000000000..05ee7a5ac11d7 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/AbstractJobOrTaskStage.java @@ -0,0 +1,942 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.util.OperationDuration; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.hadoop.util.functional.TaskPool; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_GET_FILE_STATUS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_IS_FILE; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_LIST_STATUS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_MKDIRS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_IO_RATE_LIMITED; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.createTracker; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME_RECOVERED; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_MANIFEST; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_MSYNC; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_RENAME_FILE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_SAVE_TASK_MANIFEST; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration.enterStageWorker; + +/** + * A Stage in Task/Job Commit. + * A stage can be executed once only, creating the return value of the + * {@link #apply(Object)} method, and, potentially, updating the state of the + * store via {@link ManifestStoreOperations}. + * IOStatistics will also be updated. + * Stages are expected to be combined to form the commit protocol. + * @param Type of arguments to the stage. + * @param Type of result. + */ +public abstract class AbstractJobOrTaskStage + implements JobOrTaskStage { + + private static final Logger LOG = LoggerFactory.getLogger( + AbstractJobOrTaskStage.class); + + /** + * Error text on rename failure: {@value}. + */ + public static final String FAILED_TO_RENAME_PREFIX = "Failed to "; + + /** + * Is this a task stage? If so, toString() includes task + * info.. + */ + private final boolean isTaskStage; + + /** + * Configuration of all the stages in the ongoing committer + * operation. + */ + private final StageConfig stageConfig; + + /** + * Name of the stage for statistics and logging. + */ + private final String stageStatisticName; + + /** + * Callbacks to update store. + * This is not made visible to the stages; they must + * go through the wrapper classes in this class, which + * add statistics and logging. + */ + private final ManifestStoreOperations operations; + + /** + * Submitter for doing IO against the store. + */ + private final TaskPool.Submitter ioProcessors; + + /** + * Used to stop any re-entrancy of the rename. + * This is an execute-once operation. + */ + private final AtomicBoolean executed = new AtomicBoolean(false); + + /** + * Tracker of the duration of the execution of the stage. + * set after {@link #executeStage(Object)} completes. + */ + private DurationTracker stageExecutionTracker; + + /** + * Name for logging. + */ + private final String name; + + /** + * Constructor. + * @param isTaskStage Is this a task stage? + * @param stageConfig stage-independent configuration. + * @param stageStatisticName name of the stage for statistics/logging + * @param requireIOProcessors are the IO processors required? + */ + protected AbstractJobOrTaskStage( + final boolean isTaskStage, + final StageConfig stageConfig, + final String stageStatisticName, + final boolean requireIOProcessors) { + this.isTaskStage = isTaskStage; + this.stageStatisticName = stageStatisticName; + this.stageConfig = stageConfig; + requireNonNull(stageConfig.getDestinationDir(), "Destination Directory"); + requireNonNull(stageConfig.getJobId(), "Job ID"); + requireNonNull(stageConfig.getJobAttemptDir(), "Job attempt directory"); + this.operations = requireNonNull(stageConfig.getOperations(), + "Operations callbacks"); + // and the processors of work if required. + this.ioProcessors = bindProcessor( + requireIOProcessors, + stageConfig.getIoProcessors()); + String stageName; + if (isTaskStage) { + // force fast failure. + getRequiredTaskId(); + getRequiredTaskAttemptId(); + getRequiredTaskAttemptDir(); + stageName = String.format("[Task-Attempt %s]", getRequiredTaskAttemptId()); + } else { + stageName = String.format("[Job-Attempt %s/%02d]", + stageConfig.getJobId(), + stageConfig.getJobAttemptNumber()); + } + name = stageName; + } + + /** + * Bind to the processor if it is required. + * @param required is the processor required? + * @param processor processor + * @return the processor binding + * @throws NullPointerException if required == true and processor is null. + */ + private TaskPool.Submitter bindProcessor( + final boolean required, + final TaskPool.Submitter processor) { + return required + ? requireNonNull(processor, "required IO processor is null") + : null; + } + + /** + * Stage entry point. + * Verifies that this is the first and only time the stage is invoked, + * then calls {@link #executeStage(Object)} for the subclass + * to perform its part of the commit protocol. + * The duration of the stage is collected as a statistic, and its + * entry/exit logged at INFO. + * @param arguments arguments to the function. + * @return the result. + * @throws IOException failures. + */ + @Override + public final OUT apply(final IN arguments) throws IOException { + executeOnlyOnce(); + progress(); + String stageName = getStageName(arguments); + getStageConfig().enterStage(stageName); + String statisticName = getStageStatisticName(arguments); + // The tracker here + LOG.info("{}: Executing Stage {}", getName(), stageName); + stageExecutionTracker = createTracker(getIOStatistics(), statisticName); + try { + // exec the input function and return its value + final OUT out = executeStage(arguments); + LOG.info("{}: Stage {} completed after {}", + getName(), + stageName, + OperationDuration.humanTime( + stageExecutionTracker.asDuration().toMillis())); + return out; + } catch (IOException | RuntimeException e) { + LOG.error("{}: Stage {} failed: after {}: {}", + getName(), + stageName, + OperationDuration.humanTime( + stageExecutionTracker.asDuration().toMillis()), + e.toString()); + LOG.debug("{}: Stage failure:", getName(), e); + // input function failed: note it + stageExecutionTracker.failed(); + // and rethrow + throw e; + } finally { + // update the tracker. + // this is called after the catch() call would have + // set the failed flag. + stageExecutionTracker.close(); + progress(); + getStageConfig().exitStage(stageName); + } + } + + /** + * The work of a stage. + * Executed exactly once. + * @param arguments arguments to the function. + * @return the result. + * @throws IOException failures. + */ + protected abstract OUT executeStage(IN arguments) throws IOException; + + /** + * Check that the operation has not been invoked twice. + * This is an atomic check. + * @throws IllegalStateException on a second invocation. + */ + private void executeOnlyOnce() { + Preconditions.checkState( + !executed.getAndSet(true), + "Stage attempted twice"); + } + + /** + * The stage statistic name. + * @param arguments args to the invocation. + * @return stage name. + */ + protected String getStageStatisticName(IN arguments) { + return stageStatisticName; + } + + /** + * Stage name for reporting; defaults to + * call {@link #getStageStatisticName(IN)}. + * @param arguments args to the invocation. + * @return name used in updating reports. + */ + protected String getStageName(IN arguments) { + return getStageStatisticName(arguments); + } + + /** + * Get the execution tracker; non-null + * after stage execution. + * @return a tracker or null. + */ + public DurationTracker getStageExecutionTracker() { + return stageExecutionTracker; + } + + /** + * Adds the duration of the job to an IOStatistics store + * (such as the manifest to be saved). + * @param iostats store + * @param statistic statistic name. + */ + public void addExecutionDurationToStatistics(IOStatisticsStore iostats, + String statistic) { + iostats.addTimedOperation( + statistic, + getStageExecutionTracker().asDuration()); + } + + /** + * Note any rate limiting to the given timing statistic. + * If the wait was 0, no statistics are updated. + * @param statistic statistic key. + * @param wait wait duration. + */ + private void noteAnyRateLimiting(String statistic, Duration wait) { + if (!wait.isZero()) { + // rate limiting took place + getIOStatistics().addTimedOperation( + statistic, + wait.toMillis()); + } + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "AbstractJobOrTaskStage{"); + sb.append(isTaskStage ? "Task Stage" : "Job Stage"); + sb.append(" name='").append(name).append('\''); + sb.append(" stage='").append(stageStatisticName).append('\''); + sb.append('}'); + return sb.toString(); + } + + /** + * The stage configuration. + * @return the stage configuration used by this stage. + */ + protected StageConfig getStageConfig() { + return stageConfig; + } + + /** + * Update the thread context with the stage name and + * job ID. + * This MUST be invoked at the start of methods invoked in helper threads, + * to ensure that they are all annotated with job and stage. + * @param stage stage name. + */ + protected void updateAuditContext(final String stage) { + enterStageWorker(stageConfig.getJobId(), stage); + } + + /** + * The IOStatistics are shared across all uses of the + * StageConfig. + * @return the (possibly shared) IOStatistics. + */ + @Override + public final IOStatisticsStore getIOStatistics() { + return stageConfig.getIOStatistics(); + } + + /** + * Call progress() on any Progressable passed in. + */ + protected final void progress() { + if (stageConfig.getProgressable() != null) { + stageConfig.getProgressable().progress(); + } + } + + /** + * Get a file status value or, if the path doesn't exist, return null. + * @param path path + * @return status or null + * @throws IOException IO Failure. + */ + protected final FileStatus getFileStatusOrNull( + final Path path) + throws IOException { + try { + return getFileStatus(path); + } catch (FileNotFoundException e) { + return null; + } + } + + /** + * Get a file status value or, if the path doesn't exist, return null. + * @param path path + * @return status or null + * @throws IOException IO Failure. + */ + protected final FileStatus getFileStatus( + final Path path) + throws IOException { + LOG.trace("{}: getFileStatus('{}')", getName(), path); + requireNonNull(path, + () -> String.format("%s: Null path for getFileStatus() call", getName())); + return trackDuration(getIOStatistics(), OP_GET_FILE_STATUS, () -> + operations.getFileStatus(path)); + } + + /** + * Get a file status value or, if the path doesn't exist, return null. + * @param path path + * @return true if the path resolves to a file + * @throws IOException IO Failure. + */ + protected final boolean isFile( + final Path path) + throws IOException { + LOG.trace("{}: isFile('{}')", getName(), path); + return trackDuration(getIOStatistics(), OP_IS_FILE, () -> { + return operations.isFile(path); + }); + } + + /** + * Delete a path. + * @param path path + * @param recursive recursive delete. + * @return status or null + * @throws IOException IO Failure. + */ + protected final boolean delete( + final Path path, + final boolean recursive) + throws IOException { + LOG.trace("{}: delete('{}, {}')", getName(), path, recursive); + return delete(path, recursive, OP_DELETE); + } + + /** + * Delete a path. + * @param path path + * @param recursive recursive delete. + * @param statistic statistic to update + * @return status or null + * @throws IOException IO Failure. + */ + protected Boolean delete( + final Path path, + final boolean recursive, + final String statistic) + throws IOException { + return trackDuration(getIOStatistics(), statistic, () -> { + return operations.delete(path, recursive); + }); + } + + /** + * Create a directory. + * @param path path + * @param escalateFailure escalate "false" to PathIOE + * @return true if the directory was created/exists. + * @throws IOException IO Failure. + */ + protected final boolean mkdirs( + final Path path, + final boolean escalateFailure) + throws IOException { + LOG.trace("{}: mkdirs('{}')", getName(), path); + return trackDuration(getIOStatistics(), OP_MKDIRS, () -> { + boolean success = operations.mkdirs(path); + if (!success && escalateFailure) { + throw new PathIOException(path.toUri().toString(), + stageStatisticName + ": mkdirs() returned false"); + } + return success; + }); + + } + + /** + * List all directly files under a path. + * Async implementations may under-report their durations. + * @param path path + * @return iterator over the results. + * @throws IOException IO Failure. + */ + protected final RemoteIterator listStatusIterator( + final Path path) + throws IOException { + LOG.trace("{}: listStatusIterator('{}')", getName(), path); + return trackDuration(getIOStatistics(), OP_LIST_STATUS, () -> + operations.listStatusIterator(path)); + } + + /** + * Load a manifest file. + * @param status source. + * @return the manifest. + * @throws IOException IO Failure. + */ + protected final TaskManifest loadManifest( + final FileStatus status) + throws IOException { + LOG.trace("{}: loadManifest('{}')", getName(), status); + return trackDuration(getIOStatistics(), OP_LOAD_MANIFEST, () -> + operations.loadTaskManifest( + stageConfig.currentManifestSerializer(), + status)); + } + + /** + * List all the manifests in the task manifest dir. + * @return a iterator of manifests. + * @throws IOException IO Failure. + */ + protected final RemoteIterator listManifests() + throws IOException { + return RemoteIterators.filteringRemoteIterator( + listStatusIterator(getTaskManifestDir()), + st -> st.getPath().toUri().toString().endsWith(MANIFEST_SUFFIX)); + } + + /** + * Make an msync() call; swallow when unsupported. + * @param path path + * @throws IOException IO failure + */ + protected final void msync(Path path) throws IOException { + LOG.trace("{}: msync('{}')", getName(), path); + trackDurationOfInvocation(getIOStatistics(), OP_MSYNC, () -> + operations.msync(path)); + } + + /** + * Create a directory -failing if it exists or if + * mkdirs() failed. + * @param operation operation for error reporting. + * @param path path path to create. + * @return the path. + * @throws IOException failure + * @throws PathIOException mkdirs failed. + * @throws FileAlreadyExistsException destination exists. + */ + protected final Path createNewDirectory( + final String operation, + final Path path) throws IOException { + LOG.trace("{}: {} createNewDirectory('{}')", getName(), operation, path); + requireNonNull(path, + () -> String.format("%s: Null path for operation %s", getName(), operation)); + // check for dir existence before trying to create. + try { + final FileStatus status = getFileStatus(path); + // no exception, so the path exists. + throw new FileAlreadyExistsException(operation + + ": path " + path + + " already exists and has status " + status); + } catch (FileNotFoundException e) { + // the path does not exist, so create it. + mkdirs(path, true); + return path; + } + } + + /** + * Assert that a path is a directory which must exist. + * @param operation operation for error reporting. + * @param path path path to create. + * @return the path + * @throws IOException failure + * @throws PathIOException mkdirs failed. + * @throws FileAlreadyExistsException destination exists. + */ + protected final Path directoryMustExist( + final String operation, + final Path path) throws IOException { + final FileStatus status = getFileStatus(path); + if (!status.isDirectory()) { + throw new PathIOException(path.toString(), + operation + + ": Path is not a directory; its status is :" + status); + } + return path; + } + + /** + * Save a task manifest or summary. This will be done by + * writing to a temp path and then renaming. + * If the destination path exists: Delete it. + * @param manifestData the manifest/success file + * @param tempPath temp path for the initial save + * @param finalPath final path for rename. + * @throws IOException failure to load/parse + */ + @SuppressWarnings("unchecked") + protected final void save(T manifestData, + final Path tempPath, + final Path finalPath) throws IOException { + LOG.trace("{}: save('{}, {}, {}')", getName(), manifestData, tempPath, finalPath); + trackDurationOfInvocation(getIOStatistics(), OP_SAVE_TASK_MANIFEST, () -> + operations.save(manifestData, tempPath, true)); + renameFile(tempPath, finalPath); + } + + /** + * Get an etag from a FileStatus which MUST BE + * an implementation of EtagSource and + * whose etag MUST NOT BE null/empty. + * @param status the status; may be null. + * @return the etag or null if not provided + */ + public String getEtag(FileStatus status) { + return operations.getEtag(status); + } + + /** + * Rename a file from source to dest; if the underlying FS API call + * returned false that's escalated to an IOE. + * @param source source file. + * @param dest dest file + * @throws IOException failure + * @throws PathIOException if the rename() call returned false. + */ + protected final void renameFile(final Path source, final Path dest) + throws IOException { + maybeDeleteDest(true, dest); + executeRenamingOperation("renameFile", source, dest, + OP_RENAME_FILE, () -> + operations.renameFile(source, dest)); + } + + /** + * Rename a file from source to dest; if the underlying FS API call + * returned false that's escalated to an IOE. + * @param source source file. + * @param dest dest file + * @throws IOException failure + * @throws PathIOException if the rename() call returned false. + */ + protected final void renameDir(final Path source, final Path dest) + throws IOException { + + maybeDeleteDest(true, dest); + executeRenamingOperation("renameDir", source, dest, + OP_RENAME_FILE, () -> + operations.renameDir(source, dest) + ); + } + + /** + * Commit a file from the manifest using rename or, if available, resilient renaming. + * @param entry entry from manifest + * @throws PathIOException if the rename() call returned false and was uprated. + * @throws IOException failure + */ + protected final CommitOutcome commitFile(FileEntry entry, + boolean deleteDest) + throws IOException { + + final Path source = entry.getSourcePath(); + final Path dest = entry.getDestPath(); + + maybeDeleteDest(deleteDest, dest); + if (storeSupportsResilientCommit()) { + // get the commit permits + final ManifestStoreOperations.CommitFileResult result = trackDuration(getIOStatistics(), + OP_COMMIT_FILE_RENAME, () -> + operations.commitFile(entry)); + if (result.recovered()) { + // recovery took place. + getIOStatistics().incrementCounter(OP_COMMIT_FILE_RENAME_RECOVERED); + } + if (result.getWaitTime() != null) { + // note any delay which took place + noteAnyRateLimiting(STORE_IO_RATE_LIMITED, result.getWaitTime()); + } + } else { + // commit with a simple rename; failures will be escalated. + executeRenamingOperation("renameFile", source, dest, + OP_COMMIT_FILE_RENAME, () -> + operations.renameFile(source, dest)); + } + return new CommitOutcome(); + } + + /** + * Does this store support resilient commit. + * @return true if resilient commit operations are available. + */ + protected boolean storeSupportsResilientCommit() { + return operations.storeSupportsResilientCommit(); + } + + private void maybeDeleteDest(final boolean deleteDest, final Path dest) throws IOException { + if (deleteDest) { + // delete the destination, always, knowing that it's a no-op if + // the data isn't there. Skipping the change saves one round trip + // to actually look for the file/object + boolean deleted = delete(dest, true); + // log the outcome in case of emergency diagnostics traces + // being needed. + LOG.debug("{}: delete('{}') returned {}'", getName(), dest, deleted); + } + } + + /** + * Execute an operation to rename a file/dir, commit a manifest entry. + * The statistic is tracked; returning false from the operation is considered + * a failure from the statistics perspective. + * @param operation operation name + * @param source source path + * @param dest dest path + * @param statistic statistic to track + * @param action callable of the operation + * @throws IOException on any failure + */ + private void executeRenamingOperation(String operation, + Path source, + Path dest, + String statistic, + CallableRaisingIOE action) throws IOException { + + LOG.debug("{}: {} '{}' to '{}')", getName(), operation, source, dest); + requireNonNull(source, "Null source"); + requireNonNull(dest, "Null dest"); + + // duration tracking is a bit convoluted as it + // ensures that rename failures as well as IOEs are + // treated as failures from a statistics perspective. + + DurationTracker tracker = createTracker(getIOStatistics(), statistic); + boolean success; + try { + success = action.apply(); + if (!success) { + // record failure in the tracker before closing it + tracker.failed(); + } + } catch (IOException | RuntimeException e) { + LOG.info("{}: {} raised an exception: {}", getName(), operation, e.toString()); + LOG.debug("{}: {} stack trace", getName(), operation, e); + tracker.failed(); + throw e; + } finally { + // success + // update the tracker. + tracker.close(); + } + // escalate the failure; this is done out of the duration tracker + // so its file status probes aren't included. + if (!success) { + throw escalateRenameFailure(operation, source, dest); + } + } + + /** + * Escalate a rename failure to an exception. + * Returns an error exception to throw if one was not + * triggered when probing for the source. + * @param operation operation name + * @param source source path + * @param dest dest path + * @return an exception to throw + * @throws IOException raised probing for source or dest + */ + private PathIOException escalateRenameFailure(String operation, + Path source, Path dest) throws IOException { + // rename just returned false. + // collect information for a meaningful error message + // and include in an exception raised. + + // get the source status; this will implicitly raise a FNFE. + final FileStatus sourceStatus = getFileStatus(source); + + // and look to see if there is anything at the destination + final FileStatus destStatus = getFileStatusOrNull(dest); + + LOG.error("{}: failure to {} {} to {} with" + + " source status {} " + + " and destination status {}", + getName(), operation, source, dest, + sourceStatus, destStatus); + + return new PathIOException(source.toString(), + FAILED_TO_RENAME_PREFIX + operation + " to " + dest); + } + + /** + * Outcome from the commit. + */ + public static final class CommitOutcome { + + } + + /** + * Job ID: never null. + */ + protected final String getJobId() { + return stageConfig.getJobId(); + } + + /** + * Job attempt number. + */ + protected final int getJobAttemptNumber() { + return stageConfig.getJobAttemptNumber(); + } + + /** + * ID of the task. + */ + protected final String getTaskId() { + return stageConfig.getTaskId(); + } + + /** + * Get the task ID; raise an NPE + * if it is null. + * @return a non-null task ID. + */ + protected final String getRequiredTaskId() { + return requireNonNull(getTaskId(), + "No Task ID in stage config"); + } + + /** + * ID of this specific attempt at a task. + */ + protected final String getTaskAttemptId() { + return stageConfig.getTaskAttemptId(); + } + + /** + * Get the task attempt ID; raise an NPE + * if it is null. + * @return a non-null task attempt ID. + */ + protected final String getRequiredTaskAttemptId() { + return requireNonNull(getTaskAttemptId(), + "No Task Attempt ID in stage config"); + } + + /** + * Job attempt dir. + */ + protected final Path getJobAttemptDir() { + return stageConfig.getJobAttemptDir(); + } + + /** + * Directory to put task manifests into. + * @return a path under the job attempt dir. + */ + protected final Path getTaskManifestDir() { + return stageConfig.getTaskManifestDir(); + } + + + /** + * Task attempt dir. + */ + protected final Path getTaskAttemptDir() { + return stageConfig.getTaskAttemptDir(); + } + + /** + * Get the task attemptDir; raise an NPE + * if it is null. + * @return a non-null task attempt dir. + */ + protected final Path getRequiredTaskAttemptDir() { + return requireNonNull(getTaskAttemptDir(), + "No Task Attempt Dir"); + } + + /** + * Destination of job. + */ + protected final Path getDestinationDir() { + return stageConfig.getDestinationDir(); + } + + /** + * Stage confog name, for logging. + * @return name. + */ + public final String getName() { + return name; + } + + /** + * Submitter for doing IO against the store other than + * manifest processing. + */ + protected final TaskPool.Submitter getIOProcessors() { + return ioProcessors; + } + + /** + * Submitter for doing IO against the store other than + * manifest processing. + * The size parameter is used to select between sequential + * and parallel runners. + * no data, or one entry: serial. + * everything else, parallel. + * @param size number of items. + * @return a submitter or null + */ + protected final TaskPool.Submitter getIOProcessors(int size) { + return size > 1 + ? getIOProcessors() + : null; + } + + /** + * Delete a directory, possibly suppressing exceptions. + * @param dir directory. + * @param suppressExceptions should exceptions be suppressed? + * @throws IOException exceptions raised in delete if not suppressed. + * @return any exception caught and suppressed + */ + protected IOException deleteDir( + final Path dir, + final Boolean suppressExceptions) + throws IOException { + try { + delete(dir, true); + return null; + } catch (IOException ex) { + LOG.info("Error deleting {}: {}", dir, ex.toString()); + if (!suppressExceptions) { + throw ex; + } else { + return ex; + } + } + } + + /** + * Create an entry for a file to rename under the destination. + * If the store operations supports extracting etags from file status + * entries, that is included in the entry + * @param status source file + * @param destDir destination directory + * @return an entry which includes the rename path + */ + protected FileEntry fileEntry(FileStatus status, Path destDir) { + // generate a new path under the dest dir + Path dest = new Path(destDir, status.getPath().getName()); + return new FileEntry(status.getPath(), + dest, + status.getLen(), + getEtag(status)); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java new file mode 100644 index 0000000000000..4a5971a5b1cb0 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CleanupJobStage.java @@ -0,0 +1,511 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.hadoop.util.functional.TaskPool; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED; +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED_DEFAULT; +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED; +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED_DEFAULT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_CLEANUP_PARALLEL_DELETE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_CLEANUP_PARALLEL_DELETE_DIRS_DEFAULT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CLEANUP; + +/** + * Clean up a job's temporary directory through parallel delete, + * base _temporary delete. + * Returns: the outcome of the overall operation + * The result is detailed purely for the benefit of tests, which need + * to make assertions about error handling and fallbacks. + * + * There's a few known issues with the azure and GCS stores which + * this stage tries to address. + * - Google GCS directory deletion is O(entries), so is slower for big jobs. + * - Azure storage directory delete, when using OAuth authentication or + * when not the store owner triggers a scan down the tree to verify the + * caller has the permission to delete each subdir. + * If this scan takes over 90s, the operation can time out. + * + * The main solution for both of these is that task attempts are + * deleted in parallel, in different threads. + * This will speed up GCS cleanup and reduce the risk of + * abfs related timeouts. + * Exceptions during cleanup can be suppressed, + * so that these do not cause the job to fail. + * + * Also, some users want to be able to run multiple independent jobs + * targeting the same output directory simultaneously. + * If one job deletes the directory `__temporary` all the others + * will fail. + * + * This can be addressed by disabling cleanup entirely. + * + */ +public class CleanupJobStage extends + AbstractJobOrTaskStage< + CleanupJobStage.Arguments, + CleanupJobStage.Result> { + + private static final Logger LOG = LoggerFactory.getLogger( + CleanupJobStage.class); + + /** + * Count of deleted directories. + */ + private final AtomicInteger deleteDirCount = new AtomicInteger(); + + /** + * Count of delete failures. + */ + private final AtomicInteger deleteFailureCount = new AtomicInteger(); + + /** + * Last delete exception; non null if deleteFailureCount is not zero. + */ + private IOException lastDeleteException; + + /** + * Stage name as passed in from arguments. + */ + private String stageName = OP_STAGE_JOB_CLEANUP; + + public CleanupJobStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_CLEANUP, true); + } + + /** + * Statistic name is extracted from the arguments. + * @param arguments args to the invocation. + * @return stage name. + */ + @Override + protected String getStageStatisticName(Arguments arguments) { + return arguments.statisticName; + } + + /** + * Clean up the job attempt directory tree. + * @param args arguments built up. + * @return the result. + * @throws IOException failure was raised an exceptions weren't surpressed. + */ + @Override + protected Result executeStage( + final Arguments args) + throws IOException { + stageName = getStageName(args); + // this is $dest/_temporary + final Path baseDir = requireNonNull(getStageConfig().getOutputTempSubDir()); + LOG.debug("{}: Cleaup of directory {} with {}", getName(), baseDir, args); + if (!args.enabled) { + LOG.info("{}: Cleanup of {} disabled", getName(), baseDir); + return new Result(Outcome.DISABLED, baseDir, + 0, null); + } + // shortcut of a single existence check before anything else + if (getFileStatusOrNull(baseDir) == null) { + return new Result(Outcome.NOTHING_TO_CLEAN_UP, + baseDir, + 0, null); + } + + Outcome outcome = null; + IOException exception; + + + // to delete. + LOG.info("{}: Deleting job directory {}", getName(), baseDir); + + if (args.deleteTaskAttemptDirsInParallel) { + // Attempt to do a parallel delete of task attempt dirs; + // don't overreact if a delete fails, but stop trying + // to delete the others, and fall back to deleting the + // job dir. + Path taskSubDir + = getStageConfig().getJobAttemptTaskSubDir(); + try (DurationInfo info = new DurationInfo(LOG, + "parallel deletion of task attempts in %s", + taskSubDir)) { + RemoteIterator dirs = + RemoteIterators.filteringRemoteIterator( + listStatusIterator(taskSubDir), + FileStatus::isDirectory); + TaskPool.foreach(dirs) + .executeWith(getIOProcessors()) + .stopOnFailure() + .suppressExceptions(false) + .run(this::rmTaskAttemptDir); + getIOStatistics().aggregate((retrieveIOStatistics(dirs))); + + if (getLastDeleteException() != null) { + // one of the task attempts failed. + throw getLastDeleteException(); + } + // success: record this as the outcome. + outcome = Outcome.PARALLEL_DELETE; + } catch (FileNotFoundException ex) { + // not a problem if there's no dir to list. + LOG.debug("{}: Task attempt dir {} not found", getName(), taskSubDir); + outcome = Outcome.DELETED; + } catch (IOException ex) { + // failure. Log and continue + LOG.info( + "{}: Exception while listing/deleting task attempts under {}; continuing", + getName(), + taskSubDir, ex); + // not overreacting here as the base delete will still get executing + outcome = Outcome.DELETED; + } + } + // Now the top-level deletion; exception gets saved + exception = deleteOneDir(baseDir); + if (exception != null) { + // failure, report and continue + // assume failure. + outcome = Outcome.FAILURE; + } else { + // if the outcome isn't already recorded as parallel delete, + // mark is a simple delete. + if (outcome == null) { + outcome = Outcome.DELETED; + } + } + + Result result = new Result( + outcome, + baseDir, + deleteDirCount.get(), + exception); + if (!result.succeeded() && !args.suppressExceptions) { + result.maybeRethrowException(); + } + + return result; + } + + /** + * Delete a single TA dir in a parallel task. + * Updates the audit context. + * Exceptions are swallowed so that attempts are still made + * to delete the others, but the first exception + * caught is saved in a field which can be retrieved + * via {@link #getLastDeleteException()}. + * + * @param status dir to be deleted. + * @throws IOException delete failure. + */ + private void rmTaskAttemptDir(FileStatus status) throws IOException { + // stage name in audit context is the one set in the arguments. + updateAuditContext(stageName); + // update the progress callback in case delete is really slow. + progress(); + deleteOneDir(status.getPath()); + } + + /** + * Delete a directory. + * The {@link #deleteFailureCount} counter. + * is incremented on every failure. + * @param dir directory + * @throws IOException if an IOE was raised + * @return any IOE raised. + */ + private IOException deleteOneDir(final Path dir) + throws IOException { + + deleteDirCount.incrementAndGet(); + IOException ex = deleteDir(dir, true); + if (ex != null) { + deleteFailure(ex); + } + return ex; + } + + /** + * Note a failure. + * @param ex exception + */ + private synchronized void deleteFailure(IOException ex) { + // excaption: add the count + deleteFailureCount.incrementAndGet(); + lastDeleteException = ex; + } + + /** + * Get the last delete exception; synchronized. + * @return the last delete exception or null. + */ + public synchronized IOException getLastDeleteException() { + return lastDeleteException; + } + + /** + * Options to pass down to the cleanup stage. + */ + public static final class Arguments { + + /** + * Statistic to update. + */ + private final String statisticName; + + /** Delete is enabled? */ + private final boolean enabled; + + /** Attempt parallel delete of task attempt dirs? */ + private final boolean deleteTaskAttemptDirsInParallel; + + /** Ignore failures? */ + private final boolean suppressExceptions; + + /** + * Arguments to the stage. + * @param statisticName stage name to report + * @param enabled is the stage enabled? + * @param deleteTaskAttemptDirsInParallel delete task attempt dirs in + * parallel? + * @param suppressExceptions suppress exceptions? + */ + public Arguments( + final String statisticName, + final boolean enabled, + final boolean deleteTaskAttemptDirsInParallel, + final boolean suppressExceptions) { + this.statisticName = statisticName; + this.enabled = enabled; + this.deleteTaskAttemptDirsInParallel = deleteTaskAttemptDirsInParallel; + this.suppressExceptions = suppressExceptions; + } + + public String getStatisticName() { + return statisticName; + } + + public boolean isEnabled() { + return enabled; + } + + public boolean isDeleteTaskAttemptDirsInParallel() { + return deleteTaskAttemptDirsInParallel; + } + + public boolean isSuppressExceptions() { + return suppressExceptions; + } + + @Override + public String toString() { + return "Arguments{" + + "statisticName='" + statisticName + '\'' + + ", enabled=" + enabled + + ", deleteTaskAttemptDirsInParallel=" + + deleteTaskAttemptDirsInParallel + + ", suppressExceptions=" + suppressExceptions + + '}'; + } + } + + /** + * Static disabled arguments. + */ + public static final Arguments DISABLED = new Arguments(OP_STAGE_JOB_CLEANUP, + false, + false, + false + ); + + /** + * Build an options argument from a configuration, using the + * settings from FileOutputCommitter and manifest committer. + * @param statisticName statistic name to use in duration tracking. + * @param conf configuration to use. + * @return the options to process + */ + public static Arguments cleanupStageOptionsFromConfig( + String statisticName, Configuration conf) { + + boolean enabled = !conf.getBoolean(FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED, + FILEOUTPUTCOMMITTER_CLEANUP_SKIPPED_DEFAULT); + boolean suppressExceptions = conf.getBoolean( + FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED, + FILEOUTPUTCOMMITTER_CLEANUP_FAILURES_IGNORED_DEFAULT); + boolean deleteTaskAttemptDirsInParallel = conf.getBoolean( + OPT_CLEANUP_PARALLEL_DELETE, + OPT_CLEANUP_PARALLEL_DELETE_DIRS_DEFAULT); + return new Arguments( + statisticName, + enabled, + deleteTaskAttemptDirsInParallel, + suppressExceptions + ); + } + + /** + * Enum of outcomes. + */ + public enum Outcome { + DISABLED("Disabled", false), + NOTHING_TO_CLEAN_UP("Nothing to clean up", true), + PARALLEL_DELETE("Parallel Delete of Task Attempt Directories", true), + DELETED("Delete of job directory", true), + FAILURE("Delete failed", false); + + private final String description; + + private final boolean success; + + Outcome(String description, boolean success) { + this.description = description; + this.success = success; + } + + @Override + public String toString() { + return "Outcome{" + name() + + " '" + description + '\'' + + "}"; + } + + /** + * description. + * @return text for logging + */ + public String getDescription() { + return description; + } + + /** + * Was this a success? + * @return true if this outcome is good. + */ + public boolean isSuccess() { + return success; + } + } + + /** + * Result of the cleanup. + * If the outcome == FAILURE but exceptions were suppressed + * (which they are implicitly if an instance of this object + * is created and returned), then the exception + * MUST NOT be null. + */ + public static final class Result { + + /** Outcome. */ + private final Outcome outcome; + + /** Directory cleaned up. */ + private final Path directory; + + /** + * Number of delete calls made across all threads. + */ + private final int deleteCalls; + + /** + * Any IOE raised. + */ + private final IOException exception; + + public Result( + final Outcome outcome, + final Path directory, + final int deleteCalls, + IOException exception) { + this.outcome = requireNonNull(outcome, "outcome"); + this.directory = directory; + this.deleteCalls = deleteCalls; + this.exception = exception; + if (outcome == Outcome.FAILURE) { + requireNonNull(exception, "No exception in failure result"); + } + } + + public Path getDirectory() { + return directory; + } + + public boolean wasExecuted() { + return outcome != Outcome.DISABLED; + } + + /** + * Was the outcome a success? + * That is: either the dir wasn't there or through + * delete/rename it is no longer there. + * @return true if the temporary dir no longer exists. + */ + public boolean succeeded() { + return outcome.isSuccess(); + } + + public Outcome getOutcome() { + return outcome; + } + + public int getDeleteCalls() { + return deleteCalls; + } + + public IOException getException() { + return exception; + } + + /** + * If there was an IOE caught, throw it. + * For ease of use in (meaningful) lambda expressions + * in tests, returns the string value if there + * was no exception to throw (for use in tests) + * @throws IOException exception. + */ + public String maybeRethrowException() throws IOException { + if (exception != null) { + throw exception; + } + return toString(); + } + + @Override + public String toString() { + return "CleanupResult{" + + "outcome=" + outcome + + ", directory=" + directory + + ", deleteCalls=" + deleteCalls + + ", exception=" + exception + + '}'; + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java new file mode 100644 index 0000000000000..a754f2a2da8df --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitJobStage.java @@ -0,0 +1,245 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; +import java.util.List; + +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; + +import static java.util.Objects.requireNonNull; +import static org.apache.commons.io.FileUtils.byteCountToDisplaySize; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.MANIFESTS; + +/** + * Commit the Job. + * Arguments (save manifest, validate output) + * Inputs: saveMarker: boolean, validateOutput: boolean + * Outputs: SuccessData + */ +public class CommitJobStage extends + AbstractJobOrTaskStage< + CommitJobStage.Arguments, + CommitJobStage.Result> { + + private static final Logger LOG = LoggerFactory.getLogger( + CommitJobStage.class); + + public CommitJobStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_COMMIT, true); + } + + @Override + protected CommitJobStage.Result executeStage( + final CommitJobStage.Arguments arguments) throws IOException { + + LOG.info("{}: Committing job \"{}\". resilient commit supported = {}", + getName(), + getJobId(), + storeSupportsResilientCommit()); + + boolean createMarker = arguments.isCreateMarker(); + + // load the manifests + final StageConfig stageConfig = getStageConfig(); + LoadManifestsStage.Result result + = new LoadManifestsStage(stageConfig).apply(true); + List manifests = result.getManifests(); + LoadManifestsStage.SummaryInfo summary = result.getSummary(); + + LOG.debug("{}: Job Summary {}", getName(), summary); + LOG.info("{}: Committing job with file count: {}; total size {} bytes", + getName(), + summary.getFileCount(), + byteCountToDisplaySize(summary.getTotalFileSize())); + + + // add in the manifest statistics to our local IOStatistics for + // reporting. + IOStatisticsStore iostats = getIOStatistics(); + iostats.aggregate(summary.getIOStatistics()); + + // prepare destination directories. + final CreateOutputDirectoriesStage.Result dirStageResults = + new CreateOutputDirectoriesStage(stageConfig) + .apply(manifests); + + // commit all the tasks. + // The success data includes a snapshot of the IO Statistics + // and hence all aggregate stats from the tasks. + ManifestSuccessData successData; + successData = new RenameFilesStage(stageConfig).apply( + Pair.of(manifests, dirStageResults.getCreatedDirectories())); + if (LOG.isDebugEnabled()) { + LOG.debug("{}: _SUCCESS file summary {}", getName(), successData.toJson()); + } + // update the counter of bytes committed and files. + // use setCounter so as to ignore any values accumulated when + // aggregating tasks. + iostats.setCounter( + COMMITTER_FILES_COMMITTED_COUNT, + summary.getFileCount()); + iostats.setCounter( + COMMITTER_BYTES_COMMITTED_COUNT, + summary.getTotalFileSize()); + successData.snapshotIOStatistics(iostats); + + + // rename manifests. Only warn on failure here. + final String manifestRenameDir = arguments.getManifestRenameDir(); + if (isNotBlank(manifestRenameDir)) { + Path manifestRenamePath = new Path( + new Path(manifestRenameDir), + getJobId()); + LOG.info("{}: Renaming manifests to {}", getName(), manifestRenamePath); + try { + renameDir(getTaskManifestDir(), manifestRenamePath); + + // save this path in the summary diagnostics + successData.getDiagnostics().put(MANIFESTS, manifestRenamePath.toUri().toString()); + } catch (IOException | IllegalArgumentException e) { + // rename failure, including path for wrong filesystem + LOG.warn("{}: Failed to rename manifests to {}", getName(), manifestRenamePath, e); + } + } + + // save the _SUCCESS if the option is enabled. + Path successPath = null; + if (createMarker) { + // save a snapshot of the IO Statistics + + successPath = new SaveSuccessFileStage(stageConfig) + .apply(successData); + LOG.debug("{}: Saving _SUCCESS file to {}", getName(), successPath); + } + + // optional cleanup + new CleanupJobStage(stageConfig).apply(arguments.getCleanupArguments()); + + // and then, after everything else: optionally validate. + if (arguments.isValidateOutput()) { + // cache and restore the active stage field + LOG.info("{}: Validating output.", getName()); + new ValidateRenamedFilesStage(stageConfig) + .apply(result.getManifests()); + } + + // restore the active stage so that when the report is saved + // it is declared as job commit, not cleanup or validate. + stageConfig.enterStage(getStageName(arguments)); + + // the result + return new CommitJobStage.Result(successPath, successData); + } + + /** + * Arguments for job commit. + */ + public static final class Arguments { + + /** create the _SUCCESS marker? */ + private final boolean createMarker; + + /** perform validation checks on the files? */ + private final boolean validateOutput; + + /** optional directory to rename the task manifests to. */ + private final String manifestRenameDir; + + /** cleanup arguments.. */ + private final CleanupJobStage.Arguments cleanupArguments; + + /** + * + * @param createMarker create the _SUCCESS marker? + * @param validateOutput perform validation checks on the files? + * @param manifestRenameDir optional directory to rename the task manifests to + * @param cleanupArguments cleanup arguments. + */ + public Arguments( + boolean createMarker, + boolean validateOutput, + @Nullable String manifestRenameDir, + CleanupJobStage.Arguments cleanupArguments) { + + this.createMarker = createMarker; + this.validateOutput = validateOutput; + this.manifestRenameDir = manifestRenameDir; + this.cleanupArguments = requireNonNull(cleanupArguments); + } + + public boolean isCreateMarker() { + return createMarker; + } + + public boolean isValidateOutput() { + return validateOutput; + } + + public String getManifestRenameDir() { + return manifestRenameDir; + } + + public CleanupJobStage.Arguments getCleanupArguments() { + return cleanupArguments; + } + } + + /** + * Result of the stage. + */ + public static final class Result { + /** + * Manifest success data. + */ + private final ManifestSuccessData jobSuccessData; + + /** + * Success file path. null if not saved. + */ + private final Path successPath; + + public Result(final Path successPath, + ManifestSuccessData jobSuccessData) { + this.successPath = successPath; + this.jobSuccessData = jobSuccessData; + } + + public ManifestSuccessData getJobSuccessData() { + return jobSuccessData; + } + + public Path getSuccessPath() { + return successPath; + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitTaskStage.java new file mode 100644 index 0000000000000..bf5ba27ab8ad5 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CommitTaskStage.java @@ -0,0 +1,115 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; + +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_COMMIT; + +/** + * Commit a task attempt. + * Scan the task attempt directories through + * {@link TaskAttemptScanDirectoryStage} + * and then save to the task manifest path at + * {@link SaveTaskManifestStage}. + */ + +public class CommitTaskStage extends + AbstractJobOrTaskStage { + private static final Logger LOG = LoggerFactory.getLogger( + CommitTaskStage.class); + + public CommitTaskStage(final StageConfig stageConfig) { + super(true, stageConfig, OP_STAGE_TASK_COMMIT, false); + } + + /** + * Scan the task attempt dir then save the manifest. + * A snapshot of the IOStats will be included in the manifest; + * this includes the scan time. + * @param arguments arguments to the function. + * @return the path the manifest was saved to, and the manifest. + * @throws IOException IO failure. + */ + @Override + protected CommitTaskStage.Result executeStage(final Void arguments) + throws IOException { + LOG.info("{}: Committing task \"{}\"", getName(), getTaskAttemptId()); + + // execute the scan + final TaskAttemptScanDirectoryStage scanStage = + new TaskAttemptScanDirectoryStage(getStageConfig()); + TaskManifest manifest = scanStage.apply(arguments); + + // add the scan as task commit. It's not quite, as it doesn't include + // the saving, but ... + scanStage.addExecutionDurationToStatistics(getIOStatistics(), OP_STAGE_TASK_COMMIT); + + // save a snapshot of the IO Statistics + final IOStatisticsSnapshot manifestStats = snapshotIOStatistics(); + manifestStats.aggregate(getIOStatistics()); + manifest.setIOStatistics(manifestStats); + + // Now save with rename + Path manifestPath = new SaveTaskManifestStage(getStageConfig()) + .apply(manifest); + return new CommitTaskStage.Result(manifestPath, manifest); + } + + /** + * Result of the stage. + */ + public static final class Result { + /** The path the manifest was saved to. */ + private final Path path; + /** The manifest. */ + private final TaskManifest taskManifest; + + public Result(Path path, + TaskManifest taskManifest) { + this.path = path; + this.taskManifest = taskManifest; + } + + /** + * Get the manifest path. + * @return The path the manifest was saved to. + */ + public Path getPath() { + return path; + } + + /** + * Get the manifest. + * @return The manifest. + */ + public TaskManifest getTaskManifest() { + return taskManifest; + } + + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java new file mode 100644 index 0000000000000..927004e619742 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/CreateOutputDirectoriesStage.java @@ -0,0 +1,423 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.EntryStatus; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.functional.TaskPool; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.measureDurationOfInvocation; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_CREATE_DIRECTORIES; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_DELETE_FILE_UNDER_DESTINATION; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_MKDIRS_RETURNED_FALSE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_PREPARE_DIR_ANCESTORS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CREATE_TARGET_DIRS; +import static org.apache.hadoop.util.OperationDuration.humanTime; + +/** + * Prepare the destination directory tree, as efficiently as possible. + * possible -and doing those IO operations in the thread pool. + * + * The classic FileOutputCommitter does a recursive treewalk and + * deletes any files found at paths where directories are to be created. + * + * Each task manifest's directories are combined with those of the other tasks + * to build a set of all directories which are needed, without duplicates. + * + * This stage requires the aggregate set of manifests to contain + * all directories to create, including level, + * and expects them to have been probed for existence/state. + * + * For each level, all dirs are processed in parallel to + * be created or, if files, deleted. + * + * The stage returns the list of directories created, and for testing, + * the map of paths to outcomes. + * + * Directory creation can be surprisingly slow against object stores, + * do use benchmarks from real test runs when tuning this algorithm. + */ +public class CreateOutputDirectoriesStage extends + AbstractJobOrTaskStage< + List, + CreateOutputDirectoriesStage.Result> { + + private static final Logger LOG = LoggerFactory.getLogger( + CreateOutputDirectoriesStage.class); + + /** + * Directories as a map of (path, path). + * Using a map rather than any set for efficient concurrency; the + * concurrent sets don't do lookups so fast. + */ + private final Map dirMap = new ConcurrentHashMap<>(); + + /** + * A list of created paths for the results. + */ + private final List createdDirectories = new ArrayList<>(); + + public CreateOutputDirectoriesStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_CREATE_TARGET_DIRS, true); + // add the dest dir to the dir map as we expect the job setup to create it. + dirMap.put(getDestinationDir(), DirMapState.dirWasCreated); + } + + @Override + protected Result executeStage( + final List taskManifests) + throws IOException { + + final List directories = createAllDirectories(taskManifests); + LOG.debug("{}: Created {} directories", getName(), directories.size()); + return new Result(new HashSet<>(directories), dirMap); + } + + /** + * For each task, build the list of directories it wants. + * @param taskManifests task manifests + * @return the list of paths which have been created. + */ + private List createAllDirectories(final List taskManifests) + throws IOException { + + // all directories which need to exist across all + // tasks. + // leaf directories + final Map leaves = new HashMap<>(); + // parent directories. these do not need to be + // explicitly created. + final Map parents = new HashMap<>(); + // the files which must be deleted as a directory + // will be created at that path. + final Set filesToDelete = new HashSet<>(); + + // iterate through the task manifests + // and all output dirs into the set of dirs to + // create. + // hopefully there is a lot of overlap, so the + // final number of dirs to create is small. + for (TaskManifest task: taskManifests) { + final List destDirectories = task.getDestDirectories(); + Collections.sort(destDirectories, (o1, o2) -> + o1.getLevel() - o2.getLevel()); + for (DirEntry entry: destDirectories) { + // add the dest entry + final Path path = entry.getDestPath(); + if (!leaves.containsKey(path)) { + leaves.put(path, entry); + + // if it is a file to delete, record this. + if (entry.getStatus() == EntryStatus.file) { + filesToDelete.add(path); + } + final Path parent = path.getParent(); + if (parent != null && leaves.containsKey(parent)) { + // there's a parent dir, move it from the leaf list + // to parent list + parents.put(parent, + leaves.remove(parent)); + } + } + } + } + + // at this point then there is a map of all directories which + // are leaf entries and so need to be created if not present, + // and the maximum level is known. + // we can iterate through all levels deleting any files if there are any. + + // Prepare parent directories. + deleteFiles(filesToDelete); + + // Now the real work. + final int createCount = leaves.size(); + LOG.info("Preparing {} directory/directories", createCount); + // now probe for and create the leaf dirs, which are those at the + // bottom level + Duration d = measureDurationOfInvocation(getIOStatistics(), OP_CREATE_DIRECTORIES, () -> + TaskPool.foreach(leaves.values()) + .executeWith(getIOProcessors(createCount)) + .onFailure(this::reportMkDirFailure) + .stopOnFailure() + .run(this::createOneDirectory)); + LOG.info("Time to prepare directories {}", humanTime(d.toMillis())); + return createdDirectories; + } + + /** + * How many failures have been reported. + */ + private final AtomicInteger failureCount = new AtomicInteger(); + + /** + * report a single directory failure. + * @param path path which could not be deleted + * @param e exception raised. + */ + private void reportMkDirFailure(DirEntry dirEntry, Exception e) { + Path path = dirEntry.getDestPath(); + final int count = failureCount.incrementAndGet(); + LOG.warn("{}: mkdir failure #{} Failed to create directory \"{}\": {}", + getName(), count, path, e.toString()); + LOG.debug("{}: Full exception details", + getName(), e); + } + + /** + * Delete all directories where there is a file. + * @param filesToDelete set of dirs to where there is a file. + * @throws IOException IO problem + */ + private void deleteFiles(final Set filesToDelete) + throws IOException { + + final int size = filesToDelete.size(); + if (size == 0) { + // nothing to delete. + return; + } + LOG.info("{}: Directory entries containing files to delete: {}", getName(), size); + Duration d = measureDurationOfInvocation(getIOStatistics(), + OP_PREPARE_DIR_ANCESTORS, () -> + TaskPool.foreach(filesToDelete) + .executeWith(getIOProcessors(size)) + .stopOnFailure() + .run(dir -> { + updateAuditContext(OP_PREPARE_DIR_ANCESTORS); + deleteDirWithFile(dir); + })); + LOG.info("Time to delete files {}", humanTime(d.toMillis())); + } + + /** + * Prepare a parent directory. + * @param dir directory to probe + * @throws IOException failure in probe other than FNFE + */ + private void deleteDirWithFile(Path dir) throws IOException { + // report progress back + progress(); + LOG.info("{}: Deleting file {}", getName(), dir); + delete(dir, false, OP_DELETE); + // note its final state + addToDirectoryMap(dir, DirMapState.fileNowDeleted); + } + + + /** + * Create a directory is required, updating the directory map + * and, if the operation took place, the list of created dirs. + * Reports progress on invocation. + * @param dirEntry entry + * @throws IOException failure. + */ + private void createOneDirectory(final DirEntry dirEntry) throws IOException { + // report progress back + progress(); + final Path dir = dirEntry.getDestPath(); + updateAuditContext(OP_STAGE_JOB_CREATE_TARGET_DIRS); + final DirMapState state = maybeCreateOneDirectory(dirEntry); + switch (state) { + case dirFoundInStore: + addToDirectoryMap(dir, state); + break; + case dirWasCreated: + case dirCreatedOnSecondAttempt: + addCreatedDirectory(dir); + addToDirectoryMap(dir, state); + break; + default: + break; + } + + } + + + /** + * Try to efficiently and robustly create a directory in a method which is + * expected to be executed in parallel with operations creating + * peer directories. + * @param path path to create + * @return true if dir created/found + * @throws IOException IO Failure. + */ + private DirMapState maybeCreateOneDirectory(DirEntry dirEntry) throws IOException { + final EntryStatus status = dirEntry.getStatus(); + if (status == EntryStatus.dir) { + return DirMapState.dirFoundInStore; + } + // present in case directories are ever created in task commits + if (status == EntryStatus.created_dir) { + return DirMapState.dirWasCreated; + } + + // here the dir doesn't exist because + // it was a file and has been deleted, or + // checks failed. create it. + final Path path = dirEntry.getDestPath(); + + LOG.info("Creating directory {}", path); + + try { + if (mkdirs(path, false)) { + // success -return immediately. + return DirMapState.dirWasCreated; + } + getIOStatistics().incrementCounter(OP_MKDIRS_RETURNED_FALSE); + + LOG.info("{}: mkdirs({}) returned false, attempting to recover", + getName(), path); + } catch (IOException e) { + // can be caused by file existing, etc. + LOG.info("{}: mkdir({}) raised exception {}", getName(), path, e.toString()); + LOG.debug("{}: Mkdir stack", getName(), e); + } + + // fallback to checking the FS, in case a different process did it. + FileStatus st = getFileStatusOrNull(path); + if (st != null) { + if (!st.isDirectory()) { + // is bad: delete a file + LOG.info("{}: Deleting file where a directory should go: {}", + getName(), st); + delete(path, false, OP_DELETE_FILE_UNDER_DESTINATION); + } else { + // is good. + LOG.warn("{}: Even though mkdirs({}) failed, there is now a directory there", + getName(), path); + return DirMapState.dirFoundInStore; + } + } else { + // nothing found. This should never happen. + LOG.warn("{}: Although mkdirs({}) returned false, there's nothing at that path to prevent it", + getName(), path); + + } + + // try to create the directory again + // if this fails, and IOE is still raised, that + // propagate to the caller. + if (!mkdirs(path, false)) { + + // mkdirs failed again + getIOStatistics().incrementCounter(OP_MKDIRS_RETURNED_FALSE); + + // require the dir to exist, raising an exception if it does not. + directoryMustExist("Creating directory ", path); + } + + // we only get here if the second attempt recovered + return DirMapState.dirCreatedOnSecondAttempt; + + } + + /** + * Add a created dir to the list of created dirs. + * @param dir new dir. + */ + private void addCreatedDirectory(final Path dir) { + synchronized (createdDirectories) { + createdDirectories.add(dir); + } + } + + /** + * Add a dir to the directory map if there is not already an entry there. + * @param dir directory. + * @param state state of entry + */ + private void addToDirectoryMap(final Path dir, + DirMapState state) { + if (!dirMap.containsKey(dir)) { + dirMap.put(dir, state); + } + } + + + /** + * Result of the operation. + */ + public static final class Result { + + /** directories created. */ + private final Set createdDirectories; + + /** + * Map of dirs built up during preparation. + */ + private final Map dirMap; + + public Result(Set createdDirectories, + Map dirMap) { + this.createdDirectories = requireNonNull(createdDirectories); + this.dirMap = requireNonNull(dirMap); + } + + public Set getCreatedDirectories() { + return createdDirectories; + } + + public Map getDirMap() { + return dirMap; + } + + @Override + public String toString() { + return "Result{" + + "directory count=" + createdDirectories.size() + + '}'; + } + } + + /** + * Enumeration of dir states in the dir map. + */ + public enum DirMapState { + dirFoundInStore, + dirFoundInMap, + dirWasCreated, + dirCreatedOnSecondAttempt, + fileNowDeleted, + ancestorWasDirOrMissing, + parentWasNotFile, + parentOfCreatedDir + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/JobOrTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/JobOrTaskStage.java new file mode 100644 index 0000000000000..98975bf7f195c --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/JobOrTaskStage.java @@ -0,0 +1,33 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.util.functional.FunctionRaisingIOE; + +/** + * Job Stage is a function raising an IOE. and which also + * provides IOStats. + * @param Type of arguments to the stage. + * @param Type of result. + */ +public interface JobOrTaskStage extends FunctionRaisingIOE, + IOStatisticsSource { + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java new file mode 100644 index 0000000000000..3720f549d89c3 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/LoadManifestsStage.java @@ -0,0 +1,293 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.functional.TaskPool; + +import static org.apache.commons.io.FileUtils.byteCountToDisplaySize; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_MANIFEST_FILE_SIZE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_ALL_MANIFESTS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_LOAD_MANIFESTS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.maybeAddIOStatistics; + +/** + * Stage to load all the task manifests in the job attempt directory. + * Invoked in Job Commit. + * Manifests are loaded in parallel. + * The IOStatistics snapshot passed in is built up with the statistics + * and the statistics stripped from the manifest if prune == true. + * This keeps the memory footprint of each manifest down. + */ +public class LoadManifestsStage extends + AbstractJobOrTaskStage< + Boolean, + LoadManifestsStage.Result> { + + private static final Logger LOG = LoggerFactory.getLogger( + LoadManifestsStage.class); + + /** + * Summary of manifest loading. + */ + private final SummaryInfo summaryInfo = new SummaryInfo(); + + /** + * Should manifests be pruned of IOStatistics? + */ + private boolean pruneManifests; + + /** + * List of loaded manifests. + */ + private final List manifests = new ArrayList<>(); + + public LoadManifestsStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_LOAD_MANIFESTS, true); + } + + /** + * Load the manifests. + * @param prune should manifests be pruned of IOStatistics? + * @return the summary and a list of manifests. + * @throws IOException IO failure. + */ + @Override + protected LoadManifestsStage.Result executeStage( + final Boolean prune) throws IOException { + + final Path manifestDir = getTaskManifestDir(); + LOG.info("{}: Executing Manifest Job Commit with manifests in {}", + getName(), + manifestDir); + pruneManifests = prune; + // build a list of all task manifests successfully committed + // + msync(manifestDir); + final RemoteIterator manifestFiles = listManifests(); + + final List manifestList = loadAllManifests(manifestFiles); + LOG.info("{}: Summary of {} manifests loaded in {}: {}", + getName(), + manifestList.size(), + manifestDir, + summaryInfo); + + // collect any stats + maybeAddIOStatistics(getIOStatistics(), manifestFiles); + return new LoadManifestsStage.Result(summaryInfo, manifestList); + } + + /** + * Load all the manifests. + * @param manifestFiles list of manifest files. + * @return the loaded manifests. + * @throws IOException IO Failure. + */ + private List loadAllManifests( + final RemoteIterator manifestFiles) throws IOException { + + trackDurationOfInvocation(getIOStatistics(), OP_LOAD_ALL_MANIFESTS, () -> + TaskPool.foreach(manifestFiles) + .executeWith(getIOProcessors()) + .stopOnFailure() + .run(this::processOneManifest)); + return manifests; + } + + /** + * Method invoked to process one manifest. + * @param status file to process. + * @throws IOException failure to load/parse + */ + private void processOneManifest(FileStatus status) + throws IOException { + updateAuditContext(OP_LOAD_ALL_MANIFESTS); + + TaskManifest m = fetchTaskManifest(status); + progress(); + + // update the manifest list in a synchronized block. + + synchronized (manifests) { + manifests.add(m); + // and the summary info in the same block, to + // eliminate the need to acquire a second lock. + summaryInfo.add(m); + } + if (pruneManifests) { + m.setIOStatistics(null); + m.getExtraData().clear(); + } + } + + /** + * Precommit preparation of a single manifest file. + * To reduce the memory foot print, the IOStatistics and + * extra data of each manifest is cleared. + * @param status status of file. + * @return number of files. + * @throws IOException IO Failure. + */ + private TaskManifest fetchTaskManifest(FileStatus status) + throws IOException { + if (status.getLen() == 0 || !status.isFile()) { + throw new PathIOException(status.getPath().toString(), + "Not a valid manifest file; file status = " + status); + } + // load the manifest, which includes validation. + final TaskManifest manifest = loadManifest(status); + final String id = manifest.getTaskAttemptID(); + final int filecount = manifest.getFilesToCommit().size(); + final long size = manifest.getTotalFileSize(); + LOG.info("{}: Task Attempt {} file {}: File count: {}; data size={}", + getName(), id, status.getPath(), filecount, size); + // record file size for tracking of memory consumption. + getIOStatistics().addMeanStatisticSample(COMMITTER_TASK_MANIFEST_FILE_SIZE, + status.getLen()); + return manifest; + } + + /** + * Result of the stage. + */ + public static final class Result { + private final SummaryInfo summary; + + private final List manifests; + + public Result(SummaryInfo summary, + List manifests) { + this.summary = summary; + this.manifests = manifests; + } + + public SummaryInfo getSummary() { + return summary; + } + + public List getManifests() { + return manifests; + } + + } + + /** + * Summary information. + */ + public static final class SummaryInfo implements IOStatisticsSource { + + /** + * Aggregate IOStatistics. + */ + private IOStatisticsSnapshot iostatistics = snapshotIOStatistics(); + + /** + * How many manifests were loaded. + */ + private long manifestCount; + + /** + * Total number of files to rename. + */ + private long fileCount; + + /** + * Total number of directories which may need + * to be created. + * As there is no dedup, this is likely to be + * a (major) overestimate. + */ + private long directoryCount; + + /** + * Total amount of data to be committed. + */ + private long totalFileSize; + + /** + * Get the IOStatistics. + * @return aggregate IOStatistics + */ + @Override + public IOStatisticsSnapshot getIOStatistics() { + return iostatistics; + } + + public long getFileCount() { + return fileCount; + } + + public long getDirectoryCount() { + return directoryCount; + } + + public long getTotalFileSize() { + return totalFileSize; + } + + public long getManifestCount() { + return manifestCount; + } + + /** + * Add all statistics. + * @param manifest manifest to add. + */ + public void add(TaskManifest manifest) { + manifestCount++; + iostatistics.aggregate(manifest.getIOStatistics()); + fileCount += manifest.getFilesToCommit().size(); + directoryCount += manifest.getDestDirectories().size(); + totalFileSize += manifest.getTotalFileSize(); + } + + /** + * To String includes all summary info except statistics. + * @return string value + */ + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "SummaryInfo{"); + sb.append("manifestCount=").append(manifestCount); + sb.append(", fileCount=").append(fileCount); + sb.append(", directoryCount=").append(directoryCount); + sb.append(", totalFileSize=").append( + byteCountToDisplaySize(totalFileSize)); + sb.append('}'); + return sb.toString(); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java new file mode 100644 index 0000000000000..e8124c11465e5 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/RenameFilesStage.java @@ -0,0 +1,173 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.functional.TaskPool; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER_FILE_LIMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_RENAME_FILES; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createManifestOutcome; +import static org.apache.hadoop.thirdparty.com.google.common.collect.Iterables.concat; + +/** + * This stage renames all the files. + * Input: the manifests and the set of directories created, as returned by + * {@link CreateOutputDirectoriesStage}. + * If the job is configured to delete target files, if the parent dir + * had to be created, the delete() call can be skipped. + * It returns a manifest success data file summarizing the + * output, but does not add iostatistics to it. + */ +public class RenameFilesStage extends + AbstractJobOrTaskStage< + Pair, Set>, + ManifestSuccessData> { + + private static final Logger LOG = LoggerFactory.getLogger( + RenameFilesStage.class); + + /** + * List of all files committed. + */ + private final List filesCommitted = new ArrayList<>(); + + /** + * Total file size. + */ + private long totalFileSize = 0; + + private Set createdDirectories; + + public RenameFilesStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_RENAME_FILES, true); + } + + /** + * Get the list of files committed. + * Access is not synchronized. + * @return direct access to the list of files. + */ + public synchronized List getFilesCommitted() { + return filesCommitted; + } + + /** + * Get the total file size of the committed task. + * @return a number greater than or equal to zero. + */ + public synchronized long getTotalFileSize() { + return totalFileSize; + } + + /** + * Rename files in job commit. + * @param taskManifests a list of task manifests containing files. + * @return the job report. + * @throws IOException failure + */ + @Override + protected ManifestSuccessData executeStage( + Pair, Set> args) + throws IOException { + + final List taskManifests = args.getLeft(); + createdDirectories = args.getRight(); + + final ManifestSuccessData success = createManifestOutcome(getStageConfig(), + OP_STAGE_JOB_COMMIT); + final int manifestCount = taskManifests.size(); + + LOG.info("{}: Executing Manifest Job Commit with {} manifests in {}", + getName(), manifestCount, getTaskManifestDir()); + + // first step is to aggregate the output of all manifests into a single + // list of files to commit. + // Which Guava can do in a zero-copy concatenated iterator + + final Iterable filesToCommit = concat(taskManifests.stream() + .map(TaskManifest::getFilesToCommit) + .collect(Collectors.toList())); + + TaskPool.foreach(filesToCommit) + .executeWith(getIOProcessors()) + .stopOnFailure() + .run(this::commitOneFile); + + // synchronized block to keep spotbugs happy. + List committed = getFilesCommitted(); + LOG.info("{}: Files committed: {}. Total size {}", + getName(), committed.size(), getTotalFileSize()); + + // Add a subset of the destination files to the success file; + // enough for simple testing + success.setFilenamePaths( + committed + .subList(0, Math.min(committed.size(), SUCCESS_MARKER_FILE_LIMIT)) + .stream().map(FileEntry::getDestPath) + .collect(Collectors.toList())); + + success.setSuccess(true); + + return success; + } + + /** + * Commit one file by rename, then, if that doesn't fail, + * add to the files committed list. + * @param entry entry to commit. + * @throws IOException faiure. + */ + private void commitOneFile(FileEntry entry) throws IOException { + updateAuditContext(OP_STAGE_JOB_RENAME_FILES); + + // report progress back + progress(); + + // if the dest dir is to be deleted, + // look to see if the parent dir was created. + // if it was. we know that the file doesn't exist. + final boolean deleteDest = getStageConfig().getDeleteTargetPaths() + && !createdDirectories.contains(entry.getDestPath().getParent()); + // do the rename + commitFile(entry, deleteDest); + + // update the list and IOStats + synchronized (this) { + filesCommitted.add(entry); + totalFileSize += entry.getSize(); + } + + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveSuccessFileStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveSuccessFileStage.java new file mode 100644 index 0000000000000..eb9c82f2ae739 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveSuccessFileStage.java @@ -0,0 +1,79 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.TMP_SUFFIX; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_SAVE_SUCCESS; + +/** + * Save the _SUCCESS file to the destination directory + * via a temp file in the job attempt dir. + * Returns the path of the file + */ +public class SaveSuccessFileStage extends + AbstractJobOrTaskStage { + + private static final Logger LOG = LoggerFactory.getLogger( + SaveSuccessFileStage.class); + + public SaveSuccessFileStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_SAVE_SUCCESS, false); + } + + /** + * Stage name is always job commit. + * @param arguments args to the invocation. + * @return stage name + */ + @Override + protected String getStageName(ManifestSuccessData arguments) { + // set it to the job commit stage, always. + return OP_STAGE_JOB_COMMIT; + } + + /** + * Execute. + * @param successData success data to save + * @return path saved to. + * @throws IOException failure + */ + @Override + protected Path executeStage(final ManifestSuccessData successData) + throws IOException { + // Save the marker + Path successFile = getStageConfig().getJobSuccessMarkerPath(); + Path successTempFile = new Path(getJobAttemptDir(), SUCCESS_MARKER + TMP_SUFFIX); + LOG.debug("{}: Saving _SUCCESS file to {} via {}", successFile, + getName(), + successTempFile); + save(successData, successTempFile, successFile); + return successFile; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveTaskManifestStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveTaskManifestStage.java new file mode 100644 index 0000000000000..fdaf0184cda20 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SaveTaskManifestStage.java @@ -0,0 +1,81 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_SAVE_MANIFEST; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestPathForTask; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestTempPathForTaskAttempt; + +/** + * Save a task manifest to the job attempt dir, using the task + * ID for the name of the final file. + * For atomic writes, the manifest is saved + * by writing to a temp file and then renaming it. + * Uses both the task ID and task attempt ID to determine the temp filename; + * Before the rename of (temp, final-path), any file at the final path + * is deleted. + * This is so that when this stage is invoked in a task commit, its output + * overwrites any of the first commit. + * When it succeeds, therefore, unless there is any subsequent commit of + * another task, the task manifest at the final path is from this + * operation. + * + * Returns the path where the manifest was saved. + */ +public class SaveTaskManifestStage extends + AbstractJobOrTaskStage { + + private static final Logger LOG = LoggerFactory.getLogger( + SaveTaskManifestStage.class); + + public SaveTaskManifestStage(final StageConfig stageConfig) { + super(true, stageConfig, OP_STAGE_TASK_SAVE_MANIFEST, false); + } + + /** + * Save the manifest to a temp file and rename to the final + * manifest destination. + * @param manifest manifest + * @return the path to the final entry + * @throws IOException IO failure. + */ + @Override + protected Path executeStage(final TaskManifest manifest) + throws IOException { + + final Path manifestDir = getTaskManifestDir(); + // final manifest file is by task ID + Path manifestFile = manifestPathForTask(manifestDir, + getRequiredTaskId()); + Path manifestTempFile = manifestTempPathForTaskAttempt(manifestDir, + getRequiredTaskAttemptId()); + LOG.info("{}: Saving manifest file to {}", getName(), manifestFile); + save(manifest, manifestTempFile, manifestFile); + return manifestFile; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupJobStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupJobStage.java new file mode 100644 index 0000000000000..9b873252df2cb --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupJobStage.java @@ -0,0 +1,63 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_SETUP; + +/** + * Stage to set up a job by creating the job attempt directory. + * The job attempt directory must not exist before the call. + */ +public class SetupJobStage extends + AbstractJobOrTaskStage { + + private static final Logger LOG = LoggerFactory.getLogger( + SetupJobStage.class); + + public SetupJobStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_SETUP, false); + } + + /** + * Execute the job setup stage. + * @param deleteMarker: should any success marker be deleted. + * @return the job attempted directory. + * @throws IOException failure. + */ + @Override + protected Path executeStage(final Boolean deleteMarker) throws IOException { + final Path path = getJobAttemptDir(); + LOG.info("{}: Creating Job Attempt directory {}", getName(), path); + createNewDirectory("Job setup", path); + createNewDirectory("Creating task manifest dir", getTaskManifestDir()); + // delete any success marker if so instructed. + if (deleteMarker) { + delete(getStageConfig().getJobSuccessMarkerPath(), false); + } + return path; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupTaskStage.java new file mode 100644 index 0000000000000..e909ee183ec80 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/SetupTaskStage.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_SETUP; + +/** + * Stage to set up task. + * This creates the task attempt directory, after verifying + * that the job attempt dir exists (i.e. this is invoked + * after the job is started and before any cleanup. + * Argument passed in is task name:only for logging. + */ +public class SetupTaskStage extends + AbstractJobOrTaskStage { + + public SetupTaskStage(final StageConfig stageConfig) { + super(true, stageConfig, OP_STAGE_TASK_SETUP, false); + } + + /** + * Set up a task. + * @param name task name (for logging) + * @return task attempt directory + * @throws IOException IO failure. + */ + @Override + protected Path executeStage(final String name) throws IOException { + return createNewDirectory("Task setup " + name, + requireNonNull(getTaskAttemptDir(), "No task attempt directory")); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java new file mode 100644 index 0000000000000..d5bbba5b761e3 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageConfig.java @@ -0,0 +1,556 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.util.JsonSerialization; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.functional.TaskPool; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; + +/** + * Stage Config. + * Everything to configure a stage which is common to all. + * + * It's isolated from the details of MR datatypes (taskID, taskattempt etc); + * at this point it expects parsed values. + * + * It uses the builder API, but once {@link #build()} is called it goes + * read only. This is to ensure that changes cannot + * take place when shared across stages. + */ +public class StageConfig { + + /** + * A flag which freezes the config for + * further updates. + */ + private boolean frozen; + + /** + * IOStatistics to update. + */ + private IOStatisticsStore iostatistics; + + /** + * Job ID; constant over multiple attempts. + */ + private String jobId; + + /** + * Where did the job Unique ID come from? + */ + private String jobIdSource = ""; + + /** + * Number of the job attempt; starts at zero. + */ + private int jobAttemptNumber; + + /** + * ID of the task. + */ + private String taskId; + + /** + * ID of this specific attempt at a task. + */ + private String taskAttemptId; + + /** + * Destination of job. + */ + private Path destinationDir; + + /** + * Job attempt dir. + */ + private Path jobAttemptDir; + + /** + * temp directory under job dest dir. + */ + private Path outputTempSubDir; + + /** + * Task attempt dir. + */ + private Path taskAttemptDir; + + /** + * directory where task manifests must go. + */ + private Path taskManifestDir; + + /** + * Subdir under the job attempt dir where task + * attempts will have subdirectories. + */ + private Path jobAttemptTaskSubDir; + + /** + * Callbacks to update store. + * This is not made visible to the stages; they must + * go through the superclass which + * adds statistics and logging. + */ + private ManifestStoreOperations operations; + + /** + * Submitter for doing IO against the store other than + * manifest processing. + */ + private TaskPool.Submitter ioProcessors; + + /** + * Optional progress callback. + */ + private Progressable progressable; + + /** + * Callback when a stage is entered. + */ + private StageEventCallbacks enterStageEventHandler; + + /** + * Thread local serializer; created on demand + * and shareable across a sequence of stages. + */ + private final ThreadLocal> threadLocalSerializer = + ThreadLocal.withInitial(TaskManifest::serializer); + + /** + * Delete target paths on commit? Stricter, but + * higher IO cost. + */ + private boolean deleteTargetPaths; + + /** + * Name for logging. + */ + private String name = ""; + + public StageConfig() { + } + + /** + * Verify that the config is not yet frozen. + */ + private void checkOpen() { + Preconditions.checkState(!frozen, + "StageConfig is now read-only"); + } + + /** + * The build command makes the config immutable. + * Idempotent. + * @return the now-frozen config + */ + public StageConfig build() { + frozen = true; + return this; + } + + /** + * Set job destination dir. + * @param dir new dir + * @return this + */ + public StageConfig withDestinationDir(final Path dir) { + destinationDir = dir; + return this; + } + + /** + * Set IOStatistics store. + * @param store new store + * @return this + */ + public StageConfig withIOStatistics(final IOStatisticsStore store) { + checkOpen(); + iostatistics = store; + return this; + } + + /** + * Set builder value. + * @param value new value + * @return this + */ + public StageConfig withIOProcessors(final TaskPool.Submitter value) { + checkOpen(); + ioProcessors = value; + return this; + } + + /** + * Set Job attempt directory. + * @param dir new dir + * @return this + */ + public StageConfig withJobAttemptDir(final Path dir) { + checkOpen(); + jobAttemptDir = dir; + return this; + } + + /** + * Directory to put task manifests into. + * @return a path under the job attempt dir. + */ + public Path getTaskManifestDir() { + return taskManifestDir; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public StageConfig withTaskManifestDir(Path value) { + checkOpen(); + taskManifestDir = value; + return this; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public StageConfig withJobAttemptTaskSubDir(Path value) { + jobAttemptTaskSubDir = value; + return this; + } + + /** + * Get the path to the subdirectory under $jobID where task + * attempts are. List this dir to find all task attempt dirs. + * @return a path under the job attempt dir. + */ + public Path getJobAttemptTaskSubDir() { + return jobAttemptTaskSubDir; + } + + /** + * Set the job directories from the attempt directories + * information. Does not set task attempt fields. + * @param dirs source of directories. + * @return this + */ + public StageConfig withJobDirectories( + final ManifestCommitterSupport.AttemptDirectories dirs) { + + checkOpen(); + withJobAttemptDir(dirs.getJobAttemptDir()) + .withJobAttemptTaskSubDir(dirs.getJobAttemptTaskSubDir()) + .withDestinationDir(dirs.getOutputPath()) + .withOutputTempSubDir(dirs.getOutputTempSubDir()) + .withTaskManifestDir(dirs.getTaskManifestDir()); + + return this; + } + + /** + * Set job ID with no attempt included. + * @param value new value + * @return this + */ + public StageConfig withJobId(final String value) { + checkOpen(); + jobId = value; + return this; + } + + public Path getOutputTempSubDir() { + return outputTempSubDir; + } + + /** + * Set builder value. + * @param value new value + * @return this + */ + public StageConfig withOutputTempSubDir(final Path value) { + checkOpen(); + outputTempSubDir = value; + return this; + } + + /** + * Set builder value. + * @param value new value + * @return this + */ + public StageConfig withOperations(final ManifestStoreOperations value) { + checkOpen(); + operations = value; + return this; + } + + /** + * Set builder value. + * @param value new value + * @return this + */ + public StageConfig withTaskAttemptId(final String value) { + checkOpen(); + taskAttemptId = value; + return this; + } + + /** + * Set builder value. + * @param value new value + * @return this + */ + public StageConfig withTaskId(final String value) { + checkOpen(); + taskId = value; + return this; + } + + /** + * Set handler for stage entry events.. + * @param value new value + * @return this + */ + public StageConfig withStageEventCallbacks(StageEventCallbacks value) { + checkOpen(); + enterStageEventHandler = value; + return this; + } + + /** + * Optional progress callback. + * @param value new value + * @return this + */ + public StageConfig withProgressable(final Progressable value) { + checkOpen(); + progressable = value; + return this; + } + + /** + * Set the Task attempt directory. + * @param value new value + * @return this + */ + public StageConfig withTaskAttemptDir(final Path value) { + checkOpen(); + taskAttemptDir = value; + return this; + } + + /** + * Set the job attempt number. + * @param value new value + * @return this + */ + public StageConfig withJobAttemptNumber(final int value) { + checkOpen(); + jobAttemptNumber = value; + return this; + } + + /** + * Set the Job ID source. + * @param value new value + * @return this + */ + public StageConfig withJobIdSource(final String value) { + checkOpen(); + jobIdSource = value; + return this; + } + + /** + * Set name of task/job. + * @param value new value + * @return the builder + */ + public StageConfig withName(String value) { + name = value; + return this; + } + + /** + * Get name of task/job. + * @return name for logging. + */ + public String getName() { + return name; + } + + /** + * Handler for stage entry events. + * @return the handler. + */ + public StageEventCallbacks getEnterStageEventHandler() { + return enterStageEventHandler; + } + + /** + * IOStatistics to update. + */ + public IOStatisticsStore getIOStatistics() { + return iostatistics; + } + + /** + * Job ID. + */ + public String getJobId() { + return jobId; + } + + /** + * ID of the task. + */ + public String getTaskId() { + return taskId; + } + + /** + * ID of this specific attempt at a task. + */ + public String getTaskAttemptId() { + return taskAttemptId; + } + + /** + * Job attempt dir. + */ + public Path getJobAttemptDir() { + return jobAttemptDir; + } + + /** + * Destination of job. + */ + public Path getDestinationDir() { + return destinationDir; + } + + /** + * Get the location of the success marker. + * @return a path under the destination directory. + */ + public Path getJobSuccessMarkerPath() { + return new Path(destinationDir, SUCCESS_MARKER); + } + + /** + * Callbacks to update store. + * This is not made visible to the stages; they must + * go through the wrapper classes in this class, which + * add statistics and logging. + */ + public ManifestStoreOperations getOperations() { + return operations; + } + + /** + * Submitter for doing IO against the store other than + * manifest processing. + */ + public TaskPool.Submitter getIoProcessors() { + return ioProcessors; + } + + /** + * Get optional progress callback. + * @return callback or null + */ + public Progressable getProgressable() { + return progressable; + } + + /** + * Task attempt directory. + * @return the task attempt dir. + */ + public Path getTaskAttemptDir() { + return taskAttemptDir; + } + + /** + * Get the job attempt number. + * @return the value + */ + public int getJobAttemptNumber() { + return jobAttemptNumber; + } + + public String getJobIdSource() { + return jobIdSource; + } + + /** + * Get a thread local task manifest serializer. + * @return a serializer. + */ + public JsonSerialization currentManifestSerializer() { + return threadLocalSerializer.get(); + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public StageConfig withDeleteTargetPaths(boolean value) { + checkOpen(); + deleteTargetPaths = value; + return this; + } + + public boolean getDeleteTargetPaths() { + return deleteTargetPaths; + } + + /** + * Enter the stage; calls back to + * {@link #enterStageEventHandler} if non-null. + * @param stage stage entered + */ + public void enterStage(String stage) { + if (enterStageEventHandler != null) { + enterStageEventHandler.enterStage(stage); + } + } + + /** + * Exit the stage; calls back to + * {@link #enterStageEventHandler} if non-null. + * @param stage stage entered + */ + public void exitStage(String stage) { + if (enterStageEventHandler != null) { + enterStageEventHandler.exitStage(stage); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageEventCallbacks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageEventCallbacks.java new file mode 100644 index 0000000000000..e23f72092fb02 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/StageEventCallbacks.java @@ -0,0 +1,37 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +/** + * Callbacks on stage entry/exit. + */ +public interface StageEventCallbacks { + + /** + * Callback on stage entry. + * @param stage new stage + */ + void enterStage(String stage); + + /** + * Callback on stage exit. + * @param stage stage exited. + */ + void exitStage(String stage); +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/TaskAttemptScanDirectoryStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/TaskAttemptScanDirectoryStage.java new file mode 100644 index 0000000000000..4765bcc0865d4 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/TaskAttemptScanDirectoryStage.java @@ -0,0 +1,214 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.LongSummaryStatistics; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.EntryStatus; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.DurationInfo; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_DIRECTORY_COUNT_MEAN; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_DIRECTORY_DEPTH_MEAN; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_FILE_COUNT_MEAN; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASK_FILE_SIZE_MEAN; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_TASK_SCAN_DIRECTORY; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createTaskManifest; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.maybeAddIOStatistics; + +/** + * Stage to scan a directory tree and build a task manifest. + * This is executed by the task committer. + */ +public final class TaskAttemptScanDirectoryStage + extends AbstractJobOrTaskStage { + + private static final Logger LOG = LoggerFactory.getLogger( + TaskAttemptScanDirectoryStage.class); + + public TaskAttemptScanDirectoryStage( + final StageConfig stageConfig) { + super(true, stageConfig, OP_STAGE_TASK_SCAN_DIRECTORY, false); + } + + /** + * Build the Manifest. + * @return the manifest + * @throws IOException failure. + */ + @Override + protected TaskManifest executeStage(final Void arguments) + throws IOException { + + final Path taskAttemptDir = getRequiredTaskAttemptDir(); + final TaskManifest manifest = createTaskManifest(getStageConfig()); + + LOG.info("{}: scanning directory {}", + getName(), taskAttemptDir); + + final int depth = scanDirectoryTree(manifest, + taskAttemptDir, + getDestinationDir(), + 0, true); + List filesToCommit = manifest.getFilesToCommit(); + LongSummaryStatistics fileSummary = filesToCommit.stream() + .mapToLong(FileEntry::getSize) + .summaryStatistics(); + long fileDataSize = fileSummary.getSum(); + long fileCount = fileSummary.getCount(); + int dirCount = manifest.getDestDirectories().size(); + LOG.info("{}: directory {} contained {} file(s); data size {}", + getName(), + taskAttemptDir, + fileCount, + fileDataSize); + LOG.info("{}: Directory count = {}; maximum depth {}", + getName(), + dirCount, + depth); + // add statistics about the task output which, when aggregated, provides + // insight into structure of job, task skew, etc. + IOStatisticsStore iostats = getIOStatistics(); + iostats.addSample(COMMITTER_TASK_DIRECTORY_COUNT_MEAN, dirCount); + iostats.addSample(COMMITTER_TASK_DIRECTORY_DEPTH_MEAN, depth); + iostats.addSample(COMMITTER_TASK_FILE_COUNT_MEAN, fileCount); + iostats.addSample(COMMITTER_TASK_FILE_SIZE_MEAN, fileDataSize); + + return manifest; + } + + /** + * Recursively scan a directory tree. + * The manifest will contain all files to rename + * (source and dest) and directories to create. + * All files are processed before any of the subdirs are. + * This helps in statistics gathering. + * There's some optimizations which could be done with async + * fetching of the iterators of those subdirs, but as this + * is generally off-critical path then that "enhancement" + * can be postponed until data suggests this needs improvement. + * @param manifest manifest to update + * @param srcDir dir to scan + * @param destDir destination directory + * @param depth depth from the task attempt dir. + * @param parentDirExists does the parent dir exist? + * @return the maximum depth of child directories + * @throws IOException IO failure. + */ + private int scanDirectoryTree( + TaskManifest manifest, + Path srcDir, + Path destDir, + int depth, + boolean parentDirExists) throws IOException { + + // generate some task progress in case directory scanning is very slow. + progress(); + + int maxDepth = 0; + int files = 0; + boolean dirExists = parentDirExists; + List subdirs = new ArrayList<>(); + try (DurationInfo ignored = new DurationInfo(LOG, false, + "Task Attempt %s source dir %s, dest dir %s", + getTaskAttemptId(), srcDir, destDir)) { + + // list the directory. This may block until the listing is complete, + // or, if the FS does incremental or asynchronous fetching, + // then the next()/hasNext() call will block for the results + // unless turned off, ABFS does to this async + final RemoteIterator listing = listStatusIterator(srcDir); + + // when the FS (especially ABFS) does an asyn fetch of the listing, + // we can probe for the status of the destination dir while that + // page is being fetched. + // probe for and add the dest dir entry for all but + // the base dir + + if (depth > 0) { + final EntryStatus status; + if (parentDirExists) { + final FileStatus destDirStatus = getFileStatusOrNull(destDir); + status = EntryStatus.toEntryStatus(destDirStatus); + dirExists = destDirStatus != null; + } else { + // if there is no parent dir, then there is no need to look + // for this directory -report it as missing automatically. + status = EntryStatus.not_found; + } + manifest.addDirectory(DirEntry.dirEntry( + destDir, + status, + depth)); + } + + // process the listing; this is where abfs will block + // to wait the result of the list call. + while (listing.hasNext()) { + final FileStatus st = listing.next(); + if (st.isFile()) { + // this is a file, so add to the list of files to commit. + files++; + final FileEntry entry = fileEntry(st, destDir); + manifest.addFileToCommit(entry); + LOG.debug("To rename: {}", entry); + } else { + if (st.isDirectory()) { + // will need to scan this directory too. + subdirs.add(st); + } else { + // some other object. ignoring + LOG.info("Ignoring FS object {}", st); + } + } + } + // add any statistics provided by the listing. + maybeAddIOStatistics(getIOStatistics(), listing); + } + + // now scan the subdirectories + LOG.debug("{}: Number of subdirectories under {} found: {}; file count {}", + getName(), srcDir, subdirs.size(), files); + + for (FileStatus st : subdirs) { + Path destSubDir = new Path(destDir, st.getPath().getName()); + final int d = scanDirectoryTree(manifest, + st.getPath(), + destSubDir, + depth + 1, + dirExists); + maxDepth = Math.max(maxDepth, d); + } + + return 1 + maxDepth; + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java new file mode 100644 index 0000000000000..8b72112a3dad8 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/ValidateRenamedFilesStage.java @@ -0,0 +1,200 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.OutputValidationException; +import org.apache.hadoop.util.functional.TaskPool; + +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_VALIDATE_OUTPUT; +import static org.apache.hadoop.thirdparty.com.google.common.collect.Iterables.concat; + +/** + * This stage validates all files by scanning the manifests + * and verifying every file in every manifest is of the given size. + * Returns a list of all files committed. + * + * Its cost is one getFileStatus() call (parallelized) per file. + * Raises a {@link OutputValidationException} on a validation failure. + */ +public class ValidateRenamedFilesStage extends + AbstractJobOrTaskStage< + List, + List> { + + private static final Logger LOG = LoggerFactory.getLogger( + ValidateRenamedFilesStage.class); + + /** + * Set this to halt all workers. + */ + private final AtomicBoolean halt = new AtomicBoolean(); + + /** + * List of all files committed. + */ + private List filesCommitted = new ArrayList<>(); + + public ValidateRenamedFilesStage(final StageConfig stageConfig) { + super(false, stageConfig, OP_STAGE_JOB_VALIDATE_OUTPUT, true); + } + + /** + * Get the list of files committed. + * @return a possibly empty list. + */ + private synchronized List getFilesCommitted() { + return filesCommitted; + } + + /** + * Add a file entry to the list of committed files. + * @param entry entry + */ + private synchronized void addFileCommitted(FileEntry entry) { + filesCommitted.add(entry); + } + + /** + * Validate the task manifests. + * This is done by listing all the directories + * and verifying that every file in the source list + * has a file in the destination of the same size. + * If two tasks have both written the same file or + * a source file was changed after the task was committed, + * then a mistmatch will be detected -provided the file + * length is now different. + * @param taskManifests list of manifests. + * @return list of files committed. + */ + @Override + protected List executeStage( + final List taskManifests) + throws IOException { + + // set the list of files to be as big as the number of tasks. + // synchronized to stop complaints. + synchronized (this) { + filesCommitted = new ArrayList<>(taskManifests.size()); + } + + // validate all the files. + + final Iterable filesToCommit = concat(taskManifests.stream() + .map(TaskManifest::getFilesToCommit) + .collect(Collectors.toList())); + + TaskPool.foreach(filesToCommit) + .executeWith(getIOProcessors()) + .stopOnFailure() + .run(this::validateOneFile); + + return getFilesCommitted(); + } + + /** + * Validate a file. + * @param entry entry to probe for + * @throws IOException IO problem. + * @throws OutputValidationException if the entry is not valid + */ + private void validateOneFile(FileEntry entry) throws IOException { + updateAuditContext(OP_STAGE_JOB_VALIDATE_OUTPUT); + + if (halt.get()) { + // told to stop + return; + } + // report progress back + progress(); + // look validate the file. + // raising an FNFE if the file isn't there. + FileStatus destStatus; + final Path sourcePath = entry.getSourcePath(); + Path destPath = entry.getDestPath(); + try { + destStatus = getFileStatus(destPath); + + // it must be a file + if (!destStatus.isFile()) { + throw new OutputValidationException(destPath, + "Expected a file renamed from " + sourcePath + + "; found " + destStatus); + } + final long sourceSize = entry.getSize(); + final long destSize = destStatus.getLen(); + + // etags, if the source had one. + final String sourceEtag = entry.getEtag(); + if (isNotBlank(sourceEtag)) { + final String destEtag = ManifestCommitterSupport.getEtag(destStatus); + if (!sourceEtag.equals(destEtag)) { + LOG.warn("Etag of dest file {}: {} does not match that of manifest entry {}", + destPath, destStatus, entry); + throw new OutputValidationException(destPath, + String.format("Expected the file" + + " renamed from %s" + + " with etag %s and length %s" + + " but found a file with etag %s and length %d", + sourcePath, + sourceEtag, + sourceSize, + destEtag, + destSize)); + + } + } + // check the expected length after any etag validation + if (destSize != sourceSize) { + LOG.warn("Length of dest file {}: {} does not match that of manifest entry {}", + destPath, destStatus, entry); + throw new OutputValidationException(destPath, + String.format("Expected the file" + + " renamed from %s" + + " with length %d" + + " but found a file of length %d", + sourcePath, + sourceSize, + destSize)); + } + + } catch (FileNotFoundException e) { + // file didn't exist + throw new OutputValidationException(destPath, + "Expected a file, but it was not found", e); + } + addFileCommitted(entry); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/package-info.java new file mode 100644 index 0000000000000..8b86046268d1f --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/stages/package-info.java @@ -0,0 +1,30 @@ +/* + * 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. + */ + +/** + * Intermediate manifest committer. + * + * The individual and aggregate stages of the task and job commit process. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml index 844c91c5ce081..d315a00ba4a6c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml @@ -2231,4 +2231,23 @@ + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md new file mode 100644 index 0000000000000..b446be29ddd11 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer.md @@ -0,0 +1,605 @@ + + + +# The Manifest Committer for Azure and Google Cloud Storage + +This document how to use the _Manifest Committer_. + +The _Manifest_ committer is a committer for work which provides +performance on ABFS for "real world" queries, +and performance and correctness on GCS. +It also works with other filesystems, including HDFS. +However, the design is optimized for object stores where +listing operatons are slow and expensive. + +The architecture and implementation of the committer is covered in +[Manifest Committer Architecture](manifest_committer_architecture.html). + + +The protocol and its correctness are covered in +[Manifest Committer Protocol](manifest_committer_protocol.html). + +It was added in March 2022, and should be considered unstable +in early releases. + + + +## Problem: + +The only committer of work from Spark to Azure ADLS Gen 2 "abfs://" storage +which is safe to use is the "v1 file committer". + +This is "correct" in that if a task attempt fails, its output is guaranteed not +to be included in the final out. The "v2" commit algorithm cannot meet that +guarantee, which is why it is no longer the default. + +But: it is slow, especially on jobs where deep directory trees of output are used. +Why is it slow? It's hard to point at a particular cause, primarily because of +the lack of any instrumentation in the `FileOutputCommitter`. +Stack traces of running jobs generally show `rename()`, though list operations +do surface too. + +On Google GCS, neither the v1 nor v2 algorithm are _safe_ because the google +filesystem doesn't have the atomic directory rename which the v1 algorithm +requires. + +A further issue is that both Azure and GCS storage may encounter scale issues +with deleting directories with many descendants. +This can trigger timeouts because the FileOutputCommitter assumes that +cleaning up after the job is a fast call to `delete("_temporary", true)`. + +## Solution. + +The _Intermediate Manifest_ committer is a new committer for +work which should deliver performance on ABFS +for "real world" queries, and performance and correctness on GCS. + +This committer uses the extension point which came in for the S3A committers. +Users can declare a new committer factory for abfs:// and gcs:// URLs. +A suitably configured spark deployment will pick up the new committer. + +Directory performance issues in job cleanup can be addressed by two options +1. The committer will parallelize deletion of task attempt directories before + deleting the `_temporary` directory. +1. Cleanup can be disabled. . + +The committer can be used with any filesystem client which has a "real" file rename() +operation. +It has been optimised for remote object stores where listing and file probes +are expensive -the design is less likely to offer such signifcant speedup +on HDFS -though the parallel renaming operations will speed up jobs +there compared to the classic v1 algorithm. + +# How it works + +The full details are covered in [Manifest Committer Architecture](manifest_committer_architecture.html). + +# Using the committer + +The hooks put in to support the S3A committers were designed to allow every +filesystem schema to provide their own committer. +See [Switching To an S3A Committer](../../hadoop-aws/tools/hadoop-aws/committers.html#Switching_to_an_S3A_Committer) + +A factory for the abfs schema would be defined in +`mapreduce.outputcommitter.factory.scheme.abfs` ; and a similar one for `gcs`. + +Some matching spark configuration changes, especially for parquet binding, will be required. +These can be done in `core-site.xml`, if it is not defined in the `mapred-default.xml` JAR. + + +```xml + + mapreduce.outputcommitter.factory.scheme.abfs + org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory + + + mapreduce.outputcommitter.factory.scheme.gs + org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory + +``` + +## Binding to the manifest committer in Spark. + +In Apache Spark, the configuration can be done either with command line options (after the '--conf') or by using the `spark-defaults.conf` file. The following is an example of using `spark-defaults.conf` also including the configuration for Parquet with a subclass of the parquet +committer which uses the factory mechansim internally. + +``` +spark.hadoop.mapreduce.outputcommitter.factory.scheme.abfs org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory +spark.hadoop.mapreduce.outputcommitter.factory.scheme.gs org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory +spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter +spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol +``` + + +### Using the Cloudstore `committerinfo` command to probe committer bindings. + +The hadoop committer settings can be validated in a recent build of [cloudstore](https://github.com/steveloughran/cloudstore) +and its `committerinfo` command. +This command instantiates a committer for that path through the same factory mechanism as MR and spark jobs use, +then prints its `toString` value. + +``` +hadoop jar cloudstore-1.0.jar committerinfo abfs://testing@ukwest.dfs.core.windows.net/ + +2021-09-16 19:42:59,731 [main] INFO commands.CommitterInfo (StoreDurationInfo.java:(53)) - Starting: Create committer +Committer factory for path abfs://testing@ukwest.dfs.core.windows.net/ is + org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory@3315d2d7 + (classname org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory) +2021-09-16 19:43:00,897 [main] INFO manifest.ManifestCommitter (ManifestCommitter.java:(144)) - Created ManifestCommitter with + JobID job__0000, Task Attempt attempt__0000_r_000000_1 and destination abfs://testing@ukwest.dfs.core.windows.net/ +Created committer of class org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitter: + ManifestCommitter{ManifestCommitterConfig{destinationDir=abfs://testing@ukwest.dfs.core.windows.net/, + role='task committer', + taskAttemptDir=abfs://testing@ukwest.dfs.core.windows.net/_temporary/manifest_job__0000/0/_temporary/attempt__0000_r_000000_1, + createJobMarker=true, + jobUniqueId='job__0000', + jobUniqueIdSource='JobID', + jobAttemptNumber=0, + jobAttemptId='job__0000_0', + taskId='task__0000_r_000000', + taskAttemptId='attempt__0000_r_000000_1'}, + iostatistics=counters=(); + +gauges=(); + +minimums=(); + +maximums=(); + +means=(); +} + +``` + + +## Verifying that the committer was used + +The new committer will write a JSON summary of the operation, including statistics, in the `_SUCCESS` file. + +If this file exists and is zero bytes long: the classic `FileOutputCommitter` was used. + +If this file exists and is greater than zero bytes long, either the manifest committer was used, +or in the case of S3A filesystems, one of the S3A committers. They all use the same JSON format. + +# Configuration options + +Here are the main configuration options of the committer. + + +| Option | Meaning | Default Value | +|--------|---------|---------------| +| `mapreduce.manifest.committer.delete.target.files` | Delete target files? | `false` | +| `mapreduce.manifest.committer.io.threads` | Thread count for parallel operations | `64` | +| `mapreduce.manifest.committer.summary.report.directory` | directory to save reports. | `""` | +| `mapreduce.manifest.committer.cleanup.parallel.delete` | Delete temporary directories in parallel | `true` | +| `mapreduce.fileoutputcommitter.cleanup.skipped` | Skip cleanup of `_temporary` directory| `false` | +| `mapreduce.fileoutputcommitter.cleanup-failures.ignored` | Ignore errors during cleanup | `false` | +| `mapreduce.fileoutputcommitter.marksuccessfuljobs` | Create a `_SUCCESS` marker file on successful completion. (and delete any existing one in job setup) | `true` | + +There are some more, as covered in the (Advanced)[#advanced] section. + + +## Scaling jobs `mapreduce.manifest.committer.io.threads` + +The core reason that this committer is faster than the classic `FileOutputCommitter` +is that it tries to parallelize as much file IO as it can during job commit, specifically: + +* task manifest loading +* deletion of files where directories will be created +* directory creation +* file-by-file renaming +* deletion of task attempt directories in job cleanup + +These operations are all performed in the same thread pool, whose size is set +in the option `mapreduce.manifest.committer.io.threads`. + +Larger values may be used. + +XML +```xml + + mapreduce.manifest.committer.io.threads + 200 + +``` + +spark-defaults.conf +``` +spark.hadoop.mapreduce.manifest.committer.io.threads 200 +``` + +A larger value than that of the number of cores allocated to +the MapReduce AM or Spark Driver does not directly overload +the CPUs, as the threads are normally waiting for (slow) IO +against the object store/filesystem to complete. + +Caveats +* In Spark, multiple jobs may be committed in the same process, + each of which will create their own thread pool during job + commit or cleanup. +* Azure rate throttling may be triggered if too many IO requests + are made against the store. The rate throttling option + `mapreduce.manifest.committer.io.rate` can help avoid this. + + +## Optional: deleting target files in Job Commit + +The classic `FileOutputCommitter` deletes files at the destination paths +before renaming the job's files into place. + +This is optional in the manifest committers, set in the option +`mapreduce.manifest.committer.delete.target.files` with a default value of `false`. + +This increases performance and is safe to use when all files created by a job +have unique filenames. + +Apache Spark does generate unique filenames for ORC and Parquet since +[SPARK-8406](https://issues.apache.org/jira/browse/SPARK-8406) +_Adding UUID to output file name to avoid accidental overwriting_ + +Avoiding checks for/deleting target files saves one delete call per file being committed, so can +save a significant amount of store IO. + +When appending to existing tables, using formats other than ORC and parquet, +unless confident that unique identifiers +are added to each filename, enable deletion of the target files. + +``` +spark.hadoop.mapreduce.manifest.committer.delete.target.files true +``` + +*Note 1:* the committer will skip deletion operations when it +created the directory into which a file is to be renamed. +This makes it slightly more efficient, at least if jobs +appending data are creating and writing into new partitions. + +*Note 2:* the committer still requires tasks within a single +job to create unique files. This is foundational for +any job to generate correct data. + +# Job Summaries in `_SUCCESS` files + +The original hadoop committer creates a zero byte `_SUCCESS` file in the root of the output directory +unless disabled. + +This committer writes a JSON summary which includes +* The name of the committer. +* Diagnostics information. +* A list of some of the files created (for testing; a full list is excluded as it can get big). +* IO Statistics. + +If, after running a query, this `_SUCCESS` file is zero bytes long, +*the new committer has not been used* + +If it is not empty, then it can be examined. + +## Viewing `_SUCCESS` file files through the `ManifestPrinter` tool. + +The summary files are JSON, and can be viewed in any text editor. + +For a more succinct summary, including better display of statistics, use the `ManifestPrinter` tool. + +```bash +hadoop org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter +``` + +This works for the files saved at the base of an output directory, and +any reports saved to a report directory. + +## Collecting Job Summaries `mapreduce.manifest.committer.summary.report.directory` + +The committer can be configured to save the `_SUCCESS` summary files to a report directory, +irrespective of whether the job succeed or failed, by setting a fileystem path in +the option `mapreduce.manifest.committer.summary.report.directory`. + +The path does not have to be on the same +store/filesystem as the destination of work. For example, a local fileystem could be used. + +XML + +```xml + + mapreduce.manifest.committer.summary.report.directory + file:///tmp/reports + +``` + +spark-defaults.conf + +``` +spark.hadoop.mapreduce.manifest.committer.summary.report.directory file:///tmp/reports +``` + +This allows for the statistics of jobs to be collected irrespective of their outcome, Whether or not +saving the `_SUCCESS` marker is enabled, and without problems caused by a chain of queries +overwriting the markers. + + +# Cleanup + +Job cleanup is convoluted as it is designed to address a number of issues which +may surface in cloud storage. + +* Slow performance for deletion of directories. +* Timeout when deleting very deep and wide directory trees. +* General resilience to cleanup issues escalating to job failures. + + +| Option | Meaning | Default Value | +|--------|---------|---------------| +| `mapreduce.fileoutputcommitter.cleanup.skipped` | Skip cleanup of `_temporary` directory| `false` | +| `mapreduce.fileoutputcommitter.cleanup-failures.ignored` | Ignore errors during cleanup | `false` | +| `mapreduce.manifest.committer.cleanup.parallel.delete` | Delete task attempt directories in parallel | `true` | + +The algorithm is: + +``` +if `mapreduce.fileoutputcommitter.cleanup.skipped`: + return +if `mapreduce.manifest.committer.cleanup.parallel.delete`: + attempt parallel delete of task directories; catch any exception +if not `mapreduce.fileoutputcommitter.cleanup.skipped`: + delete(`_temporary`); catch any exception +if caught-exception and not `mapreduce.fileoutputcommitter.cleanup-failures.ignored`: + throw caught-exception +``` + +It's a bit complicated, but the goal is to perform a fast/scalable delete and +throw a meaningful exception if that didn't work. + +When working with ABFS and GCS, these settings should normally be left alone. +If somehow errors surface during cleanup, enabling the option to +ignore failures will ensure the job still completes. +Disabling cleanup even avoids the overhead of cleanup, but +requires a workflow or manual operation to clean up all +`_temporary` directories on a regular basis. + + +# Working with Azure ADLS Gen2 Storage + +To switch to the manifest committer, the factory for committers for destinations with `abfs://` URLs must +be switched to the manifest committer factory, either for the application or +the entire cluster. + +```xml + + mapreduce.outputcommitter.factory.scheme.abfs + org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory + +``` + +This allows for ADLS Gen2 -specific performance and consistency logic to be used from within the committer. +In particular: +* the [Etag](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/ETag) header +can be collected in listings and used in the job commit phase. +* IO rename operations are rate limited +* recovery is attempted when throttling triggers rename failures. + +*Warning* This committer is not compatible with older Azure storage services +(WASB or ADLS Gen 1). + +The core set of Azure-optimized options becomes + +```xml + + mapreduce.outputcommitter.factory.scheme.abfs + org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory + + + + spark.hadoop.fs.azure.io.rate.limit + 10000 + +``` + +And optional settings for debugging/performance analysis + +```xml + + mapreduce.manifest.committer.summary.report.directory + abfs:// Path within same store/separate store + Optional: path to where job summaries are saved + +``` + +## Full set of ABFS options for spark + +``` +spark.hadoop.mapreduce.outputcommitter.factory.scheme.abfs org.apache.hadoop.fs.azurebfs.commit.AzureManifestCommitterFactory +spark.hadoop.fs.azure.io.rate.limit 10000 +spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter +spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol + +spark.hadoop.mapreduce.manifest.committer.summary.report.directory (optional: URI of a directory for job summaries) +``` + +## Experimental: ABFS Rename Rate Limiting `fs.azure.io.rate.limit` + +To avoid triggering store throttling and backoff delays, as well as other +throttling-related failure conditions file renames during job commit +are throttled through a "rate limiter" which limits the number of +rename operations per second a single instance of the ABFS FileSystem client +may issue. + +| Option | Meaning | +|--------|---------| +| `fs.azure.io.rate.limit` | Rate limit in operations/second for IO operations. | + +Set the option to `0` remove all rate limiting. + +The default value of this is set to 10000, which is the default IO capacity for +an ADLS storage account. + +```xml + + fs.azure.io.rate.limit + 10000 + maximum number of renames attempted per second + +``` + +This capacity is set at the level of the filesystem client, and so not +shared across all processes within a single application, let +alone other applications sharing the same storage account. + +It will be shared with all jobs being committed by the same +Spark driver, as these do share that filesystem connector. + +If rate limiting is imposed, the statistic `store_io_rate_limited` will +report the time to acquire permits for committing files. + +If server-side throttling took place, signs of this can be seen in +* The store service's logs and their throttling status codes (usually 503 or 500). +* The job statistic `commit_file_rename_recovered`. This statistic indicates that + ADLS throttling manifested as failures in renames, failures which were recovered + from in the comitter. + +If these are seen -or other applications running at the same time experience +throttling/throttling-triggered problems, consider reducing the value of +`fs.azure.io.rate.limit`, and/or requesting a higher IO capacity from Microsoft. + +*Important* if you do get extra capacity from Microsoft and you want to use +it to speed up job commits, increase the value of `fs.azure.io.rate.limit` +either across the cluster, or specifically for those jobs which you wish +to allocate extra priority to. + +This is still a work in progress; it may be expanded to support +all IO operations performed by a single filesystem instance. + +# Working with Google Cloud Storage + +The manifest committer is compatible with and tested against Google cloud storage through +the gcs-connector library from google, which provides a Hadoop filesystem client for the +schema `gs`. + +Google cloud storage has the semantics needed for the commit protocol +to work safely. + +The Spark settings to switch to this committer are + +``` +spark.hadoop.mapreduce.outputcommitter.factory.scheme.gs org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory +spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter +spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol + +spark.hadoop.mapreduce.manifest.committer.summary.report.directory (optional: URI of a directory for job summaries) +``` + +The store's directory delete operations are `O(files)` so the value +of `mapreduce.manifest.committer.cleanup.parallel.delete` +SHOULD be left at the default of `true`. + +For mapreduce, declare the binding in `core-site.xml`or `mapred-site.xml` +```xml + + mapreduce.outputcommitter.factory.scheme.gcs + org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory + +``` + + +# Working with HDFS + +This committer _does_ work with HDFS, it has just been targeted at object stores with +reduced performance on some operations, especially listing and renaming, +and semantics too reduced for the classic `FileOutputCommitter` to rely on +(specifically GCS). + +To use on HDFS, set the `ManifestCommitterFactory` as the committer factory for `hdfs://` URLs. + +Because HDFS does fast directory deletion, there is no need to parallelize deletion +of task attempt directories during cleanup, so set +`mapreduce.manifest.committer.cleanup.parallel.delete` to `false` + +The final spark bindings becomes + +``` +spark.hadoop.mapreduce.outputcommitter.factory.scheme.hdfs org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory +spark.hadoop.mapreduce.manifest.committer.cleanup.parallel.delete false +spark.sql.parquet.output.committer.class org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter +spark.sql.sources.commitProtocolClass org.apache.spark.internal.io.cloud.PathOutputCommitProtocol + +spark.hadoop.mapreduce.manifest.committer.summary.report.directory (optional: URI of a directory for job summaries) +``` + +# Advanced Topics + +## Advanced Configuration options + +There are some advanced options which are intended for development and testing, +rather than production use. + +| Option | Meaning | Default Value | +|--------|---------|---------------| +| `mapreduce.manifest.committer.store.operations.classname` | Classname for Manifest Store Operations | `""` | +| `mapreduce.manifest.committer.validate.output` | Perform output validation? | `false` | + + +## Validating output `mapreduce.manifest.committer.validate.output` + +The option `mapreduce.manifest.committer.validate.output` triggers a check of every renamed file to +verify it has the expected length. + +This adds the overhead of a `HEAD` request per file, and so is recommended for testing only. + +There is no verification of the actual contents. + +## Controlling storage integration `mapreduce.manifest.committer.store.operations.classname` + +The manifest committer interacts with filesystems through implementations of the interface +`ManifestStoreOperations`. +It is possible to provide custom implementations for store-specific features. +There is one of these for ABFS; when the abfs-specific committer factory is used this +is automatically set. + +It can be explicitly set. +```xml + + mapreduce.manifest.committer.store.operations.classname + org.apache.hadoop.fs.azurebfs.commit.AbfsManifestStoreOperations + +``` + +The default implementation may also be configured. + +```xml + + mapreduce.manifest.committer.store.operations.classname + org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem + +``` + +There is no need to alter these values, except when writing new implementations for other stores, +something which is only needed if the store provides extra integration support for the +committer. + +## Support for concurrent test runs. + +It *may* be possible to run multiple jobs targeting the same directory tree. + +For this to work, a number of conditions must be met: + +* When using spark, unique job IDs must be set. This meangs the Spark distribution + MUST contain the patches for + [SPARK-33402](https://issues.apache.org/jira/browse/SPARK-33402) + and + [SPARK-33230](https://issues.apache.org/jira/browse/SPARK-33230). +* Cleanup of the `_temporary` directory must be disabled by setting + `mapreduce.fileoutputcommitter.cleanup.skipped` to `true`. +* All jobs/tasks must create files with unique filenames. +* All jobs must create output with the same directory partition structure. +* Remember to delete the `_temporary` directory later! + +This has *NOT BEEN TESTED* diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_architecture.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_architecture.md new file mode 100644 index 0000000000000..d2b4f1ee8b446 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_architecture.md @@ -0,0 +1,335 @@ + + + +# Manifest Committer Architecture + +This document describes the architecture and other implementation/correctness +aspects of the [Manifest Committer](manifest_committer.html) + +The protocol and its correctness are covered in [Manifest Committer Protocol](manifest_committer_protocol.html). + + +The _Manifest_ committer is a committer for work which provides performance on ABFS for "real world" +queries, and performance and correctness on GCS. + +This committer uses the extension point which came in for the S3A committers. +Users can declare a new committer factory for `abfs://` and `gcs://` URLs. +It can be used through Hadoop MapReduce and Apache Spark. + +## Background + +### Terminology + +| Term | Meaning| +|------|--------| +| Committer | A class which can be invoked by MR/Spark to perform the task and job commit operations. | +| Spark Driver | The spark process scheduling the work and choreographing the commit operation.| +| Job | In MapReduce. the entire application. In spark, this is a single stage in a chain of work | +| Job Attempt | A single attempt at a job. MR supports multiple Job attempts with recovery on partial job failure. Spark says "start again from scratch" | +| Task | a subsection of a job, such as processing one file, or one part of a file | +| Task ID | ID of the task, unique within this job. Usually starts at 0 and is used in filenames (part-0000, part-001, etc.) | +| Task attempt (TA) | An attempt to perform a task. It may fail, in which case MR/spark will schedule another. | +| Task Attempt ID | A unique ID for the task attempt. The Task ID + an attempt counter.| +| Destination directory | The final destination of work.| +| Job Attempt Directory | A temporary directory used by the job attempt. This is always _underneath_ the destination directory, so as to ensure it is in the same encryption zone as HDFS, storage volume in other filesystems, etc.| +| Task Attempt directory | (also known as "Task Attempt Working Directory"). Directory exclusive for each task attempt under which files are written | +| Task Commit | Taking the output of a Task Attempt and making it the final/exclusive result of that "successful" Task.| +| Job Commit | aggregating all the outputs of all committed tasks and producing the final results of the job. | + + + +The purpose of a committer is to ensure that the complete output of +a job ends up in the destination, even in the presence of failures of tasks. + +- _Complete:_ the output includes the work of all successful tasks. +- _Exclusive:_ the output of unsuccessful tasks is not present. +- _Concurrent:_ When multiple tasks are committed in parallel the output is the same as when + the task commits are serialized. This is not a requirement of Job Commit. +- _Abortable:_ jobs and tasks may be aborted prior to job commit, after which their output is not visible. +- _Continuity of correctness:_ once a job is committed, the output of any failed, + aborted, or unsuccessful task MUST NO appear at some point in the future. + +For Hive's classic hierarchical-directory-structured tables, job committing +requires the output of all committed tasks to be put into the correct location +in the directory tree. + +The committer built into `hadoop-mapreduce-client-core` module is the `FileOutputCommitter`. + + + +## The Manifest Committer: A high performance committer for Spark on Azure and Google storage. + +The Manifest Committer is a higher performance committer for ABFS and GCS storage +for jobs which create file across deep directory trees through many tasks. + +It will also work on `hdfs://` and indeed, `file://` URLs, but +it is optimized to address listing and renaming performance and throttling +issues in cloud storage. + +It *will not* work correctly with S3, because it relies on an atomic rename-no-overwrite +operation to commit the manifest file. It will also have the performance +problems of copying rather than moving all the generated data. + +Although it will work with MapReduce +there is no handling of multiple job attempts with recovery from previous failed +attempts. + +### The Manifest + +A Manifest file is designed which contains (along with IOStatistics and some +other things) + +1. A list of destination directories which must be created if they do not exist. +1. A list of files to rename, recorded as (absolute source, absolute destination, + file-size) entries. + +### Task Commit + +Task attempts are committed by: + +1. Recursively listing the task attempt working dir to build + 1. A list of directories under which files are renamed. + 2. A list of files to rename: source, destination, size and optionally, etag. +2. Saving this information in a manifest file in the job attempt directory with + a filename derived from the Task ID. + Note: writing to a temp file and then renaming to the final path will be used + to ensure the manifest creation is atomic. + + +No renaming takes place —the files are left in their original location. + +The directory treewalk is single-threaded, then it is `O(directories)`, +with each directory listing using one or more paged LIST calls. + +This is simple, and for most tasks, the scan is off the critical path of of the job. + +Statistics analysis may justify moving to parallel scans in future. + + +### Job Commit + +Job Commit consists of: + +1. List all manifest files in the job attempt directory. +1. Load each manifest file, create directories which do not yet exist, then + rename each file in the rename list. +1. Save a JSON `_SUCCESS` file with the same format as the S3A committer (for + testing; use write and rename for atomic save) + +The job commit phase supports parallelization for many tasks and many files +per task, specifically: + +1. Manifest tasks are loaded and processed in a pool of "manifest processor" + threads. +2. Directory creation and file rename operations are each processed in a pool of " + executor" threads: many renames can execute in parallel as they use minimal + network IO. +3. job cleanup can parallelize deletion of task attempt directories. This + is relevant as directory deletion is `O(files)` on Google cloud storage, + and also on ABFS when OAuth authentication is used. + + +### Ancestor directory preparation + +Optional scan of all ancestors ...if any are files, delete. + + +### Parent directory creation + +1. Probe shared directory map for directory existing. If found: operation is + complete. +1. if the map is empty, call `getFileStatus()` on the path. Not found: create + directory, add entry and those of all parent paths Found and is directory: + add entry and those of all parent paths Found and is file: delete. then + create as before. + +Efficiently handling concurrent creation of directories (or delete+create) is going to be a +troublespot; some effort is invested there to build the set of directories to +create. + +### File Rename + +Files are renamed in parallel. + +A pre-rename check for anything being at that path (and deleting it) will be optional. +With spark creating new UUIDs for each file, this isn't going to happen, and +saves HTTP requests. + + +### Validation + +Optional scan of all committed files and verify length and, if known, +etag. For testing and diagnostics. + +## Benefits + +* Pushes the source tree list operations into the task commit phase, which is + generally off the critical path of execution +* Provides an atomic task commit to GCS, as there is no expectation that + directory rename is atomic +* It is possible to pass IOStatistics from workers in manifest. +* Allows for some pre-rename operations similar to the S3A "Partitioned Staging + committer". This can be configured to delete all existing entries in + directories scheduled to be created -or fail if those partitions are + non-empty. + See [Partitioned Staging Committer](../../hadoop-aws/tools/hadoop-aws/committers.html#The_.E2.80.9CPartitioned.E2.80.9D_Staging_Committer) +* Allows for an optional preflight validation check (verify no duplicate files created by different tasks) +* Manifests can be viewed, size of output determined, etc, during + development/debugging. + +### Disadvantages + +* Needs a new manifest file format. +* May makes task commit more complex. + +This solution is necessary for GCS and should be beneficial on ABFS as listing +overheads are paid for in the task committers. + +# Implementation Details + +### Constraints + +A key goal is to keep the manifest committer isolated and neither +touch the existing committer code nor other parts of the hadoop codebase. + +It must plug directly into MR and Spark without needing any changes +other than already implemented for the S3A Committers + +* Self-contained: MUST NOT require changes to hadoop-common, etc. +* Isolated: MUST NOT make changes to existing committers +* Integrated: MUST bind via `PathOutputCommitterFactory`. + +As a result of this there's a bit of copy and paste from elsewhere, +e.g. `org.apache.hadoop.util.functional.TaskPool` +is based on S3ACommitter's `org.apache.hadoop.fs.s3a.commit.Tasks`. + +The` _SUCCESS` file MUST be compatible with the S3A JSON file. +This is to ensure any existing test suites which validate +S3A committer output can be retargeted at jobs executed +by the manifest committer without any changes. + + +#### Progress callbacks in job commit. + +When? Proposed: heartbeat until renaming finally finishes. + +#### Error handling and aborting in job commit. + +We would want to stop the entire job commit. Some atomic boolean "abort job" +would need to be checked in the processing of each task committer thread's +iteraton through a directory (or processing of each file?) +Failures in listing or renaming will need to be escalated to halting the entire +job commit. This implies that any IOE raised in asynchronous rename operation or +in a task committer thread must: + +1. be caught +1. be stored in a shared field/variable +1. trigger the abort +1. be rethrown at the end of the `commitJob()` call + +#### Avoiding deadlocks + +If a job commit stage is using a thread pool for per-task operations, e.g. loading +files, that same thread pool MUST NOT be used for parallel operations within +the per-task stage. + +As every `JobStage` is executed in sequence within task or job commit, it +is safe to share the same thread pool across stages. + +In the current implementation, there is no parallel "per manifest" operation +in job commit other than for actually loading the files. +The operations to create directories and to rename files are actually +performed without performing parallel processing of individual manifests. + +Directory Preparation: merge the directory lists of all manifests, +then queue for creation the (hopefully very much smaller) set of unique +directories. + +Rename: iterate through all manifests and queue their renames into a pool for +renaming. + +#### Thread pool lifetimes + +The lifespan of thread pools is constrained to that of the stage configuration, +which will be limited to within each of the `PathOutputCommitter` methods +to setup, commit, abort and cleanup. + +This avoids the thread pool lifecycle problems of the S3A Committers. + +#### Scale issues similar to S3A HADOOP-16570. + +This was a failure in terasorting where many tasks each generated many files; +the full list of files to commit (and the etag of every block) was built up in memory and +validated prior to execution. + +The manifest committer assumes that the amount of data being stored in memory is less, +because there is no longer the need to store an etag for every block of every +file being committed. + + +#### Duplicate creation of directories in the dest dir + +Combine all lists of directories to create and eliminate duplicates. + +## Implementation Architecture + +The implementation architecture reflects lessons from the S3A Connector. + +* Isolate the commit stages from the MR commit class, as that's got a complex lifecycle. +* Instead, break up into series of _stages_ which can be tested in isolation + and chained to provided the final protocol. +* Don't pass in MR data types (taskID etc) down into the stages -pass down a configuration + with general types (string etc). +* Also pass in a callback for store operations, for ease of implementing a fake store. +* For each stage: define preconditions and postconditions, failure modes. Test in isolation. + +#### Statistics + +The committer collects duration statistics on all the operations it performs/invokes +against filesystems. +* Those collected during task commit are saved to the manifest (excluding the time to +save and rename that file) +* When these manifests are loaded during job commit, these statistics are merged to +form aggregate statistics of the whole job. +* Which are saved to the `_SUCCESS` file +* and to any copy of that file in the directory specified by + `mapreduce.manifest.committer.summary.report.directory`, if set. + to be saved. +* The class `org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter` + can load and print these. + +IO statistics from filsystems and input and output streams used in a query are not +collected. + + +## Auditing + +When invoking the `ManifestCommitter` via the `PathOutputCommitter` API, the following +attributes are added to the active (thread) context + +| Key | Value | +|-------|-----------------| +| `ji` | Job ID | +| `tai` | Task Attempt ID | +| `st` | Stage | + +These are also all set in all the helper threads performing work +as part of a stage's execution. + +Any store/FS which supports auditing is able to collect this data +and include in their logs. + +To ease backporting, all audit integration is in the single class +`org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.AuditingIntegration`. \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_protocol.md b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_protocol.md new file mode 100644 index 0000000000000..c5af28de70c34 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/site/markdown/manifest_committer_protocol.md @@ -0,0 +1,617 @@ + + + +# Manifest Committer Protocol + +This document describes the commit protocol + of the [Manifest Committer](manifest_committer.html) + + + +## Background + +### Terminology + +| Term | Meaning| +|------|--------| +| Committer | A class which can be invoked by MR Spark to perform the task and job commit operations. | +| Spark Driver | The spark process scheduling the work and choreographing the commit operation.| +| Job: in MapReduce | The entire application. In spark, this is a single stage in a chain of work | +| Job Attempt | A single attempt at a job. MR supports multiple Job attempts with recovery on partial job failure. Spark says "start again from scratch" | +| Task | a subsection of a job, such as processing one file, or one part of a file | +| Task ID | ID of the task, unique within this job. Usually starts at 0 and is used in filenames (part-0000, part-001, etc.) | +| Task attempt (TA) | An attempt to perform a task. It may fail, in which case MR/spark will schedule another. | +| Task Attempt ID | A unique ID for the task attempt. The Task ID + an attempt counter.| +| Destination directory | The final destination of work.| +| Job Attempt Directory | A temporary directory used by the job attempt. This is always _underneath_ the destination directory, so as to ensure it is in the same encryption zone as HDFS, storage volume in other filesystems, etc.| +| Task Attempt directory | Directory under the Job Attempt Directory where task attempts create subdiretories for their own work | +| Task Attempt Working Directory| Directory exclusive for each task attempt under which files are written | +| Task Commit | Taking the output of a Task Attempt and making it the final/exclusive result of that "successful" Task. | +| Job Commit | aggregating all the outputs of all committed tasks and producing the final results of the job. | + + + +The purpose of a committer is to ensure that the complete output of +a job ends up in the destination, even in the presence of failures of tasks. + +- _Complete:_ the output includes the work of all successful tasks. +- _Exclusive:_ the output of unsuccessful tasks is not present. +- _Concurrent:_ When multiple tasks are committed in parallel the output is the same as when + the task commits are serialized. This is not a requirement of Job Commit. +- _Abortable:_ jobs and tasks may be aborted prior to job commit, after which their output is not visible. +- _Continuity of correctness:_ once a job is committed, the output of any failed, + aborted, or unsuccessful task MUST NO appear at some point in the future. + +For Hive's classic hierarchical-directory-structured tables, job committing +requires the output of all committed tasks to be put into the correct location +in the directory tree. + +The committer built into `hadoop-mapreduce-client-core` module is the `FileOutputCommitter`. + + +It has two algorithms, v1 and v2. + +The v1 algorithm is resilient to all forms of task failure, but slow +when committing the final aggregate output as it renames each newly created file +to the correct place in the table one by one. + +The v2 algorithm is not considered safe because the output is visible when individual +tasks commit, rather than being delayed until job commit. +It is possible for multiple task attempts to get their data into the output +directory tree, and if a job fails/is aborted before the job is committed, +thie output is visible. + +## File Output Committer V1 and V2 + +### File Output Committer V1 and V2 Commit algorithms + +#### Task attempt execution (V1 and V2) + +job attempt directory in `$dest/__temporary/$jobAttemptId/` contains all output +of the job in progress every task attempt is allocated its own task attempt dir +`$dest/__temporary/$jobAttemptId/__temporary/$taskAttemptId` + +All work for a task is written under the task attempt directory. If the output +is a deep tree with files at the root, the task attempt dir will end up with a +similar structure, with the files it has generated and the directories above +them. + +### MapReduce V1 algorithm: + +#### v1 Task commit + +The task attempt dir is renamed directly underneath the job attempt dir + +``` +rename( + $dest/__temporary/$jobAttemptId/__temporary/$taskAttemptId + $dest/__temporary/$jobAttemptId/$taskId) +``` + +#### V1 Job Commit + +For each committed task, all files underneath are renamed into the destination +directory, with a filename relative from the base directory of the task remapped +to that of the dest dir. + +That is, everything under `$dest/__temporary/$jobAttemptId/$taskId` is converted +to a path under `$dest`. + +A recursive treewalk identifies the paths to rename in each TA directory. +There's some optimisation if the task directory tree contains a subdirectory +directory which does not exist under the destination: in this case the whole +directory can be renamed. If the directory already exists, a file-by-file merge +takes place for that dir, with the action for subdirectories again depending on +the presence of the destination. + +As a result, if the output of each task goes a separate final directory (e.g the +final partition is unique to a single task), the rename is O(1) for the dir, +irrespective of children. If the output is to be in the same dir as other +tasks (or updating existing directories), then the rename performance becomes O( +files). + +Finally, a 0-byte `_SUCCESS` file is written iff `mapreduce.fileoutputcommitter.marksuccessfuljobs` is true. + +### MapReduce V2 algorithm: + +#### V2 Task commit + +The files under the task attempt dir are renamed one by one into the destination +directory. There's no attempt at optimising directory renaming, because other +tasks may be committing their work at the same time. It is therefore `O(files)` + +the cost of listing the directory tree. Again: done with a recursive treewalk, +not a deep `listFiles(path, recursive=true)` API, which would be faster on HDFS +and (though not relevant here) S3. + +#### V2 Job Commit + +A 0-byte `_SUCCESS` file is written iff `mapreduce.fileoutputcommitter.marksuccessfuljobs` +is true. + + + +### Why the V2 committer is incorrect/unsafe + +If, for a Task T1, Task Attempt 1 (T1A1) fails before committing, the driver +will schedule a new attempt "T1A2", and commit it. All is good. + +But: if T1A1 was given permission to commit and it failed during the commit +process, some of its output may have been written to the destination directory. + +If attempt T1A2 was then told to commit, then if and only if its output had the +exact set of file names would any already-renamed files be overwritten. If +different filenames were generated, then the output would contain files of T1A1 +and T1A2. + +If T1A1 became partitioned during the commit process, then the job committer +would schedule another attempt and commit its work. However, if T1A1 still had +connectivity to the filesystem, it could still be renaming files. The output of +the two tasks could be intermingled even if the same filenames were used. + +## Background: the S3A Committers + +The paper, [_A Zero-Rename Committer_](https://github.com/steveloughran/zero-rename-committer/releases/), +Loughran et. al., covers these committers + +It also describes the commit problem, defines correctness, and describes the +algorithms of the v1 and v2 committers, as well as those of the S3A committers, +IBM Stocator committer and what we know of EMR's Spark committer. + +The `hadoop-aws` JAR contains a pair of committers, "Staging" and "Magic". Both +of these are implementations of the same problem: safely and rapidly committing +work to an S3 object store. + +The committers take advantage of the fact that S3 offers an atomic way to create +a file: the PUT request. + +Files either exist or they don't. A file can be uploaded direct to its +destination, and it is only when the upload completes that the file is manifest +-overwriting any existing copy. + +For large files, a multipart upload allows this upload operation to be split +into a series of POST requests + +1 `initiate-upload (path -> upload ID)` +1. `upload part(path, upload ID, data[]) -> checksum.` + This can be parallelised. Up to 10,000 parts can be uploaded to a single + object. All but the final part must be >= 5MB. +1. `complete-upload (path, upload ID, List)` + this manifests the file, building it from the parts in the sequence of blocks + defined by the ordering of the checksums. + +The secret for the S3A committers is that the final POST request can be delayed +until the job commit phase, even though the files are uploaded during task +attempt execution/commit. The task attempts need to determine the final +destination of each file, upload the data as part of a multipart operation, then +save the information needed to complete the upload in a file which is later read +by the job committer and used in a POST request. + +### Staging Committer + +The _Staging Committer_ is based on the contribution by Ryan Blue of Netflix. +it relies on HDFS to be the consistent store to propagate the `.pendingset` files. + +The working directory of each task attempt is in the local filesystem, "the +staging directory". The information needed to complete the uploads is passed +from Task Attempts to the Job Committer by using a v1 FileOutputCommitter +working with the cluster HDFS filesystem. This ensures that the committer has +the same correctness guarantees as the v1 algorithm. + +1. Task commit consists of uploading all files under the local filesystem's task + attempt working directory to their final destination path, holding back on + the final manifestation POST. +1. A JSON file containing all information needed to complete the upload of all + files in the task attempt is written to the Job Attempt directory of of the + wrapped committer working with HDFS. +1. Job commit: load in all the manifest files in the HDFS job attempt directory, + then issued the POST request to complete the uploads. These are parallelised. + + +### The Magic Committer + +The _Magic Committer_ is purely-S3A and takes advantage and of +the fact the authorts could make changes within the file system client itself. + +"Magic" paths are defined which, when opened for writing under, initiate a +multi-party upload to the final destination directory. When the output stream is +`close()`d, a zero byte marker file is written to the magic path, and a JSON +.pending file containing all the information needed to complete the upload is +saved. + +Task commit: +1. List all `.pending` files under each task attempt's magic directory; +1. Aggregate to a `.pendingset` file +1. Save to the job attempt directory with the task ID. + +Job commit: + +1. List `.pendingset` files in the job attempt directory +1. Complete the uploads with POST requests. + +The Magic committer absolutely requires a consistent S3 Store -originally with +S3Guard. Now that S3 is consistent, raw S3 can be used. It does not need HDFS +or any other filesystem with `rename()`. + +### Correctness + +The S3A committer is considered correct because + +1. Nothing is materialized until job commit. +1. Only one task attempt's manifest can be saved to the job attempt directory. + Hence: only of the TA's files of the same task ID are exclusively committed. +1. The staging committer's use of HDFS to pass manifests from TAs to the Job + committer ensures that S3's eventual consistency would not cause manifests to + be missed. +1. Until S3 was consistent, the magic committer relied on S3Guard to provide the + list consistency needed during both task- and job- commit. +1. The authors and wider community fixed all the issues related to the committers + which have surfaced in production. + +Significant issues which were fixed include: + +* [HADOOP-15961](https://issues.apache.org/jira/browse/HADOOP-15961). + S3A committers: make sure there's regular progress() calls. +* [HADOOP-16570](https://issues.apache.org/jira/browse/HADOOP-16570). + S3A committers encounter scale issues. +* [HADOOP-16798](https://issues.apache.org/jira/browse/HADOOP-16798). + S3A Committer thread pool shutdown problems. +* [HADOOP-17112](https://issues.apache.org/jira/browse/HADOOP-17112). + S3A committers can't handle whitespace in paths. +* [HADOOP-17318](https://issues.apache.org/jira/browse/HADOOP-17318). + Support concurrent S3A commit jobs with same app attempt ID. +* [HADOOP-17258](https://issues.apache.org/jira/browse/HADOOP-17258). + MagicS3GuardCommitter fails with `pendingset` already exists +* [HADOOP-17414](https://issues.apache.org/jira/browse/HADOOP-17414]). + Magic committer files don't have the count of bytes written collected by spark +* [SPARK-33230](https://issues.apache.org/jira/browse/SPARK-33230) + Hadoop committers to get unique job ID in `spark.sql.sources.writeJobUUID` +* [SPARK-33402](https://issues.apache.org/jira/browse/SPARK-33402) + Jobs launched in same second have duplicate MapReduce JobIDs +* [SPARK-33739](https://issues.apache.org/jira/browse/SPARK-33739]). + Jobs committed through the S3A Magic committer don't report + the bytes written (depends on HADOOP-17414) + +Of those which affected the correctness rather than scale/performance/UX: +HADOOP-17258 involved the recovery from a failure after TA1 task commit had +completed —but had failed to report in. SPARK-33402, SPARK-33230 and +HADOOP-17318 are all related: if two spark jobs/stages started in the +same second, they had the same job ID. This caused the HDFS directories used by +the staging committers to be intermingled. + +What is notable is this: these are all problems which the minimal integration +test suites did not discover. + +The good news: we now know of these issues and are better placed to avoid +replicating them again. And know what to write tests for. + +## The V1 committer: slow in Azure and slow and unsafe on GCS. + +The V1 committer underperforms on ABFS because: + +1. Directory listing and file renaming is somewhat slower with ABFS than it is + with HDFS. +1. The v1 committer sequentially commits the output of each task through a + listing of each committed task's output, moving directories when none exist + in the destination, merging files into extant directories. + +The V2 committer is much faster in the job commit because it performs the list +and rename process in the task commit. Which, because it is non-atomic, is why +it is considered dangerous to use. What the V2 task commit algorithm does show is +that it is possible to parallelise committing the output of different tasks by +using file-by-file rename exclusively. + +The V1 committer underperforms on GCS because even the task commit operation, +—directory rename—, is a non-atomic `O(files)` operation. +This also means that it is unsafe. + +If the task attempt has partitioned and the spark driver schedules/commits another TA, then, +the task dir may contain 1+ file from the first attempt. + +---------------------------------------------------------------------------------------- + +# The Manifest Committer Protocol + +## Requirements of the Store + +Stores/filesystems supported by this committer MUST: + +* Have consistent listings. +* Have an atomic `O(1)` file rename operation. + +Stores/filesystems supported by this committer SHOULD: + +* Rename files successfully, even under load. ABFS does not do this, + so special recovery is provided there. +* Implement the `EtagSource` interface of HADOOP-17979. + This is used for ABFS rename recovery, and for optional + validation of the final output. + +Stores/filesystems supported by this committer MAY: + +* Have list operations with high latency. +* Reject calls under load with throttling responses, + which MUST be handled in the filesystem connector. + +Stores/filesystems supported by this committer MAY NOT: + +* Support atomic directory rename. This is never used except optionally in cleanup. +* Support `O(1)` directory deletion. The `CleanupJobStage` assumes this is not + the case and so deletes task attempt directories in parallel. +* Support an atomic `create(Path, overwrite=false)` operation. + The manifests are committed by writing to a path including the task attempt ID, + then renamed to their final path. +* Support fast `listFiles(path, recursive=true)` calls. + This API call is not used. + +When compared with the `FileOutputCommitter`, the requirements +which have been removed are: + +* Atomic directory rename. +* `O(1)` directory deletion. +* Fast directory listings. +* The implicit absence of throttling behaviors. + +HDFS meets all those requirements, so does not benefit significantly from +this committer, though it will still work there. + +The S3 store does not meet the rename requirements of this committer, +even now that it is consistent. +This committer is not safe to use on S3. + +### Task and Job IDs + +Every job MUST have a unique ID. + +The implementation expects the Spark runtime to have the relevant patches to +ensure this. + +The job ID is used to name temporary directories, rather than using the classic +incrementing natural numbering scheme of `_temporary/0/`. +That scheme comes from MapReduce where job attempts of attempt ID > 1 +look for tasks committed by predecessors and incorporate that into their +results. + +This committer targets Spark, where there is no attempt at recovery. +By using the job ID in paths, if jobs are configured to _not_ delete +all of `_temporary` in job cleanup/abort, then multiple jobs +MAY be executed using the same table as their destination. + +Task IDs and Task Attempt IDs will be derived from Job IDs as usual. + +It is expected that filenames of written files SHALL be unique. +This is done in Spark for ORC and Parquet files, and allows for +checks for destination files to be omitted by default. + + +## Directory Structure + +Given a destination directory `destDir: Path` + +A job of id `jobID: String` and attempt number `jobAttemptNumber:int` +will use the directory: + +``` +$destDir/_temporary/manifest_$jobID/$jobAttemptNumber/ +``` + +For its work (note: it will actually format that final subdir with `%02d`). + +This is termed the _Job Attempt Directory_ + +Under the Job Attempt Directory, a subdirectory `tasks` is +created. This is termed the _Task Attempt Directory_. +Every task attempt will have its own subdirectory of this, +into which its work will be saved. + +Under the Job Attempt Directory, a subdirectory `manifests` is created. +This is termed the _y_. + +The manifests of all committed tasks will be saved to this +directory with the filename of +`$taskId-manifest.json` + +The full path + +``` +$destDir/_temporary/manifest_$jobID/$jobAttemptNumber/manifests/$taskId-manifest.json +``` + +Is the final location for the manifest of all files created by +a committed task. It is termed the _Manifest Path of a Committed Task_. + +Task attempts will save their manifest into this directory with +a temporary filename +`$taskAttemptId-manifest.json.tmp`. + +This is termed the _Temporary Path of a Task Attempt's Manifest_. + +For the job and task operations then, the following paths are +defined. +``` +let jobDirectory = "$destDir/_temporary/manifest_$jobID/" +let jobAttemptDirectory = jobDirectory + "$jobAttemptNumber/" +let manifestDirectory = jobAttemptDirectory + "manifests/" +let taskAttemptDirectory = jobAttemptDirectory + "tasks/" +``` + +And for each task attempt, the following paths are also defined + +``` +let taskAttemptWorkingDirectory = taskAttemptDirectory + "$taskAttemptId" +let taskManifestPath = manifestDirectory + "$taskId-manifest.json" +let taskAttemptTemporaryManifestPath = manifestDirectory + "$taskAttemptId-manifest.json" +``` + +## Core Algorithm of the Protocol + +1. Each Task attempt writes all its files to a unique directory tree under the + Task Attempt Directory. +2. Task Commit consists of a recursive scan of the directory for that task attempt, + creating a list of directories and a list of files. +3. These lists are saved as a JSON manifest file. +4. Job commit consists of listing all of the JSON manifest files, + loading their contents, creating the aggregate set of destination directories + and renaming all files into their final destinations. + + +### The Intermediate Manifest + +This is JSON file is designed which contains (along with IOStatistics and some diagnostics) + +1. A list of destination directories which must be created if they do not exist. +1. A list of files to rename as (absolute source, absolute destination, + file-size) entries. + +### Job Setup + +``` +mkdir(jobAttemptDirectory) +mkdir(manifestDirectory) +mkdir(taskAttemptDirectory) +``` + +### Task Setup + +``` +mkdir(taskAttemptWorkingDirectory) +``` + +### Task Commit + +Task attempts are committed by: + +1. Recursively listing the task attempt working dir to build + 1. A list of destination directories under which files will be renamed, + and their status (exists, not_found, file) + 2. A list of files to rename: source, destination, size and optionally, etag. +2. These lists populate a JSON file, the _Intermediate Manifest_. +3. The task attempt saves this file to its _Temporary Path of a Task Attempt's + Manifest_. +4. The task attempt then deletes the _Manifest Path of a Committed Task_ and + renames its own manifest file to that path. +5. If the rename succeeeds, the task commit is considered a success. + +No renaming takes place at this point.: the files are left in their original location until +renamed in job commit. + +``` +let (renames, directories) = scan(taskAttemptWorkingDirectory) +let manifest = new Manifest(renames, directories) + +manifest.save(taskAttemptTemporaryManifestPath) +rename(taskAttemptTemporaryManifestPath, taskManifestPath) +``` + +### Task Abort/cleanup + +``` +delete(taskAttemptWorkingDirectory) +``` + +### Job Commit + +Job Commit consists of: + +1. List all manifest files in the job attempt directory. +2. Load each manifest file, create directories which do not yet exist, then + rename each file in the rename list. +3. Optionally save a JSON `_SUCCESS` file with the same format as the S3A committer (for + testing; use write and rename for atomic save) + +The job commit phase supports parallelization for many tasks and many files +per task, specifically there is a thread pool for parallel store IO + +1. Manifest tasks are loaded and processed in parallel. +1. Deletion of files where directories are intended to be created. +1. Creation of leaf directories. +1. File rename. +1. In cleanup and abort: deletion of task attempt directories +1. If validation of output is enabled for testing/debugging: getFileStatus calls + to compare file length and, if possible etags. + +``` +let manifestPaths = list("$manifestDirectory/*-manifest.json") +let manifests = manifestPaths.map(p -> loadManifest(p)) +let directoriesToCreate = merge(manifests.directories) +let filesToRename = concat(manifests.files) + +directoriesToCreate.map(p -> mkdirs(p)) +filesToRename.map((src, dest, etag) -> rename(src, dest, etag)) + +if mapreduce.fileoutputcommitter.marksuccessfuljobs then + success.save("$destDir/_SUCCESS") + +``` + +Implementation Note: + +To aid debugging and development, the summary be saved to a location +in the same _or different_ filesystem; the intermediate +manifests may be renamed to a location in the target filesystem. + +``` +if summary.report.directory != "" then + success.save("${summary.report.directory}/$jobID.json") +if diagnostics.manifest.directory != null then + rename($manifestDirectory, "${diagnostics.manifest.directory}/$jobID") +``` + +The summary report is saved even if job commit fails for any reason + +### Job Abort/cleanup + +Job cleanup is nominally one of deleting the job directory +``` +delete(jobDirectory) +``` + +To address scale issues with the object stores, this SHALL be preceeded by +a (parallelized) deletion of all task attempt working directories + +``` +let taskAttemptWorkingDirectories = list("taskAttemptDirectory") +taskAttemptWorkingDirectories.map(p -> delete(p)) +``` + + +## Benefits of the new protocol + +* Pushes the source tree list operations into the task commit phase, which is + generally off the critical path of execution. +* Reduces the number of directories probed/created to the aggregate set of + output directories, with all duplicates eliminated. +* File rename can be parallelized, with the limits being that of configured + thread pool sizes and/or any rate limiting constraints. +* Provides an atomic task commit to GCS, as there is no expectation that + directory rename is atomic. +* Permits pass IOStatistics from tasks attempts to the job committer via the manifests. +* Allows for some pre-rename operations in the Job Committer + similar to the S3A "Partitioned Staging committer". + This can be configured to delete all existing entries in + directories scheduled to be created -or fail if those partitions are + non-empty. + See [Partitioned Staging Committer](../../hadoop-aws/tools/hadoop-aws/committers.html#The_.E2.80.9CPartitioned.E2.80.9D_Staging_Committer) +* Allows for an optional preflight validation check (verify no duplicate files created by different tasks). +* Manifests can be viewed, size of output determined, etc., during development/debugging. + +## Disadvantages of the new protocol compared to the v1 algorithm + +* Needs a new manifest file format. +* Manifests may get large if tasks create many files and/or subdirectories, or if + etags are collected and the length of these tags is significant. + The HTTP protocol limits each etag to 8 KiB, so the cost may be 8 KiB per file. +* Makes task commit more complex than the v1 algorithm. +* Possibly suboptimal on jobs where individual tasks create unique output directories, + as directory rename will never be used to commit a directory. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java new file mode 100644 index 0000000000000..bcd9d34e263de --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/AbstractManifestCommitterTest.java @@ -0,0 +1,1085 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.contract.localfs.LocalFSContract; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SaveTaskManifestStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.functional.CloseableTaskPoolSubmitter; +import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.hadoop.util.functional.TaskPool; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.readDataset; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.COMMITTER_FACTORY_CLASS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConfig.createCloseableTaskSubmitter; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ID_SOURCE_MAPREDUCE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_FACTORY; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_DIAGNOSTICS_MANIFEST_DIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_SUMMARY_REPORT_DIR; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_VALIDATE_OUTPUT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CLEANUP; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.getProjectBuildDir; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.validateSuccessFile; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.NAME_FORMAT_JOB_ATTEMPT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createIOStatisticsStore; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createTaskManifest; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; + +/** + * Tests which work with manifest committers. + * This is a filesystem contract bound to the local filesystem; + * subclasses may change the FS to test against other stores. + * Some fields are set up in + * in {@link #executeOneTaskAttempt(int, int, int)}, + * which is why fields are used. + * when synchronized access is needed; synchronize on (this) rather + * than individual fields + */ +public abstract class AbstractManifestCommitterTest + extends AbstractFSContractTestBase { + + protected static final Logger LOG = + LoggerFactory.getLogger(AbstractManifestCommitterTest.class); + + /** + * Some Job and task IDs. + */ + protected static final ManifestCommitterTestSupport.JobAndTaskIDsForTests + TASK_IDS = new ManifestCommitterTestSupport.JobAndTaskIDsForTests(2, 2); + + public static final int JOB1 = 1; + + public static final int TASK0 = 0; + + public static final int TASK1 = 1; + + /** + * Task attempt 0 index. + */ + public static final int TA0 = 0; + + /** + * Task attempt 1 index. + */ + public static final int TA1 = 1; + + /** + * Depth of dir tree to generate. + */ + public static final int DEPTH = 3; + + /** + * Width of dir tree at every level. + */ + public static final int WIDTH = 2; + + /** + * How many files to create in the leaf directories. + */ + public static final int FILES_PER_DIRECTORY = 4; + + /** + * Pool size. + */ + public static final int POOL_SIZE = 32; + + /** + * FileSystem statistics are collected across every test case. + */ + protected static final IOStatisticsSnapshot FILESYSTEM_IOSTATS = + snapshotIOStatistics(); + + /** + * Counter for creating files. Ensures that across all test suites, + * duplicate filenames are never created. Helps assign blame. + */ + private static final AtomicLong CREATE_FILE_COUNTER = new AtomicLong(); + + protected static final byte[] NO_DATA = new byte[0]; + + /** + * The thread leak tracker. + */ + private static final ThreadLeakTracker THREAD_LEAK_TRACKER = new ThreadLeakTracker(); + + private static final int MAX_LEN = 64_000; + + /** + * Submitter for tasks; may be null. + */ + private CloseableTaskPoolSubmitter submitter; + + /** + * Stage statistics. Created in test setup, and in + * teardown updates {@link #FILESYSTEM_IOSTATS}. + */ + private IOStatisticsStore stageStatistics; + + /** + * Prefer to use these to interact with the FS to + * ensure more implicit coverage. + */ + private ManifestStoreOperations storeOperations; + + /** + * Progress counter used in all stage configs. + */ + private final ProgressCounter progressCounter = new ProgressCounter(); + + /** + * Directory for job summary reports. + * This should be set up in test suites testing against real object stores. + */ + private File reportDir; + + /** + * List of task attempt IDs for those tests which create manifests. + */ + private final List taskAttemptIds = new ArrayList<>(); + + /** + * List of task IDs for those tests which create manifests. + */ + private final List taskIds = new ArrayList<>(); + + /** + * any job stage configuration created for operations. + */ + private StageConfig jobStageConfig; + + /** + * Destination dir of job. + */ + private Path destDir; + + /** + * When creating manifests, total data size. + */ + private final AtomicLong totalDataSize = new AtomicLong(); + + /** + * Where to move manifests; must be in target FS. + */ + private Path manifestDir; + + /** + * Get the contract configuration. + * @return the config used to create the FS. + */ + protected Configuration getConfiguration() { + return getContract().getConf(); + } + + /** + * Store operations to interact with.. + * @return store operations. + */ + protected ManifestStoreOperations getStoreOperations() { + return storeOperations; + } + + /** + * Set store operations. + * @param storeOperations new value + */ + protected void setStoreOperations(final ManifestStoreOperations storeOperations) { + this.storeOperations = storeOperations; + } + + public List getTaskAttemptIds() { + return taskAttemptIds; + } + + public List getTaskIds() { + return taskIds; + } + + public long getTotalDataSize() { + return totalDataSize.get(); + } + + public Path getManifestDir() { + return manifestDir; + } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public AbstractManifestCommitterTest withManifestDir(Path value) { + manifestDir = value; + return this; + } + + /** + * Describe a test in the logs. + * @param text text to print + * @param args arguments to format in the printing + */ + protected void describe(String text, Object... args) { + LOG.info("\n\n{}: {}\n", + getMethodName(), + String.format(text, args)); + } + + /** + * Local FS unless overridden. + * @param conf configuration + * @return the FS contract. + */ + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new LocalFSContract(conf); + } + + /** Enable the manifest committer options in the configuration. */ + @Override + protected Configuration createConfiguration() { + return enableManifestCommitter(super.createConfiguration()); + } + + @Override + public void setup() throws Exception { + + // set the manifest committer to a localfs path for reports across + // all threads. + // do this before superclass setup so reportDir is non-null there + // and can be used in creating the configuration. + reportDir = new File(getProjectBuildDir(), "reports"); + reportDir.mkdirs(); + + // superclass setup includes creating a filesystem instance + // for the target store. + super.setup(); + + manifestDir = path("manifests"); + + // destination directory defaults to method path in + // target FS + setDestDir(methodPath()); + + // stage statistics + setStageStatistics(createIOStatisticsStore().build()); + // thread pool for task submission. + setSubmitter(createCloseableTaskSubmitter(POOL_SIZE, TASK_IDS.getJobId())); + // store operations for the target filesystem. + storeOperations = createManifestStoreOperations(); + } + + /** + * Overrride point: create the store operations. + * @return store operations for this suite. + */ + protected ManifestStoreOperations createManifestStoreOperations() throws IOException { + final FileSystem fs = getFileSystem(); + return ManifestCommitterSupport.createManifestStoreOperations(fs.getConf(), fs, getTestPath()); + } + + @Override + public void teardown() throws Exception { + Thread.currentThread().setName("teardown"); + + IOUtils.cleanupWithLogger(LOG, storeOperations, getSubmitter()); + storeOperations = null; + super.teardown(); + FILESYSTEM_IOSTATS.aggregate(retrieveIOStatistics(getFileSystem())); + FILESYSTEM_IOSTATS.aggregate(getStageStatistics()); + } + + /** + * Add a long delay so that you don't get timeouts when working + * with object stores or debugging. + * @return a longer timeout than the base classes. + */ + @Override + protected int getTestTimeoutMillis() { + return 600_000; + } + + protected Path getTestPath() { + return getContract().getTestPath(); + } + + /** + * Get the task submitter. + * @return submitter or null + */ + protected CloseableTaskPoolSubmitter getSubmitter() { + return submitter; + } + + /** + * Set the task submitter. + * @param submitter new value. + */ + protected void setSubmitter(CloseableTaskPoolSubmitter submitter) { + this.submitter = submitter; + } + + /** + * Get the executor which the submitter also uses. + * @return an executor. + */ + protected ExecutorService getExecutorService() { + return getSubmitter().getPool(); + } + /** + * @return IOStatistics for stage. + */ + protected final IOStatisticsStore getStageStatistics() { + return stageStatistics; + } + + /** + * Set the statistics. + * @param stageStatistics statistics. + */ + protected final void setStageStatistics(IOStatisticsStore stageStatistics) { + this.stageStatistics = stageStatistics; + } + + /** + * Get the progress counter invoked during commit operations. + * @return progress counter. + */ + protected final ProgressCounter getProgressCounter() { + return progressCounter; + } + + /** + * Get the report directory. + * @return report directory. + */ + public final File getReportDir() { + return reportDir; + } + + /** + * Get the report directory as a URI. + * @return report directory. + */ + public final URI getReportDirUri() { + return getReportDir().toURI(); + } + + /** + * Get the (shared) thread leak tracker. + * @return the thread leak tracker. + */ + protected static ThreadLeakTracker getThreadLeakTracker() { + return THREAD_LEAK_TRACKER; + } + + /** + * Make sure there's no thread leakage. + */ + @AfterClass + public static void threadLeakage() { + THREAD_LEAK_TRACKER.assertNoThreadLeakage(); + } + + /** + * Dump the filesystem statistics after the class. + */ + @AfterClass + public static void dumpFileSystemIOStatistics() { + LOG.info("Aggregate FileSystem Statistics {}", + ioStatisticsToPrettyString(FILESYSTEM_IOSTATS)); + } + + /** + * Create a directory tree through an executor. + * dirs created = width^depth; + * file count = width^depth * files + * If createDirs == true, then directories are created at the bottom, + * not files. + * @param base base dir + * @param prefix prefix for filenames. + * @param executor submitter. + * @param depth depth of dirs + * @param width width of dirs + * @param files files to add in each base dir. + * @param createDirs create directories rather than files? + * @return the list of paths + * @throws IOException failure. + */ + public final List createFilesOrDirs(Path base, + String prefix, + ExecutorService executor, + int depth, + int width, + int files, + boolean createDirs) throws IOException { + + try (DurationInfo ignored = new DurationInfo(LOG, true, + "Creating Files %s (%d, %d, %d) under %s", + prefix, depth, width, files, base)) { + + assertPathExists("Task attempt dir", base); + + // create the files in the thread pool. + List> futures = createFilesOrDirs( + new ArrayList<>(), + base, prefix, + executor, + depth, width, files, + createDirs); + List result = new ArrayList<>(); + + // now wait for the creations to finish. + for (Future f : futures) { + result.add(awaitFuture(f)); + } + return result; + } + } + + /** + * Counter incremented for each file created. + */ + private final AtomicLong fileDataGenerator = new AtomicLong(); + + /** + * Create files or directories; done in a treewalk and building up + * a list of futures to wait for. The list is + * build up incrementally rather than through some merging of + * lists created down the tree. + * If createDirs == true, then directories are created at the bottom, + * not files. + * + * @param futures list of futures to build up. + * @param base base dir + * @param prefix prefix for filenames. + * @param executor submitter. + * @param depth depth of dirs + * @param width width of dirs + * @param files files to add in each base dir. + * @param createDirs create directories rather than files? + * @return the list of futures + */ + private List> createFilesOrDirs( + List> futures, + Path base, + String prefix, + ExecutorService executor, + int depth, + int width, + int files, + boolean createDirs) { + + if (depth > 0) { + // still creating directories + for (int i = 0; i < width; i++) { + Path child = new Path(base, + String.format("dir-%02d-%02d", depth, i)); + createFilesOrDirs(futures, child, prefix, executor, depth - 1, width, files, false); + } + } else { + // time to create files + for (int i = 0; i < files; i++) { + Path file = new Path(base, + String.format("%s-%04d", prefix, + CREATE_FILE_COUNTER.incrementAndGet())); + // buld the data. Not actually used in mkdir. + long entry = fileDataGenerator.incrementAndGet() & 0xffff; + byte[] data = new byte[2]; + data[0] = (byte) (entry & 0xff); + data[1] = (byte) ((entry & 0xff00) >> 8); + // the async operation. + Future f = executor.submit(() -> { + if (!createDirs) { + // create files + ContractTestUtils.createFile(getFileSystem(), file, true, data); + } else { + // create directories + mkdirs(file); + } + return file; + }); + futures.add(f); + } + } + return futures; + } + + /** + * Create a list of paths under a dir. + * @param base base dir + * @param count count + * @return the list + */ + protected List subpaths(Path base, int count) { + return IntStream.rangeClosed(1, count) + .mapToObj(i -> new Path(base, String.format("entry-%02d", i))) + .collect(Collectors.toList()); + } + + /** + * Submit a mkdir call to the executor pool. + * @param path path of dir to create. + * @return future + */ + protected CompletableFuture asyncMkdir(final Path path) { + CompletableFuture f = new CompletableFuture<>(); + getExecutorService().submit(() -> { + try { + mkdirs(path); + f.complete(path); + } catch (IOException e) { + f.completeExceptionally(e); + } + }); + return f; + } + + /** + * Given a list of paths, create the dirs async. + * @param paths path list + * @throws IOException failure + */ + protected void asyncMkdirs(Collection paths) throws IOException { + List> futures = new ArrayList<>(); + // initiate + for (Path path: paths) { + futures.add(asyncMkdir(path)); + } + // await + for (Future f : futures) { + awaitFuture(f); + } + } + + /** + * Submit an oepration to create a file to the executor pool. + * @param path path of file to create. + * @return future + */ + protected CompletableFuture asyncPut(final Path path, byte[] data) { + CompletableFuture f = new CompletableFuture<>(); + getExecutorService().submit(() -> { + try { + ContractTestUtils.createFile(getFileSystem(), path, true, data); + f.complete(path); + } catch (IOException e) { + f.completeExceptionally(e); + } + }); + return f; + } + + /** + * Convert the manifest list to a map by task attempt ID. + * @param list manifests + * @return a map, indexed by task attempt ID. + */ + protected Map toMap(List list) { + return list.stream() + .collect(Collectors.toMap(TaskManifest::getTaskAttemptID, x -> x)); + } + + /** + * Verify the manifest files match the list of paths. + * @param manifest manifest to audit + * @param files list of files. + */ + protected void verifyManifestFilesMatch(final TaskManifest manifest, + final List files) { + // get the list of source paths + Set filesToRename = manifest.getFilesToCommit() + .stream() + .map(FileEntry::getSourcePath) + .collect(Collectors.toSet()); + // which must match that of all the files created + Assertions.assertThat(filesToRename) + .containsExactlyInAnyOrderElementsOf(files); + } + + /** + * Verify that a task manifest has a given attempt ID. + * @param manifest manifest, may be null. + * @param attemptId expected attempt ID + * @return the manifest, guaranteed to be non-null and of task attempt. + */ + protected TaskManifest verifyManifestTaskAttemptID( + final TaskManifest manifest, + final String attemptId) { + Assertions.assertThat(manifest) + .describedAs("Manifest of task %s", attemptId) + .isNotNull(); + Assertions.assertThat(manifest.getTaskAttemptID()) + .describedAs("Task Attempt ID of manifest %s", manifest) + .isEqualTo(attemptId); + return manifest; + } + + /** + * Assert that a path must exist; return the path. + * @param message text for error message. + * @param path path to validate. + * @return the path + * @throws IOException IO Failure + */ + Path pathMustExist(final String message, + final Path path) throws IOException { + assertPathExists(message, path); + return path; + } + + /** + * Assert that a path must exist; return the path. + * It must also equal the expected value. + * @param message text for error message. + * @param expectedPath expected path. + * @param actualPath path to validate. + * @return the path + * @throws IOException IO Failure + */ + Path verifyPath(final String message, + final Path expectedPath, + final Path actualPath) throws IOException { + Assertions.assertThat(actualPath) + .describedAs(message) + .isEqualTo(expectedPath); + return pathMustExist(message, actualPath); + } + + /** + * Verify that the specified dir has the {@code _SUCCESS} marker + * and that it can be loaded. + * The contents will be logged and returned. + * @param dir directory to scan + * @param jobId job ID, only verified if non-empty + * @return the loaded success data + * @throws IOException IO Failure + */ + protected ManifestSuccessData verifySuccessMarker(Path dir, String jobId) + throws IOException { + return validateSuccessFile(getFileSystem(), dir, 0, jobId); + } + + /** + * Read a UTF-8 file. + * @param path path to read + * @return string value + * @throws IOException IO failure + */ + protected String readFile(Path path) throws IOException { + return ContractTestUtils.readUTF8(getFileSystem(), path, -1); + } + + /** + * Modify a (job) config to switch to the manifest committer; + * output validation is also enabled. + * @param conf config to patch. + * @return the updated configuration. + */ + protected Configuration enableManifestCommitter(final Configuration conf) { + conf.set(COMMITTER_FACTORY_CLASS, MANIFEST_COMMITTER_FACTORY); + // always create a job marker + conf.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true); + // and validate the output, for extra rigorousness + conf.setBoolean(OPT_VALIDATE_OUTPUT, true); + + // set the manifest rename dir if non-null + if (getManifestDir() != null) { + conf.set(OPT_DIAGNOSTICS_MANIFEST_DIR, + getManifestDir().toUri().toString()); + } + + // and bind the report dir + conf.set(OPT_SUMMARY_REPORT_DIR, getReportDirUri().toString()); + return conf; + } + + /** + * Create the stage config for a job but don't finalize it. + * Uses {@link #TASK_IDS} for job/task ID. + * @param jobAttemptNumber job attempt number + * @param outputPath path where the final output goes + * @return the config + */ + protected StageConfig createStageConfigForJob( + final int jobAttemptNumber, + final Path outputPath) { + return createStageConfig(jobAttemptNumber, -1, 0, outputPath); + } + + /** + * Create the stage config for job or task but don't finalize it. + * Uses {@link #TASK_IDS} for job/task ID. + * @param jobAttemptNumber job attempt number + * @param taskIndex task attempt index; -1 for job attempt only. + * @param taskAttemptNumber task attempt number + * @param outputPath path where the final output goes + * @return the config + */ + protected StageConfig createStageConfig( + final int jobAttemptNumber, + final int taskIndex, + final int taskAttemptNumber, + final Path outputPath) { + final String jobId = TASK_IDS.getJobId(); + ManifestCommitterSupport.AttemptDirectories attemptDirs = + new ManifestCommitterSupport.AttemptDirectories(outputPath, + jobId, jobAttemptNumber); + StageConfig config = new StageConfig(); + config + .withIOProcessors(getSubmitter()) + .withIOStatistics(getStageStatistics()) + .withJobId(jobId) + .withJobIdSource(JOB_ID_SOURCE_MAPREDUCE) + .withJobAttemptNumber(jobAttemptNumber) + .withJobDirectories(attemptDirs) + .withName(String.format(NAME_FORMAT_JOB_ATTEMPT, jobId)) + .withOperations(getStoreOperations()) + .withProgressable(getProgressCounter()); + + // if there's a task attempt ID set, set up its details + if (taskIndex >= 0) { + String taskAttempt = TASK_IDS.getTaskAttempt(taskIndex, + taskAttemptNumber); + config + .withTaskAttemptId(taskAttempt) + .withTaskId(TASK_IDS.getTaskIdType(taskIndex).toString()) + .withTaskAttemptDir( + attemptDirs.getTaskAttemptPath(taskAttempt)); + } + return config; + } + + /** + * A job stage config. + * @return stage config or null. + */ + protected StageConfig getJobStageConfig() { + return jobStageConfig; + } + + protected void setJobStageConfig(StageConfig jobStageConfig) { + this.jobStageConfig = jobStageConfig; + } + + protected Path getDestDir() { + return destDir; + } + + protected void setDestDir(Path destDir) { + this.destDir = destDir; + } + + /** + * Execute a set of tasks; task ID is a simple count. + * task attempt is lowest 2 bits of task ID. + * @param taskAttemptCount number of tasks. + * @param filesPerTaskAttempt number of files to include in manifest. + * @return the manifests. + * @throws IOException IO failure. + */ + protected List executeTaskAttempts(int taskAttemptCount, + int filesPerTaskAttempt) throws IOException { + + try (DurationInfo di = new DurationInfo(LOG, true, "create manifests")) { + + // build a list of the task IDs. + // it's really hard to create a list of Integers; the java8 + // IntStream etc doesn't quite fit as they do their best + // keep things unboxed, trying to map(Integer::valueOf) doesn't help. + List taskIdList = new ArrayList<>(taskAttemptCount); + for (int t = 0; t < taskAttemptCount; t++) { + taskIdList.add(t); + } + + /// execute the tasks + List manifests = Collections.synchronizedList( + new ArrayList<>()); + + // then submit their creation/save to the pool. + TaskPool.foreach(taskIdList) + .executeWith(getSubmitter()) + .stopOnFailure() + .run(i -> { + manifests.add( + executeOneTaskAttempt(i, i & 0x03, filesPerTaskAttempt)); + }); + return manifests; + + } + } + + /** + * Create at task ID and attempt (adding to taskIDs and taskAttemptIds) + * setup the task, create a manifest with fake task entries + * and save that manifest to the job attempt dir. + * No actual files are created. + * @param task task index + * @param taskAttempt task attempt value + * @param filesPerTaskAttempt number of files to include in manifest. + * @return the manifest + * @throws IOException failure + */ + protected TaskManifest executeOneTaskAttempt(final int task, + int taskAttempt, final int filesPerTaskAttempt) throws IOException { + + String tid = String.format("task_%03d", task); + String taskAttemptId = String.format("%s_%02d", + tid, taskAttempt); + synchronized (this) { + taskIds.add(tid); + taskAttemptIds.add(taskAttemptId); + } + // for each task, a job config is created then patched with the task info + StageConfig taskStageConfig = createTaskStageConfig(JOB1, tid, taskAttemptId); + + LOG.info("Generating manifest for {}", taskAttemptId); + + // task setup: create dest dir. + // This helps generate a realistic + // workload for the parallelized job cleanup. + new SetupTaskStage(taskStageConfig).apply("task " + taskAttemptId); + + final TaskManifest manifest = createTaskManifest(taskStageConfig); + + Path taDir = taskStageConfig.getTaskAttemptDir(); + long size = task * 1000_0000L; + + // for each task, 10 dirs, one file per dir. + for (int i = 0; i < filesPerTaskAttempt; i++) { + Path in = new Path(taDir, "dir-" + i); + Path out = new Path(getDestDir(), "dir-" + i); + manifest.addDirectory(DirEntry.dirEntry(out, 0, 1)); + String name = taskStageConfig.getTaskAttemptId() + ".csv"; + Path src = new Path(in, name); + Path dest = new Path(out, name); + long fileSize = size + i * 1000L; + manifest.addFileToCommit( + new FileEntry(src, dest, fileSize, Long.toString(fileSize, 16))); + totalDataSize.addAndGet(fileSize); + } + + // save the manifest for this stage. + new SaveTaskManifestStage(taskStageConfig).apply(manifest); + return manifest; + } + + public StageConfig createTaskStageConfig(final int jobId, final String tid, + final String taskAttemptId) { + Path jobAttemptTaskSubDir = getJobStageConfig().getJobAttemptTaskSubDir(); + StageConfig taskStageConfig = createStageConfigForJob(jobId, getDestDir()) + .withTaskId(tid) + .withTaskAttemptId(taskAttemptId) + .withTaskAttemptDir(new Path(jobAttemptTaskSubDir, taskAttemptId)); + return taskStageConfig; + } + + /** + * Verify that the job directories have been cleaned up. + * @throws IOException IO failure + */ + protected void verifyJobDirsCleanedUp() throws IOException { + StageConfig stageConfig = getJobStageConfig(); + assertPathDoesNotExist("Job attempt dir", stageConfig.getJobAttemptDir()); + assertPathDoesNotExist("dest temp dir", stageConfig.getOutputTempSubDir()); + } + + /** + * List a directory/directory tree and print files. + * @param fileSystem FS + * @param path path + * @param recursive do a recursive listing? + * @return the number of files found. + * @throws IOException failure. + */ + public static long lsR(FileSystem fileSystem, Path path, boolean recursive) + throws Exception { + if (path == null) { + // surfaces when someone calls getParent() on something at the top + // of the path + LOG.info("Empty path"); + return 0; + } else { + LOG.info("Listing of {}", path); + final long count = RemoteIterators.foreach( + fileSystem.listFiles(path, recursive), + (status) -> LOG.info("{}", status)); + LOG.info("Count of entries: {}", count); + return count; + } + } + + /** + * Assert that a cleanup stage coursehad a given outcome and + * deleted the given number of directories. + * @param result result to analyze + * @param outcome expected outcome + * @param expectedDirsDeleted #of directories deleted. -1 for no checks + */ + protected void assertCleanupResult( + CleanupJobStage.Result result, + CleanupJobStage.Outcome outcome, + int expectedDirsDeleted) { + Assertions.assertThat(result.getOutcome()) + .describedAs("Outcome of cleanup() in %s", result) + .isEqualTo(outcome); + if (expectedDirsDeleted >= 0) { + Assertions.assertThat(result.getDeleteCalls()) + .describedAs("Number of directories deleted in cleanup %s", result) + .isEqualTo(expectedDirsDeleted); + } + } + + /** + * Create and execute a cleanup stage. + * @param enabled is the stage enabled? + * @param deleteTaskAttemptDirsInParallel delete task attempt dirs in + * parallel? + * @param suppressExceptions suppress exceptions? + * @param outcome expected outcome. + * @param expectedDirsDeleted #of directories deleted. -1 for no checks + * @return the result + * @throws IOException non-suppressed exception + */ + protected CleanupJobStage.Result cleanup( + final boolean enabled, + final boolean deleteTaskAttemptDirsInParallel, + final boolean suppressExceptions, + final CleanupJobStage.Outcome outcome, + final int expectedDirsDeleted) throws IOException { + StageConfig stageConfig = getJobStageConfig(); + CleanupJobStage.Result result = new CleanupJobStage(stageConfig) + .apply(new CleanupJobStage.Arguments(OP_STAGE_JOB_CLEANUP, + enabled, deleteTaskAttemptDirsInParallel, suppressExceptions)); + assertCleanupResult(result, outcome, expectedDirsDeleted); + return result; + } + + /** + * Read the UTF_8 text in the file. + * @param path path to read + * @return the string + * @throws IOException failure + */ + protected String readText(final Path path) throws IOException { + + final FileSystem fs = getFileSystem(); + final FileStatus st = fs.getFileStatus(path); + Assertions.assertThat(st.getLen()) + .describedAs("length of file %s", st) + .isLessThanOrEqualTo(MAX_LEN); + + return new String( + readDataset(fs, path, (int) st.getLen()), + StandardCharsets.UTF_8); + } + + /** + * Counter. + */ + protected static final class ProgressCounter implements Progressable { + + private final AtomicLong counter = new AtomicLong(); + + /** + * Increment the counter. + */ + @Override + public void progress() { + counter.incrementAndGet(); + } + + /** + * Get the counter value. + * @return the current value. + */ + public long value() { + return counter.get(); + } + + /** + * Reset the counter. + */ + public void reset() { + counter.set(0); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ProgressCounter{"); + sb.append("counter=").append(counter.get()); + sb.append('}'); + return sb.toString(); + } + } + + /** + * Get the progress counter of a stage. + * @param stageConfig stage + * @return its progress counter. + */ + ProgressCounter progressOf(StageConfig stageConfig) { + return (ProgressCounter) stageConfig.getProgressable(); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/CommitterTestBinding.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/CommitterTestBinding.java new file mode 100644 index 0000000000000..c6586ac17599c --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/CommitterTestBinding.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.hadoop.mapreduce.lib.output.committer.manifest; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskID; +import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitter.TASK_COMMITTER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.randomJobId; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createIOStatisticsStore; + +/** + * This class represents a binding to a job in the target dir with TA, JA + * and associated paths. + * It's self contained so as to be usable in any test suite. + */ +class CommitterTestBinding implements + IOStatisticsSource { + + /** + * IOStatistics counter for progress events. + */ + public static final String PROGRESS_EVENTS = "progress_events"; + + /** + * IOStatistics to update with progress events. + */ + private final IOStatisticsStore iostatistics; + + /** + * Job attempt ID:. + */ + private final String jobAttemptId; + + /** + * Job ID. + */ + private final JobID jobId; + + /** + * Task Attempt ID, under the job attempt. + */ + private final TaskAttemptID taskAttemptId; + + /** + * Task ID. + */ + private final TaskID taskId; + + /** + * Task attempt context for the given task Attempt. + */ + private final TaskAttemptContext taskAttemptContext; + + /** + * Construct. + * @param conf job/task config. This is patched with the app attempt. + * @param appAttempt application attempt. + * @param taskNumber task number + * @param taskAttemptNumber which attempt on this task is it + */ + CommitterTestBinding( + Configuration conf, + int appAttempt, int taskNumber, int taskAttemptNumber) { + iostatistics = createIOStatisticsStore() + .withCounters(PROGRESS_EVENTS) + .build(); + + + // this is the job ID, with no attempt info. + jobId = JobID.forName(randomJobId()); + jobAttemptId = jobId.toString() + "_ " + appAttempt; + taskId = new TaskID(jobId, TaskType.MAP, taskNumber); + taskAttemptId = new TaskAttemptID(taskId, + taskAttemptNumber); + conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, appAttempt); + taskAttemptContext = new TaskAttemptContextImpl(conf, taskAttemptId); + + } + + /** + * Create a committer config for the given output path. + * @param outputPath output path in destFS. + * @return a committer for the active task. + */ + ManifestCommitterConfig createCommitterConfig( + Path outputPath) { + return new ManifestCommitterConfig(outputPath, + TASK_COMMITTER, + taskAttemptContext, + iostatistics, + null); + } + + /** + * Create a stage config from the committer config. + * All stats go to the local IOStatisticsStore; + * there's a progress callback also set to increment + * the counter {@link #PROGRESS_EVENTS} + * @return a stage config + */ + StageConfig createStageConfig(Path outputPath) { + return createCommitterConfig(outputPath) + .createStageConfig() + .withProgressable(new ProgressCallback()); + } + + @Override + public IOStatisticsStore getIOStatistics() { + return iostatistics; + } + + /** + * Whenever this progress callback is invoked, the progress_events + * counter is incremented. This allows for tests to verify that + * callbacks have occurred by asserting on the event counter. + */ + private final class ProgressCallback implements Progressable { + + @Override + public void progress() { + iostatistics.incrementCounter(PROGRESS_EVENTS, 1); + } + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java new file mode 100644 index 0000000000000..7297fdd174ec6 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterTestSupport.java @@ -0,0 +1,420 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import org.assertj.core.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskID; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.util.functional.RemoteIterators; + +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Support for committer tests. + */ +public final class ManifestCommitterTestSupport { + + private static final Logger LOG = LoggerFactory.getLogger( + ManifestCommitterTestSupport.class); + + private static final DateTimeFormatter FORMATTER = + DateTimeFormatter.ofPattern("yyyyMMddHHmmss"); + + /** + * Build directory property. + * Value: {@value}. + */ + public static final String PROJECT_BUILD_DIRECTORY_PROPERTY + = "project.build.directory"; + + /** + * default number of task attempts for some tests. + * Value: {@value}. + */ + public static final int NUMBER_OF_TASK_ATTEMPTS = 200; + + private ManifestCommitterTestSupport() { + } + + /** + * Create a random Job ID using the fork ID as part of the number if + * set in the current process. + * @return fork ID string in a format parseable by Jobs + */ + public static String randomJobId() { + String testUniqueForkId = System.getProperty("test.unique.fork.id", "0001"); + int l = testUniqueForkId.length(); + String trailingDigits = testUniqueForkId.substring(l - 4, l); + int digitValue; + try { + digitValue = Integer.valueOf(trailingDigits); + } catch (NumberFormatException e) { + digitValue = 0; + } + + return String.format("%s%04d_%04d", + FORMATTER.format(LocalDateTime.now()), + (long) (Math.random() * 1000), + digitValue); + } + + public static File getProjectBuildDir() { + String propval = System.getProperty(PROJECT_BUILD_DIRECTORY_PROPERTY); + if (StringUtils.isEmpty(propval)) { + propval = "target"; + } + return new File(propval).getAbsoluteFile(); + } + + /** + * Load a success file; fail if the file is empty/nonexistent. + * @param fs filesystem + * @param outputPath directory containing the success file. + * @return the loaded file. + * @throws IOException failure to find/load the file + * @throws AssertionError file is 0-bytes long, + */ + public static ManifestSuccessData loadSuccessFile(final FileSystem fs, + final Path outputPath) throws IOException { + Path success = new Path(outputPath, SUCCESS_MARKER); + return ManifestSuccessData.load(fs, success); + } + + /** + * Load in the success data marker. + * @param fs filesystem + * @param outputDir ouptu path of job + * @param minimumFileCount minimum number of files to have been created + * @param jobId job ID, only verified if non-empty + * @return the success data + * @throws IOException IO failure + */ + public static ManifestSuccessData validateSuccessFile( + final FileSystem fs, + final Path outputDir, + final int minimumFileCount, + final String jobId) throws IOException { + Path successPath = new Path(outputDir, SUCCESS_MARKER); + ManifestSuccessData successData + = loadAndPrintSuccessData(fs, successPath); + assertThat(successData.getCommitter()) + .describedAs("Committer field") + .isEqualTo(MANIFEST_COMMITTER_CLASSNAME); + assertThat(successData.getFilenames()) + .describedAs("Files committed") + .hasSizeGreaterThanOrEqualTo(minimumFileCount); + if (isNotEmpty(jobId)) { + assertThat(successData.getJobId()) + .describedAs("JobID") + .isEqualTo(jobId); + } + return successData; + } + + /** + * Load in and print a success data manifest. + * @param fs filesystem + * @param successPath full path to success file. + * @return the success data + * @throws IOException IO failure + */ + public static ManifestSuccessData loadAndPrintSuccessData( + FileSystem fs, + Path successPath) throws IOException { + LOG.info("Manifest {}", successPath); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintStream ps = new PrintStream(baos); + final ManifestPrinter showManifest = new ManifestPrinter(fs.getConf(), ps); + ManifestSuccessData successData = showManifest.loadAndPrintManifest(fs, successPath); + LOG.info("{}", baos); + return successData; + } + + /** + * Validate all generated files from the manifest. + * All files in the manifest must exist. + * If the exclusive flag is set, only those must exist + * (ignoring all temp files and everything in the _temporary + * dir) + * @param fs filesystem + * @param destDir dest dir to scan + * @param successData manifest + * @param exclusive expect exclusive and complete data. + * @return the files and their status + * @throws IOException IO failure. + */ + public static Map validateGeneratedFiles( + FileSystem fs, + Path destDir, + ManifestSuccessData successData, + boolean exclusive) throws IOException { + Map map = new HashMap<>(); + RemoteIterators.foreach(fs.listFiles(destDir, true), + e -> { + if (!e.getPath().getName().startsWith("_")) { + map.put(e.getPath(), e); + } + }); + // map has all files other than temp ones and the success marker + // what do we expect + final List expected = filesInManifest(successData); + + // all of those must be found + Assertions.assertThat(map.keySet()) + .describedAs("Files in FS compared to manifest") + .containsAll(expected); + + // and if exclusive, that too + if (exclusive) { + Assertions.assertThat(map.keySet()) + .describedAs("Files in FS compared to manifest") + .containsExactlyInAnyOrderElementsOf(expected); + } + return map; + } + + /** + * Given a manifest, get the list of filenames + * and convert to paths. + * @param successData data + * @return the paths. + */ + public static List filesInManifest(ManifestSuccessData successData) { + return successData.getFilenames().stream() + .map(AbstractManifestData::unmarshallPath) + .collect(Collectors.toList()); + } + + /** + * List a directory/directory tree. + * @param fileSystem FS + * @param path path + * @param recursive do a recursive listing? + * @return the number of files found. + * @throws IOException failure. + */ + public static long lsR(FileSystem fileSystem, Path path, boolean recursive) + throws Exception { + if (path == null) { + // surfaces when someone calls getParent() on something at the top + // of the path + LOG.info("Empty path"); + return 0; + } + return RemoteIterators.foreach(fileSystem.listFiles(path, recursive), + (status) -> LOG.info("{}", status)); + } + + /** + * Assert that a file or dir entry matches the given parameters. + * Matching on paths, not strings, helps validate marshalling. + * @param fileOrDir file or directory + * @param src source path + * @param dest dest path + * @param l length + */ + static void assertFileEntryMatch( + final FileEntry fileOrDir, + final Path src, + final Path dest, + final long l) { + String entry = fileOrDir.toString(); + assertThat(fileOrDir.getSourcePath()) + .describedAs("Source path of " + entry) + .isEqualTo(src); + assertThat(fileOrDir.getDestPath()) + .describedAs("Dest path of " + entry) + .isEqualTo(dest); + assertThat(fileOrDir.getSize()) + .describedAs("Size of " + entry) + .isEqualTo(l); + } + + /** + * Assert that a dir entry matches the given parameters. + * Matching on paths, not strings, helps validate marshalling. + * @param fileOrDir file or directory + * @param dest dest path + * @param type type + */ + static void assertDirEntryMatch( + final DirEntry fileOrDir, + final Path dest, + final long type) { + String entry = fileOrDir.toString(); + assertThat(fileOrDir.getDestPath()) + .describedAs("Dest path of " + entry) + .isEqualTo(dest); + assertThat(fileOrDir.getType()) + .describedAs("type of " + entry) + .isEqualTo(type); + } + + /** + * Closeable which can be used to safely close writers in + * a try-with-resources block.. + */ + public static final class CloseWriter implements AutoCloseable { + + private final RecordWriter writer; + + private final TaskAttemptContext context; + + public CloseWriter(RecordWriter writer, + TaskAttemptContext context) { + this.writer = writer; + this.context = context; + } + + @Override + public void close() { + try { + writer.close(context); + } catch (IOException | InterruptedException e) { + LOG.error("When closing {} on context {}", + writer, context, e); + } + } + } + + public static final String ATTEMPT_STRING = + "attempt_%s_m_%06d_%d"; + + /** + * Creates a random JobID and then as many tasks + * with the specific number of task attempts. + */ + public static final class JobAndTaskIDsForTests { + + /** Job ID; will be created uniquely for each instance. */ + private final String jobId; + + /** + * Store the details as strings; generate + * IDs on demand. + */ + private final String[][] taskAttempts; + + /** + * Constructor. + * @param tasks number of tasks. + * @param attempts number of attempts. + */ + public JobAndTaskIDsForTests(int tasks, int attempts) { + this(randomJobId(), tasks, attempts); + } + + public JobAndTaskIDsForTests(final String jobId, + int tasks, int attempts) { + this.jobId = jobId; + this.taskAttempts = new String[tasks][attempts]; + for (int i = 0; i < tasks; i++) { + for (int j = 0; j < attempts; j++) { + String a = String.format(ATTEMPT_STRING, + jobId, i, j); + this.taskAttempts[i][j] = a; + } + } + } + + /** + * Get the job ID. + * @return job ID string. + */ + public String getJobId() { + return jobId; + } + + /** + * Get the job ID as the MR type. + * @return job ID type. + */ + public JobID getJobIdType() { + return getTaskIdType(0).getJobID(); + } + + /** + * Get a task attempt ID. + * @param task task index + * @param attempt attempt number. + * @return the task attempt. + */ + public String getTaskAttempt(int task, int attempt) { + return taskAttempts[task][attempt]; + } + + /** + * Get task attempt ID as the MR type. + * @param task task index + * @param attempt attempt number. + * @return the task attempt type + */ + public TaskAttemptID getTaskAttemptIdType(int task, int attempt) { + return TaskAttemptID.forName(getTaskAttempt(task, attempt)); + } + + /** + * Get task ID as the MR type. + * @param task task index + * @return the task ID type + */ + public TaskID getTaskIdType(int task) { + return TaskAttemptID.forName(getTaskAttempt(task, 0)).getTaskID(); + } + + /** + * Get task ID as a string. + * @param task task index + * @return the task ID + */ + public String getTaskId(int task) { + return getTaskIdType(task).toString(); + } + + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/StubStoreOperations.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/StubStoreOperations.java new file mode 100644 index 0000000000000..d9269767f0761 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/StubStoreOperations.java @@ -0,0 +1,109 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.IOException; +import java.util.NoSuchElementException; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.util.JsonSerialization; + +/** + * Stub Store operations. + * Everything "works" provided you don't look too close. + * Files have etags of their filename; if you move a file without changing its + * name, the etag is preserved. + */ +public class StubStoreOperations extends ManifestStoreOperations { + + @Override + public FileStatus getFileStatus(final Path path) throws IOException { + return new TaggedFileStatus(0, false, 1, 1024, 0, path, path.getName()); + } + + @Override + public boolean delete(final Path path, final boolean recursive) + throws IOException { + return true; + } + + @Override + public boolean mkdirs(final Path path) throws IOException { + return true; + } + + @Override + public boolean renameFile(final Path source, final Path dest) + throws IOException { + return true; + } + + @Override + public RemoteIterator listStatusIterator(final Path path) + throws IOException { + return new EmptyRemoteIterator<>(); + } + + @Override + public TaskManifest loadTaskManifest(JsonSerialization serializer, + final FileStatus st) throws IOException { + return new TaskManifest(); + } + + @Override + public > void save(T manifestData, + final Path path, + final boolean overwrite) throws IOException { + + } + + @Override + public void close() throws IOException { + + } + + @Override + public boolean storePreservesEtagsThroughRenames(final Path path) { + return true; + } + + /** + * Always empty rempte iterator. + * @param type of iterator. + */ + private static final class EmptyRemoteIterator + implements RemoteIterator { + + @Override + public boolean hasNext() throws IOException { + return false; + } + + @Override + public T next() throws IOException { + throw new NoSuchElementException(); + } + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TaggedFileStatus.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TaggedFileStatus.java new file mode 100644 index 0000000000000..4a1115b667c1e --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TaggedFileStatus.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.IOException; + +import org.apache.hadoop.fs.EtagSource; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +/** + * File Status with etag derived from the filename, if not explicitly set. + */ +public final class TaggedFileStatus extends FileStatus implements EtagSource { + + private final String etag; + + public TaggedFileStatus(final long length, + final boolean isdir, + final int blockReplication, + final long blocksize, + final long modificationTime, + final Path path, + final String etag) { + super(length, isdir, blockReplication, blocksize, modificationTime, path); + this.etag = etag; + } + + public TaggedFileStatus(final FileStatus other, final String etag) throws IOException { + super(other); + this.etag = etag; + } + + @Override + public String getEtag() { + return etag; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCleanupStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCleanupStage.java new file mode 100644 index 0000000000000..8d551c505209c --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCleanupStage.java @@ -0,0 +1,142 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.util.List; + +import org.junit.Test; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.UnreliableManifestStoreOperations; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; + + +/** + * Test the cleanup stage. + */ +public class TestCleanupStage extends AbstractManifestCommitterTest { + + /** + * Number of task attempts to create. Manifests are created and written + * as well as test dirs, but no actual files. + */ + protected static final int TASK_ATTEMPT_COUNT = 10; + + /** + * How many delete calls for the root job delete? + */ + protected static final int ROOT_DELETE_COUNT = 1; + + /** + * Tocal invocation count for a successful parallel delete job. + */ + protected static final int PARALLEL_DELETE_COUNT = + TASK_ATTEMPT_COUNT + ROOT_DELETE_COUNT; + + /** + * Fault Injection. + */ + private UnreliableManifestStoreOperations failures; + + /** + * Manifests created. + */ + private List manifests; + + @Override + public void setup() throws Exception { + super.setup(); + failures = new UnreliableManifestStoreOperations( + createManifestStoreOperations()); + setStoreOperations(failures); + Path destDir = methodPath(); + StageConfig stageConfig = createStageConfigForJob(JOB1, destDir); + setJobStageConfig(stageConfig); + new SetupJobStage(stageConfig).apply(true); + + // lots of tasks, but don't bother creating mock files. + manifests = executeTaskAttempts(TASK_ATTEMPT_COUNT, 0); + } + + @Test + public void testCleanupInParallelHealthy() throws Throwable { + describe("parallel cleanup of TA dirs."); + cleanup(true, true, false, + CleanupJobStage.Outcome.PARALLEL_DELETE, + PARALLEL_DELETE_COUNT); + verifyJobDirsCleanedUp(); + } + + @Test + public void testCleanupSingletonHealthy() throws Throwable { + describe("Cleanup with a single delete. Not the default; would be best on HDFS"); + + cleanup(true, false, false, + CleanupJobStage.Outcome.DELETED, ROOT_DELETE_COUNT); + verifyJobDirsCleanedUp(); + } + + @Test + public void testCleanupNoDir() throws Throwable { + describe("parallel cleanup MUST not fail if there's no dir"); + // first do the cleanup + cleanup(true, true, false, + CleanupJobStage.Outcome.PARALLEL_DELETE, PARALLEL_DELETE_COUNT); + + // now expect cleanup by single delete still works + // the delete count is 0 as pre check skips it + cleanup(true, false, false, + CleanupJobStage.Outcome.NOTHING_TO_CLEAN_UP, 0); + + // if skipped, that happens first + cleanup(false, true, false, + CleanupJobStage.Outcome.DISABLED, 0); + } + + @Test + public void testFailureInParallelDelete() throws Throwable { + describe("A parallel delete fails, but the base delete works"); + + // pick one of the manifests + TaskManifest manifest = manifests.get(4); + Path taPath = new Path(manifest.getTaskAttemptDir()); + failures.addDeletePathToFail(taPath); + cleanup(true, true, false, + CleanupJobStage.Outcome.DELETED, PARALLEL_DELETE_COUNT); + } + + /** + * If there's no job task attempt subdir then the list of it will raise + * and FNFE; this MUST be caught and the base delete executed. + */ + @Test + public void testParallelDeleteNoTaskAttemptDir() throws Throwable { + describe("Execute parallel delete where" + + " the job task directory does not exist"); + StageConfig stageConfig = getJobStageConfig(); + // TA dir doesn't exist, so listing will fail. + failures.addPathNotFound(stageConfig.getJobAttemptTaskSubDir()); + cleanup(true, true, false, + CleanupJobStage.Outcome.DELETED, ROOT_DELETE_COUNT); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCommitTaskStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCommitTaskStage.java new file mode 100644 index 0000000000000..4f4162d46cb9f --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCommitTaskStage.java @@ -0,0 +1,126 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.FileNotFoundException; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CLEANUP; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test committing a task. + */ +public class TestCommitTaskStage extends AbstractManifestCommitterTest { + + @Override + public void setup() throws Exception { + super.setup(); + + Path destDir = methodPath(); + StageConfig stageConfig = createStageConfigForJob(JOB1, destDir); + setJobStageConfig(stageConfig); + new SetupJobStage(stageConfig).apply(true); + } + + @Test + public void testCommitMissingDirectory() throws Throwable { + + String tid = String.format("task_%03d", 1); + String taskAttemptId = String.format("%s_%02d", + tid, 1); + StageConfig taskStageConfig = createTaskStageConfig(JOB1, tid, + taskAttemptId); + + // the task attempt dir does not exist + Path taDir = taskStageConfig.getTaskAttemptDir(); + assertPathDoesNotExist("task attempt path", taDir); + + // so the task commit fails + intercept(FileNotFoundException.class, () -> + new CommitTaskStage(taskStageConfig).apply(null)); + } + + @Test + public void testCommitEmptyDirectory() throws Throwable { + + describe("Commit an empty directory as task then job"); + String tid = String.format("task_%03d", 2); + String taskAttemptId = String.format("%s_%02d", + tid, 1); + StageConfig taskStageConfig = createTaskStageConfig(JOB1, tid, + taskAttemptId); + + // set up the task + new SetupTaskStage(taskStageConfig).apply("setup"); + + CommitTaskStage.Result result = new CommitTaskStage(taskStageConfig) + .apply(null); + + final TaskManifest manifest = result.getTaskManifest(); + Assertions.assertThat(manifest.getDestDirectories()) + .as("directories to create") + .isEmpty(); + Assertions.assertThat(manifest.getFilesToCommit()) + .as("files to commit") + .isEmpty(); + + final Path path = result.getPath(); + + final String manifestBody = readText(path); + + LOG.info("manifest at {} of length {}:\n{}", + path, manifestBody.length(), manifestBody); + + // now commit + final CommitJobStage.Result outcome = new CommitJobStage(getJobStageConfig()) + .apply(new CommitJobStage.Arguments( + true, true, null, + new CleanupJobStage.Arguments( + OP_STAGE_JOB_CLEANUP, + true, + true, + false + ))); + + // review success file + final Path successPath = outcome.getSuccessPath(); + String successBody = readText(successPath); + LOG.info("successBody at {} of length {}:\n{}", + successPath, successBody.length(), successBody); + + final ManifestSuccessData successData = outcome.getJobSuccessData(); + Assertions.assertThat(successData.getFilenames()) + .as("Filenames in _SUCCESS") + .isEmpty(); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java new file mode 100644 index 0000000000000..4b93ce937f80c --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestCreateOutputDirectoriesStage.java @@ -0,0 +1,307 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.EntryStatus; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CreateOutputDirectoriesStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; +import org.apache.hadoop.util.Lists; + +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_DELETE; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_GET_FILE_STATUS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_IS_FILE; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_MKDIRS; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_FAILURES; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_DELETE_FILE_UNDER_DESTINATION; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_PREPARE_DIR_ANCESTORS; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test directory creation. + * As the directory creation phase relies on input from the task manifest to + * determine which directories to explicitly create, delete files at + * etc, these tests build up manifests and assert that the output + * of the directory creation stage matches that of the combination + * of the manifest and the filesystem state. + */ +public class TestCreateOutputDirectoriesStage extends AbstractManifestCommitterTest { + + /** + * Deep tree width, subclasses (including in external projects) + * may change. + */ + protected static final int DEEP_TREE_WIDTH = 4; + + /** + * The number of directories created in test setup; this must be + * added to all assertions of the value of OP_MKDIRS. + */ + private static final int DIRECTORIES_CREATED_IN_SETUP = 2; + + private Path destDir; + private CreateOutputDirectoriesStage mkdirStage; + private StageConfig stageConfig; + private IOStatisticsStore iostats; + + @Override + public void setup() throws Exception { + super.setup(); + destDir = methodPath(); + // clean up dest dir completely + destDir.getFileSystem(getConfiguration()).delete(destDir, true); + setStoreOperations(createManifestStoreOperations()); + stageConfig = createStageConfigForJob(JOB1, destDir) + .withDeleteTargetPaths(true); + setJobStageConfig(stageConfig); + // creates the job directories. + new SetupJobStage(stageConfig).apply(true); + mkdirStage = new CreateOutputDirectoriesStage(stageConfig); + iostats = stageConfig.getIOStatistics(); + // assert original count of dirs created == 2 : job and task manifest + verifyStatisticCounterValue(iostats, OP_MKDIRS, + DIRECTORIES_CREATED_IN_SETUP); + // reset the value to simplify future work + iostats.getCounterReference(OP_MKDIRS).set(0); + } + + @Test + public void testPrepareSomeDirs() throws Throwable { + + final long initialFileStatusCount = lookupCounterStatistic(iostats, OP_GET_FILE_STATUS); + final int dirCount = 8; + final List dirs = subpaths(destDir, dirCount); + final List dirEntries = dirEntries(dirs, 1, EntryStatus.not_found); + + // two manifests with duplicate entries + final List manifests = Lists.newArrayList( + manifestWithDirsToCreate(dirEntries), + manifestWithDirsToCreate(dirEntries)); + final CreateOutputDirectoriesStage.Result result = mkdirStage.apply(manifests); + Assertions.assertThat(result.getCreatedDirectories()) + .describedAs("output of %s", mkdirStage) + .containsExactlyInAnyOrderElementsOf(dirs); + + LOG.info("Job Statistics\n{}", ioStatisticsToPrettyString(iostats)); + + // now dirCount new dirs are added. + verifyStatisticCounterValue(iostats, OP_MKDIRS, dirCount); + + // now rerun the same preparation sequence, but this + // time declare that the directories exist (as they do) + final CreateOutputDirectoriesStage s2 = + new CreateOutputDirectoriesStage(stageConfig); + final CreateOutputDirectoriesStage.Result r2 = s2.apply( + Lists.newArrayList( + manifestWithDirsToCreate(dirEntries(dirs, 1, EntryStatus.dir)))); + + // no directories are now created. + Assertions.assertThat(r2.getCreatedDirectories()) + .describedAs("output of %s", s2) + .isEmpty(); + LOG.info("Job Statistics after second pass\n{}", ioStatisticsToPrettyString(iostats)); + + // second run probed no dest dirs + verifyStatisticCounterValue(iostats, OP_GET_FILE_STATUS, initialFileStatusCount); + // and no new mkdir calls were made + verifyStatisticCounterValue(iostats, OP_MKDIRS, dirCount); + verifyStatisticCounterValue(iostats, OP_DELETE_FILE_UNDER_DESTINATION, 0); + verifyStatisticCounterValue(iostats, OP_IS_FILE, 0); + } + + /** + * Given a list of paths, build a list of DirEntry entries. + * @param paths list of paths + * @param level Level in the treewalk. + * @param entryStatus status of dirs + * @return list of entries with the given level and entry status. + */ + protected List dirEntries(Collection paths, + int level, + EntryStatus entryStatus) { + return paths.stream() + .map(p -> DirEntry.dirEntry(p, entryStatus, level)) + .collect(Collectors.toList()); + } + + /** + * Create a manifest with the list of directory entries added. + * Job commit requires the entries to have been probed for, and + * for the entire tree under the dest path to be included. + * @param dirEntries list of directory entries. + * @return the manifest. + */ + protected TaskManifest manifestWithDirsToCreate(List dirEntries) { + final TaskManifest taskManifest = new TaskManifest(); + taskManifest.getDestDirectories().addAll(dirEntries); + return taskManifest; + } + + /** + * Assert the directory map status of a path. + * @param result stage result + * @param path path to look up + * @param expected expected value. + */ + private static void assertDirMapStatus( + CreateOutputDirectoriesStage.Result result, + Path path, + CreateOutputDirectoriesStage.DirMapState expected) { + Assertions.assertThat(result.getDirMap()) + .describedAs("Directory Map entry for %s", path) + .isNotNull() + .containsKey(path) + .containsEntry(path, expected); + } + + /** + * Prepare a deep tree {@code c ^ 3} of entries. + * Make one of the parent dirs a file. + * + * From a test-purity perspective, this should + * be separate tests. But attempting + * operations in the same test cases spreads the + * directory setup costs across both, rather than + * duplicating it. + */ + @Test + public void testPrepareDirtyTree() throws Throwable { + + // build the lists of paths for the different levels + final int c = getDeepTreeWidth(); + final List level1 = subpaths(destDir, c); + final List level2 = level1.stream().flatMap(p -> + subpaths(p, c).stream()) + .collect(Collectors.toList()); + final List level3 = level2.stream().flatMap(p -> + subpaths(p, c).stream()) + .collect(Collectors.toList()); + // manifest dir entry list contains all levels > 0 + // adding them out of order verifies sorting takes place + // before the merge routine which is intended to strip + // out parent dirs + final List directories = new ArrayList<>(); + final List l1 = dirEntries(level1, 1, EntryStatus.not_found); + directories.addAll(l1); + final List l3 = dirEntries(level3, 3, EntryStatus.not_found); + directories.addAll(l3); + final List l2 = dirEntries(level2, 2, EntryStatus.not_found); + directories.addAll(l2); + + // one of the level 0 paths is going to be a file + final DirEntry parentIsFile = l1.get(1); + // one entry has a dir already + final DirEntry parentIsDir = l2.get(0); + // and one of the dest dirs is a file. + final DirEntry leafIsFile = l3.get(0); + + // prepare the output + CompletableFuture.allOf( + asyncPut(parentIsFile.getDestPath(), NO_DATA), + asyncPut(leafIsFile.getDestPath(), NO_DATA), + asyncMkdir(parentIsDir.getDestPath())) + .join(); + + // patch the entries, which, as they are references + // into the lists, updates the values there. + parentIsFile.setStatus(EntryStatus.file); + parentIsDir.setStatus(EntryStatus.dir); + leafIsFile.setStatus(EntryStatus.file); + + final List manifests = Lists.newArrayList( + manifestWithDirsToCreate(directories)); + + // first attempt will succeed. + final CreateOutputDirectoriesStage.Result result = + mkdirStage.apply(manifests); + + LOG.info("Job Statistics\n{}", ioStatisticsToPrettyString(iostats)); + + assertDirMapStatus(result, leafIsFile.getDestPath(), + CreateOutputDirectoriesStage.DirMapState.fileNowDeleted); + + // for the parent dir, all is good + assertDirMapStatus(result, parentIsFile.getDestPath(), + CreateOutputDirectoriesStage.DirMapState.fileNowDeleted); + Assertions.assertThat(result.getCreatedDirectories()) + .describedAs("output of %s", mkdirStage) + .containsExactlyInAnyOrderElementsOf(level3); + verifyStatisticCounterValue(iostats, OP_MKDIRS, level3.size()); + // do a rerun. where the directory setup will fail because + // a directory is present where the manifest says there is + // a file. + CreateOutputDirectoriesStage attempt2 = + new CreateOutputDirectoriesStage( + createStageConfigForJob(JOB1, destDir) + .withDeleteTargetPaths(true)); + // attempt will fail because one of the entries marked as + // a file to delete is now a non-empty directory + LOG.info("Executing failing attempt to create the directories"); + intercept(IOException.class, () -> attempt2.apply(manifests)); + verifyStatisticCounterValue(iostats, OP_PREPARE_DIR_ANCESTORS + SUFFIX_FAILURES, 1); + verifyStatisticCounterValue(iostats, OP_DELETE + SUFFIX_FAILURES, 1); + + // build a new directory list where everything is declared a directory; + // no dirs will be created this time as they all exist. + final List directories3 = new ArrayList<>(); + directories3.addAll(dirEntries(level1, 1, EntryStatus.dir)); + directories3.addAll(dirEntries(level2, 2, EntryStatus.dir)); + directories3.addAll(dirEntries(level3, 3, EntryStatus.dir)); + + final List manifests3 = Lists.newArrayList( + manifestWithDirsToCreate(directories3)); + CreateOutputDirectoriesStage attempt3 = + new CreateOutputDirectoriesStage( + createStageConfigForJob(JOB1, destDir) + .withDeleteTargetPaths(true)); + final CreateOutputDirectoriesStage.Result r3 = + attempt3.apply(manifests3); + assertDirMapStatus(r3, leafIsFile.getDestPath(), + CreateOutputDirectoriesStage.DirMapState.dirFoundInStore); + Assertions.assertThat(r3.getCreatedDirectories()) + .describedAs("created directories") + .isEmpty(); + } + + /** + * Get the width of the deep tree; subclasses may tune for test performance, though + * a wide one is more realistic of real jobs. + * @return number of subdirs to create at each level. Must be at least 2 + */ + protected int getDeepTreeWidth() { + return DEEP_TREE_WIDTH; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java new file mode 100644 index 0000000000000..fa3527f11cc2b --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestJobThroughManifestCommitter.java @@ -0,0 +1,601 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import org.assertj.core.api.Assertions; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; + +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.OutputValidationException; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbortTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CommitTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupTaskStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.ValidateRenamedFilesStage; +import org.apache.hadoop.net.NetUtils; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.rm; +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ID_SOURCE_MAPREDUCE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_CLEANUP; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.loadAndPrintSuccessData; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.validateGeneratedFiles; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.PRINCIPAL; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.manifestPathForTask; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage.DISABLED; +import static org.apache.hadoop.security.UserGroupInformation.getCurrentUser; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test IO through the stages. + * This mimics the workflow of a job with two tasks, + * the first task has two attempts where the second attempt + * is committed after the first attempt (simulating the + * failure-during-task-commit which the v2 algorithm cannot + * handle). + * + * The test is ordered and the output dir is not cleaned up + * after each test case. + * The last test case MUST perform the cleanup. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class TestJobThroughManifestCommitter + extends AbstractManifestCommitterTest { + + /** Destination directory. */ + private Path destDir; + + /** directory names for the tests. */ + private ManifestCommitterSupport.AttemptDirectories dirs; + + /** + * To ensure that the local FS has a shared root path, this is static. + */ + @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") + private static Path sharedTestRoot = null; + + /** + * Job ID. + */ + private String jobId; + + /** + * Task 0 attempt 0 ID. + */ + private String taskAttempt00; + + /** + * Task 0 attempt 1 ID. + */ + private String taskAttempt01; + + /** + * Task 1 attempt 0 ID. + */ + private String taskAttempt10; + + /** + * Task 1 attempt 1 ID. + */ + private String taskAttempt11; + + /** + * Stage config for TA00. + */ + private StageConfig ta00Config; + + /** + * Stage config for TA01. + */ + private StageConfig ta01Config; + + /** + * Stage config for TA10. + */ + private StageConfig ta10Config; + + /** + * Stage config for TA11. + */ + private StageConfig ta11Config; + + @Override + public void setup() throws Exception { + super.setup(); + taskAttempt00 = TASK_IDS.getTaskAttempt(TASK0, TA0); + taskAttempt01 = TASK_IDS.getTaskAttempt(TASK0, TA1); + taskAttempt10 = TASK_IDS.getTaskAttempt(TASK1, TA0); + taskAttempt11 = TASK_IDS.getTaskAttempt(TASK1, TA1); + setSharedPath(path("TestJobThroughManifestCommitter")); + // add a dir with a space in. + destDir = new Path(sharedTestRoot, "out put"); + jobId = TASK_IDS.getJobId(); + // then the specific path underneath that for the attempt. + dirs = new ManifestCommitterSupport.AttemptDirectories(destDir, + jobId, 1); + + // config for job attempt 1, task 00 + setJobStageConfig(createStageConfigForJob(JOB1, destDir).build()); + ta00Config = createStageConfig(JOB1, TASK0, TA0, destDir).build(); + ta01Config = createStageConfig(JOB1, TASK0, TA1, destDir).build(); + ta10Config = createStageConfig(JOB1, TASK1, TA0, destDir).build(); + ta11Config = createStageConfig(JOB1, TASK1, TA1, destDir).build(); + } + + /** + * Test dir deletion is removed from test case teardown so the + * subsequent tests see the output. + * @throws IOException failure + */ + @Override + protected void deleteTestDirInTeardown() throws IOException { + /* no-op */ + } + + /** + * Override point and something to turn on/off when exploring what manifests look like. + * Stores where storage is billed MUST enable this. + * @return true if, at the end of the run, the test dir should be deleted. + */ + protected boolean shouldDeleteTestRootAtEndOfTestRun() { + return false; + } + + /** + * Invoke this to clean up the test directories. + */ + private void deleteSharedTestRoot() throws IOException { + describe("Deleting shared test root %s", sharedTestRoot); + + rm(getFileSystem(), sharedTestRoot, true, false); + } + + /** + * Set the shared test root if not already set. + * @param path path to set. + * @return true if the path was set + */ + private static synchronized boolean setSharedPath(final Path path) { + if (sharedTestRoot == null) { + // set this as needed + LOG.info("Set shared path to {}", path); + sharedTestRoot = path; + return true; + } + return false; + } + + @Test + public void test_0000_setupTestDir() throws Throwable { + describe("always ensure directory setup is empty"); + deleteSharedTestRoot(); + } + + @Test + public void test_0100_setupJobStage() throws Throwable { + describe("Set up a job"); + verifyPath("Job attempt dir", + dirs.getJobAttemptDir(), + new SetupJobStage(getJobStageConfig()).apply(true)); + } + + /** + * And the check that the stage worked. + * @throws IOException failure. + */ + private void verifyJobSetupCompleted() throws IOException { + assertPathExists("Job attempt dir from test_0100", dirs.getJobAttemptDir()); + } + + @Test + public void test_0110_setupJobOnlyAllowedOnce() throws Throwable { + describe("a second creation of a job attempt must fail"); + verifyJobSetupCompleted(); + intercept(FileAlreadyExistsException.class, "", () -> + new SetupJobStage(getJobStageConfig()).apply(true)); + // job is still there + assertPathExists("Job attempt dir", dirs.getJobAttemptDir()); + } + + @Test + public void test_0120_setupJobNewAttemptNumber() throws Throwable { + describe("Creating a new job attempt is supported"); + verifyJobSetupCompleted(); + Path path = pathMustExist("Job attempt 2 dir", + new SetupJobStage(createStageConfig(2, -1, 0, destDir)) + .apply(false)); + Assertions.assertThat(path) + .describedAs("Stage created path") + .isNotEqualTo(dirs.getJobAttemptDir()); + } + + @Test + public void test_0200_setupTask00() throws Throwable { + describe("Set up a task; job must have been set up first"); + verifyJobSetupCompleted(); + verifyPath("Task attempt 00", + dirs.getTaskAttemptPath(taskAttempt00), + new SetupTaskStage(ta00Config).apply("first")); + } + + /** + * Verify TA00 is set up. + */ + private void verifyTaskAttempt00SetUp() throws IOException { + pathMustExist("Dir from taskAttempt00 setup", + dirs.getTaskAttemptPath(taskAttempt00)); + } + + @Test + public void test_0210_setupTask00OnlyAllowedOnce() throws Throwable { + describe("Second attempt to set up task00 must fail."); + verifyTaskAttempt00SetUp(); + intercept(FileAlreadyExistsException.class, "second", () -> + new SetupTaskStage(ta00Config).apply("second")); + } + + @Test + public void test_0220_setupTask01() throws Throwable { + describe("Setup task attempt 01"); + verifyTaskAttempt00SetUp(); + verifyPath("Task attempt 01", + dirs.getTaskAttemptPath(taskAttempt01), + new SetupTaskStage(ta01Config) + .apply("01")); + } + + @Test + public void test_0230_setupTask10() throws Throwable { + describe("Setup task attempt 10"); + verifyJobSetupCompleted(); + verifyPath("Task attempt 10", + dirs.getTaskAttemptPath(taskAttempt10), + new SetupTaskStage(ta10Config) + .apply("10")); + } + + /** + * Setup then abort task 11 before creating any files; + * verify that commit fails before creating a manifest file. + */ + @Test + public void test_0240_setupThenAbortTask11() throws Throwable { + describe("Setup then abort task attempt 11"); + verifyJobSetupCompleted(); + Path ta11Path = new SetupTaskStage(ta11Config).apply("11"); + Path deletedDir = new AbortTaskStage(ta11Config).apply(false); + Assertions.assertThat(ta11Path) + .isEqualTo(deletedDir); + assertPathDoesNotExist("aborted directory", ta11Path); + // execute will fail as there's no dir to list. + intercept(FileNotFoundException.class, () -> + new CommitTaskStage(ta11Config).apply(null)); + assertPathDoesNotExist("task manifest", + manifestPathForTask(dirs.getTaskManifestDir(), + TASK_IDS.getTaskId(TASK1))); + } + + /** + * Execute TA01 by generating a lot of files in its directory + * then committing the task attempt. + * The manifest at the task path (i.e. the record of which attempt's + * output is to be used) MUST now have been generated by this TA. + */ + @Test + public void test_0300_executeTask00() throws Throwable { + describe("Create the files for Task 00, then commit the task"); + List files = createFilesOrDirs(dirs.getTaskAttemptPath(taskAttempt00), + "part-00", getExecutorService(), + DEPTH, WIDTH, FILES_PER_DIRECTORY, false); + // saves the task manifest to the job dir + CommitTaskStage.Result result = new CommitTaskStage(ta00Config) + .apply(null); + verifyPathExists(getFileSystem(), "manifest", + result.getPath()); + + TaskManifest manifest = result.getTaskManifest(); + manifest.validate(); + // clear the IOStats to reduce the size of the printed JSON. + manifest.setIOStatistics(null); + LOG.info("Task Manifest {}", manifest.toJson()); + validateTaskAttemptManifest(this.taskAttempt00, files, manifest); + } + + /** + * Validate the manifest of a task attempt. + * @param attemptId attempt ID + * @param files files which were created. + * @param manifest manifest + * @throws IOException IO problem + */ + protected void validateTaskAttemptManifest( + String attemptId, + List files, + TaskManifest manifest) throws IOException { + + verifyManifestTaskAttemptID(manifest, attemptId); + + // validate the manifest + verifyManifestFilesMatch(manifest, files); + } + + /** + * Execute TA01 by generating a lot of files in its directory + * then committing the task attempt. + * The manifest at the task path (i.e. the record of which attempt's + * output is to be used) MUST now have been generated by this TA. + * Any existing manifest will have been overwritten. + */ + @Test + public void test_0310_executeTask01() throws Throwable { + describe("Create the files for Task 01, then commit the task"); + List files = createFilesOrDirs(dirs.getTaskAttemptPath(taskAttempt01), + "part-00", getExecutorService(), + DEPTH, WIDTH, FILES_PER_DIRECTORY, false); + // saves the task manifest to the job dir + CommitTaskStage.Result result = new CommitTaskStage(ta01Config) + .apply(null); + Path manifestPath = verifyPathExists(getFileSystem(), "manifest", + result.getPath()).getPath(); + + // load the manifest from the FS, not the return value, + // so we can verify that last task to commit wins. + TaskManifest manifest = TaskManifest.load(getFileSystem(), manifestPath); + manifest.validate(); + // clear the IOStats to reduce the size of the printed JSON. + manifest.setIOStatistics(null); + LOG.info("Task Manifest {}", manifest.toJson()); + + validateTaskAttemptManifest(taskAttempt01, files, manifest); + + } + + /** + * Second task writes to more directories, but fewer files per dir. + * This ensures that there will dirs here which aren't in the first + * attempt. + */ + @Test + public void test_0320_executeTask10() throws Throwable { + describe("Create the files for Task 10, then commit the task"); + List files = createFilesOrDirs( + dirs.getTaskAttemptPath(ta10Config.getTaskAttemptId()), + "part-01", getExecutorService(), + DEPTH, WIDTH + 1, FILES_PER_DIRECTORY - 1, false); + // saves the task manifest to the job dir + CommitTaskStage.Result result = new CommitTaskStage(ta10Config) + .apply(null); + TaskManifest manifest = result.getTaskManifest(); + validateTaskAttemptManifest(taskAttempt10, files, manifest); + } + + @Test + public void test_0340_setupThenAbortTask11() throws Throwable { + describe("Setup then abort task attempt 11"); + Path ta11Path = new SetupTaskStage(ta11Config).apply("11"); + createFilesOrDirs( + ta11Path, + "part-01", getExecutorService(), + 2, 1, 1, false); + + new AbortTaskStage(ta11Config).apply(false); + assertPathDoesNotExist("aborted directory", ta11Path); + // execute will fail as there's no dir to list. + intercept(FileNotFoundException.class, () -> + new CommitTaskStage(ta11Config).apply(null)); + + // and the manifest MUST be unchanged from the previous stage + Path manifestPathForTask1 = manifestPathForTask(dirs.getTaskManifestDir(), + TASK_IDS.getTaskId(TASK1)); + verifyManifestTaskAttemptID( + TaskManifest.load(getFileSystem(), manifestPathForTask1), + taskAttempt10); + + } + + /** + * Load all the committed manifests, which must be TA01 (last of + * task 0 to commit) and TA10. + */ + @Test + public void test_0400_loadManifests() throws Throwable { + describe("Load all manifests; committed must be TA01 and TA10"); + LoadManifestsStage.Result result + = new LoadManifestsStage(getJobStageConfig()).apply(true); + String summary = result.getSummary().toString(); + LOG.info("Manifest summary {}", summary); + List manifests = result.getManifests(); + Assertions.assertThat(manifests) + .describedAs("Loaded manifests in %s", summary) + .hasSize(2); + Map manifestMap = toMap(manifests); + verifyManifestTaskAttemptID( + manifestMap.get(taskAttempt01), taskAttempt01); + verifyManifestTaskAttemptID( + manifestMap.get(taskAttempt10), taskAttempt10); + } + + @Test + public void test_0410_commitJob() throws Throwable { + describe("Commit the job"); + CommitJobStage stage = new CommitJobStage(getJobStageConfig()); + stage.apply(new CommitJobStage.Arguments(true, false, null, DISABLED)); + } + + /** + * Validate that the job output is good by invoking the + * {@link ValidateRenamedFilesStage} stage to + * validate all the manifests. + */ + @Test + public void test_0420_validateJob() throws Throwable { + describe("Validate the output of the job through the validation" + + " stage"); + + + // load in the success data. + ManifestSuccessData successData = loadAndPrintSuccessData( + getFileSystem(), + getJobStageConfig().getJobSuccessMarkerPath()); + + // load manifests stage will load all the task manifests again + List manifests = new LoadManifestsStage(getJobStageConfig()) + .apply(true).getManifests(); + // Now verify their files exist, returning the list of renamed files. + List committedFiles = new ValidateRenamedFilesStage(getJobStageConfig()) + .apply(manifests) + .stream().map(FileEntry::getDest) + .collect(Collectors.toList()); + + // verify that the list of committed files also matches + // that in the _SUCCESS file + // note: there's a limit to the #of files in the SUCCESS file + // to stop writing it slowing down jobs; therefore we don't + // make a simple "all must match check + Assertions.assertThat(committedFiles) + .containsAll(successData.getFilenames()); + + // now patch one of the manifest files by editing an entry + FileEntry entry = manifests.get(0).getFilesToCommit().get(0); + // no longer exists. + String oldName = entry.getDest(); + String newName = oldName + ".missing"; + entry.setDest(newName); + + // validation will now fail + intercept(OutputValidationException.class, ".missing", () -> + new ValidateRenamedFilesStage(getJobStageConfig()) + .apply(manifests)); + + // restore the name, but change the size + entry.setDest(oldName); + entry.setSize(128_000_000); + intercept(OutputValidationException.class, () -> + new ValidateRenamedFilesStage(getJobStageConfig()) + .apply(manifests)); + } + + @Test + public void test_0430_validateStatistics() throws Throwable { + // load in the success data. + ManifestSuccessData successData = ManifestSuccessData.load( + getFileSystem(), + getJobStageConfig().getJobSuccessMarkerPath()); + String json = successData.toJson(); + LOG.info("Success data is {}", json); + Assertions.assertThat(successData) + .describedAs("Manifest " + json) + .returns(NetUtils.getLocalHostname(), + ManifestSuccessData::getHostname) + .returns(MANIFEST_COMMITTER_CLASSNAME, + ManifestSuccessData::getCommitter) + .returns(jobId, + ManifestSuccessData::getJobId) + .returns(true, + ManifestSuccessData::getSuccess) + .returns(JOB_ID_SOURCE_MAPREDUCE, + ManifestSuccessData::getJobIdSource); + // diagnostics + Assertions.assertThat(successData.getDiagnostics()) + .containsEntry(PRINCIPAL, + getCurrentUser().getShortUserName()) + .containsEntry(STAGE, OP_STAGE_JOB_COMMIT); + + // and stats + IOStatisticsSnapshot iostats = successData.getIOStatistics(); + + int files = successData.getFilenames().size(); + verifyStatisticCounterValue(iostats, + OP_STAGE_JOB_COMMIT, 1); + assertThatStatisticCounter(iostats, + COMMITTER_FILES_COMMITTED_COUNT) + .isGreaterThanOrEqualTo(files); + Long totalFiles = iostats.counters().get(COMMITTER_FILES_COMMITTED_COUNT); + verifyStatisticCounterValue(iostats, + COMMITTER_BYTES_COMMITTED_COUNT, totalFiles * 2); + } + + @Test + public void test_440_validateSuccessFiles() throws Throwable { + + // load in the success data. + final FileSystem fs = getFileSystem(); + ManifestSuccessData successData = loadAndPrintSuccessData( + fs, + getJobStageConfig().getJobSuccessMarkerPath()); + validateGeneratedFiles(fs, + getJobStageConfig().getDestinationDir(), + successData, false); + } + + @Test + public void test_0900_cleanupJob() throws Throwable { + describe("Cleanup job"); + CleanupJobStage.Arguments arguments = new CleanupJobStage.Arguments( + OP_STAGE_JOB_CLEANUP, true, true, false); + // the first run will list the three task attempt dirs and delete each + // one before the toplevel dir. + CleanupJobStage.Result result = new CleanupJobStage( + getJobStageConfig()).apply(arguments); + assertCleanupResult(result, CleanupJobStage.Outcome.PARALLEL_DELETE, 1 + 3); + assertPathDoesNotExist("Job attempt dir", result.getDirectory()); + + // not an error if we retry and the dir isn't there + result = new CleanupJobStage(getJobStageConfig()).apply(arguments); + assertCleanupResult(result, CleanupJobStage.Outcome.NOTHING_TO_CLEAN_UP, 0); + } + + /** + * Needed to clean up the shared test root, as test case teardown + * does not do it. + */ + //@Test + public void test_9999_cleanupTestDir() throws Throwable { + if (shouldDeleteTestRootAtEndOfTestRun()) { + deleteSharedTestRoot(); + } + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java new file mode 100644 index 0000000000000..00372c5941efa --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestLoadManifestsStage.java @@ -0,0 +1,141 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CleanupJobStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.CreateOutputDirectoriesStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.LoadManifestsStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.SetupJobStage; + +/** + * Test loading manifests from a store. + * By not creating files we can simulate a large job just by + * creating the manifests. + * The SaveTaskManifestStage stage is used for the save operation; + * this does a save + rename. + * For better test performance against a remote store, a thread + * pool is used to save the manifests in parallel. + */ +public class TestLoadManifestsStage extends AbstractManifestCommitterTest { + + private int taskAttemptCount; + + /** + * How many task attempts to make? + * Override point. + * @return a number greater than 0. + */ + protected int numberOfTaskAttempts() { + return ManifestCommitterTestSupport.NUMBER_OF_TASK_ATTEMPTS; + } + + @Override + public void setup() throws Exception { + super.setup(); + taskAttemptCount = numberOfTaskAttempts(); + Assertions.assertThat(taskAttemptCount) + .describedAs("Task attempt count") + .isGreaterThan(0); + } + + /** + * Build a large number of manifests, but without the real files + * and directories. + * Save the manifests under the job attempt dir, then load + * them via the {@link LoadManifestsStage}. + * The directory preparation process is then executed after this. + * Because we know each task attempt creates the same number of directories, + * they will all be merged and so only a limited number of output dirs + * will be created. + */ + @Test + public void testSaveThenLoadManyManifests() throws Throwable { + + describe("Creating many manifests with fake file/dir entries," + + " load them and prepare the output dirs."); + + int filesPerTaskAttempt = 10; + LOG.info("Number of task attempts: {}, files per task attempt {}", + taskAttemptCount, filesPerTaskAttempt); + + setJobStageConfig(createStageConfigForJob(JOB1, getDestDir())); + + // set up the job. + new SetupJobStage(getJobStageConfig()).apply(false); + + LOG.info("Creating manifest files for {}", taskAttemptCount); + + executeTaskAttempts(taskAttemptCount, filesPerTaskAttempt); + + LOG.info("Loading in the manifests"); + + // Load in the manifests + LoadManifestsStage stage = new LoadManifestsStage( + getJobStageConfig()); + + LoadManifestsStage.Result result = stage.apply(true); + LoadManifestsStage.SummaryInfo summary = result.getSummary(); + List loadedManifests = result.getManifests(); + + Assertions.assertThat(summary.getManifestCount()) + .describedAs("Manifest count of %s", summary) + .isEqualTo(taskAttemptCount); + Assertions.assertThat(summary.getFileCount()) + .describedAs("File count of %s", summary) + .isEqualTo(taskAttemptCount * (long) filesPerTaskAttempt); + Assertions.assertThat(summary.getTotalFileSize()) + .describedAs("File Size of %s", summary) + .isEqualTo(getTotalDataSize()); + + // now that manifest list. + List manifestTaskIds = loadedManifests.stream() + .map(TaskManifest::getTaskID) + .collect(Collectors.toList()); + Assertions.assertThat(getTaskIds()) + .describedAs("Task IDs of all tasks") + .containsExactlyInAnyOrderElementsOf(manifestTaskIds); + + // now let's see about aggregating a large set of directories + Set createdDirectories = new CreateOutputDirectoriesStage( + getJobStageConfig()) + .apply(loadedManifests) + .getCreatedDirectories(); + + // but after the merge process, only one per generated file output + // dir exists + Assertions.assertThat(createdDirectories) + .describedAs("Directories created") + .hasSize(filesPerTaskAttempt); + + // and skipping the rename stage (which is going to fail), + // go straight to cleanup + new CleanupJobStage(getJobStageConfig()).apply( + new CleanupJobStage.Arguments("", true, true, false)); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestManifestCommitProtocol.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestManifestCommitProtocol.java new file mode 100644 index 0000000000000..2212fabe54acd --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestManifestCommitProtocol.java @@ -0,0 +1,1801 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.fs.statistics.IOStatisticsSupport; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.MapFile; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.MapFileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.task.JobContextImpl; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.functional.RemoteIterators; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.contract.ContractTestUtils.listChildren; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.COMMITTER_FACTORY_CLASS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_FACTORY; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SPARK_WRITE_UUID; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUCCESS_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_BYTES_COMMITTED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_FILES_COMMITTED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_COMPLETED_COUNT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_LOAD_MANIFEST; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_ABORT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_STAGE_JOB_COMMIT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createJobSummaryFilename; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.randomJobId; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.validateSuccessFile; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * This is a contract test for the commit protocol on a target filesystem. + * It is subclassed in the ABFS integration tests and elsewhere. + * Derived from the S3A protocol suite, which was itself based off + * the test suite {@code TestFileOutputCommitter}. + * + * Some of the methods trigger java warnings about unchecked casts; + * it's impossible to remove them, so the checks are suppressed. + */ +@SuppressWarnings("unchecked") +public class TestManifestCommitProtocol + extends AbstractManifestCommitterTest { + + private static final Logger LOG = + LoggerFactory.getLogger(TestManifestCommitProtocol.class); + + private static final String SUB_DIR = "SUB_DIR"; + + /** + * Part of the name of the output of task attempt 0. + */ + protected static final String PART_00000 = "part-m-00000"; + + private static final Text KEY_1 = new Text("key1"); + + private static final Text KEY_2 = new Text("key2"); + + private static final Text VAL_1 = new Text("val1"); + + private static final Text VAL_2 = new Text("val2"); + + /** + * Snapshot of stats, which will be collected from + * committers. + */ + private static final IOStatisticsSnapshot IOSTATISTICS = + IOStatisticsSupport.snapshotIOStatistics(); + + /** + * Job ID for jobs. + */ + private final String jobId; + + /** + * A random task attempt id for testing. + */ + private final String attempt0; + + /** + * Attempt 0's task attempt ID. + */ + private final TaskAttemptID taskAttempt0; + + /** + * TA 1. + */ + private final TaskAttemptID taskAttempt1; + + /** + * Attempt 1 string value. + */ + private final String attempt1; + + + /** A job to abort in test case teardown. */ + private final List abortInTeardown = new ArrayList<>(1); + + /** + * Output directory. + * This is the directory into which output goes; + * all the job files go in _temporary underneath. + */ + private Path outputDir; + + /** + * Committer factory which calls back into + * {@link #createCommitter(Path, TaskAttemptContext)}. + */ + private final LocalCommitterFactory + localCommitterFactory = new LocalCommitterFactory(); + + /** + * Clean up the output dir. No-op if + * {@link #outputDir} is null. + * @throws IOException failure to delete + */ + private void cleanupOutputDir() throws IOException { + if (outputDir != null) { + getFileSystem().delete(outputDir, true); + } + } + + /** + * Constructor. + */ + public TestManifestCommitProtocol() { + ManifestCommitterTestSupport.JobAndTaskIDsForTests taskIDs + = new ManifestCommitterTestSupport.JobAndTaskIDsForTests(2, 2); + jobId = taskIDs.getJobId(); + attempt0 = taskIDs.getTaskAttempt(0, 0); + taskAttempt0 = taskIDs.getTaskAttemptIdType(0, 0); + attempt1 = taskIDs.getTaskAttempt(0, 1); + taskAttempt1 = taskIDs.getTaskAttemptIdType(0, 1); + } + + /** + * This must return the name of a suite which is unique to the test. + * @return a string which must be unique and a valid path. + */ + protected String suitename() { + return "TestManifestCommitProtocolLocalFS"; + } + + /** + * Get the log; can be overridden for test case log. + * @return a log. + */ + public Logger log() { + return LOG; + } + + /** + * Overridden method returns the suitename as well as the method name, + * so if more than one committer test is run in parallel, paths are + * isolated. + * @return a name for a method, unique across the suites and test cases. + */ + @Override + protected String getMethodName() { + return suitename() + "-" + super.getMethodName(); + } + + @Override + public void setup() throws Exception { + super.setup(); + + outputDir = path(getMethodName()); + cleanupOutputDir(); + } + + @Override + public void teardown() throws Exception { + describe("teardown"); + Thread.currentThread().setName("teardown"); + for (JobData jobData : abortInTeardown) { + // stop the job + abortJobQuietly(jobData); + // and then get its statistics + IOSTATISTICS.aggregate(jobData.committer.getIOStatistics()); + } + try { + cleanupOutputDir(); + } catch (IOException e) { + log().info("Exception during cleanup", e); + } + super.teardown(); + } + + @AfterClass + public static void logAggregateIOStatistics() { + LOG.info("Final IOStatistics {}", + ioStatisticsToPrettyString(IOSTATISTICS)); + } + + /** + * Add the specified job to the current list of jobs to abort in teardown. + * @param jobData job data. + */ + protected void abortInTeardown(JobData jobData) { + abortInTeardown.add(jobData); + } + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + bindCommitter(conf); + return conf; + } + + /*** + * Set job up to use the manifest committer. + * @param conf configuration to set up + */ + protected void bindCommitter(Configuration conf) { + conf.set(COMMITTER_FACTORY_CLASS, MANIFEST_COMMITTER_FACTORY); + } + + /** + * Create a committer for a task. + * @param context task context + * @return new committer + * @throws IOException failure + */ + protected ManifestCommitter createCommitter( + TaskAttemptContext context) throws IOException { + return createCommitter(getOutputDir(), context); + } + + /** + * Create a committer for a task and a given output path. + * @param outputPath path + * @param context task context + * @return new committer + * @throws IOException failure + */ + protected ManifestCommitter createCommitter( + Path outputPath, + TaskAttemptContext context) throws IOException { + return new ManifestCommitter(outputPath, context); + } + + protected Path getOutputDir() { + return outputDir; + } + + protected String getJobId() { + return jobId; + } + + protected String getAttempt0() { + return attempt0; + } + + protected TaskAttemptID getTaskAttempt0() { + return taskAttempt0; + } + + protected String getAttempt1() { + return attempt1; + } + + protected TaskAttemptID getTaskAttempt1() { + return taskAttempt1; + } + + /** + * Functional interface for creating committers, designed to allow + * different factories to be used to create different failure modes. + */ + @FunctionalInterface + public interface CommitterFactory { + + /** + * Create a committer for a task. + * @param context task context + * @return new committer + * @throws IOException failure + */ + ManifestCommitter createCommitter( + TaskAttemptContext context) throws IOException; + } + + /** + * The normal committer creation factory, uses the abstract methods + * in the class. + */ + protected class LocalCommitterFactory implements CommitterFactory { + + @Override + public ManifestCommitter createCommitter(TaskAttemptContext context) + throws IOException { + return TestManifestCommitProtocol.this + .createCommitter(context); + } + } + + /** + * Assert that for a given output, the job context returns a manifest + * committer factory. This is what FileOutputFormat does internally, + * and is needed to make sure that the relevant settings are being passed + * around. + * @param context job/task context + * @param output destination path. + */ + protected void assertCommitterFactoryIsManifestCommitter( + JobContext context, Path output) { + + final Configuration conf = context.getConfiguration(); + // check one: committer + assertConfigurationUsesManifestCommitter(conf); + final String factoryName = conf.get(COMMITTER_FACTORY_CLASS, ""); + final PathOutputCommitterFactory factory + = PathOutputCommitterFactory.getCommitterFactory( + output, + conf); + Assertions.assertThat(factory) + .describedAs("Committer for output path %s" + + " and factory name \"%s\"", + output, factoryName) + .isInstanceOf(ManifestCommitterFactory.class); + } + + /** + * This is to debug situations where the test committer factory + * on tasks was binding to FileOutputCommitter even when + * tests were overriding it. + * @param conf configuration to probe. + */ + private void assertConfigurationUsesManifestCommitter( + Configuration conf) { + final String factoryName = conf.get(COMMITTER_FACTORY_CLASS, null); + Assertions.assertThat(factoryName) + .describedAs("Value of %s", COMMITTER_FACTORY_CLASS) + .isEqualTo(MANIFEST_COMMITTER_FACTORY); + } + + /** + * Write some text out. + * @param context task + * @throws IOException IO failure + * @throws InterruptedException write interrupted + * @return the path written to + */ + protected Path writeTextOutput(TaskAttemptContext context) + throws IOException, InterruptedException { + describe("write output"); + try (DurationInfo d = new DurationInfo(LOG, + "Writing Text output for task %s", context.getTaskAttemptID())) { + TextOutputForTests.LoggingLineRecordWriter writer + = new TextOutputForTests().getRecordWriter(context); + writeOutput(writer, context); + return writer.getDest(); + } + } + + /** + * Write the standard output. + * @param writer record writer + * @param context task context + * @throws IOException IO failure + * @throws InterruptedException write interrupted + */ + private void writeOutput( + RecordWriter writer, + TaskAttemptContext context) throws IOException, InterruptedException { + NullWritable nullWritable = NullWritable.get(); + try (ManifestCommitterTestSupport.CloseWriter cw = + new ManifestCommitterTestSupport.CloseWriter<>(writer, context)) { + writer.write(KEY_1, VAL_1); + writer.write(null, nullWritable); + writer.write(null, VAL_1); + writer.write(nullWritable, VAL_2); + writer.write(KEY_2, nullWritable); + writer.write(KEY_1, null); + writer.write(null, null); + writer.write(KEY_2, VAL_2); + writer.close(context); + } + } + + /** + * Write the output of a map. + * @param writer record writer + * @param context task context + * @throws IOException IO failure + * @throws InterruptedException write interrupted + */ + private void writeMapFileOutput(RecordWriter, Writable> writer, + TaskAttemptContext context) throws IOException, InterruptedException { + describe("\nWrite map output"); + try (DurationInfo d = new DurationInfo(LOG, + "Writing Text output for task %s", context.getTaskAttemptID()); + ManifestCommitterTestSupport.CloseWriter, Writable> cw = + new ManifestCommitterTestSupport.CloseWriter<>(writer, context)) { + for (int i = 0; i < 10; ++i) { + Text val = ((i & 1) == 1) ? VAL_1 : VAL_2; + writer.write(new LongWritable(i), val); + } + LOG.debug("Closing writer {}", writer); + writer.close(context); + } + } + + /** + * Details on a job for use in {@code startJob} and elsewhere. + */ + protected static final class JobData { + + private final Job job; + + private final JobContext jContext; + + private final TaskAttemptContext tContext; + + private final ManifestCommitter committer; + + private final Configuration conf; + + private Path writtenTextPath; // null if not written to + + public JobData(Job job, + JobContext jContext, + TaskAttemptContext tContext, + ManifestCommitter committer) { + this.job = job; + this.jContext = jContext; + this.tContext = tContext; + this.committer = committer; + conf = job.getConfiguration(); + } + + public String jobId() { + return committer.getJobUniqueId(); + } + } + + /** + * Create a new job. Sets the task attempt ID, + * and output dir; asks for a success marker. + * @return the new job + * @throws IOException failure + */ + public Job newJob() throws IOException { + return newJob(outputDir, getConfiguration(), attempt0); + } + + /** + * Create a new job. Sets the task attempt ID, + * and output dir; asks for a success marker. + * Committer factory is set to manifest factory, so is independent + * of FS schema. + * @param dir dest dir + * @param configuration config to get the job from + * @param taskAttemptId task attempt + * @return the new job + * @throws IOException failure + */ + private Job newJob(Path dir, Configuration configuration, + String taskAttemptId) throws IOException { + Job job = Job.getInstance(configuration); + Configuration conf = job.getConfiguration(); + conf.set(MRJobConfig.TASK_ATTEMPT_ID, taskAttemptId); + enableManifestCommitter(conf); + FileOutputFormat.setOutputPath(job, dir); + return job; + } + + /** + * Start a job with a committer; optionally write the test data. + * Always register the job to be aborted (quietly) in teardown. + * This is, from an "OO-purity perspective" the wrong kind of method to + * do: it's setting things up, mixing functionality, registering for teardown. + * Its aim is simple though: a common body of code for starting work + * in test cases. + * @param writeText should the text be written? + * @return the job data 4-tuple + * @throws IOException IO problems + * @throws InterruptedException interruption during write + */ + protected JobData startJob(boolean writeText) + throws IOException, InterruptedException { + return startJob(localCommitterFactory, writeText); + } + + /** + * Start a job with a committer; optionally write the test data. + * Always register the job to be aborted (quietly) in teardown. + * This is, from an "OO-purity perspective" the wrong kind of method to + * do: it's setting things up, mixing functionality, registering for teardown. + * Its aim is simple though: a common body of code for starting work + * in test cases. + * @param factory the committer factory to use + * @param writeText should the text be written? + * @return the job data 4-tuple + * @throws IOException IO problems + * @throws InterruptedException interruption during write + */ + protected JobData startJob(CommitterFactory factory, boolean writeText) + throws IOException, InterruptedException { + Job job = newJob(); + Configuration conf = job.getConfiguration(); + assertConfigurationUsesManifestCommitter(conf); + conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt0); + conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1); + JobContext jContext = new JobContextImpl(conf, taskAttempt0.getJobID()); + TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, + taskAttempt0); + ManifestCommitter committer = factory.createCommitter(tContext); + + // setup + JobData jobData = new JobData(job, jContext, tContext, committer); + setupJob(jobData); + abortInTeardown(jobData); + + if (writeText) { + // write output + jobData.writtenTextPath = writeTextOutput(tContext); + } + return jobData; + } + + /** + * Set up the job and task. + * @param jobData job data + * @throws IOException problems + */ + protected void setupJob(JobData jobData) throws IOException { + ManifestCommitter committer = jobData.committer; + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + describe("\nsetup job"); + try (DurationInfo d = new DurationInfo(LOG, + "setup job %s", jContext.getJobID())) { + committer.setupJob(jContext); + } + setupCommitter(committer, tContext); + describe("setup complete"); + } + + private void setupCommitter( + final ManifestCommitter committer, + final TaskAttemptContext tContext) throws IOException { + try (DurationInfo d = new DurationInfo(LOG, + "setup task %s", tContext.getTaskAttemptID())) { + committer.setupTask(tContext); + } + } + + /** + * Abort a job quietly. + * @param jobData job info + */ + protected void abortJobQuietly(JobData jobData) { + abortJobQuietly(jobData.committer, jobData.jContext, jobData.tContext); + } + + /** + * Abort a job quietly: first task, then job. + * @param committer committer + * @param jContext job context + * @param tContext task context + */ + protected void abortJobQuietly(ManifestCommitter committer, + JobContext jContext, + TaskAttemptContext tContext) { + describe("\naborting task"); + try { + committer.abortTask(tContext); + } catch (Exception e) { + log().warn("Exception aborting task:", e); + } + describe("\naborting job"); + try { + committer.abortJob(jContext, JobStatus.State.KILLED); + } catch (Exception e) { + log().warn("Exception aborting job", e); + } + } + + /** + * Commit the task and then the job. + * @param committer committer + * @param jContext job context + * @param tContext task context + * @throws IOException problems + */ + protected void commitTaskAndJob(ManifestCommitter committer, + JobContext jContext, + TaskAttemptContext tContext) throws IOException { + try (DurationInfo d = new DurationInfo(LOG, + "committing Job %s", jContext.getJobID())) { + describe("\ncommitting task"); + committer.commitTask(tContext); + describe("\ncommitting job"); + committer.commitJob(jContext); + describe("commit complete\n"); + } + } + + /** + * Execute work as part of a test, after creating the job. + * After the execution, {@link #abortJobQuietly(JobData)} is + * called for abort/cleanup. + * @param name name of work (for logging) + * @param action action to execute + * @throws Exception failure + */ + protected void executeWork(String name, ActionToTest action) + throws Exception { + executeWork(name, startJob(false), action); + } + + /** + * Execute work as part of a test, against the created job. + * After the execution, {@link #abortJobQuietly(JobData)} is + * called for abort/cleanup. + * @param name name of work (for logging) + * @param jobData job info + * @param action action to execute + * @throws Exception failure + */ + public void executeWork(String name, + JobData jobData, + ActionToTest action) throws Exception { + try (DurationInfo d = new DurationInfo(LOG, "Executing %s", name)) { + action.exec(jobData.job, + jobData.jContext, + jobData.tContext, + jobData.committer); + } finally { + abortJobQuietly(jobData); + } + } + + /** + * Load a manifest from the test FS. + * @param path path + * @return the manifest + * @throws IOException failure to load + */ + TaskManifest loadManifest(Path path) throws IOException { + return TaskManifest.load(getFileSystem(), path); + } + + /** + * Verify that recovery doesn't work for these committers. + */ + @Test + @SuppressWarnings("deprecation") + public void testRecoveryAndCleanup() throws Exception { + describe("Test (unsupported) task recovery."); + JobData jobData = startJob(true); + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + + Assertions.assertThat(committer.getWorkPath()) + .as("null workPath in committer " + committer) + .isNotNull(); + Assertions.assertThat(committer.getOutputPath()) + .as("null outputPath in committer " + committer) + .isNotNull(); + + // Commit the task. + commitTask(committer, tContext); + + // load and log the manifest + final TaskManifest manifest = loadManifest( + committer.getTaskManifestPath(tContext)); + LOG.info("Manifest {}", manifest); + + Configuration conf2 = jobData.job.getConfiguration(); + conf2.set(MRJobConfig.TASK_ATTEMPT_ID, attempt0); + conf2.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 2); + JobContext jContext2 = new JobContextImpl(conf2, taskAttempt0.getJobID()); + TaskAttemptContext tContext2 = new TaskAttemptContextImpl(conf2, + taskAttempt0); + ManifestCommitter committer2 = createCommitter(tContext2); + committer2.setupJob(tContext2); + + Assertions.assertThat(committer2.isRecoverySupported()) + .as("recoverySupported in " + committer2) + .isFalse(); + intercept(IOException.class, "recover", + () -> committer2.recoverTask(tContext2)); + + // at this point, task attempt 0 has failed to recover + // it should be abortable though. This will be a no-op as it already + // committed + describe("aborting task attempt 2; expect nothing to clean up"); + committer2.abortTask(tContext2); + describe("Aborting job 2; expect pending commits to be aborted"); + committer2.abortJob(jContext2, JobStatus.State.KILLED); + } + + /** + * Assert that the task attempt FS Doesn't have a task attempt + * directory. + * @param committer committer + * @param context task context + * @throws IOException IO failure. + */ + protected void assertTaskAttemptPathDoesNotExist( + ManifestCommitter committer, TaskAttemptContext context) + throws IOException { + Path attemptPath = committer.getTaskAttemptPath(context); + ContractTestUtils.assertPathDoesNotExist( + attemptPath.getFileSystem(context.getConfiguration()), + "task attempt dir", + attemptPath); + } + + protected void assertJobAttemptPathDoesNotExist( + ManifestCommitter committer, JobContext context) + throws IOException { + Path attemptPath = committer.getJobAttemptPath(context); + ContractTestUtils.assertPathDoesNotExist( + attemptPath.getFileSystem(context.getConfiguration()), + "job attempt dir", + attemptPath); + } + + /** + * Verify the output of the directory. + * That includes the {@code part-m-00000-*} + * file existence and contents, as well as optionally, the success marker. + * @param dir directory to scan. + * @param expectSuccessMarker check the success marker? + * @param expectedJobId job ID, verified if non-empty and success data loaded + * @throws Exception failure. + * @return the success data + */ + private ManifestSuccessData validateContent(Path dir, + boolean expectSuccessMarker, + String expectedJobId) throws Exception { + lsR(getFileSystem(), dir, true); + ManifestSuccessData successData; + if (expectSuccessMarker) { + successData = verifySuccessMarker(dir, expectedJobId); + } else { + successData = null; + } + Path expectedFile = getPart0000(dir); + log().debug("Validating content in {}", expectedFile); + StringBuilder expectedOutput = new StringBuilder(); + expectedOutput.append(KEY_1).append('\t').append(VAL_1).append("\n"); + expectedOutput.append(VAL_1).append("\n"); + expectedOutput.append(VAL_2).append("\n"); + expectedOutput.append(KEY_2).append("\n"); + expectedOutput.append(KEY_1).append("\n"); + expectedOutput.append(KEY_2).append('\t').append(VAL_2).append("\n"); + String output = readFile(expectedFile); + Assertions.assertThat(output) + .describedAs("Content of %s", expectedFile) + .isEqualTo(expectedOutput.toString()); + return successData; + } + + /** + * Identify any path under the directory which begins with the + * {@code "part-m-00000"} sequence. There's some compensation for + * eventual consistency here. + * @param dir directory to scan + * @return the full path + * @throws FileNotFoundException the path is missing. + * @throws Exception failure. + */ + protected Path getPart0000(final Path dir) throws Exception { + final FileSystem fs = dir.getFileSystem(getConfiguration()); + FileStatus[] statuses = fs.listStatus(dir, + path -> path.getName().startsWith(PART_00000)); + if (statuses.length != 1) { + // fail, with a listing of the parent dir + ContractTestUtils.assertPathExists(fs, "Output file", + new Path(dir, PART_00000)); + } + return statuses[0].getPath(); + } + + /** + * Look for the partFile subdir of the output dir + * and the ma and data entries. + * @param fs filesystem + * @param dir output dir + * @throws Exception failure. + */ + private void validateMapFileOutputContent( + FileSystem fs, Path dir) throws Exception { + // map output is a directory with index and data files + assertPathExists("Map output", dir); + Path expectedMapDir = getPart0000(dir); + assertPathExists("Map output", expectedMapDir); + assertIsDirectory(expectedMapDir); + FileStatus[] files = fs.listStatus(expectedMapDir); + Assertions.assertThat(files) + .as("No files found in " + expectedMapDir) + .isNotEmpty(); + assertPathExists("index file in " + expectedMapDir, + new Path(expectedMapDir, MapFile.INDEX_FILE_NAME)); + assertPathExists("data file in " + expectedMapDir, + new Path(expectedMapDir, MapFile.DATA_FILE_NAME)); + } + + /** + * Full test of the expected lifecycle: start job, task, write, commit task, + * commit job. + * @throws Exception on a failure + */ + @Test + public void testCommitLifecycle() throws Exception { + describe("Full test of the expected lifecycle:\n" + + " start job, task, write, commit task, commit job.\n" + + "Verify:\n" + + "* no files are visible after task commit\n" + + "* the expected file is visible after job commit\n"); + JobData jobData = startJob(false); + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + assertCommitterFactoryIsManifestCommitter(tContext, + tContext.getWorkingDirectory()); + validateTaskAttemptWorkingDirectory(committer, tContext); + + // write output + describe("1. Writing output"); + final Path textOutputPath = writeTextOutput(tContext); + describe("Output written to %s", textOutputPath); + + describe("2. Committing task"); + Assertions.assertThat(committer.needsTaskCommit(tContext)) + .as("No files to commit were found by " + committer) + .isTrue(); + commitTask(committer, tContext); + final TaskManifest taskManifest = requireNonNull( + committer.getTaskAttemptCommittedManifest(), "committerTaskManifest"); + final String manifestJSON = taskManifest.toJson(); + LOG.info("Task manifest {}", manifestJSON); + int filesCreated = 1; + Assertions.assertThat(taskManifest.getFilesToCommit()) + .describedAs("Files to commit in task manifest %s", manifestJSON) + .hasSize(filesCreated); + Assertions.assertThat(taskManifest.getDestDirectories()) + .describedAs("Directories to create in task manifest %s", + manifestJSON) + .isEmpty(); + + // this is only task commit; there MUST be no part- files in the dest dir + try { + RemoteIterators.foreach(getFileSystem().listFiles(outputDir, false), + (status) -> + Assertions.assertThat(status.getPath().toString()) + .as("task committed file to dest :" + status) + .contains("part")); + } catch (FileNotFoundException ignored) { + log().info("Outdir {} is not created by task commit phase ", + outputDir); + } + + describe("3. Committing job"); + + commitJob(committer, jContext); + + // validate output + describe("4. Validating content"); + String jobUniqueId = jobData.jobId(); + ManifestSuccessData successData = validateContent(outputDir, + true, + jobUniqueId); + // look in the SUMMARY + Assertions.assertThat(successData.getDiagnostics()) + .describedAs("Stage entry in SUCCESS") + .containsEntry(STAGE, OP_STAGE_JOB_COMMIT); + IOStatisticsSnapshot jobStats = successData.getIOStatistics(); + // manifest + verifyStatisticCounterValue(jobStats, + OP_LOAD_MANIFEST, 1); + FileStatus st = getFileSystem().getFileStatus(getPart0000(outputDir)); + verifyStatisticCounterValue(jobStats, + COMMITTER_FILES_COMMITTED_COUNT, filesCreated); + verifyStatisticCounterValue(jobStats, + COMMITTER_BYTES_COMMITTED_COUNT, st.getLen()); + + // now load and examine the job report. + // this MUST contain all the stats of the summary, plus timings on + // job commit itself + + ManifestSuccessData report = loadReport(jobUniqueId, true); + Map diag = report.getDiagnostics(); + Assertions.assertThat(diag) + .describedAs("Stage entry in report") + .containsEntry(STAGE, OP_STAGE_JOB_COMMIT); + IOStatisticsSnapshot reportStats = report.getIOStatistics(); + verifyStatisticCounterValue(reportStats, + OP_LOAD_MANIFEST, 1); + verifyStatisticCounterValue(reportStats, + OP_STAGE_JOB_COMMIT, 1); + verifyStatisticCounterValue(reportStats, + COMMITTER_FILES_COMMITTED_COUNT, filesCreated); + verifyStatisticCounterValue(reportStats, + COMMITTER_BYTES_COMMITTED_COUNT, st.getLen()); + + } + + /** + * Load a summary from the report dir. + * @param jobUniqueId job ID + * @param expectSuccess is the job expected to have succeeded. + * @throws IOException failure to load + * @return the report + */ + private ManifestSuccessData loadReport(String jobUniqueId, + boolean expectSuccess) throws IOException { + File file = new File(getReportDir(), + createJobSummaryFilename(jobUniqueId)); + ContractTestUtils.assertIsFile(FileSystem.getLocal(getConfiguration()), + new Path(file.toURI())); + ManifestSuccessData report = ManifestSuccessData.serializer().load(file); + LOG.info("Report for job {}:\n{}", jobUniqueId, report.toJson()); + Assertions.assertThat(report.getSuccess()) + .describedAs("success flag in report") + .isEqualTo(expectSuccess); + return report; + } + + /** + * Repeated commit call after job commit. + */ + @Test + public void testCommitterWithDuplicatedCommit() throws Exception { + describe("Call a task then job commit twice;" + + "expect the second task commit to fail."); + JobData jobData = startJob(true); + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + + // do commit + describe("committing task"); + committer.commitTask(tContext); + + // repeated commit while TA dir exists fine/idempotent + committer.commitTask(tContext); + + describe("committing job"); + committer.commitJob(jContext); + describe("commit complete\n"); + + describe("cleanup"); + committer.cleanupJob(jContext); + // validate output + validateContent(outputDir, shouldExpectSuccessMarker(), + committer.getJobUniqueId()); + + // commit task to fail on retry as task attempt dir doesn't exist + describe("Attempting commit of the same task after job commit -expecting failure"); + expectFNFEonTaskCommit(committer, tContext); + } + + /** + * HADOOP-17258. If a second task attempt is committed, it + * must succeed, and the output of the first TA, even if already + * committed, MUST NOT be visible in the final output. + *

+ * What's important is not just that only one TA must succeed, + * but it must be the last one executed. + */ + @Test + public void testTwoTaskAttemptsCommit() throws Exception { + describe("Commit two task attempts;" + + " expect the second attempt to succeed."); + JobData jobData = startJob(false); + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + // do commit + describe("\ncommitting task"); + // write output for TA 1, + Path outputTA1 = writeTextOutput(tContext); + + // speculatively execute committer 2. + + // jobconf with a different base to its parts. + Configuration conf2 = jobData.conf; + conf2.set("mapreduce.output.basename", "attempt2"); + String attempt2 = "attempt_" + jobId + "_m_000000_1"; + TaskAttemptID ta2 = TaskAttemptID.forName(attempt2); + TaskAttemptContext tContext2 = new TaskAttemptContextImpl( + conf2, ta2); + + ManifestCommitter committer2 = localCommitterFactory + .createCommitter(tContext2); + setupCommitter(committer2, tContext2); + + // verify working dirs are different + Assertions.assertThat(committer.getWorkPath()) + .describedAs("Working dir of %s", committer) + .isNotEqualTo(committer2.getWorkPath()); + + // write output for TA 2, + Path outputTA2 = writeTextOutput(tContext2); + + // verify the names are different. + String name1 = outputTA1.getName(); + String name2 = outputTA2.getName(); + Assertions.assertThat(name1) + .describedAs("name of task attempt output %s", outputTA1) + .isNotEqualTo(name2); + + // commit task 1 + committer.commitTask(tContext); + + // then pretend that task1 didn't respond, so + // commit task 2 + committer2.commitTask(tContext2); + + // and the job + committer2.commitJob(tContext); + + // validate output + FileSystem fs = getFileSystem(); + ManifestSuccessData successData = validateSuccessFile(fs, outputDir, + 1, + ""); + Assertions.assertThat(successData.getFilenames()) + .describedAs("Files committed") + .hasSize(1); + + assertPathExists("attempt2 output", new Path(outputDir, name2)); + assertPathDoesNotExist("attempt1 output", new Path(outputDir, name1)); + + } + + protected boolean shouldExpectSuccessMarker() { + return true; + } + + /** + * Simulate a failure on the first job commit; expect the + * second to succeed. + */ + /*@Test + public void testCommitterWithFailure() throws Exception { + describe("Fail the first job commit then retry"); + JobData jobData = startJob(new FailingCommitterFactory(), true); + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + + // do commit + committer.commitTask(tContext); + + // now fail job + expectSimulatedFailureOnJobCommit(jContext, committer); + + commitJob(committer, jContext); + + // but the data got there, due to the order of operations. + validateContent(outDir, shouldExpectSuccessMarker(), + committer.getUUID()); + expectJobCommitToFail(jContext, committer); + } +*/ + + /** + * Override point: the failure expected on the attempt to commit a failed + * job. + * @param jContext job context + * @param committer committer + * @throws Exception any unexpected failure. + */ + protected void expectJobCommitToFail(JobContext jContext, + ManifestCommitter committer) throws Exception { + // next attempt will fail as there is no longer a directory to commit + expectJobCommitFailure(jContext, committer, + FileNotFoundException.class); + } + + /** + * Expect a job commit operation to fail with a specific exception. + * @param jContext job context + * @param committer committer + * @param clazz class of exception + * @return the caught exception + * @throws Exception any unexpected failure. + */ + protected static E expectJobCommitFailure( + JobContext jContext, + ManifestCommitter committer, + Class clazz) + throws Exception { + + return intercept(clazz, + () -> { + committer.commitJob(jContext); + return committer.toString(); + }); + } + + protected static void expectFNFEonTaskCommit( + ManifestCommitter committer, + TaskAttemptContext tContext) throws Exception { + intercept(FileNotFoundException.class, + () -> { + committer.commitTask(tContext); + return committer.toString(); + }); + } + + /** + * Commit a task with no output. + * Dest dir should exist. + */ + @Test + public void testCommitterWithNoOutputs() throws Exception { + describe("Have a task and job with no outputs: expect success"); + JobData jobData = startJob(localCommitterFactory, false); + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + + // do commit + committer.commitTask(tContext); + Path attemptPath = committer.getTaskAttemptPath(tContext); + ContractTestUtils.assertPathExists( + attemptPath.getFileSystem(tContext.getConfiguration()), + "task attempt dir", + attemptPath); + } + + + @Test + public void testMapFileOutputCommitter() throws Exception { + describe("Test that the committer generates map output into a directory\n" + + "starting with the prefix part-"); + JobData jobData = startJob(false); + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + Configuration conf = jobData.conf; + + // write output + writeMapFileOutput(new MapFileOutputFormat() + .getRecordWriter(tContext), tContext); + + // do commit + commitTaskAndJob(committer, jContext, tContext); + FileSystem fs = getFileSystem(); + + lsR(fs, outputDir, true); + String ls = ls(outputDir); + describe("\nvalidating"); + + // validate output + verifySuccessMarker(outputDir, committer.getJobUniqueId()); + + describe("validate output of %s", outputDir); + validateMapFileOutputContent(fs, outputDir); + + // Ensure getReaders call works and also ignores + // hidden filenames (_ or . prefixes) + describe("listing"); + FileStatus[] filtered = fs.listStatus(outputDir, HIDDEN_FILE_FILTER); + Assertions.assertThat(filtered) + .describedAs("listed children under %s", ls) + .hasSize(1); + FileStatus fileStatus = filtered[0]; + Assertions.assertThat(fileStatus.getPath().getName()) + .as("Not the part file: " + fileStatus) + .startsWith(PART_00000); + + describe("getReaders()"); + Assertions.assertThat(getReaders(fs, outputDir, conf)) + .describedAs("getReaders() MapFile.Reader entries with shared FS %s %s", outputDir, ls) + .hasSize(1); + + describe("getReaders(new FS)"); + FileSystem fs2 = FileSystem.get(outputDir.toUri(), conf); + Assertions.assertThat(getReaders(fs2, outputDir, conf)) + .describedAs("getReaders(new FS) %s %s", outputDir, ls) + .hasSize(1); + + describe("MapFileOutputFormat.getReaders"); + Assertions.assertThat(MapFileOutputFormat.getReaders(outputDir, conf)) + .describedAs("MapFileOutputFormat.getReaders(%s) %s", outputDir, ls) + .hasSize(1); + + } + + /** Open the output generated by this format. */ + @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") + private static MapFile.Reader[] getReaders(FileSystem fs, + Path dir, + Configuration conf) throws IOException { + Path[] names = FileUtil.stat2Paths(fs.listStatus(dir, HIDDEN_FILE_FILTER)); + + // sort names, so that hash partitioning works + Arrays.sort(names); + + MapFile.Reader[] parts = new MapFile.Reader[names.length]; + for (int i = 0; i < names.length; i++) { + parts[i] = new MapFile.Reader(names[i], conf); + } + return parts; + } + + public static final PathFilter HIDDEN_FILE_FILTER = (path) -> + !path.getName().startsWith("_") && !path.getName().startsWith("."); + + /** + * A functional interface which an action to test must implement. + */ + @FunctionalInterface + public interface ActionToTest { + + void exec(Job job, JobContext jContext, TaskAttemptContext tContext, + ManifestCommitter committer) throws Exception; + } + + @Test + public void testAbortTaskNoWorkDone() throws Exception { + executeWork("abort task no work", + (job, jContext, tContext, committer) -> + committer.abortTask(tContext)); + } + + @Test + public void testAbortJobNoWorkDone() throws Exception { + executeWork("abort task no work", + (job, jContext, tContext, committer) -> + committer.abortJob(jContext, JobStatus.State.RUNNING)); + } + + @Test + public void testCommitJobButNotTask() throws Exception { + executeWork("commit a job while a task's work is pending, " + + "expect task writes to be cancelled.", + (job, jContext, tContext, committer) -> { + // step 1: write the text + writeTextOutput(tContext); + // step 2: commit the job + createCommitter(tContext).commitJob(tContext); + // verify that no output can be observed + assertPart0000DoesNotExist(outputDir); + } + ); + } + + @Test + public void testAbortTaskThenJob() throws Exception { + JobData jobData = startJob(true); + ManifestCommitter committer = jobData.committer; + + // do abort + committer.abortTask(jobData.tContext); + + intercept(FileNotFoundException.class, "", + () -> getPart0000(committer.getWorkPath())); + + committer.abortJob(jobData.jContext, JobStatus.State.FAILED); + assertJobAbortCleanedUp(jobData); + + } + + /** + * Extension point: assert that the job was all cleaned up after an abort. + * Base assertions + *
    + *
  • Output dir is absent or, if present, empty
  • + *
+ * @param jobData job data + * @throws Exception failure + */ + public void assertJobAbortCleanedUp(JobData jobData) throws Exception { + FileSystem fs = getFileSystem(); + try { + FileStatus[] children = listChildren(fs, outputDir); + if (children.length != 0) { + lsR(fs, outputDir, true); + } + Assertions.assertThat(children) + .as("Output directory not empty " + ls(outputDir)) + .containsExactly(new FileStatus[0]); + } catch (FileNotFoundException e) { + // this is a valid state; it means the dest dir doesn't exist yet. + } + + } + + @Test + public void testFailAbort() throws Exception { + describe("Abort the task, then job (failed), abort the job again"); + JobData jobData = startJob(true); + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + ManifestCommitter committer = jobData.committer; + + // do abort + committer.abortTask(tContext); + + committer.getJobAttemptPath(jContext); + committer.getTaskAttemptPath(tContext); + assertPart0000DoesNotExist(outputDir); + assertSuccessMarkerDoesNotExist(outputDir); + describe("Aborting job into %s", outputDir); + + committer.abortJob(jContext, JobStatus.State.FAILED); + + assertTaskAttemptPathDoesNotExist(committer, tContext); + assertJobAttemptPathDoesNotExist(committer, jContext); + + // verify a failure report + ManifestSuccessData report = loadReport(jobData.jobId(), false); + Map diag = report.getDiagnostics(); + Assertions.assertThat(diag) + .describedAs("Stage entry in report") + .containsEntry(STAGE, OP_STAGE_JOB_ABORT); + IOStatisticsSnapshot reportStats = report.getIOStatistics(); + verifyStatisticCounterValue(reportStats, + OP_STAGE_JOB_ABORT, 1); + + // try again; expect abort to be idempotent. + committer.abortJob(jContext, JobStatus.State.FAILED); + + } + + /** + * Assert that the given dir does not have the {@code _SUCCESS} marker. + * @param dir dir to scan + * @throws IOException IO Failure + */ + protected void assertSuccessMarkerDoesNotExist(Path dir) throws IOException { + assertPathDoesNotExist("Success marker", + new Path(dir, SUCCESS_MARKER)); + } + + public void assertPart0000DoesNotExist(Path dir) throws Exception { + intercept(FileNotFoundException.class, + () -> getPart0000(dir)); + assertPathDoesNotExist("expected output file", new Path(dir, PART_00000)); + } + + @Test + public void testAbortJobNotTask() throws Exception { + executeWork("abort task no work", + (job, jContext, tContext, committer) -> { + // write output + writeTextOutput(tContext); + committer.abortJob(jContext, JobStatus.State.RUNNING); + assertTaskAttemptPathDoesNotExist( + committer, tContext); + assertJobAttemptPathDoesNotExist( + committer, jContext); + }); + } + + /** + * This looks at what happens with concurrent commits. + * However, the failure condition it looks for (subdir under subdir) + * is the kind of failure you see on a rename-based commit. + * + * What it will not detect is the fact that both tasks will each commit + * to the destination directory. That is: whichever commits last wins. + * + * There's no way to stop this. Instead it is a requirement that the task + * commit operation is only executed when the committer is happy to + * commit only those tasks which it knows have succeeded, and abort those + * which have not. + * @throws Exception failure + */ + @Test + public void testConcurrentCommitTaskWithSubDir() throws Exception { + Job job = newJob(); + FileOutputFormat.setOutputPath(job, outputDir); + final Configuration conf = job.getConfiguration(); + + final JobContext jContext = + new JobContextImpl(conf, taskAttempt0.getJobID()); + ManifestCommitter amCommitter = createCommitter( + new TaskAttemptContextImpl(conf, taskAttempt0)); + amCommitter.setupJob(jContext); + + final TaskAttemptContext[] taCtx = new TaskAttemptContextImpl[2]; + taCtx[0] = new TaskAttemptContextImpl(conf, taskAttempt0); + taCtx[1] = new TaskAttemptContextImpl(conf, taskAttempt1); + + // IDE/checkstyle complain here about type casting but they + // are confused. + final TextOutputFormat[] tof = + new TextOutputForTests[2]; + + for (int i = 0; i < tof.length; i++) { + tof[i] = new TextOutputForTests() { + @Override + public Path getDefaultWorkFile( + TaskAttemptContext context, + String extension) throws IOException { + final ManifestCommitter foc = (ManifestCommitter) + getOutputCommitter(context); + return new Path(new Path(foc.getWorkPath(), SUB_DIR), + getUniqueFile(context, getOutputName(context), extension)); + } + }; + } + + final ExecutorService executor = HadoopExecutors.newFixedThreadPool(2); + try { + for (int i = 0; i < taCtx.length; i++) { + final int taskIdx = i; + executor.submit(() -> { + final OutputCommitter outputCommitter = + tof[taskIdx].getOutputCommitter(taCtx[taskIdx]); + outputCommitter.setupTask(taCtx[taskIdx]); + writeOutput(tof[taskIdx].getRecordWriter(taCtx[taskIdx]), taCtx[taskIdx]); + describe("Committing Task %d", taskIdx); + outputCommitter.commitTask(taCtx[taskIdx]); + return null; + }); + } + } finally { + executor.shutdown(); + while (!executor.awaitTermination(1, TimeUnit.SECONDS)) { + log().info("Awaiting thread termination!"); + } + } + + // if we commit here then all tasks will be committed, so there will + // be contention for that final directory: both parts will go in. + + describe("\nCommitting Job"); + amCommitter.commitJob(jContext); + assertPathExists("base output directory", outputDir); + assertPart0000DoesNotExist(outputDir); + Path outSubDir = new Path(outputDir, SUB_DIR); + assertPathDoesNotExist("Must not end up with sub_dir/sub_dir", + new Path(outSubDir, SUB_DIR)); + + // validate output + // There's no success marker in the subdirectory + validateContent(outSubDir, false, ""); + } + + @Test + public void testUnsupportedSchema() throws Throwable { + intercept(PathIOException.class, () -> + new ManifestCommitterFactory() + .createOutputCommitter(new Path("s3a://unsupported/"), null)); + } + + /** + * Factory for failing committers. + */ + + +/* + protected ManifestCommitter createFailingCommitter( + final TaskAttemptContext tContext) + throws IOException { + // TODO + return null; + } + + public class FailingCommitterFactory implements CommitterFactory { + + @Override + public ManifestCommitter createCommitter(TaskAttemptContext context) + throws IOException { + return createFailingCommitter(context); + } + }*/ + @Test + public void testOutputFormatIntegration() throws Throwable { + Configuration conf = getConfiguration(); + Job job = newJob(); + assertCommitterFactoryIsManifestCommitter(job, outputDir); + job.setOutputFormatClass(TextOutputForTests.class); + conf = job.getConfiguration(); + conf.set(MRJobConfig.TASK_ATTEMPT_ID, attempt0); + conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1); + JobContext jContext = new JobContextImpl(conf, taskAttempt0.getJobID()); + TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, + taskAttempt0); + TextOutputForTests outputFormat = + (TextOutputForTests) + ReflectionUtils.newInstance(tContext.getOutputFormatClass(), conf); + ManifestCommitter committer = (ManifestCommitter) + outputFormat.getOutputCommitter(tContext); + + // setup + JobData jobData = new JobData(job, jContext, tContext, committer); + setupJob(jobData); + abortInTeardown(jobData); + TextOutputForTests.LoggingLineRecordWriter recordWriter + = outputFormat.getRecordWriter(tContext); + IntWritable iw = new IntWritable(1); + recordWriter.write(iw, iw); + long expectedLength = 4; + Path dest = recordWriter.getDest(); + validateTaskAttemptPathDuringWrite(dest, expectedLength); + recordWriter.close(tContext); + // at this point + validateTaskAttemptPathAfterWrite(dest, expectedLength); + Assertions.assertThat(committer.needsTaskCommit(tContext)) + .as("Committer does not have data to commit " + committer) + .isTrue(); + commitTask(committer, tContext); + // at this point the committer tasks stats should be current. + IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot( + committer.getIOStatistics()); + String commitsCompleted = COMMITTER_TASKS_COMPLETED_COUNT; + LOG.info("after task commit {}", ioStatisticsToPrettyString(snapshot)); + verifyStatisticCounterValue(snapshot, + commitsCompleted, 1); + final TaskManifest manifest = loadManifest( + committer.getTaskManifestPath(tContext)); + LOG.info("Manifest {}", manifest.toJson()); + + commitJob(committer, jContext); + LOG.info("committer iostatistics {}", + ioStatisticsSourceToString(committer)); + + // validate output + ManifestSuccessData successData = verifySuccessMarker(outputDir, + committer.getJobUniqueId()); + + // the task commit count should get through the job commit + IOStatisticsSnapshot successStats = successData.getIOStatistics(); + LOG.info("loaded statistics {}", successStats); + verifyStatisticCounterValue(successStats, + commitsCompleted, 1); + } + + /** + * Create a committer through reflection then use it to abort + * a task. This mimics the action of an AM when a container fails and + * the AM wants to abort the task attempt. + */ + @Test + public void testAMWorkflow() throws Throwable { + describe("Create a committer with a null output path & use as an AM"); + JobData jobData = startJob(true); + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + + TaskAttemptContext newAttempt = new TaskAttemptContextImpl( + jContext.getConfiguration(), + taskAttempt0); + Configuration conf = jContext.getConfiguration(); + + // bind + TextOutputForTests.bind(conf); + + OutputFormat outputFormat + = ReflectionUtils.newInstance(newAttempt.getOutputFormatClass(), conf); + Path outputPath = FileOutputFormat.getOutputPath(newAttempt); + Assertions.assertThat(outputPath) + .as("null output path in new task attempt") + .isNotNull(); + + ManifestCommitter committer2 = (ManifestCommitter) + outputFormat.getOutputCommitter(newAttempt); + committer2.abortTask(tContext); + + } + + /** + * Make sure that two jobs in parallel directory trees coexist. + * Note: the two jobs are not trying to write to the same + * output directory. + * That should be possible, but cleanup must be disabled. + */ + @Test + public void testParallelJobsToAdjacentPaths() throws Throwable { + + describe("Run two jobs in parallel, assert they both complete"); + JobData jobData = startJob(true); + Job job1 = jobData.job; + ManifestCommitter committer1 = jobData.committer; + JobContext jContext1 = jobData.jContext; + TaskAttemptContext tContext1 = jobData.tContext; + + // now build up a second job + String jobId2 = randomJobId(); + String attempt20 = "attempt_" + jobId2 + "_m_000000_0"; + TaskAttemptID taskAttempt20 = TaskAttemptID.forName(attempt20); + String attempt21 = "attempt_" + jobId2 + "_m_000001_0"; + TaskAttemptID taskAttempt21 = TaskAttemptID.forName(attempt21); + + Path job1Dest = outputDir; + Path job2Dest = new Path(getOutputDir().getParent(), + getMethodName() + "job2Dest"); + // little safety check + Assertions.assertThat(job2Dest) + .describedAs("Job destinations") + .isNotEqualTo(job1Dest); + + // create the second job + Job job2 = newJob(job2Dest, + unsetUUIDOptions(new JobConf(getConfiguration())), + attempt20); + Configuration conf2 = job2.getConfiguration(); + conf2.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1); + ManifestCommitter committer2 = null; + try { + JobContext jContext2 = new JobContextImpl(conf2, + taskAttempt20.getJobID()); + TaskAttemptContext tContext2 = + new TaskAttemptContextImpl(conf2, taskAttempt20); + committer2 = createCommitter(job2Dest, tContext2); + JobData jobData2 = new JobData(job2, jContext2, tContext2, committer2); + setupJob(jobData2); + abortInTeardown(jobData2); + // make sure the directories are different + Assertions.assertThat(committer1.getOutputPath()) + .describedAs("Committer output path of %s and %s", committer1, committer2) + .isNotEqualTo(committer2.getOutputPath()); + // and job IDs + Assertions.assertThat(committer1.getJobUniqueId()) + .describedAs("JobUnique IDs of %s and %s", committer1, committer2) + .isNotEqualTo(committer2.getJobUniqueId()); + + // job2 setup, write some data there + writeTextOutput(tContext2); + + // at this point, job1 and job2 both have uncommitted tasks + + // commit tasks in order task 2, task 1. + commitTask(committer2, tContext2); + commitTask(committer1, tContext1); + + // commit jobs in order job 1, job 2 + commitJob(committer1, jContext1); + + getPart0000(job1Dest); + + commitJob(committer2, jContext2); + getPart0000(job2Dest); + + } finally { + // clean things up in test failures. + FileSystem fs = getFileSystem(); + if (committer1 != null) { + fs.delete(committer1.getOutputPath(), true); + } + if (committer2 != null) { + fs.delete(committer2.getOutputPath(), true); + } + } + + } + + /** + * Strip staging/spark UUID options. + * @param conf config + * @return the patched config + */ + protected Configuration unsetUUIDOptions(final Configuration conf) { + conf.unset(SPARK_WRITE_UUID); + return conf; + } + + /** + * Assert that a committer's job attempt path exists. + * For the staging committers, this is in the cluster FS. + * @param committer committer + * @param jobContext job context + * @throws IOException failure + */ + protected void assertJobAttemptPathExists( + final ManifestCommitter committer, + final JobContext jobContext) throws IOException { + Path attemptPath = committer.getJobAttemptPath(jobContext); + ContractTestUtils.assertIsDirectory( + attemptPath.getFileSystem(committer.getConf()), + attemptPath); + } + + /** + * Validate the path of a file being written to during the write + * itself. + * @param p path + * @param expectedLength + * @throws IOException IO failure + */ + protected void validateTaskAttemptPathDuringWrite(Path p, + final long expectedLength) throws IOException { + + } + + /** + * Validate the path of a file being written to after the write + * operation has completed. + * @param p path + * @param expectedLength + * @throws IOException IO failure + */ + protected void validateTaskAttemptPathAfterWrite(Path p, + final long expectedLength) throws IOException { + + } + + /** + * Perform any actions needed to validate the working directory of + * a committer. + * For example: filesystem, path attributes + * @param committer committer instance + * @param context task attempt context + * @throws IOException IO failure + */ + protected void validateTaskAttemptWorkingDirectory( + ManifestCommitter committer, + TaskAttemptContext context) throws IOException { + } + + /** + * Commit a task then validate the state of the committer afterwards. + * @param committer committer + * @param tContext task context + * @throws IOException IO failure + */ + protected void commitTask(final ManifestCommitter committer, + final TaskAttemptContext tContext) throws IOException { + committer.commitTask(tContext); + } + + /** + * Commit a job then validate the state of the committer afterwards. + * @param committer committer + * @param jContext job context + * @throws IOException IO failure + */ + protected void commitJob(final ManifestCommitter committer, + final JobContext jContext) throws IOException { + committer.commitJob(jContext); + + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java new file mode 100644 index 0000000000000..b794f5814df36 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestRenameStageFailure.java @@ -0,0 +1,379 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.Assume; +import org.junit.Test; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.CommonPathCapabilities; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.UnreliableManifestStoreOperations; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.RenameFilesStage; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.StageConfig; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents; +import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.OP_COMMIT_FILE_RENAME; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.getEtag; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.UnreliableManifestStoreOperations.SIMULATED_FAILURE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.stages.AbstractJobOrTaskStage.FAILED_TO_RENAME_PREFIX; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test renaming files with fault injection. + * This explores etag support and overwrite-on-rename semantics + * of the target FS, so some of the tests behave differently + * on different stores. + */ +public class TestRenameStageFailure extends AbstractManifestCommitterTest { + + /** + * Statistic to look for. + */ + public static final String RENAME_FAILURES = OP_COMMIT_FILE_RENAME + ".failures"; + private static final int FAILING_FILE_INDEX = 5; + + /** + * Fault Injection. + */ + private UnreliableManifestStoreOperations failures; + + /** etags returned in listing/file status operations? */ + private boolean etagsSupported; + + /** etags preserved through rename? */ + private boolean etagsPreserved; + + /** resilient commit expected? */ + private boolean resilientCommit; + + protected boolean isResilientCommit() { + return resilientCommit; + } + + protected boolean isEtagsPreserved() { + return etagsPreserved; + } + + protected boolean isEtagsSupported() { + return etagsSupported; + } + + @Override + public void setup() throws Exception { + super.setup(); + final FileSystem fs = getFileSystem(); + final Path methodPath = methodPath(); + etagsSupported = fs.hasPathCapability(methodPath, + CommonPathCapabilities.ETAGS_AVAILABLE); + etagsPreserved = fs.hasPathCapability(methodPath, + CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME); + + final ManifestStoreOperations wrappedOperations = getStoreOperations(); + failures + = new UnreliableManifestStoreOperations(wrappedOperations); + setStoreOperations(failures); + resilientCommit = wrappedOperations.storeSupportsResilientCommit(); + } + + /** + * Does this test suite require rename resilience in the store/FS? + * @return true if the store operations are resilient. + */ + protected boolean requireRenameResilience() { + return false; + } + + @Test + public void testResilienceAsExpected() throws Throwable { + Assertions.assertThat(isResilientCommit()) + .describedAs("resilient commit support") + .isEqualTo(requireRenameResilience()); + } + + @Test + public void testRenameSourceException() throws Throwable { + describe("rename fails raising an IOE -expect stage to fail" + + " and exception message preserved"); + + // destination directory. + Path destDir = methodPath(); + StageConfig stageConfig = createStageConfigForJob(JOB1, destDir); + Path jobAttemptTaskSubDir = stageConfig.getJobAttemptTaskSubDir(); + + // create a manifest with a lot of files, but for + // which one of whose renames will fail + TaskManifest manifest = new TaskManifest(); + createFileset(destDir, jobAttemptTaskSubDir, manifest, filesToCreate()); + final List filesToCommit = manifest.getFilesToCommit(); + final FileEntry entry = filesToCommit.get(FAILING_FILE_INDEX); + failures.addRenameSourceFilesToFail(entry.getSourcePath()); + + // rename MUST fail + expectRenameFailure( + new RenameFilesStage(stageConfig), + manifest, + filesToCommit.size(), + SIMULATED_FAILURE, + PathIOException.class); + } + + /** + * Number of files to create; must be more than + * {@link #FAILING_FILE_INDEX}. + */ + protected int filesToCreate() { + return 100; + } + + @Test + public void testCommitMissingFile() throws Throwable { + describe("commit a file which doesn't exist. Expect FNFE always"); + // destination directory. + Path destDir = methodPath(); + StageConfig stageConfig = createStageConfigForJob(JOB1, destDir); + Path jobAttemptTaskSubDir = stageConfig.getJobAttemptTaskSubDir(); + TaskManifest manifest = new TaskManifest(); + final List filesToCommit = manifest.getFilesToCommit(); + + Path source = new Path(jobAttemptTaskSubDir, "source.parquet"); + Path dest = new Path(destDir, "destdir.parquet"); + filesToCommit.add(new FileEntry(source, dest, 0, null)); + final FileNotFoundException ex = expectRenameFailure( + new RenameFilesStage(stageConfig), + manifest, + 0, + "", + FileNotFoundException.class); + LOG.info("Exception raised: {}", ex.toString()); + } + + /** + * Verify that when a job is configured to delete target paths, + * renaming will overwrite them. + * This test has to use FileSystem contract settings to determine + * whether or not the FS will actually permit file-over-file rename. + * As POSIX does, local filesystem tests will not fail if the + * destination exists. + * As ABFS and GCS do reject it, they are required to fail the + * first rename sequence, but succeed once delete.target.paths + * is true. + */ + @Test + public void testDeleteTargetPaths() throws Throwable { + describe("Verify that target path deletion works"); + // destination directory. + Path destDir = methodPath(); + StageConfig stageConfig = createStageConfigForJob(JOB1, destDir) + .withDeleteTargetPaths(true); + Path jobAttemptTaskSubDir = stageConfig.getJobAttemptTaskSubDir(); + final Path source = new Path(jobAttemptTaskSubDir, "source.txt"); + final Path dest = new Path(destDir, "source.txt"); + final byte[] sourceData = "data".getBytes(StandardCharsets.UTF_8); + final FileSystem fs = getFileSystem(); + ContractTestUtils.createFile(fs, source, false, sourceData); + touch(fs, dest); + TaskManifest manifest = new TaskManifest(); + final FileEntry entry = createEntryWithEtag(source, dest); + manifest.addFileToCommit(entry); + + List manifests = new ArrayList<>(); + manifests.add(manifest); + + // local POSIX filesystems allow rename of file onto file, so + // don't fail on the rename. + boolean renameOverwritesDest = isSupported(RENAME_OVERWRITES_DEST); + + if (!renameOverwritesDest) { + // HDFS, ABFS and GCS do all reject rename of file onto file. + // ABFS will use its rename operation so will even raise a + // meaningful exception here. + final IOException ex = expectRenameFailure( + new RenameFilesStage(stageConfig.withDeleteTargetPaths(false)), + manifest, + 0, + "", + IOException.class); + LOG.info("Exception raised: {}", ex.toString()); + } + + // delete target paths and it works + new RenameFilesStage(stageConfig.withDeleteTargetPaths(true)) + .apply(Pair.of(manifests, Collections.emptySet())); + + // and the new data made it over + verifyFileContents(fs, dest, sourceData); + + // lets check the etag too, for completeness + if (isEtagsPreserved()) { + Assertions.assertThat(getEtag(fs.getFileStatus(dest))) + .describedAs("Etag of destination file %s", dest) + .isEqualTo(entry.getEtag()); + } + + } + + @Test + public void testRenameReturnsFalse() throws Throwable { + describe("commit where rename() returns false for one file." + + " Expect failure to be escalated to an IOE"); + + Assume.assumeTrue("not used when resilient commits are available", + !resilientCommit); + // destination directory. + Path destDir = methodPath(); + StageConfig stageConfig = createStageConfigForJob(JOB1, destDir); + Path jobAttemptTaskSubDir = stageConfig.getJobAttemptTaskSubDir(); + + // create a manifest with a lot of files, but for + // which one of whose renames will fail + TaskManifest manifest = new TaskManifest(); + createFileset(destDir, jobAttemptTaskSubDir, manifest, filesToCreate()); + + final List filesToCommit = manifest.getFilesToCommit(); + final FileEntry entry = filesToCommit.get(FAILING_FILE_INDEX); + failures.addRenameSourceFilesToFail(entry.getSourcePath()); + + // switch to rename returning false.; again, this must + // be escalated to a failure. + failures.setRenameToFailWithException(false); + expectRenameFailure( + new RenameFilesStage(stageConfig), + manifest, + filesToCommit.size(), + FAILED_TO_RENAME_PREFIX, + PathIOException.class); + } + + /** + * Create the source files for a task. + * @param destDir destination directory + * @param taskAttemptDir directory of the task attempt + * @param manifest manifest to update. + * @param fileCount how many files. + */ + private void createFileset( + final Path destDir, + final Path taskAttemptDir, + final TaskManifest manifest, + final int fileCount) throws IOException { + final FileSystem fs = getFileSystem(); + for (int i = 0; i < fileCount; i++) { + String name = String.format("file%04d", i); + Path src = new Path(taskAttemptDir, name); + Path dest = new Path(destDir, name); + touch(fs, src); + + final FileEntry entry = createEntryWithEtag(src, dest); + manifest.addFileToCommit(entry); + } + } + + /** + * Create a manifest entry, including size. + * If the FS supports etags, one is retrieved. + * @param source source + * @param dest dest + * @return entry + * @throws IOException if getFileStatus failed. + */ + private FileEntry createEntryWithEtag(final Path source, + final Path dest) + throws IOException { + final FileStatus st = getFileSystem().getFileStatus(source); + final String etag = isEtagsSupported() + ? getEtag(st) + : null; + + return new FileEntry(source, dest, st.getLen(), etag); + } + + /** + * Execute rename, expecting a failure. + * The number of files renamed MUST be less than the value of {@code files} + * @param stage stage + * @param manifest task manifests + * @param files number of files being renamed. + * @param errorText text which must be in the exception string + * @param exceptionClass class of the exception + * @return the caught exception + * @throws Exception if anything else went wrong, or no exception was raised. + */ + private E expectRenameFailure( + RenameFilesStage stage, + TaskManifest manifest, + int files, + String errorText, + Class exceptionClass) throws Exception { + + List manifests = new ArrayList<>(); + manifests.add(manifest); + ProgressCounter progressCounter = getProgressCounter(); + progressCounter.reset(); + IOStatisticsStore iostatistics = stage.getIOStatistics(); + long failures0 = iostatistics.counters().get(RENAME_FAILURES); + + // rename MUST raise an exception. + E ex = intercept(exceptionClass, errorText, () -> + stage.apply(Pair.of(manifests, Collections.emptySet()))); + + LOG.info("Statistics {}", ioStatisticsToPrettyString(iostatistics)); + // the IOStatistics record the rename as a failure. + assertThatStatisticCounter(iostatistics, RENAME_FAILURES) + .isEqualTo(failures0 + 1); + + // count of files committed MUST be less than expected. + if (files > 0) { + + Assertions.assertThat(stage.getFilesCommitted()) + .describedAs("Files Committed by stage") + .isNotEmpty() + .hasSizeLessThan(files); + + } + + // the progress counter will show that the rename did invoke it. + // there's no assertion on the actual value as it depends on + // execution time of the threads. + + Assertions.assertThat(progressCounter.value()) + .describedAs("Progress counter %s", progressCounter) + .isGreaterThan(0); + return ex; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestTaskManifestFileIO.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestTaskManifestFileIO.java new file mode 100644 index 0000000000000..3bd75e5750c75 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TestTaskManifestFileIO.java @@ -0,0 +1,185 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.IOException; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DirEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.JsonSerialization; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.assertDirEntryMatch; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.assertFileEntryMatch; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.marshallPath; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData.unmarshallPath; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Test Reading/writing manifest files. + */ +public class TestTaskManifestFileIO extends AbstractManifestCommitterTest { + + private TaskManifest source; + + private ManifestCommitterTestSupport.JobAndTaskIDsForTests taskIDs; + + private String taskAttempt00; + + private Path testPath; + + private Path taPath; + + @Override + public void setup() throws Exception { + super.setup(); + taskIDs = new ManifestCommitterTestSupport.JobAndTaskIDsForTests(2, 2); + source = new TaskManifest(); + taskAttempt00 = taskIDs.getTaskAttempt(0, 0); + source.setTaskAttemptID(taskAttempt00); + testPath = methodPath(); + taPath = new Path(testPath, " " + taskAttempt00); + source.setTaskAttemptDir(marshallPath(taPath)); + } + + /** + * Test marshalling, paying attention to paths with spaces in them + * as they've been a source of trouble in the S3A committers. + */ + @Test + public void testJsonRoundTrip() throws Throwable { + describe("Save manifest file to string and back"); + Path subdirS = new Path(taPath, "subdir"); + Path subdirD = new Path(testPath, "subdir"); + source.addDirectory(DirEntry.dirEntry(subdirD, 0, 0)); + + // a file + Path subfileS = new Path(subdirS, "file"); + Path subfileD = new Path(subdirD, "file"); + long len = 256L; + FileEntry subFileEntry = new FileEntry(subfileS, + subfileD, len, "etag"); + source.addFileToCommit(subFileEntry); + + + JsonSerialization serializer + = TaskManifest.serializer(); + + String json = serializer.toJson(source); + LOG.info("serialized form\n{}", json); + TaskManifest deser = serializer.fromJson(json); + deser.validate(); + + Assertions.assertThat(deser.getTaskAttemptID()) + .describedAs("Task attempt ID") + .isEqualTo(taskAttempt00); + + Assertions.assertThat(unmarshallPath(deser.getTaskAttemptDir())) + .describedAs("Task attempt Dir %s", + deser.getTaskAttemptDir()) + .isEqualTo(taPath); + + Assertions.assertThat(deser.getDestDirectories()) + .hasSize(1) + .allSatisfy(d -> assertDirEntryMatch(d, subdirD, 0)); + Assertions.assertThat(deser.getFilesToCommit()) + .hasSize(1) + .allSatisfy(d -> assertFileEntryMatch(d, subfileS, subfileD, len)); + final FileEntry entry = deser.getFilesToCommit().get(0); + assertFileEntryMatch(entry, subfileS, subfileD, len); + Assertions.assertThat(entry.getEtag()) + .describedAs("etag of %s", entry) + .isEqualTo("etag"); + + } + + /** + * The manifest validation logic has a safety check that only one + * file can rename to the same destination, and that the entries + * are valid. + */ + @Test + public void testValidateRejectsTwoCommitsToSameDest() throws Throwable { + + Path subdirS = new Path(taPath, "subdir"); + Path subdirD = new Path(testPath, "subdir"); + source.addDirectory(DirEntry.dirEntry(subdirD, 0, 0)); + + // a file + Path subfileS = new Path(subdirS, "file"); + Path subfileS2 = new Path(subdirS, "file2"); + Path subfileD = new Path(subdirD, "file"); + long len = 256L; + source.addFileToCommit( + new FileEntry(subfileS, subfileD, len, "tag1")); + source.addFileToCommit( + new FileEntry(subfileS2, subfileD, len, "tag2")); + assertValidationFailureOnRoundTrip(source); + } + + /** + * The manifest validation logic has a safety check that only one + * file can rename to the same destination, and that the entries + * are valid. + */ + @Test + public void testValidateRejectsIncompleteFileEntry() throws Throwable { + source.addFileToCommit( + new FileEntry(taPath, null, 0, null)); + assertValidationFailureOnRoundTrip(source); + } + + /** + * negative lengths are not allowed. + */ + @Test + public void testValidateRejectsInvalidFileLength() throws Throwable { + source.addFileToCommit( + new FileEntry(taPath, testPath, -1, null)); + assertValidationFailureOnRoundTrip(source); + } + + @Test + public void testRejectIncompatibleVersion() throws Throwable { + source.setVersion(5); + assertValidationFailureOnRoundTrip(source); + } + + @Test + public void testRejectIncompatibleType() throws Throwable { + source.setType("Incompatible type"); + assertValidationFailureOnRoundTrip(source); + } + + private void assertValidationFailureOnRoundTrip( + final TaskManifest manifest) throws Exception { + JsonSerialization serializer + = TaskManifest.serializer(); + String json = serializer.toJson(manifest); + LOG.info("serialized form\n{}", json); + TaskManifest deser = serializer.fromJson(json); + intercept(IOException.class, deser::validate); + } + + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TextOutputForTests.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TextOutputForTests.java new file mode 100644 index 0000000000000..14c57c451ab86 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/TextOutputForTests.java @@ -0,0 +1,136 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.DataOutputStream; +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * A subclass of {@link TextOutputFormat} which logs what is happening, and + * returns a {@link LoggingLineRecordWriter} which allows the caller + * to get the destination path. + * @param key + * @param value + */ +public class TextOutputForTests extends TextOutputFormat { + + private static final Logger LOG = + LoggerFactory.getLogger(TextOutputForTests.class); + + public static final String NAME + = "org.apache.hadoop.fs.s3a.commit.LoggingTextOutputFormat"; + + @Override + public LoggingLineRecordWriter getRecordWriter(TaskAttemptContext job) + throws IOException, InterruptedException { + Configuration conf = job.getConfiguration(); + boolean isCompressed = getCompressOutput(job); + String keyValueSeparator = conf.get(SEPARATOR, "\t"); + CompressionCodec codec = null; + String extension = ""; + if (isCompressed) { + Class codecClass = + getOutputCompressorClass(job, GzipCodec.class); + codec = ReflectionUtils.newInstance(codecClass, conf); + extension = codec.getDefaultExtension(); + } + Path file = getDefaultWorkFile(job, extension); + FileSystem fs = file.getFileSystem(conf); + FSDataOutputStream fileOut = fs.create(file, true); + LOG.debug("Creating LineRecordWriter with destination {}", file); + if (isCompressed) { + return new LoggingLineRecordWriter<>( + file, new DataOutputStream(codec.createOutputStream(fileOut)), + keyValueSeparator); + } else { + return new LoggingLineRecordWriter<>(file, fileOut, keyValueSeparator); + } + } + + /** + * Write a line; counts the number of lines written and logs @ debug in the + * {@code close()} call. + * @param key + * @param value + */ + public static class LoggingLineRecordWriter + extends LineRecordWriter { + + private final Path dest; + + private long lines; + + public LoggingLineRecordWriter(Path dest, DataOutputStream out, + String keyValueSeparator) { + super(out, keyValueSeparator); + this.dest = dest; + } + + public LoggingLineRecordWriter(DataOutputStream out, Path dest) { + super(out); + this.dest = dest; + } + + @Override + public synchronized void write(K key, V value) throws IOException { + super.write(key, value); + lines++; + } + + public synchronized void close(TaskAttemptContext context) + throws IOException { + LOG.debug("Closing output file {} with {} lines :{}", + dest, lines, out); + out.close(); + } + + public Path getDest() { + return dest; + } + + public long getLines() { + return lines; + } + } + + /** + * Bind to a configuration for job submission. + * @param conf configuration + */ + public static void bind(Configuration conf) { + conf.setClass(MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR, + TextOutputForTests.class, + OutputFormat.class); + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ThreadLeakTracker.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ThreadLeakTracker.java new file mode 100644 index 0000000000000..4245ef3aabddb --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ThreadLeakTracker.java @@ -0,0 +1,83 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.util.Set; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import org.assertj.core.api.Assertions; + +/** + * Thread Leak tracker. + * Initialized with the current set of threads and some known-long-life + * threads whose presence must not be considered a failure. + */ +public class ThreadLeakTracker { + + private final Set trackedThreads = + listInitialThreadsForLifecycleChecks(); + + public void assertNoThreadLeakage() { + Assertions.assertThat(getCurrentThreadNames()) + .describedAs("The threads at the end of the test run") + .isSubsetOf(trackedThreads); + } + + public Set getTrackedThreads() { + return trackedThreads; + } + + /** + * Get a set containing the names of all active threads, + * stripping out all test runner threads. + * @return the current set of threads. + */ + public static Set getCurrentThreadNames() { + TreeSet threads = Thread.getAllStackTraces().keySet() + .stream() + .map(Thread::getName) + .filter(n -> n.startsWith("JUnit")) + .filter(n -> n.startsWith("surefire")) + .collect(Collectors.toCollection(TreeSet::new)); + return threads; + } + + /** + * This creates a set containing all current threads and some well-known + * thread names whose existence should not fail test runs. + * They are generally static cleaner threads created by various classes + * on instantiation. + * @return a set of threads to use in later assertions. + */ + public static Set listInitialThreadsForLifecycleChecks() { + Set threadSet = getCurrentThreadNames(); + // static filesystem statistics cleaner + threadSet.add("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner"); + + // java.lang.UNIXProcess. maybe if chmod is called? + threadSet.add("process reaper"); + // once a quantile has been scheduled, the mutable quantile thread pool + // is initialized; it has a minimum thread size of 1. + threadSet.add("MutableQuantiles-0"); + // IDE? + threadSet.add("Attach Listener"); + return threadSet; + } +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/UnreliableManifestStoreOperations.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/UnreliableManifestStoreOperations.java new file mode 100644 index 0000000000000..811fc704a2a33 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/UnreliableManifestStoreOperations.java @@ -0,0 +1,380 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest.impl; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.HashSet; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.AbstractManifestData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.util.JsonSerialization; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.OPERATION_TIMED_OUT; + +/** + * Wrap an existing {@link ManifestStoreOperations} implementation and fail on + * specific paths. + * This is for testing. It could be implemented via + * Mockito 2 spy code but is not so that: + * 1. It can be backported to Hadoop versions using Mockito 1.x. + * 2. It can be extended to use in production. This is why it is in + * the production module -to allow for downstream tests to adopt it. + * 3. You can actually debug what's going on. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class UnreliableManifestStoreOperations extends ManifestStoreOperations { + + private static final Logger LOG = LoggerFactory.getLogger( + UnreliableManifestStoreOperations.class); + + /** + * The timeout message ABFS raises. + */ + public static final String E_TIMEOUT + = "Operation could not be completed within the specified time"; + + /** + * Text to use in simulated failure exceptions. + */ + public static final String SIMULATED_FAILURE = "Simulated failure"; + + /** + * Underlying store operations to wrap. + */ + private final ManifestStoreOperations wrappedOperations; + + /** + * Paths of delete operations to fail. + */ + private final Set deletePathsToFail = new HashSet<>(); + + /** + * Paths of delete operations to time out, as ABFS may. + */ + private final Set deletePathsToTimeOut = new HashSet<>(); + + /** + * Paths of List operations to fail. + */ + private final Set listToFail = new HashSet<>(); + + /** + * Paths of mkdirs operations to fail. + */ + private final Set mkdirsToFail = new HashSet<>(); + + /** + * Paths which don't exist. + */ + private final Set pathNotFound = new HashSet<>(); + + /** + * Source file whose rename/commit will fail. + */ + private final Set renameSourceFilesToFail = new HashSet<>(); + + /** + * Dest dir into which all renames/commits will fail. + * Subdirectories under this are not checked. + */ + private final Set renameDestDirsToFail = new HashSet<>(); + + /** + * Path of save() to fail. + */ + private final Set saveToFail = new HashSet<>(); + + /** + * timeout sleep. + */ + private int timeoutSleepTimeMillis; + + /** + * Should rename thrown an exception or just return false. + */ + private boolean renameToFailWithException = true; + + /** + * Constructor. + * @param wrappedOperations operations to wrap. + */ + public UnreliableManifestStoreOperations(final ManifestStoreOperations wrappedOperations) { + this.wrappedOperations = wrappedOperations; + } + + + /** + * Reset everything. + */ + public void reset() { + deletePathsToFail.clear(); + deletePathsToTimeOut.clear(); + pathNotFound.clear(); + renameSourceFilesToFail.clear(); + renameDestDirsToFail.clear(); + timeoutSleepTimeMillis = 0; + } + + public int getTimeoutSleepTimeMillis() { + return timeoutSleepTimeMillis; + } + + public void setTimeoutSleepTimeMillis(final int timeoutSleepTimeMillis) { + this.timeoutSleepTimeMillis = timeoutSleepTimeMillis; + } + + public boolean getRenameToFailWithException() { + return renameToFailWithException; + } + + public void setRenameToFailWithException( + final boolean renameToFailWithException) { + this.renameToFailWithException = renameToFailWithException; + } + + /** + * Add a path to the list of delete paths to fail. + * @param path path to add. + */ + public void addDeletePathToFail(Path path) { + deletePathsToFail.add(requireNonNull(path)); + } + + /** + * Add a path to the list of delete paths to time out. + * @param path path to add. + */ + public void addDeletePathToTimeOut(Path path) { + deletePathsToTimeOut.add(requireNonNull(path)); + } + + /** + * Add a path to the list of paths where list will fail. + * @param path path to add. + */ + public void addListToFail(Path path) { + listToFail.add(requireNonNull(path)); + } + + /** + * Add a path to the list of mkdir calls to fail. + * @param path path to add. + */ + public void addMkdirsToFail(Path path) { + mkdirsToFail.add(requireNonNull(path)); + } + + /** + * Add a path not found. + * @param path path + */ + public void addPathNotFound(Path path) { + pathNotFound.add(requireNonNull(path)); + } + + /** + * Add a path to the list of rename source paths to fail. + * @param path path to add. + */ + public void addRenameSourceFilesToFail(Path path) { + renameSourceFilesToFail.add(requireNonNull(path)); + } + + /** + * Add a path to the list of dest dirs to fail. + * @param path path to add. + */ + public void addRenameDestDirsFail(Path path) { + renameDestDirsToFail.add(requireNonNull(path)); + } + + /** + * Add a path to the list of paths where save will fail. + * @param path path to add. + */ + public void addSaveToFail(Path path) { + saveToFail.add(requireNonNull(path)); + } + + /** + * Raise an exception if the path is in the set of target paths. + * @param operation operation which failed. + * @param path path to check + * @param paths paths to probe for {@code path} being in. + * @throws IOException simulated failure + */ + private void maybeRaiseIOE(String operation, Path path, Set paths) + throws IOException { + if (paths.contains(path)) { + LOG.info("Simulating failure of {} with {}", operation, path); + throw new PathIOException(path.toString(), + SIMULATED_FAILURE + " of " + operation); + } + } + + /** + * Verify that a path is not on the file not found list. + * @param path path + * @throws FileNotFoundException if configured to fail. + */ + private void verifyExists(Path path) throws FileNotFoundException { + if (pathNotFound.contains(path)) { + throw new FileNotFoundException(path.toString()); + } + } + + /** + * Time out if the path is in the list of timeout paths. + * Will sleep first, to help simulate delays. + * @param operation operation which failed. + * @param path path to check + * @param paths paths to probe for {@code path} being in. + * @throws IOException simulated timeout + */ + private void maybeTimeout(String operation, Path path, Set paths) + throws IOException { + if (paths.contains(path)) { + LOG.info("Simulating timeout of {} with {}", operation, path); + try { + if (timeoutSleepTimeMillis > 0) { + Thread.sleep(timeoutSleepTimeMillis); + } + } catch (InterruptedException e) { + throw new InterruptedIOException(e.toString()); + } + throw new PathIOException(path.toString(), + "ErrorCode=" + OPERATION_TIMED_OUT + + " ErrorMessage=" + E_TIMEOUT); + } + } + + @Override + public FileStatus getFileStatus(final Path path) throws IOException { + verifyExists(path); + return wrappedOperations.getFileStatus(path); + } + + @Override + public boolean delete(final Path path, final boolean recursive) + throws IOException { + String op = "delete"; + maybeTimeout(op, path, deletePathsToTimeOut); + maybeRaiseIOE(op, path, deletePathsToFail); + return wrappedOperations.delete(path, recursive); + } + + @Override + public boolean mkdirs(final Path path) throws IOException { + maybeRaiseIOE("mkdirs", path, mkdirsToFail); + return wrappedOperations.mkdirs(path); + } + + @Override + public boolean renameFile(final Path source, final Path dest) + throws IOException { + String op = "rename"; + if (renameToFailWithException) { + maybeRaiseIOE(op, source, renameSourceFilesToFail); + maybeRaiseIOE(op, dest.getParent(), renameDestDirsToFail); + } else { + if (renameSourceFilesToFail.contains(source) + || renameDestDirsToFail.contains(dest.getParent())) { + LOG.info("Failing rename({}, {})", source, dest); + return false; + } + } + return wrappedOperations.renameFile(source, dest); + } + + @Override + public RemoteIterator listStatusIterator(final Path path) + throws IOException { + verifyExists(path); + maybeRaiseIOE("listStatus", path, listToFail); + return wrappedOperations.listStatusIterator(path); + } + + @Override + public TaskManifest loadTaskManifest(JsonSerialization serializer, + final FileStatus st) throws IOException { + verifyExists(st.getPath()); + return wrappedOperations.loadTaskManifest(serializer, st); + } + + @Override + public > void save(T manifestData, + final Path path, + final boolean overwrite) throws IOException { + maybeRaiseIOE("save", path, saveToFail); + wrappedOperations.save(manifestData, path, overwrite); + } + + @Override + public void msync(Path path) throws IOException { + wrappedOperations.msync(path); + } + + @Override + public String getEtag(FileStatus status) { + return wrappedOperations.getEtag(status); + } + + @Override + public boolean storeSupportsResilientCommit() { + return wrappedOperations.storeSupportsResilientCommit(); + } + + @Override + public CommitFileResult commitFile(final FileEntry entry) + throws IOException { + if (renameToFailWithException) { + maybeRaiseIOE("commitFile", + entry.getSourcePath(), renameSourceFilesToFail); + maybeRaiseIOE("commitFile", + entry.getDestPath().getParent(), renameDestDirsToFail); + } + return wrappedOperations.commitFile(entry); + } + + @Override + public boolean storePreservesEtagsThroughRenames(Path path) { + return wrappedOperations.storePreservesEtagsThroughRenames(path); + } + + @Override + public void close() throws IOException { + wrappedOperations.close(); + } + +} diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java new file mode 100644 index 0000000000000..61c4073e1bdb6 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/package-info.java @@ -0,0 +1,29 @@ +/* + * 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. + */ + +/** + * These test suites are subclassed in internal modules (hadoop-azure) and + * externally for gcs integration testing. + * Do test with both before making changes. + */ +@InterfaceAudience.LimitedPrivate("integration tests") +@InterfaceStability.Unstable +package org.apache.hadoop.mapreduce.lib.output.committer.manifest; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/contract/localfs.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/contract/localfs.xml new file mode 100644 index 0000000000000..e956a50e8d307 --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/contract/localfs.xml @@ -0,0 +1,138 @@ + + + + + + + + fs.contract.is-case-sensitive + true + + + + + fs.contract.supports-unix-permissions + true + + + + + + fs.contract.test.root-tests-enabled + false + + + + fs.contract.test.random-seek-count + 1000 + + + + fs.contract.rename-creates-dest-dirs + true + + + + fs.contract.rename-overwrites-dest + true + + + + fs.contract.rename-remove-dest-if-empty-dir + true + + + + + fs.contract.supports-append + false + + + + fs.contract.supports-atomic-directory-delete + true + + + + fs.contract.supports-atomic-rename + true + + + + fs.contract.supports-block-locality + false + + + + fs.contract.supports-concat + false + + + + fs.contract.supports-seek + true + + + + fs.contract.supports-seek-on-closed-file + true + + + + + fs.contract.rejects-seek-past-eof + true + + + + fs.contract.supports-strict-exceptions + false + + + + fs.contract.supports-settimes + true + + + + fs.contract.supports-getfilestatus + true + + + + fs.contract.supports-settimes + true + + + + fs.contract.supports-getfilestatus + true + + + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/core-site.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/core-site.xml new file mode 100644 index 0000000000000..9312483e6efba --- /dev/null +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/resources/core-site.xml @@ -0,0 +1,33 @@ + + + + + + + + + + hadoop.tmp.dir + target/build/test + A base for other temporary directories. + false + + + From 8294bd5a37c0de15af576700c6cba46791eddd07 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 16 Mar 2022 15:41:03 +0000 Subject: [PATCH 042/145] HADOOP-18163. hadoop-azure support for the Manifest Committer of MAPREDUCE-7341 Follow-on patch to MAPREDUCE-7341, adding ABFS support and tests * resilient rename * tests for job commit through the manifest committer. contains - HADOOP-17976. ABFS etag extraction inconsistent between LIST and HEAD calls - HADOOP-16204. ABFS tests to include terasort Contributed by Steve Loughran. Change-Id: I0a7d4043bdf19bcb00c033fc389730109b93b77f --- hadoop-project/pom.xml | 7 + hadoop-tools/hadoop-azure/pom.xml | 81 +++- .../src/config/checkstyle-suppressions.xml | 3 + .../hadoop/fs/azurebfs/AbfsConfiguration.java | 9 + .../fs/azurebfs/AzureBlobFileSystem.java | 197 +++++++--- .../fs/azurebfs/AzureBlobFileSystemStore.java | 34 +- .../commit/AbfsManifestStoreOperations.java | 130 +++++++ .../commit/AzureManifestCommitterFactory.java | 58 +++ .../commit/ResilientCommitByRename.java | 101 +++++ .../fs/azurebfs/commit/package-info.java | 28 ++ .../azurebfs/constants/ConfigurationKeys.java | 3 + .../constants/FileSystemConfigurations.java | 5 + .../fs/azurebfs/services/AbfsClient.java | 99 ++++- .../azurebfs/services/SimpleKeyProvider.java | 10 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 11 +- .../azurebfs/ITestAbfsReadWriteAndSeek.java | 13 +- ...ITestAzureBlobFileSystemDelegationSAS.java | 5 +- .../fs/azurebfs/ITestCustomerProvidedKey.java | 3 +- .../azurebfs/commit/AbfsCommitTestHelper.java | 49 +++ .../commit/AbstractAbfsClusterITest.java | 260 +++++++++++++ .../commit/ITestAbfsCleanupStage.java | 54 +++ .../commit/ITestAbfsCommitTaskStage.java | 54 +++ ...ITestAbfsCreateOutputDirectoriesStage.java | 54 +++ .../ITestAbfsJobThroughManifestCommitter.java | 101 +++++ .../commit/ITestAbfsLoadManifestsStage.java | 55 +++ .../ITestAbfsManifestCommitProtocol.java | 62 +++ .../ITestAbfsManifestStoreOperations.java | 175 +++++++++ .../commit/ITestAbfsRenameStageFailure.java | 69 ++++ .../commit/ITestAbfsTaskManifestFileIO.java | 54 +++ .../fs/azurebfs/commit/ITestAbfsTerasort.java | 353 ++++++++++++++++++ .../fs/azurebfs/commit/package-info.java | 24 ++ .../contract/AbfsFileSystemContract.java | 2 +- .../src/test/resources/core-site.xml | 25 ++ 33 files changed, 2106 insertions(+), 82 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AzureManifestCommitterFactory.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/ResilientCommitByRename.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbstractAbfsClusterITest.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCleanupStage.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCommitTaskStage.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCreateOutputDirectoriesStage.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsJobThroughManifestCommitter.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestCommitProtocol.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestStoreOperations.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsRenameStageFailure.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTaskManifestFileIO.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTerasort.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java create mode 100644 hadoop-tools/hadoop-azure/src/test/resources/core-site.xml diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 193f898fc7f66..80e07f0ac1d7a 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -384,6 +384,13 @@ ${hadoop.version}
+ + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + test-jar + + org.apache.hadoop hadoop-mapreduce-client-jobclient diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml index a9f58e39c6a4b..40aeec07026cc 100644 --- a/hadoop-tools/hadoop-azure/pom.xml +++ b/hadoop-tools/hadoop-azure/pom.xml @@ -219,6 +219,67 @@ test + + + org.apache.hadoop + hadoop-mapreduce-client-core + provided + + + + org.apache.hadoop + hadoop-mapreduce-client-core + test + test-jar + + + org.apache.hadoop + hadoop-yarn-server-tests + test + test-jar + + + org.apache.hadoop + hadoop-mapreduce-client-hs + test + + + org.apache.hadoop + hadoop-mapreduce-examples + test + jar + + + + org.apache.hadoop + hadoop-mapreduce-client-app + test + + + org.apache.hadoop + hadoop-mapreduce-client-app + test-jar + test + + + org.apache.hadoop + hadoop-mapreduce-client-jobclient + test + test-jar + org.apache.hadoop hadoop-distcp @@ -319,7 +380,7 @@ ${test.build.data}/${surefire.forkNumber} ${test.build.dir}/${surefire.forkNumber} ${hadoop.tmp.dir}/${surefire.forkNumber} - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.huge.filesize} ${fs.azure.scale.test.huge.partitionsize} @@ -350,7 +411,7 @@ ${test.build.data}/${surefire.forkNumber} ${test.build.dir}/${surefire.forkNumber} ${hadoop.tmp.dir}/${surefire.forkNumber} - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.huge.filesize} ${fs.azure.scale.test.huge.partitionsize} @@ -392,7 +453,7 @@ - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.huge.filesize} @@ -482,7 +543,7 @@ ${test.build.data}/${surefire.forkNumber} ${test.build.dir}/${surefire.forkNumber} ${hadoop.tmp.dir}/${surefire.forkNumber} - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.huge.filesize} ${fs.azure.scale.test.huge.partitionsize} @@ -526,7 +587,7 @@ - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.timeout} @@ -544,6 +605,7 @@ **/azurebfs/extensions/ITestAbfsDelegationTokens.java **/azurebfs/ITestSmallWriteOptimization.java **/azurebfs/services/ITestReadBufferManager.java + **/azurebfs/commit/*.java @@ -572,7 +634,7 @@ - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.timeout} @@ -585,6 +647,7 @@ **/azurebfs/extensions/ITestAbfsDelegationTokens.java **/azurebfs/ITestSmallWriteOptimization.java **/azurebfs/services/ITestReadBufferManager.java + **/azurebfs/commit/*.java @@ -634,7 +697,7 @@ ${test.build.data}/${surefire.forkNumber} ${test.build.dir}/${surefire.forkNumber} ${hadoop.tmp.dir}/${surefire.forkNumber} - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.huge.filesize} ${fs.azure.scale.test.huge.partitionsize} @@ -664,7 +727,7 @@ ${test.build.data}/${surefire.forkNumber} ${test.build.dir}/${surefire.forkNumber} ${hadoop.tmp.dir}/${surefire.forkNumber} - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.huge.filesize} ${fs.azure.scale.test.huge.partitionsize} @@ -706,7 +769,7 @@ - fork-${surefire.forkNumber} + fork-000${surefire.forkNumber} ${fs.azure.scale.test.enabled} ${fs.azure.scale.test.huge.filesize} diff --git a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml index 070c8c1fe827a..fd2a7c210e706 100644 --- a/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml +++ b/hadoop-tools/hadoop-azure/src/config/checkstyle-suppressions.xml @@ -48,4 +48,7 @@ files="org[\\/]apache[\\/]hadoop[\\/]fs[\\/]azurebfs[\\/]utils[\\/]Base64.java"/> + + diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 9719da7dc168a..fafc30372b4a5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -260,6 +260,11 @@ public class AbfsConfiguration{ DefaultValue = DEFAULT_ENABLE_AUTOTHROTTLING) private boolean enableAutoThrottling; + @IntegerConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_ABFS_IO_RATE_LIMIT, + MinValue = 0, + DefaultValue = RATE_LIMIT_DEFAULT) + private int rateLimit; + @StringConfigurationValidatorAnnotation(ConfigurationKey = FS_AZURE_USER_AGENT_PREFIX_KEY, DefaultValue = DEFAULT_FS_AZURE_USER_AGENT_PREFIX) private String userAgentId; @@ -726,6 +731,10 @@ public boolean isAutoThrottlingEnabled() { return this.enableAutoThrottling; } + public int getRateLimit() { + return rateLimit; + } + public String getCustomUserAgentPrefix() { return this.userAgentId; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index ae70b8dc53b02..46141e7c4a838 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -27,6 +27,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.nio.file.AccessDeniedException; +import java.time.Duration; import java.util.Hashtable; import java.util.List; import java.util.ArrayList; @@ -42,13 +43,17 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.io.IOUtils; +import javax.annotation.Nullable; + import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.azurebfs.commit.ResilientCommitByRename; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsClientThrottlingIntercept; import org.apache.hadoop.fs.azurebfs.services.AbfsListStatusRemoteIterator; @@ -94,9 +99,12 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.fs.store.DataBlocks; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.RateLimiting; +import org.apache.hadoop.util.RateLimitingFactory; import org.apache.hadoop.util.functional.RemoteIterators; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.LambdaUtils; @@ -143,6 +151,9 @@ public class AzureBlobFileSystem extends FileSystem /** Maximum Active blocks per OutputStream. */ private int blockOutputActiveBlocks; + /** Rate limiting for operations which use it to throttle their IO. */ + private RateLimiting rateLimiting; + @Override public void initialize(URI uri, Configuration configuration) throws IOException { @@ -215,7 +226,7 @@ public void initialize(URI uri, Configuration configuration) } AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); - + rateLimiting = RateLimitingFactory.create(abfsConfiguration.getRateLimit()); LOG.debug("Initializing AzureBlobFileSystem for {} complete", uri); } @@ -261,7 +272,7 @@ private FSDataInputStream open(final Path path, InputStream inputStream = abfsStore .openFileForRead(qualifiedPath, parameters, statistics, tracingContext); return new FSDataInputStream(inputStream); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(path, ex); return null; } @@ -290,8 +301,13 @@ protected CompletableFuture openFileWithOptions( } @Override - public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, - final short replication, final long blockSize, final Progressable progress) throws IOException { + public FSDataOutputStream create(final Path f, + final FsPermission permission, + final boolean overwrite, + final int bufferSize, + final short replication, + final long blockSize, + final Progressable progress) throws IOException { LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", f, permission, @@ -311,7 +327,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi FsPermission.getUMask(getConf()), tracingContext); statIncrement(FILES_CREATED); return new FSDataOutputStream(outputStream, statistics); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(f, ex); return null; } @@ -340,8 +356,12 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe @Override @SuppressWarnings("deprecation") - public FSDataOutputStream createNonRecursive(final Path f, final FsPermission permission, - final EnumSet flags, final int bufferSize, final short replication, final long blockSize, + public FSDataOutputStream createNonRecursive(final Path f, + final FsPermission permission, + final EnumSet flags, + final int bufferSize, + final short replication, + final long blockSize, final Progressable progress) throws IOException { // Check if file should be appended or overwritten. Assume that the file @@ -365,7 +385,8 @@ public FSDataOutputStream createNonRecursive(final Path f, } @Override - public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { + public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) + throws IOException { LOG.debug( "AzureBlobFileSystem.append path: {} bufferSize: {}", f.toString(), @@ -380,7 +401,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr OutputStream outputStream = abfsStore .openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(f, ex); return null; } @@ -403,7 +424,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat, listener); // rename under same folder; - if(makeQualified(parentFolder).equals(qualifiedDstPath)) { + if (makeQualified(parentFolder).equals(qualifiedDstPath)) { return tryGetFileStatus(qualifiedSrcPath, tracingContext) != null; } @@ -438,24 +459,99 @@ public boolean rename(final Path src, final Path dst) throws IOException { qualifiedDstPath = makeQualified(adjustedDst); - abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext); + abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext, null); return true; - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); checkException( - src, - ex, - AzureServiceErrorCode.PATH_ALREADY_EXISTS, - AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, - AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, - AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, - AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, - AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); + src, + ex, + AzureServiceErrorCode.PATH_ALREADY_EXISTS, + AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, + AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, + AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, + AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, + AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); return false; } } + /** + * Private method to create resilient commit support. + * @return a new instance + * @param path destination path + * @throws IOException problem probing store capabilities + * @throws UnsupportedOperationException if the store lacks this support + */ + @InterfaceAudience.Private + public ResilientCommitByRename createResilientCommitSupport(final Path path) + throws IOException { + + if (!hasPathCapability(path, + CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME)) { + throw new UnsupportedOperationException( + "Resilient commit support not available for " + path); + } + return new ResilientCommitByRenameImpl(); + } + + /** + * Resilient commit support. + * Provided as a nested class to avoid contaminating the + * FS instance with too many private methods which end up + * being used widely (as has happened to the S3A FS) + */ + public class ResilientCommitByRenameImpl implements ResilientCommitByRename { + + /** + * Perform the rename. + * This will be rate limited, as well as able to recover + * from rename errors if the etag was passed in. + * @param source path to source file + * @param dest destination of rename. + * @param sourceEtag etag of source file. may be null or empty + * @return the outcome of the operation + * @throws IOException any rename failure which was not recovered from. + */ + public Pair commitSingleFileByRename( + final Path source, + final Path dest, + @Nullable final String sourceEtag) throws IOException { + + LOG.debug("renameFileWithEtag source: {} dest: {} etag {}", source, dest, sourceEtag); + statIncrement(CALL_RENAME); + + trailingPeriodCheck(dest); + Path qualifiedSrcPath = makeQualified(source); + Path qualifiedDstPath = makeQualified(dest); + + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat, + listener); + + if (qualifiedSrcPath.equals(qualifiedDstPath)) { + // rename to itself is forbidden + throw new PathIOException(qualifiedSrcPath.toString(), "cannot rename object onto self"); + } + + // acquire one IO permit + final Duration waitTime = rateLimiting.acquire(1); + + try { + final boolean recovered = abfsStore.rename(qualifiedSrcPath, + qualifiedDstPath, tracingContext, sourceEtag); + return Pair.of(recovered, waitTime); + } catch (AzureBlobFileSystemException ex) { + LOG.debug("Rename operation failed. ", ex); + checkException(source, ex); + // never reached + return null; + } + + } + } + @Override public boolean delete(final Path f, final boolean recursive) throws IOException { LOG.debug( @@ -533,7 +629,7 @@ private void incrementStatistic(AbfsStatistic statistic) { * @throws IllegalArgumentException if the path has a trailing period (.) */ private void trailingPeriodCheck(Path path) throws IllegalArgumentException { - while (!path.isRoot()){ + while (!path.isRoot()) { String pathToString = path.toString(); if (pathToString.length() != 0) { if (pathToString.charAt(pathToString.length() - 1) == '.') { @@ -541,8 +637,7 @@ private void trailingPeriodCheck(Path path) throws IllegalArgumentException { "ABFS does not allow files or directories to end with a dot."); } path = path.getParent(); - } - else { + } else { break; } } @@ -601,10 +696,10 @@ public synchronized void close() throws IOException { @Override public FileStatus getFileStatus(final Path f) throws IOException { - TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.GET_FILESTATUS, tracingHeaderFormat, - listener); - return getFileStatus(f, tracingContext); + TracingContext tracingContext = new TracingContext(clientCorrelationId, + fileSystemId, FSOperationType.GET_FILESTATUS, tracingHeaderFormat, + listener); + return getFileStatus(f, tracingContext); } private FileStatus getFileStatus(final Path path, @@ -615,7 +710,7 @@ private FileStatus getFileStatus(final Path path, try { return abfsStore.getFileStatus(qualifiedPath, tracingContext); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(path, ex); return null; } @@ -639,7 +734,7 @@ public void breakLease(final Path f) throws IOException { fileSystemId, FSOperationType.BREAK_LEASE, tracingHeaderFormat, listener); abfsStore.breakLease(qualifiedPath, tracingContext); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(f, ex); } } @@ -666,7 +761,6 @@ public Path makeQualified(Path path) { return super.makeQualified(path); } - @Override public Path getWorkingDirectory() { return this.workingDir; @@ -689,8 +783,8 @@ public String getScheme() { @Override public Path getHomeDirectory() { return makeQualified(new Path( - FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX - + "/" + abfsStore.getUser())); + FileSystemConfigurations.USER_HOME_DIRECTORY_PREFIX + + "/" + abfsStore.getUser())); } /** @@ -714,8 +808,8 @@ public BlockLocation[] getFileBlockLocations(FileStatus file, } final String blobLocationHost = abfsStore.getAbfsConfiguration().getAzureBlockLocationHost(); - final String[] name = { blobLocationHost }; - final String[] host = { blobLocationHost }; + final String[] name = {blobLocationHost}; + final String[] host = {blobLocationHost}; long blockSize = file.getBlockSize(); if (blockSize <= 0) { throw new IllegalArgumentException( @@ -790,15 +884,14 @@ public Void call() throws Exception { } }); } - } - finally { + } finally { executorService.shutdownNow(); } return true; } - /** + /** * Set owner of a path (i.e. a file or a directory). * The parameters owner and group cannot both be null. * @@ -828,9 +921,9 @@ public void setOwner(final Path path, final String owner, final String group) try { abfsStore.setOwner(qualifiedPath, - owner, - group, - tracingContext); + owner, + group, + tracingContext); } catch (AzureBlobFileSystemException ex) { checkException(path, ex); } @@ -847,7 +940,10 @@ public void setOwner(final Path path, final String owner, final String group) * @throws IllegalArgumentException If name is null or empty or if value is null */ @Override - public void setXAttr(final Path path, final String name, final byte[] value, final EnumSet flag) + public void setXAttr(final Path path, + final String name, + final byte[] value, + final EnumSet flag) throws IOException { LOG.debug("AzureBlobFileSystem.setXAttr path: {}", path); @@ -971,7 +1067,7 @@ public void modifyAclEntries(final Path path, final List aclSpec) if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "modifyAclEntries is only supported by storage accounts with the " - + "hierarchical namespace enabled."); + + "hierarchical namespace enabled."); } if (aclSpec == null || aclSpec.isEmpty()) { @@ -1006,7 +1102,7 @@ public void removeAclEntries(final Path path, final List aclSpec) if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "removeAclEntries is only supported by storage accounts with the " - + "hierarchical namespace enabled."); + + "hierarchical namespace enabled."); } if (aclSpec == null || aclSpec.isEmpty()) { @@ -1038,7 +1134,7 @@ public void removeDefaultAcl(final Path path) throws IOException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "removeDefaultAcl is only supported by storage accounts with the " - + "hierarchical namespace enabled."); + + "hierarchical namespace enabled."); } Path qualifiedPath = makeQualified(path); @@ -1068,7 +1164,7 @@ public void removeAcl(final Path path) throws IOException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "removeAcl is only supported by storage accounts with the " - + "hierarchical namespace enabled."); + + "hierarchical namespace enabled."); } Path qualifiedPath = makeQualified(path); @@ -1101,7 +1197,7 @@ public void setAcl(final Path path, final List aclSpec) if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "setAcl is only supported by storage accounts with the hierarchical " - + "namespace enabled."); + + "namespace enabled."); } if (aclSpec == null || aclSpec.size() == 0) { @@ -1133,7 +1229,7 @@ public AclStatus getAclStatus(final Path path) throws IOException { if (!getIsNamespaceEnabled(tracingContext)) { throw new UnsupportedOperationException( "getAclStatus is only supported by storage account with the " - + "hierarchical namespace enabled."); + + "hierarchical namespace enabled."); } Path qualifiedPath = makeQualified(path); @@ -1243,7 +1339,7 @@ private FileStatus tryGetFileStatus(final Path f, TracingContext tracingContext) private boolean fileSystemExists() throws IOException { LOG.debug( - "AzureBlobFileSystem.fileSystemExists uri: {}", uri); + "AzureBlobFileSystem.fileSystemExists uri: {}", uri); try { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.TEST_OP, tracingHeaderFormat, listener); @@ -1534,8 +1630,9 @@ public boolean hasPathCapability(final Path path, final String capability) case CommonPathCapabilities.FS_PERMISSIONS: case CommonPathCapabilities.FS_APPEND: case CommonPathCapabilities.ETAGS_AVAILABLE: - case CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME: return true; + + case CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME: case CommonPathCapabilities.FS_ACLS: return getIsNamespaceEnabled( new TracingContext(clientCorrelationId, fileSystemId, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index f4f895996447c..046f9f0b56240 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -62,6 +62,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -878,7 +879,22 @@ public void breakLease(final Path path, final TracingContext tracingContext) thr client.breakLease(getRelativePath(path), tracingContext); } - public void rename(final Path source, final Path destination, TracingContext tracingContext) throws + /** + * Rename a file or directory. + * If a source etag is passed in, the operation will attempt to recover + * from a missing source file by probing the destination for + * existence and comparing etags. + * @param source path to source file + * @param destination destination of rename. + * @param tracingContext trace context + * @param sourceEtag etag of source file. may be null or empty + * @throws AzureBlobFileSystemException failure, excluding any recovery from overload failures. + * @return true if recovery was needed and succeeded. + */ + public boolean rename(final Path source, + final Path destination, + final TracingContext tracingContext, + final String sourceEtag) throws AzureBlobFileSystemException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; @@ -898,23 +914,29 @@ public void rename(final Path source, final Path destination, TracingContext tra String sourceRelativePath = getRelativePath(source); String destinationRelativePath = getRelativePath(destination); + // was any operation recovered from? + boolean recovered = false; do { try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) { - AbfsRestOperation op = client - .renamePath(sourceRelativePath, destinationRelativePath, - continuation, tracingContext); + final Pair pair = + client.renamePath(sourceRelativePath, destinationRelativePath, + continuation, tracingContext, sourceEtag); + + AbfsRestOperation op = pair.getLeft(); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); countAggregate++; shouldContinue = continuation != null && !continuation.isEmpty(); - + // update the recovery flag. + recovered |= pair.getRight(); if (!shouldContinue) { perfInfo.registerAggregates(startAggregate, countAggregate); } } } while (shouldContinue); + return recovered; } public void delete(final Path path, final boolean recursive, @@ -1932,7 +1954,7 @@ boolean areLeasesFreed() { * @param result response to process. * @return the quote-unwrapped etag. */ - private static String extractEtagHeader(AbfsHttpOperation result) { + public static String extractEtagHeader(AbfsHttpOperation result) { String etag = result.getResponseHeader(HttpHeaderConfigurations.ETAG); if (etag != null) { // strip out any wrapper "" quotes which come back, for consistency with diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java new file mode 100644 index 0000000000000..efba9244af700 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AbfsManifestStoreOperations.java @@ -0,0 +1,130 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import java.io.IOException; +import java.time.Duration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem; + +/** + * Extension of StoreOperationsThroughFileSystem with ABFS awareness. + * Purely for use by jobs committing work through the manifest committer. + * The {@link AzureManifestCommitterFactory} will configure + * this as the binding to the FS. + * + * ADLS Gen2 stores support etag-recovery on renames, but not WASB + * stores. + */ +@InterfaceAudience.LimitedPrivate("mapreduce") +@InterfaceStability.Unstable +public class AbfsManifestStoreOperations extends + ManifestStoreOperationsThroughFileSystem { + + private static final Logger LOG = LoggerFactory.getLogger( + AbfsManifestStoreOperations.class); + + /** + * Classname, which can be declared in jpb configurations. + */ + public static final String NAME = AbfsManifestStoreOperations.class.getName(); + + /** + * Resilient rename calls; only available on an ADLS Gen2 store. + * Will be null after binding if the FS isn't compatible. + */ + private ResilientCommitByRename resilientCommitByRename; + + @Override + public AzureBlobFileSystem getFileSystem() { + return (AzureBlobFileSystem) super.getFileSystem(); + } + + /** + * Bind to the store. + * + * @param filesystem FS. + * @param path path to work under + * @throws IOException binding problems. + */ + @Override + public void bindToFileSystem(FileSystem filesystem, Path path) throws IOException { + if (!(filesystem instanceof AzureBlobFileSystem)) { + throw new PathIOException(path.toString(), + "Not an abfs filesystem: " + filesystem.getClass()); + } + super.bindToFileSystem(filesystem, path); + try { + resilientCommitByRename = getFileSystem().createResilientCommitSupport(path); + LOG.debug("Bonded to filesystem with resilient commits under path {}", path); + } catch (UnsupportedOperationException e) { + LOG.debug("No resilient commit support under path {}", path); + } + } + + @Override + public boolean storePreservesEtagsThroughRenames(final Path path) { + return true; + } + + /** + * Resilient commits available on hierarchical stores. + * @return true if the FS can use etags on renames. + */ + @Override + public boolean storeSupportsResilientCommit() { + return resilientCommitByRename != null; + } + + /** + * Commit a file through an internal ABFS operation. + * If resilient commit is unavailable, invokes the superclass, which + * will raise an UnsupportedOperationException + * @param entry entry to commit + * @return the outcome + * @throws IOException any failure in resilient commit. + * @throws UnsupportedOperationException if not available. + */ + @Override + public CommitFileResult commitFile(final FileEntry entry) throws IOException { + + if (resilientCommitByRename != null) { + final Pair result = + resilientCommitByRename.commitSingleFileByRename( + entry.getSourcePath(), + entry.getDestPath(), + entry.getEtag()); + return CommitFileResult.fromResilientCommit(result.getLeft(), + result.getRight()); + } else { + return super.commitFile(entry); + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AzureManifestCommitterFactory.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AzureManifestCommitterFactory.java new file mode 100644 index 0000000000000..b760fa7a4ac53 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/AzureManifestCommitterFactory.java @@ -0,0 +1,58 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterFactory; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS; + +/** + * A Committer for the manifest committer which performs all bindings needed + * to work best with abfs. + * This includes, at a minimum, switching to the abfs-specific manifest store operations. + * + * This classname is referenced in configurations, so MUST NOT change. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class AzureManifestCommitterFactory extends ManifestCommitterFactory { + + /** + * Classname, which can be declared in job configurations. + */ + public static final String NAME = ManifestCommitterFactory.class.getName(); + + @Override + public ManifestCommitter createOutputCommitter(final Path outputPath, + final TaskAttemptContext context) throws IOException { + final Configuration conf = context.getConfiguration(); + // use ABFS Store operations + conf.set(OPT_STORE_OPERATIONS_CLASS, + AbfsManifestStoreOperations.NAME); + return super.createOutputCommitter(outputPath, context); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/ResilientCommitByRename.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/ResilientCommitByRename.java new file mode 100644 index 0000000000000..2e91392a661b1 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/ResilientCommitByRename.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.hadoop.fs.azurebfs.commit; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.time.Duration; +import javax.annotation.Nullable; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; + +/** + * API exclusively for committing files. + * + * This is only for use by (@link {@link AbfsManifestStoreOperations}, + * and is intended to be implemented by ABFS. + * To ensure that there is no need to add mapreduce JARs to the + * classpath just to work with ABFS, this interface + * MUST NOT refer to anything in the + * {@code org.apache.hadoop.mapreduce} package. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public interface ResilientCommitByRename extends IOStatisticsSource { + + /** + * Rename source file to dest path *Exactly*; no subdirectory games here. + * if the method does not raise an exception,then + * the data at dest is the data which was at source. + * + * Requirements + * + *
+   *   exists(FS, source) else raise FileNotFoundException
+   *   source != dest else raise PathIOException
+   *   not exists(FS, dest)
+   *   isDir(FS, dest.getParent)
+   * 
+ *
    + *
  1. source != dest else raise PathIOException
  2. + *
  3. source must exist else raise FileNotFoundException
  4. + *
  5. source must exist and be a file
  6. + *
  7. dest must not exist;
  8. + *
  9. dest.getParent() must be a dir
  10. + *
  11. if sourceEtag is non-empty, it MAY be used to qualify/validate the rename.
  12. + *
+ * + * The outcome of the operation is undefined if source is not a file, dest exists, + * dest.getParent() doesn't exist/is a file. + * That is: implementations SHOULD assume that the code calling this method has + * set up the destination directory tree and is only invoking this call on a file. + * Accordingly: implementations MAY skip validation checks + * + * Post Conditions on a successful operation: + *
+   * FS' where:
+   *     not exists(FS', source)
+   *     and exists(FS', dest)
+   *     and data(FS', dest) == data (FS, source)
+   * 
+ * This is exactly the same outcome as `FileSystem.rename()` when the same preconditions + * are met. This API call simply restricts the operation to file rename with strict + * conditions, (no need to be 'clever' about dest path calculation) and the ability + * to pass in etags, modtimes and file status values. + * + * @param source path to source file + * @param dest destination of rename. + * @param sourceEtag etag of source file. may be null or empty + * @return true if recovery was needed. + * @throws FileNotFoundException source file not found + * @throws PathIOException failure, including source and dest being the same path + * @throws IOException any other exception + */ + Pair commitSingleFileByRename( + Path source, + Path dest, + @Nullable String sourceEtag) throws IOException; + + +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java new file mode 100644 index 0000000000000..3567377350d6b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java @@ -0,0 +1,28 @@ +/* + * 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. + */ + +/** + * Support for manifest committer. + * Unless otherwise stated: classes are private. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 12beb5a9bbabe..9d3b2d5e82c6e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -220,6 +220,9 @@ public final class ConfigurationKeys { /** Key for enabling the tracking of ABFS API latency and sending the latency numbers to the ABFS API service */ public static final String FS_AZURE_ABFS_LATENCY_TRACK = "fs.azure.abfs.latency.track"; + /** Key for rate limit capacity, as used by IO operations which try to throttle themselves. */ + public static final String FS_AZURE_ABFS_IO_RATE_LIMIT = "fs.azure.io.rate.limit"; + public static String accountProperty(String property, String account) { return property + "." + account; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java index f58c61e8908a6..63d62a33b1819 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/FileSystemConfigurations.java @@ -133,5 +133,10 @@ public final class FileSystemConfigurations { public static final String DATA_BLOCKS_BUFFER_DEFAULT = DATA_BLOCKS_BUFFER_DISK; + /** + * IO rate limit. Value: {@value} + */ + public static final int RATE_LIMIT_DEFAULT = 10_000; + private FileSystemConfigurations() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 08142a17a8aa1..b701037d0fc41 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -51,6 +51,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; @@ -67,6 +68,8 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.SERVER_SIDE_ENCRYPTION_ALGORITHM; @@ -478,8 +481,30 @@ public AbfsRestOperation breakLease(final String path, return op; } - public AbfsRestOperation renamePath(String source, final String destination, - final String continuation, TracingContext tracingContext) + + /** + * Rename a file or directory. + * If a source etag is passed in, the operation will attempt to recover + * from a missing source file by probing the destination for + * existence and comparing etags. + * The second value in the result will be true to indicate that this + * took place. + * As rename recovery is only attempted if the source etag is non-empty, + * in normal rename operations rename recovery will never happen. + * @param source path to source file + * @param destination destination of rename. + * @param continuation continuation. + * @param tracingContext trace context + * @param sourceEtag etag of source file. may be null or empty + * @return pair of (the rename operation, flag indicating recovery took place) + * @throws AzureBlobFileSystemException failure, excluding any recovery from overload failures. + */ + public Pair renamePath( + final String source, + final String destination, + final String continuation, + final TracingContext tracingContext, + final String sourceEtag) throws AzureBlobFileSystemException { final List requestHeaders = createDefaultHeaders(); @@ -505,9 +530,73 @@ public AbfsRestOperation renamePath(String source, final String destination, HTTP_METHOD_PUT, url, requestHeaders); - // no attempt at recovery using timestamps as it was not reliable. - op.execute(tracingContext); - return op; + try { + op.execute(tracingContext); + return Pair.of(op, false); + } catch (AzureBlobFileSystemException e) { + // If we have no HTTP response, throw the original exception. + if (!op.hasResult()) { + throw e; + } + boolean etagCheckSucceeded = renameIdempotencyCheckOp( + source, + sourceEtag, op, destination, tracingContext); + if (!etagCheckSucceeded) { + // idempotency did not return different result + // throw back the exception + throw e; + } + return Pair.of(op, true); + } + } + + /** + * Check if the rename request failure is post a retry and if earlier rename + * request might have succeeded at back-end. + * + * If a source etag was passed in, and the error was 404, get the + * etag of any file at the destination. + * If it matches the source etag, then the rename is considered + * a success. + * Exceptions raised in the probe of the destination are swallowed, + * so that they do not interfere with the original rename failures. + * @param source source path + * @param op Rename request REST operation response with non-null HTTP response + * @param destination rename destination path + * @param sourceEtag etag of source file. may be null or empty + * @param tracingContext Tracks identifiers for request header + * @return true if the file was successfully copied + */ + public boolean renameIdempotencyCheckOp( + final String source, + final String sourceEtag, + final AbfsRestOperation op, + final String destination, + TracingContext tracingContext) { + Preconditions.checkArgument(op.hasResult(), "Operations has null HTTP response"); + + if ((op.isARetriedRequest()) + && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) + && isNotEmpty(sourceEtag)) { + + // Server has returned HTTP 404, which means rename source no longer + // exists. Check on destination status and if its etag matches + // that of the source, consider it to be a success. + LOG.debug("rename {} to {} failed, checking etag of destination", + source, destination); + + try { + final AbfsRestOperation destStatusOp = getPathStatus(destination, + false, tracingContext); + final AbfsHttpOperation result = destStatusOp.getResult(); + + return result.getStatusCode() == HttpURLConnection.HTTP_OK + && sourceEtag.equals(extractEtagHeader(result)); + } catch (AzureBlobFileSystemException ignored) { + // GetFileStatus on the destination failed, the rename did not take place + } + } + return false; } public AbfsRestOperation append(final String path, final byte[] buffer, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java index bb1ec9e4a3fb5..e3adc59afac5e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/SimpleKeyProvider.java @@ -48,9 +48,15 @@ public String getStorageAccountKey(String accountName, Configuration rawConfig) // Validating the key. validateStorageAccountKey(key); } catch (IllegalAccessException | InvalidConfigurationValueException e) { - throw new KeyProviderException("Failure to initialize configuration", e); + LOG.debug("Failure to retrieve storage account key for {}", accountName, + e); + throw new KeyProviderException("Failure to initialize configuration for " + + accountName + + " key =\"" + key + "\"" + + ": " + e, e); } catch(IOException ioe) { - LOG.warn("Unable to get key from credential providers. {}", ioe); + LOG.warn("Unable to get key for {} from credential providers. {}", + accountName, ioe, ioe); } return key; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 56d553819feae..4a5507526c3a1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -271,11 +271,12 @@ protected void createFilesystemForSASTests() throws Exception { // The SAS tests do not have permission to create a filesystem // so first create temporary instance of the filesystem using SharedKey // then re-use the filesystem it creates with SAS auth instead of SharedKey. - AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig); - ContractTestUtils.assertPathExists(tempFs, "This path should exist", - new Path("/")); - abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SAS.name()); - usingFilesystemForSASTests = true; + try (AzureBlobFileSystem tempFs = (AzureBlobFileSystem) FileSystem.newInstance(rawConfig)){ + ContractTestUtils.assertPathExists(tempFs, "This path should exist", + new Path("/")); + abfsConfig.set(FS_AZURE_ACCOUNT_AUTH_TYPE_PROPERTY_NAME, AuthType.SAS.name()); + usingFilesystemForSASTests = true; + } } public AzureBlobFileSystem getFileSystem() throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java index b0e82444afb34..5bd6eaff42e84 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsReadWriteAndSeek.java @@ -32,7 +32,10 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_READ_BUFFER_SIZE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.MAX_BUFFER_SIZE; @@ -76,13 +79,19 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { new Random().nextBytes(b); Path testPath = path(TEST_PATH); - try (FSDataOutputStream stream = fs.create(testPath)) { + FSDataOutputStream stream = fs.create(testPath); + try { stream.write(b); + } finally{ + stream.close(); } + IOStatisticsLogging.logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, stream); final byte[] readBuffer = new byte[2 * bufferSize]; int result; + IOStatisticsSource statisticsSource = null; try (FSDataInputStream inputStream = fs.open(testPath)) { + statisticsSource = inputStream; ((AbfsInputStream) inputStream.getWrappedStream()).registerListener( new TracingHeaderValidator(abfsConfiguration.getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.READ, true, 0, @@ -100,6 +109,8 @@ private void testReadWriteAndSeek(int bufferSize) throws Exception { inputStream.seek(0); result = inputStream.read(readBuffer, 0, bufferSize); } + IOStatisticsLogging.logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, statisticsSource); + assertNotEquals("data read in final read()", -1, result); assertArrayEquals(readBuffer, b); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java index ea9fba62579f6..965e02a0a3ebe 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java @@ -401,7 +401,8 @@ public void testSignatureMask() throws Exception { fs.create(new Path(src)).close(); AbfsRestOperation abfsHttpRestOperation = fs.getAbfsClient() .renamePath(src, "/testABC" + "/abc.txt", null, - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false), null) + .getLeft(); AbfsHttpOperation result = abfsHttpRestOperation.getResult(); String url = result.getMaskedUrl(); String encodedUrl = result.getMaskedEncodedUrl(); @@ -418,7 +419,7 @@ public void testSignatureMaskOnExceptionMessage() throws Exception { intercept(IOException.class, "sig=XXXX", () -> getFileSystem().getAbfsClient() .renamePath("testABC/test.xt", "testABC/abc.txt", null, - getTestTracingContext(getFileSystem(), false))); + getTestTracingContext(getFileSystem(), false), null)); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 2198a6ab35852..02260310bb813 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -526,7 +526,8 @@ private void testRenamePath(final boolean isWithCPK) throws Exception { AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient .renamePath(testFileName, newName, null, - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false), null) + .getLeft(); assertCPKHeaders(abfsRestOperation, false); assertNoCPKResponseHeadersPresent(abfsRestOperation); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java new file mode 100644 index 0000000000000..8160cdc64c546 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbfsCommitTestHelper.java @@ -0,0 +1,49 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; + +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_STORE_OPERATIONS_CLASS; + +/** + * Helper methods for committer tests on ABFS. + */ +final class AbfsCommitTestHelper { + private AbfsCommitTestHelper() { + } + + /** + * Prepare the test configuration. + * @param contractTestBinding test binding + * @return an extracted and patched configuration. + */ + static Configuration prepareTestConfiguration( + ABFSContractTestBinding contractTestBinding) { + final Configuration conf = + contractTestBinding.getRawConfiguration(); + + // use ABFS Store operations + conf.set(OPT_STORE_OPERATIONS_CLASS, + AbfsManifestStoreOperations.NAME); + + return conf; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbstractAbfsClusterITest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbstractAbfsClusterITest.java new file mode 100644 index 0000000000000..55752055f0c31 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/AbstractAbfsClusterITest.java @@ -0,0 +1,260 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; + +import org.junit.AfterClass; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azure.integration.AzureTestConstants; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.AbstractManifestCommitterTest; +import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster; +import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig; +import org.apache.hadoop.util.DurationInfo; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.azure.integration.AzureTestUtils.assumeScaleTestsEnabled; +import static org.apache.hadoop.io.IOUtils.closeStream; + +/** + * Tests which create a yarn minicluster. + * These are all considered scale tests; the probe for + * scale tests being enabled is executed before the cluster + * is set up to avoid wasting time on non-scale runs. + */ +public abstract class AbstractAbfsClusterITest extends + AbstractManifestCommitterTest { + + public static final int NO_OF_NODEMANAGERS = 2; + + private final ABFSContractTestBinding binding; + + + /** + * The static cluster binding with the lifecycle of this test; served + * through instance-level methods for sharing across methods in the + * suite. + */ + @SuppressWarnings("StaticNonFinalField") + private static ClusterBinding clusterBinding; + + protected AbstractAbfsClusterITest() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + protected int getTestTimeoutMillis() { + return AzureTestConstants.SCALE_TEST_TIMEOUT_MILLIS; + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + requireScaleTestsEnabled(); + if (getClusterBinding() == null) { + clusterBinding = demandCreateClusterBinding(); + } + assertNotNull("cluster is not bound", getClusterBinding()); + } + + @AfterClass + public static void teardownClusters() throws IOException { + terminateCluster(clusterBinding); + clusterBinding = null; + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + + @Override + protected Configuration createConfiguration() { + return AbfsCommitTestHelper.prepareTestConfiguration(binding); + } + + /** + * This is the cluster binding which every subclass must create. + */ + protected static final class ClusterBinding { + + private String clusterName; + + private final MiniMRYarnCluster yarn; + + public ClusterBinding( + final String clusterName, + final MiniMRYarnCluster yarn) { + this.clusterName = clusterName; + this.yarn = requireNonNull(yarn); + } + + + /** + * Get the cluster FS, which will either be HDFS or the local FS. + * @return a filesystem. + * @throws IOException failure + */ + public FileSystem getClusterFS() throws IOException { + return FileSystem.getLocal(yarn.getConfig()); + } + + public MiniMRYarnCluster getYarn() { + return yarn; + } + + public Configuration getConf() { + return getYarn().getConfig(); + } + + public String getClusterName() { + return clusterName; + } + + public void terminate() { + closeStream(getYarn()); + } + } + + /** + * Create the cluster binding. + * The configuration will be patched by propagating down options + * from the maven build (S3Guard binding etc) and turning off unwanted + * YARN features. + * + * If an HDFS cluster is requested, + * the HDFS and YARN clusters will share the same configuration, so + * the HDFS cluster binding is implicitly propagated to YARN. + * If one is not requested, the local filesystem is used as the cluster FS. + * @param conf configuration to start with. + * @return the cluster binding. + * @throws IOException failure. + */ + protected static ClusterBinding createCluster( + final JobConf conf) throws IOException { + try (DurationInfo d = new DurationInfo(LOG, "Creating YARN MiniCluster")) { + conf.setBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, false); + // create a unique cluster name based on the current time in millis. + String timestamp = LocalDateTime.now().format( + DateTimeFormatter.ofPattern("yyyy-MM-dd-HH.mm.ss.SS")); + String clusterName = "yarn-" + timestamp; + MiniMRYarnCluster yarnCluster = + new MiniMRYarnCluster(clusterName, NO_OF_NODEMANAGERS); + yarnCluster.init(conf); + yarnCluster.start(); + return new ClusterBinding(clusterName, yarnCluster); + } + } + + /** + * Terminate the cluster if it is not null. + * @param cluster the cluster + */ + protected static void terminateCluster(ClusterBinding cluster) { + if (cluster != null) { + cluster.terminate(); + } + } + + /** + * Get the cluster binding for this subclass. + * @return the cluster binding + */ + protected ClusterBinding getClusterBinding() { + return clusterBinding; + } + + protected MiniMRYarnCluster getYarn() { + return getClusterBinding().getYarn(); + } + + + /** + * We stage work into a temporary directory rather than directly under + * the user's home directory, as that is often rejected by CI test + * runners. + */ + @Rule + public final TemporaryFolder stagingFilesDir = new TemporaryFolder(); + + + /** + * binding on demand rather than in a BeforeClass static method. + * Subclasses can override this to change the binding options. + * @return the cluster binding + */ + protected ClusterBinding demandCreateClusterBinding() throws Exception { + return createCluster(new JobConf()); + } + + /** + * Create a job configuration. + * This creates a new job conf from the yarn + * cluster configuration then calls + * {@link #applyCustomConfigOptions(JobConf)} to allow it to be customized. + * @return the new job configuration. + * @throws IOException failure + */ + protected JobConf newJobConf() throws IOException { + JobConf jobConf = new JobConf(getYarn().getConfig()); + jobConf.addResource(getConfiguration()); + applyCustomConfigOptions(jobConf); + return jobConf; + } + + /** + * Patch the (job) configuration for this committer. + * @param jobConf configuration to patch + * @return a configuration which will run this configuration. + */ + protected Configuration patchConfigurationForCommitter( + final Configuration jobConf) { + enableManifestCommitter(jobConf); + return jobConf; + } + + /** + * Override point to let implementations tune the MR Job conf. + * @param jobConf configuration + */ + protected void applyCustomConfigOptions(JobConf jobConf) throws IOException { + + } + + + /** + * Assume that scale tests are enabled. + */ + protected void requireScaleTestsEnabled() { + assumeScaleTestsEnabled(getConfiguration()); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCleanupStage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCleanupStage.java new file mode 100644 index 0000000000000..a597c35376a71 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCleanupStage.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestCleanupStage; + +/** + * Cleanup logic on ABFS. + */ +public class ITestAbfsCleanupStage extends TestCleanupStage { + + private final ABFSContractTestBinding binding; + + public ITestAbfsCleanupStage() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return AbfsCommitTestHelper.prepareTestConfiguration(binding); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCommitTaskStage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCommitTaskStage.java new file mode 100644 index 0000000000000..a0aaec8532850 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCommitTaskStage.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestCommitTaskStage; + +/** + * ABFS storage test of task committer. + */ +public class ITestAbfsCommitTaskStage extends TestCommitTaskStage { + + private final ABFSContractTestBinding binding; + + public ITestAbfsCommitTaskStage() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return AbfsCommitTestHelper.prepareTestConfiguration(binding); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCreateOutputDirectoriesStage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCreateOutputDirectoriesStage.java new file mode 100644 index 0000000000000..6621b80da00c1 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsCreateOutputDirectoriesStage.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestCreateOutputDirectoriesStage; + +/** + * ABFS storage test of directory creation. + */ +public class ITestAbfsCreateOutputDirectoriesStage extends TestCreateOutputDirectoriesStage { + + private final ABFSContractTestBinding binding; + + public ITestAbfsCreateOutputDirectoriesStage() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return AbfsCommitTestHelper.prepareTestConfiguration(binding); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsJobThroughManifestCommitter.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsJobThroughManifestCommitter.java new file mode 100644 index 0000000000000..4e4c4f5996bc2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsJobThroughManifestCommitter.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.hadoop.fs.azurebfs.commit; + +import java.io.IOException; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.FixMethodOrder; +import org.junit.runners.MethodSorters; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestJobThroughManifestCommitter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; + +import static org.apache.hadoop.fs.azurebfs.commit.AbfsCommitTestHelper.prepareTestConfiguration; + +/** + * Test the Manifest committer stages against ABFS. + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class ITestAbfsJobThroughManifestCommitter + extends TestJobThroughManifestCommitter { + + private final ABFSContractTestBinding binding; + + public ITestAbfsJobThroughManifestCommitter() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return enableManifestCommitter(prepareTestConfiguration(binding)); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + + @Override + protected boolean shouldDeleteTestRootAtEndOfTestRun() { + return true; + } + + /** + * Add read of manifest and validate of output's etags. + * @param attemptId attempt ID + * @param files files which were created. + * @param manifest manifest + * @throws IOException failure + */ + @Override + protected void validateTaskAttemptManifest(String attemptId, + List files, + TaskManifest manifest) throws IOException { + super.validateTaskAttemptManifest(attemptId, files, manifest); + final List commit = manifest.getFilesToCommit(); + final ManifestStoreOperations operations = getStoreOperations(); + for (FileEntry entry : commit) { + Assertions.assertThat(entry.getEtag()) + .describedAs("Etag of %s", entry) + .isNotEmpty(); + final FileStatus sourceStatus = operations.getFileStatus(entry.getSourcePath()); + final String etag = ManifestCommitterSupport.getEtag(sourceStatus); + Assertions.assertThat(etag) + .describedAs("Etag of %s", sourceStatus) + .isEqualTo(entry.getEtag()); + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java new file mode 100644 index 0000000000000..acd693e39a0f0 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsLoadManifestsStage.java @@ -0,0 +1,55 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestLoadManifestsStage; + +/** + * ABFS storage test of saving and loading a large number + * of manifests. + */ +public class ITestAbfsLoadManifestsStage extends TestLoadManifestsStage { + + private final ABFSContractTestBinding binding; + + public ITestAbfsLoadManifestsStage() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return AbfsCommitTestHelper.prepareTestConfiguration(binding); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestCommitProtocol.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestCommitProtocol.java new file mode 100644 index 0000000000000..aac06f952dab2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestCommitProtocol.java @@ -0,0 +1,62 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestManifestCommitProtocol; + +import static org.apache.hadoop.fs.azurebfs.commit.AbfsCommitTestHelper.prepareTestConfiguration; + +/** + * Test the Manifest protocol against ABFS. + */ +public class ITestAbfsManifestCommitProtocol extends + TestManifestCommitProtocol { + + private final ABFSContractTestBinding binding; + + public ITestAbfsManifestCommitProtocol() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return enableManifestCommitter(prepareTestConfiguration(binding)); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + + + @Override + protected String suitename() { + return "ITestAbfsManifestCommitProtocol"; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestStoreOperations.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestStoreOperations.java new file mode 100644 index 0000000000000..922782da29c5f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsManifestStoreOperations.java @@ -0,0 +1,175 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import java.nio.charset.StandardCharsets; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.AbstractManifestCommitterTest; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.FileEntry; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; + +import static org.apache.hadoop.fs.CommonPathCapabilities.ETAGS_PRESERVED_IN_RENAME; +import static org.apache.hadoop.fs.azurebfs.commit.AbfsCommitTestHelper.prepareTestConfiguration; +import static org.junit.Assume.assumeTrue; + +/** + * Test {@link AbfsManifestStoreOperations}. + * As this looks at etag handling through FS operations, it's actually testing how etags work + * in ABFS (preservation across renames) and in the client (are they consistent + * in LIST and HEAD calls). + * + * Skipped when tested against wasb-compatible stores. + */ +public class ITestAbfsManifestStoreOperations extends AbstractManifestCommitterTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestAbfsManifestStoreOperations.class); + + private final ABFSContractTestBinding binding; + + public ITestAbfsManifestStoreOperations() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + + // skip tests on non-HNS stores + assumeTrue("Resilient rename not available", + getFileSystem().hasPathCapability(getContract().getTestPath(), + ETAGS_PRESERVED_IN_RENAME)); + + } + + @Override + protected Configuration createConfiguration() { + return enableManifestCommitter(prepareTestConfiguration(binding)); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + + /** + * basic consistency across operations, as well as being non-empty. + */ + @Test + public void testEtagConsistencyAcrossListAndHead() throws Throwable { + describe("Etag values must be non-empty and consistent across LIST and HEAD Calls."); + final Path path = methodPath(); + final FileSystem fs = getFileSystem(); + ContractTestUtils.touch(fs, path); + final ManifestStoreOperations operations = createManifestStoreOperations(); + Assertions.assertThat(operations) + .describedAs("Store operations class loaded via Configuration") + .isInstanceOf(AbfsManifestStoreOperations.class); + + final FileStatus st = operations.getFileStatus(path); + final String etag = operations.getEtag(st); + Assertions.assertThat(etag) + .describedAs("Etag of %s", st) + .isNotBlank(); + LOG.info("etag of empty file is \"{}\"", etag); + + final FileStatus[] statuses = fs.listStatus(path); + Assertions.assertThat(statuses) + .describedAs("List(%s)", path) + .hasSize(1); + final FileStatus lsStatus = statuses[0]; + Assertions.assertThat(operations.getEtag(lsStatus)) + .describedAs("etag of list status (%s) compared to HEAD value of %s", lsStatus, st) + .isEqualTo(etag); + } + + @Test + public void testEtagsOfDifferentDataDifferent() throws Throwable { + describe("Verify that two different blocks of data written have different tags"); + + final Path path = methodPath(); + final FileSystem fs = getFileSystem(); + Path src = new Path(path, "src"); + + ContractTestUtils.createFile(fs, src, true, + "data1234".getBytes(StandardCharsets.UTF_8)); + final ManifestStoreOperations operations = createManifestStoreOperations(); + final FileStatus srcStatus = operations.getFileStatus(src); + final String srcTag = operations.getEtag(srcStatus); + LOG.info("etag of file 1 is \"{}\"", srcTag); + + // now overwrite with data of same length + // (ensure that path or length aren't used exclusively as tag) + ContractTestUtils.createFile(fs, src, true, + "1234data".getBytes(StandardCharsets.UTF_8)); + + // validate + final String tag2 = operations.getEtag(operations.getFileStatus(src)); + LOG.info("etag of file 2 is \"{}\"", tag2); + + Assertions.assertThat(tag2) + .describedAs("etag of updated file") + .isNotEqualTo(srcTag); + } + + @Test + public void testEtagConsistencyAcrossRename() throws Throwable { + describe("Verify that when a file is renamed, the etag remains unchanged"); + final Path path = methodPath(); + final FileSystem fs = getFileSystem(); + Path src = new Path(path, "src"); + Path dest = new Path(path, "dest"); + + ContractTestUtils.createFile(fs, src, true, + "sample data".getBytes(StandardCharsets.UTF_8)); + final ManifestStoreOperations operations = createManifestStoreOperations(); + final FileStatus srcStatus = operations.getFileStatus(src); + final String srcTag = operations.getEtag(srcStatus); + LOG.info("etag of short file is \"{}\"", srcTag); + + Assertions.assertThat(srcTag) + .describedAs("Etag of %s", srcStatus) + .isNotBlank(); + + // rename + operations.commitFile(new FileEntry(src, dest, 0, srcTag)); + + // validate + FileStatus destStatus = operations.getFileStatus(dest); + final String destTag = operations.getEtag(destStatus); + Assertions.assertThat(destTag) + .describedAs("etag of list status (%s) compared to HEAD value of %s", destStatus, srcStatus) + .isEqualTo(srcTag); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsRenameStageFailure.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsRenameStageFailure.java new file mode 100644 index 0000000000000..5547d081c963f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsRenameStageFailure.java @@ -0,0 +1,69 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestRenameStageFailure; + +/** + * Rename failure logic on ABFS. + * This will go through the resilient rename operation. + */ +public class ITestAbfsRenameStageFailure extends TestRenameStageFailure { + + /** + * How many files to create. + */ + private static final int FILES_TO_CREATE = 20; + + private final ABFSContractTestBinding binding; + + public ITestAbfsRenameStageFailure() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return AbfsCommitTestHelper.prepareTestConfiguration(binding); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + + @Override + protected boolean requireRenameResilience() { + return true; + } + + @Override + protected int filesToCreate() { + return FILES_TO_CREATE; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTaskManifestFileIO.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTaskManifestFileIO.java new file mode 100644 index 0000000000000..d2fe9de115c3b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTaskManifestFileIO.java @@ -0,0 +1,54 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.contract.ABFSContractTestBinding; +import org.apache.hadoop.fs.azurebfs.contract.AbfsFileSystemContract; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.TestTaskManifestFileIO; + +/** + * Test Reading/writing manifest file through ABFS. + */ +public class ITestAbfsTaskManifestFileIO extends TestTaskManifestFileIO { + + private final ABFSContractTestBinding binding; + + public ITestAbfsTaskManifestFileIO() throws Exception { + binding = new ABFSContractTestBinding(); + } + + @Override + public void setup() throws Exception { + binding.setup(); + super.setup(); + } + + @Override + protected Configuration createConfiguration() { + return AbfsCommitTestHelper.prepareTestConfiguration(binding); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new AbfsFileSystemContract(conf, binding.isSecureMode()); + } + +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTerasort.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTerasort.java new file mode 100644 index 0000000000000..4b21b838decc5 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/ITestAbfsTerasort.java @@ -0,0 +1,353 @@ +/* + * 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.hadoop.fs.azurebfs.commit; + +import java.io.File; +import java.io.FileNotFoundException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.function.Consumer; + +import org.junit.Assume; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.runners.MethodSorters; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.examples.terasort.TeraGen; +import org.apache.hadoop.examples.terasort.TeraSort; +import org.apache.hadoop.examples.terasort.TeraSortConfigKeys; +import org.apache.hadoop.examples.terasort.TeraValidate; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; +import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hadoop.util.functional.RemoteIterators; + +import static java.util.Optional.empty; +import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_INFO; +import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.loadSuccessFile; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterTestSupport.validateSuccessFile; + +/** + * Runs Terasort against ABFS using the manifest committer. + * The tests run in sequence, so each operation is isolated. + * Scale test only (it is big and slow) + */ +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +@SuppressWarnings({"StaticNonFinalField", "OptionalUsedAsFieldOrParameterType"}) +public class ITestAbfsTerasort extends AbstractAbfsClusterITest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestAbfsTerasort.class); + + public static final int EXPECTED_PARTITION_COUNT = 10; + + public static final int PARTITION_SAMPLE_SIZE = 1000; + + public static final int ROW_COUNT = 1000; + + /** + * This has to be common across all test methods. + */ + private static final Path TERASORT_PATH = new Path("/ITestAbfsTerasort"); + + /** + * Duration tracker created in the first of the test cases and closed + * in {@link #test_140_teracomplete()}. + */ + private static Optional terasortDuration = empty(); + + /** + * Tracker of which stages are completed and how long they took. + */ + private static final Map COMPLETED_STAGES = new HashMap<>(); + + /** + * FileSystem statistics are collected from the _SUCCESS markers. + */ + protected static final IOStatisticsSnapshot JOB_IOSTATS = + snapshotIOStatistics(); + + /** Base path for all the terasort input and output paths. */ + private Path terasortPath; + + /** Input (teragen) path. */ + private Path sortInput; + + /** Path where sorted data goes. */ + private Path sortOutput; + + /** Path for validated job's output. */ + private Path sortValidate; + + public ITestAbfsTerasort() throws Exception { + } + + + @Override + public void setup() throws Exception { + // superclass calls requireScaleTestsEnabled(); + super.setup(); + prepareToTerasort(); + } + + /** + * Set up the job conf with the options for terasort chosen by the scale + * options. + * @param conf configuration + */ + @Override + protected void applyCustomConfigOptions(JobConf conf) { + // small sample size for faster runs + conf.setInt(TeraSortConfigKeys.SAMPLE_SIZE.key(), + getSampleSizeForEachPartition()); + conf.setInt(TeraSortConfigKeys.NUM_PARTITIONS.key(), + getExpectedPartitionCount()); + conf.setBoolean( + TeraSortConfigKeys.USE_SIMPLE_PARTITIONER.key(), + false); + } + + private int getExpectedPartitionCount() { + return EXPECTED_PARTITION_COUNT; + } + + private int getSampleSizeForEachPartition() { + return PARTITION_SAMPLE_SIZE; + } + + protected int getRowCount() { + return ROW_COUNT; + } + + /** + * Set up the terasort by initializing paths variables + * The paths used must be unique across parameterized runs but + * common across all test cases in a single parameterized run. + */ + private void prepareToTerasort() { + terasortPath = getFileSystem().makeQualified(TERASORT_PATH); + sortInput = new Path(terasortPath, "sortin"); + sortOutput = new Path(terasortPath, "sortout"); + sortValidate = new Path(terasortPath, "validate"); + } + + /** + * Declare that a stage has completed. + * @param stage stage name/key in the map + * @param d duration. + */ + private static void completedStage(final String stage, + final DurationInfo d) { + COMPLETED_STAGES.put(stage, d); + } + + /** + * Declare a stage which is required for this test case. + * @param stage stage name + */ + private static void requireStage(final String stage) { + Assume.assumeTrue( + "Required stage was not completed: " + stage, + COMPLETED_STAGES.get(stage) != null); + } + + /** + * Execute a single stage in the terasort. + * Updates the completed stages map with the stage duration -if successful. + * @param stage Stage name for the stages map. + * @param jobConf job conf + * @param dest destination directory -the _SUCCESS file will be expected here. + * @param tool tool to run. + * @param args args for the tool. + * @param minimumFileCount minimum number of files to have been created + * @throws Exception any failure + */ + private void executeStage( + final String stage, + final JobConf jobConf, + final Path dest, + final Tool tool, + final String[] args, + final int minimumFileCount) throws Exception { + int result; + + // the duration info is created outside a try-with-resources + // clause as it is used later. + DurationInfo d = new DurationInfo(LOG, stage); + try { + result = ToolRunner.run(jobConf, tool, args); + } finally { + d.close(); + } + dumpOutputTree(dest); + assertEquals(stage + + "(" + StringUtils.join(", ", args) + ")" + + " failed", 0, result); + final ManifestSuccessData successFile = validateSuccessFile(getFileSystem(), dest, + minimumFileCount, ""); + JOB_IOSTATS.aggregate(successFile.getIOStatistics()); + + completedStage(stage, d); + } + + /** + * Set up terasort by cleaning out the destination, and note the initial + * time before any of the jobs are executed. + * + * This is executed first for each parameterized run. + * It is where all variables which need to be reset for each run need + * to be reset. + */ + @Test + public void test_100_terasort_setup() throws Throwable { + describe("Setting up for a terasort"); + + getFileSystem().delete(terasortPath, true); + terasortDuration = Optional.of(new DurationInfo(LOG, false, "Terasort")); + } + + @Test + public void test_110_teragen() throws Throwable { + describe("Teragen to %s", sortInput); + getFileSystem().delete(sortInput, true); + + JobConf jobConf = newJobConf(); + patchConfigurationForCommitter(jobConf); + executeStage("teragen", + jobConf, + sortInput, + new TeraGen(), + new String[]{Integer.toString(getRowCount()), sortInput.toString()}, + 1); + } + + + @Test + public void test_120_terasort() throws Throwable { + describe("Terasort from %s to %s", sortInput, sortOutput); + requireStage("teragen"); + getFileSystem().delete(sortOutput, true); + + loadSuccessFile(getFileSystem(), sortInput); + JobConf jobConf = newJobConf(); + patchConfigurationForCommitter(jobConf); + executeStage("terasort", + jobConf, + sortOutput, + new TeraSort(), + new String[]{sortInput.toString(), sortOutput.toString()}, + 1); + } + + @Test + public void test_130_teravalidate() throws Throwable { + describe("TeraValidate from %s to %s", sortOutput, sortValidate); + requireStage("terasort"); + getFileSystem().delete(sortValidate, true); + loadSuccessFile(getFileSystem(), sortOutput); + JobConf jobConf = newJobConf(); + patchConfigurationForCommitter(jobConf); + executeStage("teravalidate", + jobConf, + sortValidate, + new TeraValidate(), + new String[]{sortOutput.toString(), sortValidate.toString()}, + 1); + } + + /** + * Print the results, and save to the base dir as a CSV file. + * Why there? Makes it easy to list and compare. + */ + @Test + public void test_140_teracomplete() throws Throwable { + terasortDuration.ifPresent(d -> { + d.close(); + completedStage("overall", d); + }); + + // IO Statistics + IOStatisticsLogging.logIOStatisticsAtLevel(LOG, IOSTATISTICS_LOGGING_LEVEL_INFO, JOB_IOSTATS); + + // and the summary + final StringBuilder results = new StringBuilder(); + results.append("\"Operation\"\t\"Duration\"\n"); + + // this is how you dynamically create a function in a method + // for use afterwards. + // Works because there's no IOEs being raised in this sequence. + Consumer stage = (s) -> { + DurationInfo duration = COMPLETED_STAGES.get(s); + results.append(String.format("\"%s\"\t\"%s\"\n", + s, + duration == null ? "" : duration)); + }; + + stage.accept("teragen"); + stage.accept("terasort"); + stage.accept("teravalidate"); + stage.accept("overall"); + String text = results.toString(); + File resultsFile = File.createTempFile("results", ".csv"); + FileUtils.write(resultsFile, text, StandardCharsets.UTF_8); + LOG.info("Results are in {}\n{}", resultsFile, text); + } + + /** + * Reset the duration so if two committer tests are run sequentially. + * Without this the total execution time is reported as from the start of + * the first test suite to the end of the second. + */ + @Test + public void test_150_teracleanup() throws Throwable { + terasortDuration = Optional.empty(); + } + + @Test + public void test_200_directory_deletion() throws Throwable { + getFileSystem().delete(terasortPath, true); + } + + /** + * Dump the files under a path -but not fail if the path is not present., + * @param path path to dump + * @throws Exception any failure. + */ + protected void dumpOutputTree(Path path) throws Exception { + LOG.info("Files under output directory {}", path); + try { + RemoteIterators.foreach(getFileSystem().listFiles(path, true), + (status) -> LOG.info("{}", status)); + } catch (FileNotFoundException e) { + LOG.info("Output directory {} not found", path); + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java new file mode 100644 index 0000000000000..3d49d62eaa865 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/commit/package-info.java @@ -0,0 +1,24 @@ +/* + * 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. + */ + +/** + * Unit and integration tests for the manifest committer. + * JSON job reports will be saved to + * {@code target/reports} + */ +package org.apache.hadoop.fs.azurebfs.commit; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java index 62bcca174ef8d..1319ea44c7c07 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/contract/AbfsFileSystemContract.java @@ -34,7 +34,7 @@ public class AbfsFileSystemContract extends AbstractBondedFSContract { public static final String CONTRACT_XML = "abfs.xml"; private final boolean isSecure; - protected AbfsFileSystemContract(final Configuration conf, boolean secure) { + public AbfsFileSystemContract(final Configuration conf, boolean secure) { super(conf); //insert the base features addConfResource(CONTRACT_XML); diff --git a/hadoop-tools/hadoop-azure/src/test/resources/core-site.xml b/hadoop-tools/hadoop-azure/src/test/resources/core-site.xml new file mode 100644 index 0000000000000..7d2d11c04eff3 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/resources/core-site.xml @@ -0,0 +1,25 @@ + + + + + + + + + From da9970dd697752b4d00fe4e4760ea9cbf019ff2e Mon Sep 17 00:00:00 2001 From: Abhishek Das Date: Thu, 17 Feb 2022 20:16:19 -0800 Subject: [PATCH 043/145] HADOOP-18129: Change URI to String in INodeLink to reduce memory footprint of ViewFileSystem Fixes #3996 --- .../apache/hadoop/fs/viewfs/InodeTree.java | 28 +++++++++---------- .../hadoop/fs/viewfs/ViewFileSystem.java | 16 ++++++++--- .../org/apache/hadoop/fs/viewfs/ViewFs.java | 10 ++++--- .../fs/viewfs/ViewFileSystemBaseTest.java | 18 ++++++++++++ 4 files changed, 50 insertions(+), 22 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index bdc2b4e9d74b7..ef40f98cc02e0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -273,7 +273,7 @@ enum LinkType { * is changed later it is then ignored (a dir with null entries) */ public static class INodeLink extends INode { - final URI[] targetDirLinkList; + final String[] targetDirLinkList; private T targetFileSystem; // file system object created from the link. // Function to initialize file system. Only applicable for simple links private Function fileSystemInitMethod; @@ -283,7 +283,7 @@ public static class INodeLink extends INode { * Construct a mergeLink or nfly. */ INodeLink(final String pathToNode, final UserGroupInformation aUgi, - final T targetMergeFs, final URI[] aTargetDirLinkList) { + final T targetMergeFs, final String[] aTargetDirLinkList) { super(pathToNode, aUgi); targetFileSystem = targetMergeFs; targetDirLinkList = aTargetDirLinkList; @@ -294,11 +294,11 @@ public static class INodeLink extends INode { */ INodeLink(final String pathToNode, final UserGroupInformation aUgi, Function createFileSystemMethod, - final URI aTargetDirLink) { + final String aTargetDirLink) throws URISyntaxException { super(pathToNode, aUgi); targetFileSystem = null; - targetDirLinkList = new URI[1]; - targetDirLinkList[0] = aTargetDirLink; + targetDirLinkList = new String[1]; + targetDirLinkList[0] = new URI(aTargetDirLink).toString(); this.fileSystemInitMethod = createFileSystemMethod; } @@ -336,7 +336,8 @@ public T getTargetFileSystem() throws IOException { if (targetFileSystem != null) { return targetFileSystem; } - targetFileSystem = fileSystemInitMethod.apply(targetDirLinkList[0]); + targetFileSystem = + fileSystemInitMethod.apply(URI.create(targetDirLinkList[0])); if (targetFileSystem == null) { throw new IOException( "Could not initialize target File System for URI : " + @@ -404,7 +405,7 @@ private void createLink(final String src, final String target, switch (linkType) { case SINGLE: newLink = new INodeLink(fullPath, aUgi, - initAndGetTargetFs(), new URI(target)); + initAndGetTargetFs(), target); break; case SINGLE_FALLBACK: case MERGE_SLASH: @@ -413,10 +414,10 @@ private void createLink(final String src, final String target, throw new IllegalArgumentException("Unexpected linkType: " + linkType); case MERGE: case NFLY: - final URI[] targetUris = StringUtils.stringToURI( - StringUtils.getStrings(target)); + final String[] targetUris = StringUtils.getStrings(target); newLink = new INodeLink(fullPath, aUgi, - getTargetFileSystem(settings, targetUris), targetUris); + getTargetFileSystem(settings, StringUtils.stringToURI(targetUris)), + targetUris); break; default: throw new IllegalArgumentException(linkType + ": Infeasible linkType"); @@ -633,8 +634,7 @@ protected InodeTree(final Configuration config, final String viewName, if (isMergeSlashConfigured) { Preconditions.checkNotNull(mergeSlashTarget); root = new INodeLink(mountTableName, ugi, - initAndGetTargetFs(), - new URI(mergeSlashTarget)); + initAndGetTargetFs(), mergeSlashTarget); mountPoints.add(new MountPoint("/", (INodeLink) root)); rootFallbackLink = null; } else { @@ -652,7 +652,7 @@ protected InodeTree(final Configuration config, final String viewName, + "not allowed."); } fallbackLink = new INodeLink(mountTableName, ugi, - initAndGetTargetFs(), new URI(le.getTarget())); + initAndGetTargetFs(), le.getTarget()); continue; case REGEX: addRegexMountEntry(le); @@ -677,7 +677,7 @@ protected InodeTree(final Configuration config, final String viewName, .info("Empty mount table detected for {} and considering itself " + "as a linkFallback.", theUri); rootFallbackLink = new INodeLink(mountTableName, ugi, - initAndGetTargetFs(), theUri); + initAndGetTargetFs(), theUri.toString()); getRootDir().addFallbackLink(rootFallbackLink); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index fde0faa59b45e..b30e086cf06b0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -216,11 +216,11 @@ public static class MountPoint { /** * Array of target FileSystem URIs. */ - private final URI[] targetFileSystemURIs; + private final String[] targetFileSystemPaths; - MountPoint(Path srcPath, URI[] targetFs) { + MountPoint(Path srcPath, String[] targetFs) { mountedOnPath = srcPath; - targetFileSystemURIs = targetFs; + targetFileSystemPaths = targetFs; } public Path getMountedOnPath() { @@ -228,7 +228,15 @@ public Path getMountedOnPath() { } public URI[] getTargetFileSystemURIs() { - return targetFileSystemURIs; + URI[] targetUris = new URI[targetFileSystemPaths.length]; + for (int i = 0; i < targetFileSystemPaths.length; i++) { + targetUris[i] = URI.create(targetFileSystemPaths[i]); + } + return targetUris; + } + + public String[] getTargetFileSystemPaths() { + return targetFileSystemPaths; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java index deaf4f4108a3e..6d619b1d6779d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java @@ -185,16 +185,18 @@ static AccessControlException readOnlyMountTable(final String operation, static public class MountPoint { - private Path src; // the src of the mount - private URI[] targets; // target of the mount; Multiple targets imply mergeMount - MountPoint(Path srcPath, URI[] targetURIs) { + // the src of the mount + private Path src; + // Target of the mount; Multiple targets imply mergeMount + private String[] targets; + MountPoint(Path srcPath, String[] targetURIs) { src = srcPath; targets = targetURIs; } Path getSrc() { return src; } - URI[] getTargets() { + String[] getTargets() { return targets; } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index 90ffa06bfce30..6eb0570f9c60b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -1687,4 +1687,22 @@ public void testTargetFileSystemLazyInitializationForChecksumMethods() // viewfs inner cache is disabled assertEquals(cacheSize + 1, TestFileUtil.getCacheSize()); } + + @Test + public void testInvalidMountPoints() throws Exception { + final String clusterName = "cluster" + new Random().nextInt(); + Configuration config = new Configuration(conf); + config.set(ConfigUtil.getConfigViewFsPrefix(clusterName) + "." + + Constants.CONFIG_VIEWFS_LINK + "." + "/invalidPath", + "othermockfs:|mockauth/mockpath"); + + try { + FileSystem viewFs = FileSystem.get( + new URI("viewfs://" + clusterName + "/"), config); + fail("FileSystem should not initialize. Should fail with IOException"); + } catch (IOException ex) { + assertTrue("Should get URISyntax Exception", + ex.getMessage().startsWith("URISyntax exception")); + } + } } From a631f45a99c7abf8c9a2dcfb10afb668c8ff6b09 Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Fri, 18 Mar 2022 21:41:01 +0530 Subject: [PATCH 044/145] HDFS-16470. Make HDFS find tool cross platform (#4076) * The source files for hdfs_find uses getopt for parsing the command line arguments. getopt is available only on Linux and thus, isn't cross platform. * Thus, we need to replace getopt with boost::program_options to make hdfs_find cross platform. --- .../libhdfspp/tests/tools/CMakeLists.txt | 3 + .../tests/tools/hdfs-create-snapshot-mock.cc | 8 +- .../libhdfspp/tests/tools/hdfs-find-mock.cc | 93 +++++++++ .../libhdfspp/tests/tools/hdfs-find-mock.h | 69 +++++++ .../libhdfspp/tests/tools/hdfs-tool-tests.cc | 20 ++ .../libhdfspp/tests/tools/hdfs-tool-tests.h | 32 ++- .../native/libhdfspp/tools/CMakeLists.txt | 3 +- .../libhdfspp/tools/hdfs-find/CMakeLists.txt | 27 +++ .../libhdfspp/tools/hdfs-find/hdfs-find.cc | 193 ++++++++++++++++++ .../libhdfspp/tools/hdfs-find/hdfs-find.h | 96 +++++++++ .../native/libhdfspp/tools/hdfs-find/main.cc | 52 +++++ .../main/native/libhdfspp/tools/hdfs_find.cc | 146 ------------- 12 files changed, 590 insertions(+), 152 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/CMakeLists.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/main.cc delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_find.cc diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt index 769e5da0f1ccd..22d677f0a29c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt @@ -36,6 +36,7 @@ add_executable(hdfs_tool_tests hdfs-mkdir-mock.cc hdfs-rm-mock.cc hdfs-get-mock.cc + hdfs-find-mock.cc main.cc) target_include_directories(hdfs_tool_tests PRIVATE ../tools @@ -56,6 +57,7 @@ target_include_directories(hdfs_tool_tests PRIVATE ../../tools/hdfs-mkdir ../../tools/hdfs-rm ../../tools/hdfs-get + ../../tools/hdfs-find ../../tools/hdfs-cat) target_link_libraries(hdfs_tool_tests PRIVATE gmock_main @@ -75,5 +77,6 @@ target_link_libraries(hdfs_tool_tests PRIVATE hdfs_mkdir_lib hdfs_rm_lib hdfs_get_lib + hdfs_find_lib hdfs_cat_lib) add_test(hdfs_tool_tests hdfs_tool_tests) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-create-snapshot-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-create-snapshot-mock.cc index 323963181fc37..6a4f96a83fd61 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-create-snapshot-mock.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-create-snapshot-mock.cc @@ -48,9 +48,11 @@ void CreateSnapshotMock::SetExpectations( } if (*test_case_func == &PassNOptAndAPath) { - const auto arg1 = args[1]; - const auto arg2 = std::optional{args[0]}; - EXPECT_CALL(*this, HandleSnapshot(arg1, arg2)) + const auto opt_n = args[0]; + const auto path = args[2]; + const auto opt_n_value = std::optional{args[1]}; + ASSERT_EQ(opt_n, "-n"); + EXPECT_CALL(*this, HandleSnapshot(path, opt_n_value)) .Times(1) .WillOnce(testing::Return(true)); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.cc new file mode 100644 index 0000000000000..9fd57ec270c07 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.cc @@ -0,0 +1,93 @@ +/** + * 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. + */ + +#include +#include +#include +#include + +#include +#include + +#include "hdfs-find-mock.h" +#include "hdfs-tool-tests.h" +#include "hdfspp/hdfspp.h" + +namespace hdfs::tools::test { +FindMock::~FindMock() = default; + +void FindMock::SetExpectations( + std::function()> test_case, + const std::vector &args) const { + // Get the pointer to the function that defines the test case + const auto test_case_func = + test_case.target (*)()>(); + ASSERT_NE(test_case_func, nullptr); + + // Set the expected method calls and their corresponding arguments for each + // test case + if (*test_case_func == &CallHelp) { + EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true)); + return; + } + + if (*test_case_func == &PassAPath) { + const auto arg1 = args[0]; + EXPECT_CALL(*this, HandlePath(arg1, "*", + hdfs::FileSystem::GetDefaultFindMaxDepth())) + .Times(1) + .WillOnce(testing::Return(true)); + } + + if (*test_case_func == &PassNOptAndAPath) { + const auto arg1 = args[0]; + const auto arg2 = args[1]; + const auto arg3 = args[2]; + ASSERT_EQ(arg1, "-n"); + EXPECT_CALL(*this, HandlePath(arg3, arg2, + hdfs::FileSystem::GetDefaultFindMaxDepth())) + .Times(1) + .WillOnce(testing::Return(true)); + } + + if (*test_case_func == &PassMOptPermissionsAndAPath) { + const auto arg1 = args[0]; + const auto arg2 = args[1]; + const auto arg3 = args[2]; + ASSERT_EQ(arg1, "-m"); + EXPECT_CALL(*this, + HandlePath(arg3, "*", static_cast(std::stoi(arg2)))) + .Times(1) + .WillOnce(testing::Return(true)); + } + + if (*test_case_func == &PassNStrMNumAndAPath) { + const auto arg1 = args[0]; + const auto arg2 = args[1]; + const auto arg3 = args[2]; + const auto arg4 = args[3]; + const auto arg5 = args[4]; + ASSERT_EQ(arg1, "-n"); + ASSERT_EQ(arg3, "-m"); + EXPECT_CALL(*this, + HandlePath(arg5, arg2, static_cast(std::stoi(arg4)))) + .Times(1) + .WillOnce(testing::Return(true)); + } +} +} // namespace hdfs::tools::test diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.h new file mode 100644 index 0000000000000..7520ac7c0da03 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-find-mock.h @@ -0,0 +1,69 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_FIND_MOCK +#define LIBHDFSPP_TOOLS_HDFS_FIND_MOCK + +#include +#include +#include +#include + +#include + +#include "hdfs-find.h" + +namespace hdfs::tools::test { +/** + * {@class FindMock} is an {@class Find} whereby it mocks the + * HandleHelp and HandlePath methods for testing their functionality. + */ +class FindMock : public hdfs::tools::Find { +public: + /** + * {@inheritdoc} + */ + FindMock(const int argc, char **argv) : Find(argc, argv) {} + + // Abiding to the Rule of 5 + FindMock(const FindMock &) = delete; + FindMock(FindMock &&) = delete; + FindMock &operator=(const FindMock &) = delete; + FindMock &operator=(FindMock &&) = delete; + ~FindMock() override; + + /** + * Defines the methods and the corresponding arguments that are expected + * to be called on this instance of {@link HdfsTool} for the given test case. + * + * @param test_case An {@link std::function} object that points to the + * function defining the test case + * @param args The arguments that are passed to this test case + */ + void SetExpectations(std::function()> test_case, + const std::vector &args = {}) const; + + MOCK_METHOD(bool, HandleHelp, (), (const, override)); + + MOCK_METHOD(bool, HandlePath, + (const std::string &, const std::string &, uint32_t), + (const, override)); +}; +} // namespace hdfs::tools::test + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc index 50d555aebda0c..53df82021bd6e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc @@ -31,6 +31,7 @@ #include "hdfs-df-mock.h" #include "hdfs-disallow-snapshot-mock.h" #include "hdfs-du-mock.h" +#include "hdfs-find-mock.h" #include "hdfs-get-mock.h" #include "hdfs-mkdir-mock.h" #include "hdfs-move-to-local-mock.h" @@ -140,6 +141,14 @@ INSTANTIATE_TEST_SUITE_P( PassAPath, PassRecursivePath)); +INSTANTIATE_TEST_SUITE_P( + HdfsFind, HdfsToolBasicTest, + testing::Values(CallHelp, + PassAPath, + PassNStrMNumAndAPath, + PassMOptPermissionsAndAPath, + PassNOptAndAPath)); + // Negative tests INSTANTIATE_TEST_SUITE_P( HdfsAllowSnapshot, HdfsToolNegativeTestThrows, @@ -210,6 +219,17 @@ INSTANTIATE_TEST_SUITE_P( PassRecursiveOwnerAndAPath, PassMOpt)); +INSTANTIATE_TEST_SUITE_P( + HdfsFind, HdfsToolNegativeTestThrows, + testing::Values(Pass2Paths, + Pass3Paths, + PassRecursiveOwnerAndAPath, + PassRecursive, + PassRecursivePath, + PassMPOptsPermissionsAndAPath, + PassMOpt, + PassNOpt)); + INSTANTIATE_TEST_SUITE_P( HdfsRm, HdfsToolNegativeTestNoThrow, testing::Values(PassRecursive)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h index 12dbc6c01ce24..f27a2b022da68 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h @@ -118,7 +118,7 @@ template std::unique_ptr PassNOptAndAPath() { static char *argv[] = {exe.data(), arg1.data(), arg2.data(), arg3.data()}; auto hdfs_tool = std::make_unique(argc, argv); - hdfs_tool->SetExpectations(PassNOptAndAPath, {arg2, arg3}); + hdfs_tool->SetExpectations(PassNOptAndAPath, {arg1, arg2, arg3}); return hdfs_tool; } @@ -271,4 +271,34 @@ template std::unique_ptr PassMPOptsPermissionsAndAPath() { return hdfs_tool; } +template std::unique_ptr PassNStrMNumAndAPath() { + constexpr auto argc = 6; + static std::string exe("hdfs_tool_name"); + static std::string arg1("-n"); + static std::string arg2("some_str"); + static std::string arg3("-m"); + static std::string arg4("757"); + static std::string arg5("some/path"); + + static char *argv[] = {exe.data(), arg1.data(), arg2.data(), + arg3.data(), arg4.data(), arg5.data()}; + + auto hdfs_tool = std::make_unique(argc, argv); + hdfs_tool->SetExpectations(PassNStrMNumAndAPath, + {arg1, arg2, arg3, arg4, arg5}); + return hdfs_tool; +} + +template std::unique_ptr PassNOpt() { + constexpr auto argc = 2; + static std::string exe("hdfs_tool_name"); + static std::string arg1("-n"); + + static char *argv[] = {exe.data(), arg1.data()}; + + auto hdfs_tool = std::make_unique(argc, argv); + hdfs_tool->SetExpectations(PassNOpt, {arg1}); + return hdfs_tool; +} + #endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt index bed78b7a5777d..214d7b56dcb5c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt @@ -41,8 +41,7 @@ add_subdirectory(hdfs-chown) add_subdirectory(hdfs-chmod) -add_executable(hdfs_find hdfs_find.cc) -target_link_libraries(hdfs_find tools_common hdfspp_static) +add_subdirectory(hdfs-find) add_subdirectory(hdfs-mkdir) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/CMakeLists.txt new file mode 100644 index 0000000000000..c6ce02132875a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/CMakeLists.txt @@ -0,0 +1,27 @@ +# +# 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. +# + +add_library(hdfs_find_lib STATIC $ hdfs-find.cc) +target_include_directories(hdfs_find_lib PRIVATE ../../tools ${Boost_INCLUDE_DIRS}) +target_link_libraries(hdfs_find_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static) + +add_executable(hdfs_find main.cc) +target_include_directories(hdfs_find PRIVATE ../../tools) +target_link_libraries(hdfs_find PRIVATE hdfs_find_lib) + +install(TARGETS hdfs_find RUNTIME DESTINATION bin) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.cc new file mode 100644 index 0000000000000..a149d05c1399a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.cc @@ -0,0 +1,193 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include + +#include "hdfs-find.h" +#include "tools_common.h" + +namespace hdfs::tools { +Find::Find(const int argc, char **argv) : HdfsTool(argc, argv) {} + +bool Find::Initialize() { + auto add_options = opt_desc_.add_options(); + add_options( + "help,h", + "Finds all files recursively starting from the specified PATH and prints " + "their file paths. This hdfs_find tool mimics the POSIX find."); + add_options( + "name,n", po::value(), + "If provided, all results will be matching the NAME pattern otherwise, " + "the implicit '*' will be used NAME allows wild-cards"); + add_options( + "max-depth,m", po::value(), + "If provided, the maximum depth to recurse after the end of the path is " + "reached will be limited by MAX_DEPTH otherwise, the maximum depth to " + "recurse is unbound MAX_DEPTH can be set to 0 for pure globbing and " + "ignoring the NAME option (no recursion after the end of the path)"); + add_options("path", po::value(), + "The path where we want to start the find operation"); + + // We allow only one positional argument to be passed to this tool. An + // exception is thrown if multiple arguments are passed. + pos_opt_desc_.add("path", 1); + + po::store(po::command_line_parser(argc_, argv_) + .options(opt_desc_) + .positional(pos_opt_desc_) + .run(), + opt_val_); + po::notify(opt_val_); + return true; +} + +std::string Find::GetDescription() const { + std::stringstream desc; + desc << "Usage: hdfs_find [OPTION] PATH" << std::endl + << std::endl + << "Finds all files recursively starting from the" << std::endl + << "specified PATH and prints their file paths." << std::endl + << "This hdfs_find tool mimics the POSIX find." << std::endl + << std::endl + << "Both PATH and NAME can have wild-cards." << std::endl + << std::endl + << " -n NAME if provided all results will be matching the NAME " + "pattern" + << std::endl + << " otherwise, the implicit '*' will be used" + << std::endl + << " NAME allows wild-cards" << std::endl + << std::endl + << " -m MAX_DEPTH if provided the maximum depth to recurse after the " + "end of" + << std::endl + << " the path is reached will be limited by MAX_DEPTH" + << std::endl + << " otherwise, the maximum depth to recurse is unbound" + << std::endl + << " MAX_DEPTH can be set to 0 for pure globbing and " + "ignoring" + << std::endl + << " the NAME option (no recursion after the end of the " + "path)" + << std::endl + << std::endl + << " -h display this help and exit" << std::endl + << std::endl + << "Examples:" << std::endl + << "hdfs_find hdfs://localhost.localdomain:8020/dir?/tree* -n " + "some?file*name" + << std::endl + << "hdfs_find / -n file_name -m 3" << std::endl; + return desc.str(); +} + +bool Find::Do() { + if (!Initialize()) { + std::cerr << "Unable to initialize HDFS find tool" << std::endl; + return false; + } + + if (!ValidateConstraints()) { + std::cout << GetDescription(); + return false; + } + + if (opt_val_.count("help") > 0) { + return HandleHelp(); + } + + if (opt_val_.count("path") > 0) { + const auto path = opt_val_["path"].as(); + const auto name = + opt_val_.count("name") > 0 ? opt_val_["name"].as() : "*"; + const auto max_depth = opt_val_.count("max-depth") <= 0 + ? hdfs::FileSystem::GetDefaultFindMaxDepth() + : opt_val_["max-depth"].as(); + return HandlePath(path, name, max_depth); + } + + return false; +} + +bool Find::HandleHelp() const { + std::cout << GetDescription(); + return true; +} + +bool Find::HandlePath(const std::string &path, const std::string &name, + const uint32_t max_depth) const { + // Building a URI object from the given path + auto uri = hdfs::parse_path_or_exit(path); + + const auto fs = hdfs::doConnect(uri, true); + if (!fs) { + std::cerr << "Could not connect the file system." << std::endl; + return false; + } + + const auto promise = std::make_shared>(); + std::future future(promise->get_future()); + auto final_status = hdfs::Status::OK(); + + /** + * Keep requesting more until we get the entire listing. Set the promise + * when we have the entire listing to stop. + * + * Find guarantees that the handler will only be called once at a time, + * so we do not need any locking here. It also guarantees that the handler + * will be only called once with has_more_results set to false. + */ + auto handler = [promise, + &final_status](const hdfs::Status &status, + const std::vector &stat_info, + const bool has_more_results) -> bool { + // Print result chunks as they arrive + if (!stat_info.empty()) { + for (hdfs::StatInfo const &info : stat_info) { + std::cout << info.str() << std::endl; + } + } + if (!status.ok() && final_status.ok()) { + // We make sure we set 'status' only on the first error + final_status = status; + } + if (!has_more_results) { + promise->set_value(); // Set promise + return false; // Request stop sending results + } + return true; // request more results + }; + + // Asynchronous call to Find + fs->Find(uri.get_path(), name, max_depth, handler); + + // Block until promise is set + future.get(); + if (!final_status.ok()) { + std::cerr << "Error: " << final_status.ToString() << std::endl; + return false; + } + return true; +} +} // namespace hdfs::tools diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.h new file mode 100644 index 0000000000000..9adde3c622bf1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/hdfs-find.h @@ -0,0 +1,96 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_FIND +#define LIBHDFSPP_TOOLS_HDFS_FIND + +#include + +#include + +#include "hdfs-tool.h" + +namespace hdfs::tools { +/** + * {@class Find} is an {@class HdfsTool} finds all files recursively starting + * from the specified PATH and prints their file paths. This tool mimics the + * POSIX find. + */ +class Find : public HdfsTool { +public: + /** + * {@inheritdoc} + */ + Find(int argc, char **argv); + + // Abiding to the Rule of 5 + Find(const Find &) = default; + Find(Find &&) = default; + Find &operator=(const Find &) = delete; + Find &operator=(Find &&) = delete; + ~Find() override = default; + + /** + * {@inheritdoc} + */ + [[nodiscard]] std::string GetDescription() const override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Do() override; + +protected: + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Initialize() override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool ValidateConstraints() const override { return argc_ > 1; } + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool HandleHelp() const override; + + /** + * Handle the path argument that's passed to this tool. + * + * @param path The path to the directory to begin the find. + * @param name The pattern name of the search term. + * @param max_depth The maximum depth of the traversal while searching through + * the folders. + * + * @return A boolean indicating the result of this operation. + */ + [[nodiscard]] virtual bool HandlePath(const std::string &path, + const std::string &name, + uint32_t max_depth) const; + +private: + /** + * A boost data-structure containing the description of positional arguments + * passed to the command-line. + */ + po::positional_options_description pos_opt_desc_; +}; +} // namespace hdfs::tools +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/main.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/main.cc new file mode 100644 index 0000000000000..1f63aa7a2d8e3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-find/main.cc @@ -0,0 +1,52 @@ +/** + * 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. + */ + +#include +#include +#include + +#include + +#include "hdfs-find.h" + +int main(int argc, char *argv[]) { + const auto result = std::atexit([]() -> void { + // Clean up static data on exit and prevent valgrind memory leaks + google::protobuf::ShutdownProtobufLibrary(); + }); + if (result != 0) { + std::cerr + << "Error: Unable to schedule clean-up tasks for HDFS find tool, exiting" + << std::endl; + std::exit(EXIT_FAILURE); + } + + hdfs::tools::Find find(argc, argv); + auto success = false; + + try { + success = find.Do(); + } catch (const std::exception &e) { + std::cerr << "Error: " << e.what() << std::endl; + } + + if (!success) { + std::exit(EXIT_FAILURE); + } + return 0; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_find.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_find.cc deleted file mode 100644 index 348f851ad3835..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_find.cc +++ /dev/null @@ -1,146 +0,0 @@ -/* - 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. -*/ - -#include -#include -#include -#include "tools_common.h" - -void usage(){ - std::cout << "Usage: hdfs_find [OPTION] PATH" - << std::endl - << std::endl << "Finds all files recursively starting from the" - << std::endl << "specified PATH and prints their file paths." - << std::endl << "This hdfs_find tool mimics the POSIX find." - << std::endl - << std::endl << "Both PATH and NAME can have wild-cards." - << std::endl - << std::endl << " -n NAME if provided all results will be matching the NAME pattern" - << std::endl << " otherwise, the implicit '*' will be used" - << std::endl << " NAME allows wild-cards" - << std::endl - << std::endl << " -m MAX_DEPTH if provided the maximum depth to recurse after the end of" - << std::endl << " the path is reached will be limited by MAX_DEPTH" - << std::endl << " otherwise, the maximum depth to recurse is unbound" - << std::endl << " MAX_DEPTH can be set to 0 for pure globbing and ignoring" - << std::endl << " the NAME option (no recursion after the end of the path)" - << std::endl - << std::endl << " -h display this help and exit" - << std::endl - << std::endl << "Examples:" - << std::endl << "hdfs_find hdfs://localhost.localdomain:8020/dir?/tree* -n some?file*name" - << std::endl << "hdfs_find / -n file_name -m 3" - << std::endl; -} - -int main(int argc, char *argv[]) { - //We should have at least 2 arguments - if (argc < 2) { - usage(); - exit(EXIT_FAILURE); - } - - int input; - //If NAME is not specified we use implicit "*" - std::string name = "*"; - //If MAX_DEPTH is not specified we use the max value of uint_32_t - uint32_t max_depth = hdfs::FileSystem::GetDefaultFindMaxDepth(); - - //Using GetOpt to read in the values - opterr = 0; - while ((input = getopt(argc, argv, "hn:m:")) != -1) { - switch (input) - { - case 'h': - usage(); - exit(EXIT_SUCCESS); - case 'n': - name = optarg; - break; - case 'm': - max_depth = std::stoi(optarg); - break; - case '?': - if (optopt == 'n' || optopt == 'm') - std::cerr << "Option -" << (char) optopt << " requires an argument." << std::endl; - else if (isprint(optopt)) - std::cerr << "Unknown option `-" << (char) optopt << "'." << std::endl; - else - std::cerr << "Unknown option character `" << (char) optopt << "'." << std::endl; - usage(); - exit(EXIT_FAILURE); - default: - exit(EXIT_FAILURE); - } - } - std::string uri_path = argv[optind]; - - //Building a URI object from the given uri_path - hdfs::URI uri = hdfs::parse_path_or_exit(uri_path); - - std::shared_ptr fs = hdfs::doConnect(uri, true); - if (!fs) { - std::cerr << "Could not connect the file system. " << std::endl; - exit(EXIT_FAILURE); - } - - std::shared_ptr> promise = std::make_shared>(); - std::future future(promise->get_future()); - hdfs::Status status = hdfs::Status::OK(); - - /** - * Keep requesting more until we get the entire listing. Set the promise - * when we have the entire listing to stop. - * - * Find guarantees that the handler will only be called once at a time, - * so we do not need any locking here. It also guarantees that the handler will be - * only called once with has_more_results set to false. - */ - auto handler = [promise, &status] - (const hdfs::Status &s, const std::vector & si, bool has_more_results) -> bool { - //Print result chunks as they arrive - if(!si.empty()) { - for (hdfs::StatInfo const& s : si) { - std::cout << s.str() << std::endl; - } - } - if(!s.ok() && status.ok()){ - //We make sure we set 'status' only on the first error. - status = s; - } - if (!has_more_results) { - promise->set_value(); //set promise - return false; //request stop sending results - } - return true; //request more results - }; - - //Asynchronous call to Find - fs->Find(uri.get_path(), name, max_depth, handler); - - //block until promise is set - future.get(); - if(!status.ok()) { - std::cerr << "Error: " << status.ToString() << std::endl; - } - - // Clean up static data and prevent valgrind memory leaks - google::protobuf::ShutdownProtobufLibrary(); - return 0; -} From 62d59e516ee7df1a1c026ac8e5552eb03442ec44 Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Sun, 20 Mar 2022 14:01:45 +0800 Subject: [PATCH 045/145] HDFS-16504. Add parameter for NameNode to process getBloks request. (#4068). Contributed by Max Xie. Signed-off-by: He Xiaoqiao --- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 4 ++++ .../hdfs/server/balancer/NameNodeConnector.java | 12 ++++++------ .../hdfs/server/namenode/FSNamesystem.java | 17 +++++++++++++++-- .../src/main/resources/hdfs-default.xml | 8 ++++++++ .../balancer/TestBalancerWithHANameNodes.java | 15 +++++++-------- 5 files changed, 40 insertions(+), 16 deletions(-) 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 b14e92d42d119..38949744b4697 100755 --- 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 @@ -746,6 +746,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys { */ public static final String DFS_NAMENODE_GETBLOCKS_MAX_QPS_KEY = "dfs.namenode.get-blocks.max-qps"; public static final int DFS_NAMENODE_GETBLOCKS_MAX_QPS_DEFAULT = 20; + public static final String DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_KEY + = "dfs.namenode.get-blocks.check.operation"; + public static final boolean DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_DEFAULT + = true; public static final String DFS_BALANCER_MOVEDWINWIDTH_KEY = "dfs.balancer.movedWinWidth"; public static final long DFS_BALANCER_MOVEDWINWIDTH_DEFAULT = 5400*1000L; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java index ce0fb968bbb9f..238457bcb867a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java @@ -147,11 +147,11 @@ public static void checkOtherInstanceRunning(boolean toCheck) { private final BalancerProtocols namenode; /** - * If set requestToStandby true, Balancer will getBlocks from + * If set getBlocksToStandby true, Balancer will getBlocks from * Standby NameNode only and it can reduce the performance impact of Active * NameNode, especially in a busy HA mode cluster. */ - private boolean requestToStandby; + private boolean getBlocksToStandby; private String nsId; private Configuration config; private final KeyManager keyManager; @@ -191,9 +191,9 @@ public NameNodeConnector(String name, URI nameNodeUri, Path idPath, this.namenode = NameNodeProxies.createProxy(conf, nameNodeUri, BalancerProtocols.class, fallbackToSimpleAuth).getProxy(); - this.requestToStandby = conf.getBoolean( - DFSConfigKeys.DFS_HA_ALLOW_STALE_READ_KEY, - DFSConfigKeys.DFS_HA_ALLOW_STALE_READ_DEFAULT); + this.getBlocksToStandby = !conf.getBoolean( + DFSConfigKeys.DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_KEY, + DFSConfigKeys.DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_DEFAULT); this.config = conf; this.fs = (DistributedFileSystem)FileSystem.get(nameNodeUri, conf); @@ -318,7 +318,7 @@ public DatanodeStorageReport[] getLiveDatanodeStorageReport() private ProxyPair getProxy() throws IOException { boolean isRequestStandby = false; ClientProtocol clientProtocol = null; - if (requestToStandby && nsId != null + if (getBlocksToStandby && nsId != null && HAUtil.isHAEnabled(config, nsId)) { List namenodes = HAUtil.getProxiesForAllNameNodesInNameservice(config, nsId); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index a61028bde7c78..c03281d6a9a91 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -505,6 +505,12 @@ private boolean isClientPortInfoAbsent(CallerContext ctx){ private final int snapshotDiffReportLimit; private final int blockDeletionIncrement; + /** + * Whether enable checkOperation when call getBlocks. + * It is enabled by default. + */ + private final boolean isGetBlocksCheckOperationEnabled; + /** Interval between each check of lease to release. */ private final long leaseRecheckIntervalMs; /** Maximum time the lock is hold to release lease. */ @@ -1066,6 +1072,10 @@ static FSNamesystem loadFromDisk(Configuration conf) throws IOException { Preconditions.checkArgument(blockDeletionIncrement > 0, DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_INCREMENT_KEY + " must be a positive integer."); + this.isGetBlocksCheckOperationEnabled = conf.getBoolean( + DFSConfigKeys.DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_KEY, + DFSConfigKeys.DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_DEFAULT); + } catch(IOException e) { LOG.error(getClass().getSimpleName() + " initialization failed.", e); close(); @@ -1938,10 +1948,13 @@ public boolean isInStandbyState() { */ public BlocksWithLocations getBlocks(DatanodeID datanode, long size, long minimumBlockSize, long timeInterval) throws IOException { - checkOperation(OperationCategory.READ); + OperationCategory checkOp = + isGetBlocksCheckOperationEnabled ? OperationCategory.READ : + OperationCategory.UNCHECKED; + checkOperation(checkOp); readLock(); try { - checkOperation(OperationCategory.READ); + checkOperation(checkOp); return getBlockManager().getBlocksWithLocations(datanode, size, minimumBlockSize, timeInterval); } finally { 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 584ae24ab134a..6890b8a8412c4 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -4119,6 +4119,14 @@ Mover, and StoragePolicySatisfier. + + dfs.namenode.get-blocks.check.operation + true + + Set false to disable checkOperation and getBlocks for Balancer + will route to Standby NameNode for HA mode setup. + + dfs.balancer.dispatcherThreads 200 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java index bb9b1a1cde6b7..9b4ba5ccb73a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java @@ -18,9 +18,7 @@ package org.apache.hadoop.hdfs.server.balancer; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_ALLOW_STALE_READ_DEFAULT; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_ALLOW_STALE_READ_KEY; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.*; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -146,8 +144,9 @@ void doTest(Configuration conf, boolean withHA) throws Exception { TestBalancer.createFile(cluster, TestBalancer.filePath, totalUsedSpace / numOfDatanodes, (short) numOfDatanodes, 0); - boolean isRequestStandby = conf.getBoolean( - DFS_HA_ALLOW_STALE_READ_KEY, DFS_HA_ALLOW_STALE_READ_DEFAULT); + boolean isRequestStandby = !conf.getBoolean( + DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_KEY, + DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_DEFAULT); if (isRequestStandby) { HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0), cluster.getNameNode(1)); @@ -182,7 +181,7 @@ void doTest(Configuration conf, boolean withHA) throws Exception { @Test(timeout = 60000) public void testBalancerRequestSBNWithHA() throws Exception { Configuration conf = new HdfsConfiguration(); - conf.setBoolean(DFS_HA_ALLOW_STALE_READ_KEY, true); + conf.setBoolean(DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_KEY, false); conf.setLong(DFS_HA_TAILEDITS_PERIOD_KEY, 1); //conf.setBoolean(DFS_HA_BALANCER_REQUEST_STANDBY_KEY, true); TestBalancer.initConf(conf); @@ -329,8 +328,8 @@ nsId, new Path("/test"), nncActive.close(); // Request to standby namenode. - conf.setBoolean(DFSConfigKeys.DFS_HA_ALLOW_STALE_READ_KEY, - true); + conf.setBoolean(DFS_NAMENODE_GETBLOCKS_CHECK_OPERATION_KEY, + false); NameNodeConnector nncStandby = new NameNodeConnector( "nncStandby", namenode, nsId, new Path("/test"), From e5549a2a68a02c8b2d42c421c5688a537d32a049 Mon Sep 17 00:00:00 2001 From: litao Date: Mon, 21 Mar 2022 15:10:51 +0800 Subject: [PATCH 046/145] HDFS-16503. Should verify whether the path name is valid in the WebHDFS (#4067). Contributed by tomscut. Signed-off-by: Ayush Saxena --- .../hadoop/hdfs/web/WebHdfsFileSystem.java | 3 +- .../router/RouterWebHdfsMethods.java | 5 +++ .../router/TestRouterWebHdfsMethods.java | 24 ++++++++++++++ .../web/resources/NamenodeWebHdfsMethods.java | 4 +++ .../apache/hadoop/hdfs/web/TestWebHDFS.java | 33 +++++++++++++++++++ 5 files changed, 68 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index ee5bf14c83c09..ebc12ff04b919 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -478,7 +478,8 @@ private Path makeAbsolute(Path f) { return f.isAbsolute()? f: new Path(workingDir, f); } - static Map jsonParse(final HttpURLConnection c, + @VisibleForTesting + public static Map jsonParse(final HttpURLConnection c, final boolean useErrorStream) throws IOException { if (c.getContentLength() == 0) { return null; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java index a812f198f5e57..accec4627eda8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java @@ -19,6 +19,8 @@ import static org.apache.hadoop.util.StringUtils.getTrimmedStringCollection; +import org.apache.hadoop.fs.InvalidPathException; +import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -419,6 +421,9 @@ private URI redirectURI(final Router router, final UserGroupInformation ugi, final DoAsParam doAsUser, final String path, final HttpOpParam.Op op, final long openOffset, final String excludeDatanodes, final Param... parameters) throws URISyntaxException, IOException { + if (!DFSUtil.isValidName(path)) { + throw new InvalidPathException(path); + } final DatanodeInfo dn = chooseDatanode(router, path, op, openOffset, excludeDatanodes); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWebHdfsMethods.java index 8e82d44c4ddfe..7c3643f5a511e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWebHdfsMethods.java @@ -23,9 +23,12 @@ import static org.junit.Assert.fail; import java.io.FileNotFoundException; +import java.io.IOException; import java.net.HttpURLConnection; import java.net.URL; import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -34,6 +37,7 @@ import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster; import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder; +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -145,4 +149,24 @@ public void testGetNsFromDataNodeNetworkLocation() { assertEquals("", RouterWebHdfsMethods .getNsFromDataNodeNetworkLocation("whatever-rack-info1")); } + + @Test + public void testWebHdfsCreateWithInvalidPath() throws Exception { + // A path name include duplicated slashes. + String path = "//tmp//file"; + assertResponse(path); + } + + private void assertResponse(String path) throws IOException { + URL url = new URL(getUri(path)); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestMethod("PUT"); + // Assert response code. + assertEquals(HttpURLConnection.HTTP_BAD_REQUEST, conn.getResponseCode()); + // Assert exception. + Map response = WebHdfsFileSystem.jsonParse(conn, true); + assertEquals("InvalidPathException", + ((LinkedHashMap) response.get("RemoteException")).get("exception")); + conn.disconnect(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java index 5910a80700ec6..a3250c213caf0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java @@ -55,6 +55,7 @@ import javax.ws.rs.core.Response.ResponseBuilder; import javax.ws.rs.core.Response.Status; +import org.apache.hadoop.fs.InvalidPathException; import org.apache.hadoop.fs.QuotaUsage; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -401,6 +402,9 @@ private URI redirectURI(ResponseBuilder rb, final NameNode namenode, final String path, final HttpOpParam.Op op, final long openOffset, final long blocksize, final String excludeDatanodes, final Param... parameters) throws URISyntaxException, IOException { + if (!DFSUtil.isValidName(path)) { + throw new InvalidPathException(path); + } final DatanodeInfo dn; final NamenodeProtocols np = getRPCServer(namenode); HdfsFileStatus status = null; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java index 2461d22dde81c..c4f53b0561573 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java @@ -55,6 +55,7 @@ import java.util.Collection; import java.util.EnumSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.Map; import java.util.NoSuchElementException; import java.util.Random; @@ -522,6 +523,38 @@ public void testCreateWithNoDN() throws Exception { } } + @Test + public void testWebHdfsCreateWithInvalidPath() throws Exception { + final Configuration conf = WebHdfsTestUtil.createConf(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build(); + // A path name include duplicated slashes. + String path = "//tmp//file"; + assertResponse(path); + } + + private String getUri(String path) { + final String user = System.getProperty("user.name"); + final StringBuilder uri = new StringBuilder(cluster.getHttpUri(0)); + uri.append("/webhdfs/v1"). + append(path). + append("?op=CREATE"). + append("&user.name=" + user); + return uri.toString(); + } + + private void assertResponse(String path) throws IOException { + URL url = new URL(getUri(path)); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setRequestMethod("PUT"); + // Assert response code. + assertEquals(HttpURLConnection.HTTP_BAD_REQUEST, conn.getResponseCode()); + // Assert exception. + Map response = WebHdfsFileSystem.jsonParse(conn, true); + assertEquals("InvalidPathException", + ((LinkedHashMap) response.get("RemoteException")).get("exception")); + conn.disconnect(); + } + /** * Test allow and disallow snapshot through WebHdfs. Verifying webhdfs with * Distributed filesystem methods. From 4537b34e1c088f2b6d61c9bad8b82438cd94b944 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Mon, 21 Mar 2022 14:18:27 +0100 Subject: [PATCH 047/145] YARN-11089. Fix typo in RM audit log. Contributed by Junfan Zhang. --- .../apache/hadoop/yarn/server/resourcemanager/RMAppManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index 7ebcd656f9b70..928eeb19160f1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -294,7 +294,7 @@ protected synchronized void finishApplication(ApplicationId applicationId) { protected void writeAuditLog(ApplicationId appId) { RMApp app = rmContext.getRMApps().get(appId); - String operation = "UNKONWN"; + String operation = "UNKNOWN"; boolean success = false; switch (app.getState()) { case FAILED: From e2701e227f82b89c9dcde53ef8fa49f757a61fa8 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Mon, 21 Mar 2022 14:37:10 +0100 Subject: [PATCH 048/145] YARN-11086. Add space in debug log of ParentQueue. Contributed by Junfan Zhang. --- .../resourcemanager/scheduler/capacity/ParentQueue.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index 87ebc0b40f3b1..c0ad7f89fdd01 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -465,7 +465,7 @@ public String toString() { "numChildQueue= " + childQueues.size() + ", " + getCapacityOrWeightString() + ", " + "absoluteCapacity=" + queueCapacities.getAbsoluteCapacity() + ", " + - "usedResources=" + usageTracker.getQueueUsage().getUsed() + + "usedResources=" + usageTracker.getQueueUsage().getUsed() + ", " + "usedCapacity=" + getUsedCapacity() + ", " + "numApps=" + getNumApplications() + ", " + "numContainers=" + getNumContainers(); @@ -1148,7 +1148,7 @@ String getChildQueuesToPrint() { StringBuilder sb = new StringBuilder(); for (CSQueue q : childQueues) { sb.append(q.getQueuePath() + - "usedCapacity=(" + q.getUsedCapacity() + "), " + + " usedCapacity=(" + q.getUsedCapacity() + "), " + " label=(" + StringUtils.join(q.getAccessibleNodeLabels().iterator(), ",") + ")"); From c3124a3f6e9948f60cfc84c5b431a5cf82818a33 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Fri, 11 Mar 2022 15:37:44 +0100 Subject: [PATCH 049/145] YARN-10565. Refactor CS queue initialization to simplify weight mode calculation. Contributed by Benjamin Teke. --- .../server/resourcemanager/scheduler/capacity/ParentQueue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index c0ad7f89fdd01..671ed21c4312f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -1280,7 +1280,7 @@ public void updateClusterResource(Resource clusterResource, labelManager.getResourceByLabel(null, clusterResource), RMNodeLabelsManager.NO_LABEL, this); } catch (IOException e) { - LOG.error("Fatal issue found: e", e); + LOG.error("Error during updating cluster resource: ", e); throw new YarnRuntimeException("Fatal issue during scheduling", e); } finally { writeLock.unlock(); From 1d5650c4d0acf33f141d593762682b3603523104 Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Thu, 17 Mar 2022 14:06:55 -0700 Subject: [PATCH 050/145] HDFS-13248: Namenode needs to use the actual client IP when going through the RBF proxy. There is a new configuration knob dfs.namenode.ip-proxy-users that configures the list of users than can set their client ip address using the client context. Fixes #4081 --- .../org/apache/hadoop/ipc/CallerContext.java | 5 + .../hadoop/security/UserGroupInformation.java | 14 ++- .../federation/router/RouterRpcClient.java | 7 +- .../src/site/markdown/HDFSRouterFederation.md | 14 +++ .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 2 + .../hdfs/server/namenode/FSNamesystem.java | 5 +- .../server/namenode/NameNodeRpcServer.java | 32 ++++++- .../src/main/resources/hdfs-default.xml | 11 +++ .../namenode/TestNameNodeRpcServer.java | 96 ++++++++++++++++++- 9 files changed, 175 insertions(+), 11 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java index c8b4135d088ed..dbd9184a2b91e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallerContext.java @@ -43,6 +43,11 @@ @InterfaceStability.Evolving public final class CallerContext { public static final Charset SIGNATURE_ENCODING = StandardCharsets.UTF_8; + + // field names + public static final String CLIENT_IP_STR = "clientIp"; + public static final String CLIENT_PORT_STR = "clientPort"; + /** The caller context. * * It will be truncated if it exceeds the maximum allowed length in diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java index a3b1cbd14d822..b2efe502144cb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java @@ -1508,7 +1508,19 @@ public UserGroupInformation getRealUser() { return null; } - + /** + * If this is a proxy user, get the real user. Otherwise, return + * this user. + * @param user the user to check + * @return the real user or self + */ + public static UserGroupInformation getRealUserOrSelf(UserGroupInformation user) { + if (user == null) { + return null; + } + UserGroupInformation real = user.getRealUser(); + return real != null ? real : user; + } /** * This class is used for storing the groups for testing. It stores a local diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index 436418da00348..90d6c347ef73e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java @@ -131,9 +131,6 @@ public class RouterRpcClient { private static final Pattern STACK_TRACE_PATTERN = Pattern.compile("\\tat (.*)\\.(.*)\\((.*):(\\d*)\\)"); - private static final String CLIENT_IP_STR = "clientIp"; - private static final String CLIENT_PORT_STR = "clientPort"; - /** Fairness manager to control handlers assigned per NS. */ private RouterRpcFairnessPolicyController routerRpcFairnessPolicyController; private Map rejectedPermitsPerNs = new ConcurrentHashMap<>(); @@ -597,8 +594,8 @@ private void addClientIpToCallerContext() { byte[] origSignature = ctx == null ? null : ctx.getSignature(); CallerContext.Builder builder = new CallerContext.Builder("", contextFieldSeparator) - .append(CLIENT_IP_STR, Server.getRemoteAddress()) - .append(CLIENT_PORT_STR, + .append(CallerContext.CLIENT_IP_STR, Server.getRemoteAddress()) + .append(CallerContext.CLIENT_PORT_STR, Integer.toString(Server.getRemotePort())) .setSignature(origSignature); // Append the original caller context diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md index de92f86d63c52..f0edc14f12f7d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/site/markdown/HDFSRouterFederation.md @@ -375,6 +375,20 @@ With this setting a user can interact with `ns-fed` as a regular namespace: This federated namespace can also be set as the default one at **core-site.xml** using `fs.defaultFS`. +NameNode configuration +-------------------- + +In order for the system to support data-locality, you must configure your NameNodes so that they will trust the routers to supply the user's client IP address. `dfs.namenode.ip-proxy-users` defines a comma separated list of users that are allowed to provide the client ip address via the caller context. + +```xml + + + dfs.namenode.ip-proxy-users + hdfs + + +``` + Router configuration -------------------- 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 38949744b4697..6216f6e7a1ded 100755 --- 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 @@ -991,6 +991,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.namenode.lifeline.handler.count"; public static final String DFS_NAMENODE_SERVICE_HANDLER_COUNT_KEY = "dfs.namenode.service.handler.count"; public static final int DFS_NAMENODE_SERVICE_HANDLER_COUNT_DEFAULT = 10; + // List of users that can override their client ip + public static final String DFS_NAMENODE_IP_PROXY_USERS = "dfs.namenode.ip-proxy-users"; public static final String DFS_HTTP_POLICY_KEY = "dfs.http.policy"; public static final String DFS_HTTP_POLICY_DEFAULT = HttpConfig.Policy.HTTP_ONLY.name(); public static final String DFS_DATANODE_HTTPSERVER_FILTER_HANDLERS = "dfs.datanode.httpserver.filter.handlers"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index c03281d6a9a91..ef59b9f75ea06 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -401,7 +401,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean, @Metric final MutableRatesWithAggregation detailedLockHoldTimeMetrics = registry.newRatesWithAggregation("detailedLockHoldTimeMetrics"); - private static final String CLIENT_PORT_STR = "clientPort"; private final String contextFieldSeparator; boolean isAuditEnabled() { @@ -467,7 +466,7 @@ private void appendClientPortToCallerContextIfAbsent() { byte[] origSignature = ctx == null ? null : ctx.getSignature(); CallerContext.setCurrent( new CallerContext.Builder(origContext, contextFieldSeparator) - .append(CLIENT_PORT_STR, String.valueOf(Server.getRemotePort())) + .append(CallerContext.CLIENT_PORT_STR, String.valueOf(Server.getRemotePort())) .setSignature(origSignature) .build()); } @@ -475,7 +474,7 @@ private void appendClientPortToCallerContextIfAbsent() { private boolean isClientPortInfoAbsent(CallerContext ctx){ return ctx == null || ctx.getContext() == null - || !ctx.getContext().contains(CLIENT_PORT_STR); + || !ctx.getContext().contains(CallerContext.CLIENT_PORT_STR); } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java index c0cb5787682d7..4f400519ff416 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java @@ -21,6 +21,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_IP_PROXY_USERS; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_HANDLER_RATIO_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIFELINE_HANDLER_RATIO_KEY; @@ -45,6 +46,9 @@ import java.util.Map; import java.util.Set; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.hadoop.ipc.CallerContext; + import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -267,6 +271,9 @@ public class NameNodeRpcServer implements NamenodeProtocols { private final String defaultECPolicyName; + // Users who can override the client ip + private final String[] ipProxyUsers; + public NameNodeRpcServer(Configuration conf, NameNode nn) throws IOException { this.nn = nn; @@ -277,6 +284,7 @@ public NameNodeRpcServer(Configuration conf, NameNode nn) int handlerCount = conf.getInt(DFS_NAMENODE_HANDLER_COUNT_KEY, DFS_NAMENODE_HANDLER_COUNT_DEFAULT); + ipProxyUsers = conf.getStrings(DFS_NAMENODE_IP_PROXY_USERS); RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class, ProtobufRpcEngine2.class); @@ -1899,7 +1907,29 @@ private void verifySoftwareVersion(DatanodeRegistration dnReg) } } - private static String getClientMachine() { + private String getClientMachine() { + if (ipProxyUsers != null) { + // Get the real user (or effective if it isn't a proxy user) + UserGroupInformation user = + UserGroupInformation.getRealUserOrSelf(Server.getRemoteUser()); + if (user != null && + ArrayUtils.contains(ipProxyUsers, user.getShortUserName())) { + CallerContext context = CallerContext.getCurrent(); + if (context != null && context.isContextValid()) { + String cc = context.getContext(); + // if the rpc has a caller context of "clientIp:1.2.3.4,CLI", + // return "1.2.3.4" as the client machine. + String key = CallerContext.CLIENT_IP_STR + + CallerContext.Builder.KEY_VALUE_SEPARATOR; + int posn = cc.indexOf(key); + if (posn != -1) { + posn += key.length(); + int end = cc.indexOf(",", posn); + return end == -1 ? cc.substring(posn) : cc.substring(posn, end); + } + } + } + } String clientMachine = Server.getRemoteAddress(); if (clientMachine == null) { //not a RPC client clientMachine = ""; 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 6890b8a8412c4..d49800c34849e 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -565,6 +565,17 @@ + + dfs.namenode.ip-proxy-users + + A comma separated list of user names that are allowed by the + NameNode to specify a different client IP address in the caller context. + This is used by Router-Based Federation (RBF) to provide the actual client's + IP address to the NameNode, which is critical to preserve data locality when + using RBF. If you are using RBF, add the user that runs the routers. + + + dfs.namenode.acls.enabled true diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java index ada93e84f0ead..74d85bc637efa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRpcServer.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -24,14 +24,25 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_IP_PROXY_USERS; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_BIND_HOST_KEY; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; import java.io.IOException; +import java.nio.charset.StandardCharsets; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.ipc.CallerContext; +import org.apache.hadoop.security.UserGroupInformation; import org.junit.Test; public class TestNameNodeRpcServer { @@ -59,5 +70,88 @@ public void testNamenodeRpcBindAny() throws IOException { conf.unset(DFS_NAMENODE_RPC_BIND_HOST_KEY); } } + + /** + * Get the preferred DataNode location for the first block of the + * given file. + * @param fs The file system to use + * @param p The path to use + * @return the preferred host to get the data + */ + private static String getPreferredLocation(DistributedFileSystem fs, + Path p) throws IOException{ + // Use getLocatedBlocks because it is the basis for HDFS open, + // but provides visibility into which host will be used. + LocatedBlocks blocks = fs.getClient() + .getLocatedBlocks(p.toUri().getPath(), 0); + return blocks.get(0).getLocations()[0].getHostName(); + } + + // Because of the randomness of the NN assigning DN, we run multiple + // trials. 1/3^20=3e-10, so that should be good enough. + static final int ITERATIONS_TO_USE = 20; + + /** + * A test to make sure that if an authorized user adds "clientIp:" to their + * caller context, it will be used to make locality decisions on the NN. + */ + @Test + public void testNamenodeRpcClientIpProxy() throws IOException { + Configuration conf = new HdfsConfiguration(); + + conf.set(DFS_NAMENODE_IP_PROXY_USERS, "fake_joe"); + // Make 3 nodes & racks so that we have a decent shot of detecting when + // our change overrides the random choice of datanode. + final String[] racks = new String[]{"/rack1", "/rack2", "/rack3"}; + final String[] hosts = new String[]{"node1", "node2", "node3"}; + MiniDFSCluster cluster = null; + final CallerContext original = CallerContext.getCurrent(); + + try { + cluster = new MiniDFSCluster.Builder(conf) + .racks(racks).hosts(hosts).numDataNodes(hosts.length) + .build(); + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + // Write a sample file + final Path fooName = fs.makeQualified(new Path("/foo")); + FSDataOutputStream stream = fs.create(fooName); + stream.write("Hello world!\n".getBytes(StandardCharsets.UTF_8)); + stream.close(); + // Set the caller context to set the ip address + CallerContext.setCurrent( + new CallerContext.Builder("test", conf) + .append(CallerContext.CLIENT_IP_STR, hosts[0]) + .build()); + // Should get a random mix of DataNodes since we aren't joe. + for (int trial = 0; trial < ITERATIONS_TO_USE; ++trial) { + String host = getPreferredLocation(fs, fooName); + if (!hosts[0].equals(host)) { + // found some other host, so things are good + break; + } else if (trial == ITERATIONS_TO_USE - 1) { + assertNotEquals("Failed to get non-node1", hosts[0], host); + } + } + // Run as fake joe to authorize the test + UserGroupInformation joe = + UserGroupInformation.createUserForTesting("fake_joe", + new String[]{"fake_group"}); + DistributedFileSystem joeFs = + (DistributedFileSystem) DFSTestUtil.getFileSystemAs(joe, conf); + // As joe, we should get all node1. + for (int trial = 0; trial < ITERATIONS_TO_USE; ++trial) { + String host = getPreferredLocation(joeFs, fooName); + assertEquals("Trial " + trial + " failed", hosts[0], host); + } + } finally { + CallerContext.setCurrent(original); + if (cluster != null) { + cluster.shutdown(); + } + // Reset the config + conf.unset(DFS_NAMENODE_IP_PROXY_USERS); + } + } } From 2beb7296fb47e2d40e209c1129bf59f61398b03d Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 22 Mar 2022 12:33:17 +0100 Subject: [PATCH 051/145] YARN-11087. Introduce the config to control the refresh interval in RMDelegatedNodeLabelsUpdater. Contributed by Junfan Zhang. --- .../hadoop/yarn/conf/YarnConfiguration.java | 6 ++++++ .../src/main/resources/yarn-default.xml | 20 +++++++++++++++---- .../RMDelegatedNodeLabelsUpdater.java | 5 ++++- 3 files changed, 26 insertions(+), 5 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index 28c96de8291a3..e61b5416e22a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -4423,6 +4423,12 @@ public static boolean areNodeLabelsEnabled( public static final long DEFAULT_RM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS = 30 * 60 * 1000; + public static final String RM_NODE_LABELS_PROVIDER_UPDATE_NEWLY_REGISTERED_INTERVAL_MS = + RM_NODE_LABELS_PROVIDER_PREFIX + "update-newly-registered-nodes-interval-ms"; + + public static final long DEFAULT_RM_NODE_LABELS_PROVIDER_UPDATE_NEWLY_REGISTERED_INTERVAL_MS = + 30 * 1000; + @Private /** * This is a private feature that isn't supposed to be used by end-users. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index 39212626ae419..1416665a4fb5d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -3431,15 +3431,27 @@ When "yarn.node-labels.configuration-type" is configured with - "delegated-centralized", then periodically node labels are retrieved - from the node labels provider. This configuration is to define the - interval. If -1 is configured then node labels are retrieved from - provider only once for each node after it registers. Defaults to 30 mins. + "delegated-centralized", then node labels of all nodes + are updated by periodically retrieving node labels from the + provider. If -1 is configured then node labels are retrieved + from provider only once for each node after it registers. + Defaults to 30 mins. yarn.resourcemanager.node-labels.provider.fetch-interval-ms 1800000 + + + When "yarn.node-labels.configuration-type" is configured with + "delegated-centralized", then node labels of newly registered + nodes are updated by periodically retrieving node labels from + the provider. Defaults to 30 secs. + + yarn.resourcemanager.node-labels.provider.update-newly-registered-nodes-interval-ms + 30000 + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMDelegatedNodeLabelsUpdater.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMDelegatedNodeLabelsUpdater.java index bb95cf98f40a9..ba0c430d77f93 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMDelegatedNodeLabelsUpdater.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMDelegatedNodeLabelsUpdater.java @@ -56,7 +56,7 @@ public class RMDelegatedNodeLabelsUpdater extends CompositeService { private Timer nodeLabelsScheduler; // 30 seconds @VisibleForTesting - public long nodeLabelsUpdateInterval = 30 * 1000; + public long nodeLabelsUpdateInterval; private Set newlyRegisteredNodes = new HashSet(); // Lock to protect newlyRegisteredNodes @@ -78,6 +78,9 @@ protected void serviceInit(Configuration conf) throws Exception { allNodesLabelUpdateInterval = conf.getLong( YarnConfiguration.RM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS, YarnConfiguration.DEFAULT_RM_NODE_LABELS_PROVIDER_FETCH_INTERVAL_MS); + nodeLabelsUpdateInterval = + conf.getLong(YarnConfiguration.RM_NODE_LABELS_PROVIDER_UPDATE_NEWLY_REGISTERED_INTERVAL_MS, + YarnConfiguration.DEFAULT_RM_NODE_LABELS_PROVIDER_UPDATE_NEWLY_REGISTERED_INTERVAL_MS); rmNodeLabelsMappingProvider = createRMNodeLabelsMappingProvider(conf); addService(rmNodeLabelsMappingProvider); super.serviceInit(conf); From 708a0ce21bc1bd6164ff650e9104de70fe3f1dbb Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Tue, 22 Mar 2022 13:20:37 +0000 Subject: [PATCH 052/145] HADOOP-13704. Optimized S3A getContentSummary() Optimize the scan for s3 by performing a deep tree listing, inferring directory counts from the paths returned. Contributed by Ahmar Suhail. Change-Id: I26ffa8c6f65fd11c68a88d6e2243b0eac6ffd024 --- .../site/markdown/filesystem/filesystem.md | 20 ++++ .../AbstractContractContentSummaryTest.java | 65 +++++++++++ .../TestLocalFSContractContentSummary.java | 31 ++++++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 6 +- .../hadoop/fs/s3a/S3ObjectAttributes.java | 2 +- .../s3a/impl/GetContentSummaryOperation.java | 104 +++++++++++------- .../s3a/ITestS3AContractContentSummary.java | 70 ++++++++++++ .../ITestS3AMiscOperationCost.java | 4 +- .../scale/ITestS3ADirectoryPerformance.java | 33 ++++++ 9 files changed, 289 insertions(+), 46 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractContentSummaryTest.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractContentSummary.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractContentSummary.java diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md index 0e01aa1dc8d76..4517bd8ff4a15 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md @@ -453,6 +453,26 @@ The function `getLocatedFileStatus(FS, d)` is as defined in The atomicity and consistency constraints are as for `listStatus(Path, PathFilter)`. + +### `ContentSummary getContentSummary(Path path)` + +Given a path return its content summary. + +`getContentSummary()` first checks if the given path is a file and if yes, it returns 0 for directory count +and 1 for file count. + +#### Preconditions + + exists(FS, path) else raise FileNotFoundException + +#### Postconditions + +Returns a `ContentSummary` object with information such as directory count +and file count for a given path. + +The atomicity and consistency constraints are as for +`listStatus(Path, PathFilter)`. + ### `BlockLocation[] getFileBlockLocations(FileStatus f, int s, int l)` #### Preconditions diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractContentSummaryTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractContentSummaryTest.java new file mode 100644 index 0000000000000..5e5c917395413 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractContentSummaryTest.java @@ -0,0 +1,65 @@ +/* + * 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.hadoop.fs.contract; + +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.io.FileNotFoundException; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public abstract class AbstractContractContentSummaryTest extends AbstractFSContractTestBase { + + @Test + public void testGetContentSummary() throws Throwable { + FileSystem fs = getFileSystem(); + + Path parent = path("parent"); + Path nested = path(parent + "/a/b/c"); + Path filePath = path(nested + "file.txt"); + + fs.mkdirs(parent); + fs.mkdirs(nested); + touch(getFileSystem(), filePath); + + ContentSummary summary = fs.getContentSummary(parent); + + Assertions.assertThat(summary.getDirectoryCount()).as("Summary " + summary).isEqualTo(4); + + Assertions.assertThat(summary.getFileCount()).as("Summary " + summary).isEqualTo(1); + } + + @Test + public void testGetContentSummaryIncorrectPath() throws Throwable { + FileSystem fs = getFileSystem(); + + Path parent = path("parent"); + Path nested = path(parent + "/a"); + + fs.mkdirs(parent); + + intercept(FileNotFoundException.class, () -> fs.getContentSummary(nested)); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractContentSummary.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractContentSummary.java new file mode 100644 index 0000000000000..7555cf85158f9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractContentSummary.java @@ -0,0 +1,31 @@ +/* + * 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.hadoop.fs.contract.localfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractContentSummaryTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +public class TestLocalFSContractContentSummary extends AbstractContractContentSummaryTest { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 86da70ecdd316..4b450c4dcce5c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -3266,9 +3266,9 @@ public S3AFileStatus probePathStatus(final Path path, } @Override - public RemoteIterator listStatusIterator(final Path path) - throws IOException { - return S3AFileSystem.this.innerListStatus(path); + public RemoteIterator listFilesIterator(final Path path, + final boolean recursive) throws IOException { + return S3AFileSystem.this.innerListFiles(path, recursive, Listing.ACCEPT_ALL_BUT_S3N, null); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java index 5a8dfc7bc8c0e..275b207cd6db1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java @@ -66,7 +66,7 @@ public S3ObjectAttributes( /** * Construct from the result of a copy and those parameters * which aren't included in an AWS SDK response. - * @param path + * @param path path * @param copyResult copy result. * @param serverSideEncryptionAlgorithm current encryption algorithm * @param serverSideEncryptionKey any server side encryption key? diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java index 23631c67e30a1..248bffb9401fb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java @@ -20,6 +20,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.util.HashSet; import java.util.Set; import org.slf4j.Logger; @@ -34,22 +35,15 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; /** * GetContentSummary operation. - * This is based on {@code FileSystem.get#getContentSummary}; - * its still doing sequential treewalk with the efficiency - * issues. * - * Changes: - * 1. On the recursive calls there - * is no probe to see if the path is a file: we know the - * recursion only happens with a dir. - * 2. If a subdirectory is not found during the walk, that - * does not trigger an error. The directory is clearly - * not part of the content any more. + * It is optimized for s3 and performs a deep tree listing, + * inferring directory counts from the paths returned. * * The Operation serves up IOStatistics; this counts * the cost of all the list operations, but not the @@ -122,9 +116,7 @@ public ContentSummary execute() throws IOException { /** * Return the {@link ContentSummary} of a given directory. - * This is a recursive operation (as the original is); - * it'd be more efficient of stack and heap if it managed its - * own stack. + * * @param dir dir to scan * @throws FileNotFoundException if the path does not resolve * @throws IOException IO failure @@ -133,34 +125,65 @@ public ContentSummary execute() throws IOException { * @throws IOException failure */ public ContentSummary getDirSummary(Path dir) throws IOException { + long totalLength = 0; long fileCount = 0; long dirCount = 1; - final RemoteIterator it - = callbacks.listStatusIterator(dir); + + RemoteIterator it = callbacks.listFilesIterator(dir, true); + + Set dirSet = new HashSet<>(); + Set pathsTraversed = new HashSet<>(); while (it.hasNext()) { - final S3AFileStatus s = it.next(); - if (s.isDirectory()) { - try { - ContentSummary c = getDirSummary(s.getPath()); - totalLength += c.getLength(); - fileCount += c.getFileCount(); - dirCount += c.getDirectoryCount(); - } catch (FileNotFoundException ignored) { - // path was deleted during the scan; exclude from - // summary. - } - } else { - totalLength += s.getLen(); + S3ALocatedFileStatus fileStatus = it.next(); + Path filePath = fileStatus.getPath(); + + if (fileStatus.isDirectory() && !filePath.equals(dir)) { + dirSet.add(filePath); + buildDirectorySet(dirSet, pathsTraversed, dir, filePath.getParent()); + } else if (!fileStatus.isDirectory()) { fileCount += 1; + totalLength += fileStatus.getLen(); + buildDirectorySet(dirSet, pathsTraversed, dir, filePath.getParent()); } + } + // Add the list's IOStatistics iostatistics.aggregate(retrieveIOStatistics(it)); + return new ContentSummary.Builder().length(totalLength). - fileCount(fileCount).directoryCount(dirCount). - spaceConsumed(totalLength).build(); + fileCount(fileCount).directoryCount(dirCount + dirSet.size()). + spaceConsumed(totalLength).build(); + } + + /*** + * This method builds the set of all directories found under the base path. We need to do this + * because if the directory structure /a/b/c was created with a single mkdirs() call, it is + * stored as 1 object in S3 and the list files iterator will only return a single entry /a/b/c. + * + * We keep track of paths traversed so far to prevent duplication of work. For eg, if we had + * a/b/c/file-1.txt and /a/b/c/file-2.txt, we will only recurse over the complete path once + * and won't have to do anything for file-2.txt. + * + * @param dirSet Set of all directories found in the path + * @param pathsTraversed Set of all paths traversed so far + * @param basePath Path of directory to scan + * @param parentPath Parent path of the current file/directory in the iterator + */ + private void buildDirectorySet(Set dirSet, Set pathsTraversed, Path basePath, + Path parentPath) { + + if (parentPath == null || pathsTraversed.contains(parentPath) || parentPath.equals(basePath)) { + return; + } + + dirSet.add(parentPath); + + buildDirectorySet(dirSet, pathsTraversed, basePath, parentPath.getParent()); + + pathsTraversed.add(parentPath); } /** @@ -186,23 +209,24 @@ public interface GetContentSummaryCallbacks { /** * Get the status of a path. - * @param path path to probe. + * + * @param path path to probe. * @param probes probes to exec * @return the status * @throws IOException failure */ @Retries.RetryTranslated - S3AFileStatus probePathStatus(Path path, - Set probes) throws IOException; - - /** - * Incremental list of all entries in a directory. - * @param path path of dir - * @return an iterator + S3AFileStatus probePathStatus(Path path, Set probes) throws IOException; + + /*** + * List all entries under a path. + * + * @param path + * @param recursive if the subdirectories need to be traversed recursively + * @return an iterator over the listing. * @throws IOException failure */ - RemoteIterator listStatusIterator(Path path) + RemoteIterator listFilesIterator(Path path, boolean recursive) throws IOException; - } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractContentSummary.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractContentSummary.java new file mode 100644 index 0000000000000..ad83cfe52dadc --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractContentSummary.java @@ -0,0 +1,70 @@ +/* + * 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.hadoop.fs.contract.s3a; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractContractContentSummaryTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; + +public class ITestS3AContractContentSummary extends AbstractContractContentSummaryTest { + + @Test + public void testGetContentSummaryDir() throws Throwable { + describe("getContentSummary on test dir with children"); + S3AFileSystem fs = getFileSystem(); + Path baseDir = methodPath(); + + // Nested folders created separately will return as separate objects in listFiles() + fs.mkdirs(new Path(baseDir, "a")); + fs.mkdirs(new Path(baseDir, "a/b")); + fs.mkdirs(new Path(baseDir, "a/b/a")); + + // Will return as one object + fs.mkdirs(new Path(baseDir, "d/e/f")); + + Path filePath = new Path(baseDir, "a/b/file"); + touch(fs, filePath); + + // look at path to see if it is a file + // it is not: so LIST + final ContentSummary summary = fs.getContentSummary(baseDir); + + Assertions.assertThat(summary.getDirectoryCount()).as("Summary " + summary).isEqualTo(7); + Assertions.assertThat(summary.getFileCount()).as("Summary " + summary).isEqualTo(1); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new S3AContract(conf); + } + + @Override + public S3AFileSystem getFileSystem() { + return (S3AFileSystem) super.getFileSystem(); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java index 75701b50d2f65..6a6baa03ead41 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AMiscOperationCost.java @@ -144,8 +144,8 @@ public void testGetContentSummaryDir() throws Throwable { with(INVOCATION_GET_CONTENT_SUMMARY, 1), withAuditCount(1), always(FILE_STATUS_FILE_PROBE // look at path to see if it is a file - .plus(LIST_OPERATION) // it is not: so LIST - .plus(LIST_OPERATION))); // and a LIST on the child dir + .plus(LIST_OPERATION))); // it is not: so LIST + Assertions.assertThat(summary.getDirectoryCount()) .as("Summary " + summary) .isEqualTo(2); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index d87af3bac5d01..946e59e9e3c01 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -19,6 +19,7 @@ package org.apache.hadoop.fs.s3a.scale; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; @@ -154,6 +155,38 @@ public void testListOperations() throws Throwable { listStatusCalls, getFileStatusCalls); + describe("Get content summary for directory"); + + NanoTimer getContentSummaryTimer = new NanoTimer(); + + ContentSummary rootPathSummary = fs.getContentSummary(scaleTestDir); + ContentSummary testPathSummary = fs.getContentSummary(listDir); + + getContentSummaryTimer.end("getContentSummary of %s", created); + + // only two list operations should have taken place + print(LOG, + metadataRequests, + listRequests, + listContinueRequests, + listStatusCalls, + getFileStatusCalls); + assertEquals(listRequests.toString(), 2, listRequests.diff()); + reset(metadataRequests, + listRequests, + listContinueRequests, + listStatusCalls, + getFileStatusCalls); + + assertTrue("Root directory count should be > test path", + rootPathSummary.getDirectoryCount() > testPathSummary.getDirectoryCount()); + assertTrue("Root file count should be >= to test path", + rootPathSummary.getFileCount() >= testPathSummary.getFileCount()); + assertEquals("Incorrect directory count", created.getDirCount() + 1, + testPathSummary.getDirectoryCount()); + assertEquals("Incorrect file count", created.getFileCount(), + testPathSummary.getFileCount()); + } finally { describe("deletion"); // deletion at the end of the run From 88975496d8a076b8923999e9e9ecef13e3721e3d Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Tue, 22 Mar 2022 22:28:09 +0800 Subject: [PATCH 053/145] HDFS-14617. Improve oiv tool to parse fsimage file in parallel with delimited format. (#2918). Contributed by Hongbing Wang. Signed-off-by: He Xiaoqiao --- .../OfflineImageViewerPB.java | 10 +- .../PBImageCorruptionDetector.java | 2 +- .../PBImageDelimitedTextWriter.java | 8 +- .../offlineImageViewer/PBImageTextWriter.java | 267 ++++++++++++++++-- .../TestOfflineImageViewer.java | 59 +++- .../TestOfflineImageViewerForAcl.java | 2 +- 6 files changed, 306 insertions(+), 42 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java index dbcb452e166aa..05e687ab97e43 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java @@ -107,6 +107,7 @@ public class OfflineImageViewerPB { + " Delimited outputs. If not set, the processor\n" + " constructs the namespace in memory \n" + " before outputting text.\n" + + "-m,--multiThread Use multiThread to process sub-sections.\n" + "-h,--help Display usage information and exit\n"; /** @@ -132,6 +133,7 @@ private static Options buildOptions() { options.addOption("delimiter", true, ""); options.addOption("sp", false, ""); options.addOption("t", "temp", true, ""); + options.addOption("m", "multiThread", true, ""); return options; } @@ -185,6 +187,7 @@ public static int run(String[] args) throws Exception { String delimiter = cmd.getOptionValue("delimiter", PBImageTextWriter.DEFAULT_DELIMITER); String tempPath = cmd.getOptionValue("t", ""); + int threads = Integer.parseInt(cmd.getOptionValue("m", "1")); Configuration conf = new Configuration(); PrintStream out = null; @@ -227,15 +230,14 @@ public static int run(String[] args) throws Exception { boolean printStoragePolicy = cmd.hasOption("sp"); try (PBImageDelimitedTextWriter writer = new PBImageDelimitedTextWriter(out, delimiter, - tempPath, printStoragePolicy); - RandomAccessFile r = new RandomAccessFile(inputFile, "r")) { - writer.visit(r); + tempPath, printStoragePolicy, threads, outputFile)) { + writer.visit(inputFile); } break; case "DETECTCORRUPTION": try (PBImageCorruptionDetector detector = new PBImageCorruptionDetector(out, delimiter, tempPath)) { - detector.visit(new RandomAccessFile(inputFile, "r")); + detector.visit(inputFile); } break; default: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java index 28c450701b846..17593867bd642 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java @@ -337,7 +337,7 @@ public void afterOutput() throws IOException { if (parentId != -1) { entryBuilder.setParentId(parentId); } - printIfNotEmpty(entryBuilder.build()); + printIfNotEmpty(serialOutStream(), entryBuilder.build()); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java index 45d42f0396b1a..3e080ec8e65cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java @@ -146,7 +146,13 @@ public String build() { PBImageDelimitedTextWriter(PrintStream out, String delimiter, String tempPath, boolean printStoragePolicy) throws IOException { - super(out, delimiter, tempPath); + this(out, delimiter, tempPath, printStoragePolicy, 1, "-"); + } + + PBImageDelimitedTextWriter(PrintStream out, String delimiter, + String tempPath, boolean printStoragePolicy, int threads, + String parallelOut) throws IOException { + super(out, delimiter, tempPath, threads, parallelOut); this.printStoragePolicy = printStoragePolicy; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java index 08fe7fb943c15..2dab44a036ac6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java @@ -21,17 +21,25 @@ import java.io.Closeable; import java.io.File; import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.PrintStream; import java.io.RandomAccessFile; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -455,20 +463,22 @@ public String getParentPath(long inode) throws IOException { return "/"; } long parent = getFromDirChildMap(inode); - if (!dirPathCache.containsKey(parent)) { - byte[] bytes = dirMap.get(toBytes(parent)); - if (parent != INodeId.ROOT_INODE_ID && bytes == null) { - // The parent is an INodeReference, which is generated from snapshot. - // For delimited oiv tool, no need to print out metadata in snapshots. - throw PBImageTextWriter.createIgnoredSnapshotException(inode); + byte[] bytes = dirMap.get(toBytes(parent)); + synchronized (this) { + if (!dirPathCache.containsKey(parent)) { + if (parent != INodeId.ROOT_INODE_ID && bytes == null) { + // The parent is an INodeReference, which is generated from snapshot. + // For delimited oiv tool, no need to print out metadata in snapshots. + throw PBImageTextWriter.createIgnoredSnapshotException(inode); + } + String parentName = toString(bytes); + String parentPath = + new Path(getParentPath(parent), + parentName.isEmpty() ? "/" : parentName).toString(); + dirPathCache.put(parent, parentPath); } - String parentName = toString(bytes); - String parentPath = - new Path(getParentPath(parent), - parentName.isEmpty() ? "/" : parentName).toString(); - dirPathCache.put(parent, parentPath); + return dirPathCache.get(parent); } - return dirPathCache.get(parent); } @Override @@ -493,9 +503,12 @@ public long getParentId(long id) throws IOException { } private SerialNumberManager.StringTable stringTable; - private PrintStream out; + private final PrintStream out; private MetadataMap metadataMap = null; private String delimiter; + private File filename; + private int numThreads; + private String parallelOutputFile; /** * Construct a PB FsImage writer to generate text file. @@ -503,8 +516,8 @@ public long getParentId(long id) throws IOException { * @param tempPath the path to store metadata. If it is empty, store metadata * in memory instead. */ - PBImageTextWriter(PrintStream out, String delimiter, String tempPath) - throws IOException { + PBImageTextWriter(PrintStream out, String delimiter, String tempPath, + int numThreads, String parallelOutputFile) throws IOException { this.out = out; this.delimiter = delimiter; if (tempPath.isEmpty()) { @@ -512,6 +525,17 @@ public long getParentId(long id) throws IOException { } else { metadataMap = new LevelDBMetadataMap(tempPath); } + this.numThreads = numThreads; + this.parallelOutputFile = parallelOutputFile; + } + + PBImageTextWriter(PrintStream out, String delimiter, String tempPath) + throws IOException { + this(out, delimiter, tempPath, 1, "-"); + } + + protected PrintStream serialOutStream() { + return out; } @Override @@ -562,7 +586,9 @@ void append(StringBuffer buffer, String field) { */ abstract protected void afterOutput() throws IOException; - public void visit(RandomAccessFile file) throws IOException { + public void visit(String filePath) throws IOException { + filename = new File(filePath); + RandomAccessFile file = new RandomAccessFile(filePath, "r"); Configuration conf = new Configuration(); if (!FSImageUtil.checkFileFormat(file)) { throw new IOException("Unrecognized FSImage"); @@ -642,21 +668,122 @@ long getParentId(long id) throws IOException { private void output(Configuration conf, FileSummary summary, FileInputStream fin, ArrayList sections) throws IOException { + ArrayList allINodeSubSections = + getINodeSubSections(sections); + if (numThreads > 1 && !parallelOutputFile.equals("-") && + allINodeSubSections.size() > 1) { + outputInParallel(conf, summary, allINodeSubSections); + } else { + LOG.info("Serial output due to threads num: {}, parallel output file: {}, " + + "subSections: {}.", numThreads, parallelOutputFile, allINodeSubSections.size()); + outputInSerial(conf, summary, fin, sections); + } + } + + private void outputInSerial(Configuration conf, FileSummary summary, + FileInputStream fin, ArrayList sections) + throws IOException { InputStream is; long startTime = Time.monotonicNow(); - out.println(getHeader()); + serialOutStream().println(getHeader()); for (FileSummary.Section section : sections) { if (SectionName.fromString(section.getName()) == SectionName.INODE) { fin.getChannel().position(section.getOffset()); is = FSImageUtil.wrapInputStreamForCompression(conf, summary.getCodec(), new BufferedInputStream(new LimitInputStream( fin, section.getLength()))); - outputINodes(is); + INodeSection s = INodeSection.parseDelimitedFrom(is); + LOG.info("Found {} INodes in the INode section", s.getNumInodes()); + int count = outputINodes(is, serialOutStream()); + LOG.info("Outputted {} INodes.", count); } } afterOutput(); long timeTaken = Time.monotonicNow() - startTime; - LOG.debug("Time to output inodes: {}ms", timeTaken); + LOG.debug("Time to output inodes: {} ms", timeTaken); + } + + /** + * STEP1: Multi-threaded process sub-sections. + * Given n (n>1) threads to process k (k>=n) sections, + * output parsed results of each section to tmp file in order. + * STEP2: Merge tmp files. + */ + private void outputInParallel(Configuration conf, FileSummary summary, + ArrayList subSections) + throws IOException { + int nThreads = Integer.min(numThreads, subSections.size()); + LOG.info("Outputting in parallel with {} sub-sections using {} threads", + subSections.size(), nThreads); + final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); + CountDownLatch latch = new CountDownLatch(subSections.size()); + ExecutorService executorService = Executors.newFixedThreadPool(nThreads); + AtomicLong expectedINodes = new AtomicLong(0); + AtomicLong totalParsed = new AtomicLong(0); + String codec = summary.getCodec(); + String[] paths = new String[subSections.size()]; + + for (int i = 0; i < subSections.size(); i++) { + paths[i] = parallelOutputFile + ".tmp." + i; + int index = i; + executorService.submit(() -> { + LOG.info("Output iNodes of section-{}", index); + InputStream is = null; + try (PrintStream outStream = new PrintStream(paths[index], "UTF-8")) { + long startTime = Time.monotonicNow(); + is = getInputStreamForSection(subSections.get(index), codec, conf); + if (index == 0) { + // The first iNode section has a header which must be processed first + INodeSection s = INodeSection.parseDelimitedFrom(is); + expectedINodes.set(s.getNumInodes()); + } + totalParsed.addAndGet(outputINodes(is, outStream)); + long timeTaken = Time.monotonicNow() - startTime; + LOG.info("Time to output iNodes of section-{}: {} ms", index, timeTaken); + } catch (Exception e) { + exceptions.add(new IOException(e)); + } finally { + latch.countDown(); + try { + if (is != null) { + is.close(); + } + } catch (IOException ioe) { + LOG.warn("Failed to close the input stream, ignoring", ioe); + } + } + }); + } + + try { + latch.await(); + } catch (InterruptedException e) { + LOG.error("Interrupted waiting for countdown latch", e); + throw new IOException(e); + } + + executorService.shutdown(); + if (exceptions.size() != 0) { + LOG.error("Failed to output INode sub-sections, {} exception(s) occurred.", + exceptions.size()); + throw exceptions.get(0); + } + if (totalParsed.get() != expectedINodes.get()) { + throw new IOException("Expected to parse " + expectedINodes + " in parallel, " + + "but parsed " + totalParsed.get() + ". The image may be corrupt."); + } + LOG.info("Completed outputting all INode sub-sections to {} tmp files.", + subSections.size()); + + try (PrintStream ps = new PrintStream(parallelOutputFile, "UTF-8")) { + ps.println(getHeader()); + } + + // merge tmp files + long startTime = Time.monotonicNow(); + mergeFiles(paths, parallelOutputFile); + long timeTaken = Time.monotonicNow() - startTime; + LOG.info("Completed all stages. Time to merge files: {} ms", timeTaken); } protected PermissionStatus getPermission(long perm) { @@ -763,22 +890,27 @@ protected void buildNamespace(InputStream in, List refIdList) LOG.info("Scanned {} INode directories to build namespace.", count); } - void printIfNotEmpty(String line) { + void printIfNotEmpty(PrintStream outStream, String line) { if (!line.isEmpty()) { - out.println(line); + outStream.println(line); } } - private void outputINodes(InputStream in) throws IOException { - INodeSection s = INodeSection.parseDelimitedFrom(in); - LOG.info("Found {} INodes in the INode section", s.getNumInodes()); + private int outputINodes(InputStream in, PrintStream outStream) + throws IOException { long ignored = 0; long ignoredSnapshots = 0; - for (int i = 0; i < s.getNumInodes(); ++i) { + // As the input stream is a LimitInputStream, the reading will stop when + // EOF is encountered at the end of the stream. + int count = 0; + while (true) { INode p = INode.parseDelimitedFrom(in); + if (p == null) { + break; + } try { String parentPath = metadataMap.getParentPath(p.getId()); - printIfNotEmpty(getEntry(parentPath, p)); + printIfNotEmpty(outStream, getEntry(parentPath, p)); } catch (IOException ioe) { ignored++; if (!(ioe instanceof IgnoreSnapshotException)) { @@ -790,16 +922,16 @@ private void outputINodes(InputStream in) throws IOException { } } } - - if (LOG.isDebugEnabled() && i % 100000 == 0) { - LOG.debug("Outputted {} INodes.", i); + count++; + if (LOG.isDebugEnabled() && count % 100000 == 0) { + LOG.debug("Outputted {} INodes.", count); } } if (ignored > 0) { LOG.warn("Ignored {} nodes, including {} in snapshots. Please turn on" + " debug log for details", ignored, ignoredSnapshots); } - LOG.info("Outputted {} INodes.", s.getNumInodes()); + return count; } private static IgnoreSnapshotException createIgnoredSnapshotException( @@ -822,4 +954,79 @@ public int getStoragePolicy( } return HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED; } + + private ArrayList getINodeSubSections( + ArrayList sections) { + ArrayList subSections = new ArrayList<>(); + Iterator iter = sections.iterator(); + while (iter.hasNext()) { + FileSummary.Section s = iter.next(); + if (SectionName.fromString(s.getName()) == SectionName.INODE_SUB) { + subSections.add(s); + } + } + return subSections; + } + + /** + * Given a FSImage FileSummary.section, return a LimitInput stream set to + * the starting position of the section and limited to the section length. + * @param section The FileSummary.Section containing the offset and length + * @param compressionCodec The compression codec in use, if any + * @return An InputStream for the given section + * @throws IOException + */ + private InputStream getInputStreamForSection(FileSummary.Section section, + String compressionCodec, Configuration conf) + throws IOException { + // channel of RandomAccessFile is not thread safe, use File + FileInputStream fin = new FileInputStream(filename); + try { + FileChannel channel = fin.getChannel(); + channel.position(section.getOffset()); + InputStream in = new BufferedInputStream(new LimitInputStream(fin, + section.getLength())); + + in = FSImageUtil.wrapInputStreamForCompression(conf, + compressionCodec, in); + return in; + } catch (IOException e) { + fin.close(); + throw e; + } + } + + /** + * @param srcPaths Source files of contents to be merged + * @param resultPath Merged file path + * @throws IOException + */ + public static void mergeFiles(String[] srcPaths, String resultPath) + throws IOException { + if (srcPaths == null || srcPaths.length < 1) { + LOG.warn("no source files to merge."); + return; + } + + File[] files = new File[srcPaths.length]; + for (int i = 0; i < srcPaths.length; i++) { + files[i] = new File(srcPaths[i]); + } + + File resultFile = new File(resultPath); + try (FileChannel resultChannel = + new FileOutputStream(resultFile, true).getChannel()) { + for (File file : files) { + try (FileChannel src = new FileInputStream(file).getChannel()) { + resultChannel.transferFrom(src, resultChannel.size(), src.size()); + } + } + } + + for (File file : files) { + if (!file.delete() && file.exists()) { + LOG.warn("delete tmp file: {} returned false", file); + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java index 53031bcad90e3..9878469c89e30 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java @@ -83,8 +83,10 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto; import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion; +import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.ErasureCodeConstants; import org.apache.hadoop.net.NetUtils; @@ -122,6 +124,7 @@ import static org.apache.hadoop.fs.permission.FsAction.ALL; import static org.apache.hadoop.fs.permission.FsAction.EXECUTE; import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE; +import static org.apache.hadoop.hdfs.MiniDFSCluster.HDFS_MINIDFS_BASEDIR; import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; import static org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter.ERASURE_CODING_SECTION_NAME; import static org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter.ERASURE_CODING_SECTION_POLICY; @@ -186,6 +189,12 @@ public static void createOriginalFSImage() throws IOException { conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL, "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT"); + // fsimage with sub-section conf + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY, "true"); + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY, "1"); + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY, "4"); + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_KEY, "4"); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); cluster.waitActive(); DistributedFileSystem hdfs = cluster.getFileSystem(); @@ -791,6 +800,13 @@ public void testPBDelimitedWriter() throws IOException, InterruptedException { new FileSystemTestHelper().getTestRootDir() + "/delimited.db"); } + @Test + public void testParallelPBDelimitedWriter() throws Exception { + testParallelPBDelimitedWriter(""); // Test in memory db. + testParallelPBDelimitedWriter(new FileSystemTestHelper().getTestRootDir() + + "/parallel-delimited.db"); + } + @Test public void testCorruptionOutputEntryBuilder() throws IOException { PBImageCorruptionDetector corrDetector = @@ -882,11 +898,10 @@ private void testPBDelimitedWriter(String db) final String DELIMITER = "\t"; ByteArrayOutputStream output = new ByteArrayOutputStream(); - try (PrintStream o = new PrintStream(output); - RandomAccessFile r = new RandomAccessFile(originalFsimage, "r")) { + try (PrintStream o = new PrintStream(output)) { PBImageDelimitedTextWriter v = new PBImageDelimitedTextWriter(o, DELIMITER, db); - v.visit(r); + v.visit(originalFsimage.getAbsolutePath()); } Set fileNames = new HashSet<>(); @@ -920,6 +935,37 @@ private void testPBDelimitedWriter(String db) assertEquals(writtenFiles.keySet(), fileNames); } + private void testParallelPBDelimitedWriter(String db) throws Exception{ + String delimiter = "\t"; + int numThreads = 4; + + File parallelDelimitedOut = new File(tempDir, "parallelDelimitedOut"); + if (OfflineImageViewerPB.run(new String[] {"-p", "Delimited", + "-i", originalFsimage.getAbsolutePath(), + "-o", parallelDelimitedOut.getAbsolutePath(), + "-delimiter", delimiter, + "-t", db, + "-m", String.valueOf(numThreads)}) != 0) { + throw new IOException("oiv returned failure outputting in parallel."); + } + MD5Hash parallelMd5 = MD5FileUtils.computeMd5ForFile(parallelDelimitedOut); + + File serialDelimitedOut = new File(tempDir, "serialDelimitedOut"); + if (db != "") { + db = db + "/../serial.db"; + } + if (OfflineImageViewerPB.run(new String[] {"-p", "Delimited", + "-i", originalFsimage.getAbsolutePath(), + "-o", serialDelimitedOut.getAbsolutePath(), + "-t", db, + "-delimiter", delimiter}) != 0) { + throw new IOException("oiv returned failure outputting in serial."); + } + MD5Hash serialMd5 = MD5FileUtils.computeMd5ForFile(serialDelimitedOut); + + assertEquals(parallelMd5, serialMd5); + } + private void testPBCorruptionDetector(String db) throws IOException, InterruptedException { final String delimiter = "\t"; @@ -928,7 +974,7 @@ private void testPBCorruptionDetector(String db) try (PrintStream o = new PrintStream(output)) { PBImageCorruptionDetector v = new PBImageCorruptionDetector(o, delimiter, db); - v.visit(new RandomAccessFile(originalFsimage, "r")); + v.visit(originalFsimage.getAbsolutePath()); } try ( @@ -1024,7 +1070,7 @@ private String testCorruptionDetectorRun(int runNumber, try (PrintStream o = new PrintStream(output)) { PBImageCorruptionDetector v = new PBImageCorruptionDetector(o, ",", db); - v.visit(new RandomAccessFile(corruptedImage, "r")); + v.visit(corruptedImage.getAbsolutePath()); } return output.toString(); } @@ -1212,6 +1258,9 @@ public void testReverseXmlWithoutSnapshotDiffSection() throws Throwable { public void testFileDistributionCalculatorForException() throws Exception { File fsimageFile = null; Configuration conf = new Configuration(); + // Avoid using the same cluster dir to cause the global originalFsimage + // file to be cleared. + conf.set(HDFS_MINIDFS_BASEDIR, GenericTestUtils.getRandomizedTempPath()); HashMap files = Maps.newHashMap(); // Create a initial fsimage file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java index 4955846432a60..b23ddf4afbcfb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java @@ -239,7 +239,7 @@ public void testPBDelimitedWriterForAcl() throws Exception { try (PrintStream o = new PrintStream(output)) { PBImageDelimitedTextWriter v = new PBImageDelimitedTextWriter(o, DELIMITER, ""); // run in memory. - v.visit(new RandomAccessFile(originalFsimage, "r")); + v.visit(originalFsimage.getAbsolutePath()); } try ( From 59d07bdcc36ca4744b3a724d9874b5f36e637619 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 22 Mar 2022 18:14:29 +0100 Subject: [PATCH 054/145] HADOOP-18160 Avoid shading wildfly.openssl runtime dependency (#4074) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Contributed by André Fonseca. --- hadoop-client-modules/hadoop-client-api/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-client-modules/hadoop-client-api/pom.xml b/hadoop-client-modules/hadoop-client-api/pom.xml index 9b70f8c8b01e0..82f2f8a778087 100644 --- a/hadoop-client-modules/hadoop-client-api/pom.xml +++ b/hadoop-client-modules/hadoop-client-api/pom.xml @@ -161,6 +161,9 @@ org/xerial/snappy/* org/xerial/snappy/**/* + + org/wildfly/openssl/* + org/wildfly/openssl/**/* From 81879eb9cbfb8931294f386772b7f01e758ea514 Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Tue, 22 Mar 2022 23:12:00 +0530 Subject: [PATCH 055/145] HDFS-16471. Make HDFS ls tool cross platform (#4086) * The source files for hdfs_ls uses getopt for parsing the command line arguments. * getopt is available only on Linux and thus, isn't cross platform. * Thus, we need to replace getopt with boost::program_options to make this tool cross platform. --- .../libhdfspp/tests/tools/CMakeLists.txt | 3 + .../libhdfspp/tests/tools/hdfs-ls-mock.cc | 67 ++++++++ .../libhdfspp/tests/tools/hdfs-ls-mock.h | 68 ++++++++ .../libhdfspp/tests/tools/hdfs-tool-tests.cc | 19 +++ .../native/libhdfspp/tools/CMakeLists.txt | 3 +- .../libhdfspp/tools/hdfs-ls/CMakeLists.txt | 27 +++ .../native/libhdfspp/tools/hdfs-ls/hdfs-ls.cc | 156 ++++++++++++++++++ .../native/libhdfspp/tools/hdfs-ls/hdfs-ls.h | 92 +++++++++++ .../native/libhdfspp/tools/hdfs-ls/main.cc | 52 ++++++ .../main/native/libhdfspp/tools/hdfs_ls.cc | 130 --------------- 10 files changed, 485 insertions(+), 132 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/CMakeLists.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/main.cc delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_ls.cc diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt index 22d677f0a29c8..75c5ad1ff9816 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt @@ -37,6 +37,7 @@ add_executable(hdfs_tool_tests hdfs-rm-mock.cc hdfs-get-mock.cc hdfs-find-mock.cc + hdfs-ls-mock.cc main.cc) target_include_directories(hdfs_tool_tests PRIVATE ../tools @@ -58,6 +59,7 @@ target_include_directories(hdfs_tool_tests PRIVATE ../../tools/hdfs-rm ../../tools/hdfs-get ../../tools/hdfs-find + ../../tools/hdfs-ls ../../tools/hdfs-cat) target_link_libraries(hdfs_tool_tests PRIVATE gmock_main @@ -78,5 +80,6 @@ target_link_libraries(hdfs_tool_tests PRIVATE hdfs_rm_lib hdfs_get_lib hdfs_find_lib + hdfs_ls_lib hdfs_cat_lib) add_test(hdfs_tool_tests hdfs_tool_tests) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.cc new file mode 100644 index 0000000000000..6f1cbcf1d074d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.cc @@ -0,0 +1,67 @@ +/** + * 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. + */ + +#include +#include +#include +#include + +#include +#include + +#include "hdfs-ls-mock.h" +#include "hdfs-tool-tests.h" + +namespace hdfs::tools::test { +LsMock::~LsMock() = default; + +void LsMock::SetExpectations(std::function()> test_case, + const std::vector &args) const { + // Get the pointer to the function that defines the test case + const auto test_case_func = test_case.target (*)()>(); + ASSERT_NE(test_case_func, nullptr); + + // Set the expected method calls and their corresponding arguments for each + // test case + if (*test_case_func == &CallHelp) { + EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true)); + return; + } + + if (*test_case_func == &PassAPath) { + const auto arg1 = args[0]; + EXPECT_CALL(*this, HandlePath(arg1, false)) + .Times(1) + .WillOnce(testing::Return(true)); + } + + if (*test_case_func == &PassRecursivePath) { + const auto arg1 = args[0]; + const auto arg2 = args[1]; + ASSERT_EQ(arg1, "-R"); + EXPECT_CALL(*this, HandlePath(arg2, true)) + .Times(1) + .WillOnce(testing::Return(true)); + } + + if (*test_case_func == &PassRecursive) { + const auto arg1 = args[0]; + ASSERT_EQ(arg1, "-R"); + } +} +} // namespace hdfs::tools::test diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.h new file mode 100644 index 0000000000000..2218549e95cb7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-ls-mock.h @@ -0,0 +1,68 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_LS_MOCK +#define LIBHDFSPP_TOOLS_HDFS_LS_MOCK + +#include +#include +#include +#include + +#include + +#include "hdfs-ls.h" + +namespace hdfs::tools::test { +/** + * {@class LsMock} is an {@class Ls} whereby it mocks the + * HandleHelp and HandlePath methods for testing their functionality. + */ +class LsMock : public hdfs::tools::Ls { +public: + /** + * {@inheritdoc} + */ + LsMock(const int argc, char **argv) : Ls(argc, argv) {} + + // Abiding to the Rule of 5 + LsMock(const LsMock &) = delete; + LsMock(LsMock &&) = delete; + LsMock &operator=(const LsMock &) = delete; + LsMock &operator=(LsMock &&) = delete; + ~LsMock() override; + + /** + * Defines the methods and the corresponding arguments that are expected + * to be called on this instance of {@link HdfsTool} for the given test case. + * + * @param test_case An {@link std::function} object that points to the + * function defining the test case + * @param args The arguments that are passed to this test case + */ + void SetExpectations(std::function()> test_case, + const std::vector &args = {}) const; + + MOCK_METHOD(bool, HandleHelp, (), (const, override)); + + MOCK_METHOD(bool, HandlePath, (const std::string &, const bool), + (const, override)); +}; +} // namespace hdfs::tools::test + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc index 53df82021bd6e..97169293f495b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc @@ -33,6 +33,7 @@ #include "hdfs-du-mock.h" #include "hdfs-find-mock.h" #include "hdfs-get-mock.h" +#include "hdfs-ls-mock.h" #include "hdfs-mkdir-mock.h" #include "hdfs-move-to-local-mock.h" #include "hdfs-rename-snapshot-mock.h" @@ -81,6 +82,12 @@ INSTANTIATE_TEST_SUITE_P( CallHelp, PassRecursivePath)); +INSTANTIATE_TEST_SUITE_P( + HdfsLs, HdfsToolBasicTest, + testing::Values(PassAPath, + CallHelp, + PassRecursivePath)); + INSTANTIATE_TEST_SUITE_P( HdfsDeleteSnapshot, HdfsToolBasicTest, testing::Values(CallHelp, @@ -180,6 +187,14 @@ INSTANTIATE_TEST_SUITE_P( PassOwnerAndAPath, PassPermissionsAndAPath)); +INSTANTIATE_TEST_SUITE_P( + HdfsLs, HdfsToolNegativeTestThrows, + testing::Values(Pass2Paths, + Pass3Paths, + PassNOptAndAPath, + PassOwnerAndAPath, + PassPermissionsAndAPath)); + INSTANTIATE_TEST_SUITE_P( HdfsCat, HdfsToolNegativeTestThrows, testing::Values(Pass2Paths)); @@ -262,6 +277,10 @@ INSTANTIATE_TEST_SUITE_P( HdfsDu, HdfsToolNegativeTestNoThrow, testing::Values(PassRecursive)); +INSTANTIATE_TEST_SUITE_P( + HdfsLs, HdfsToolNegativeTestNoThrow, + testing::Values(PassRecursive)); + INSTANTIATE_TEST_SUITE_P( HdfsChown, HdfsToolNegativeTestNoThrow, testing::Values(PassAPath)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt index 214d7b56dcb5c..7cbbe49b5580f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt @@ -47,8 +47,7 @@ add_subdirectory(hdfs-mkdir) add_subdirectory(hdfs-rm) -add_executable(hdfs_ls hdfs_ls.cc) -target_link_libraries(hdfs_ls tools_common hdfspp_static) +add_subdirectory(hdfs-ls) add_executable(hdfs_stat hdfs_stat.cc) target_link_libraries(hdfs_stat tools_common hdfspp_static) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/CMakeLists.txt new file mode 100644 index 0000000000000..196d6356e2ee2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/CMakeLists.txt @@ -0,0 +1,27 @@ +# +# 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. +# + +add_library(hdfs_ls_lib STATIC $ hdfs-ls.cc) +target_include_directories(hdfs_ls_lib PRIVATE ../../tools ${Boost_INCLUDE_DIRS}) +target_link_libraries(hdfs_ls_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static) + +add_executable(hdfs_ls main.cc) +target_include_directories(hdfs_ls PRIVATE ../../tools) +target_link_libraries(hdfs_ls PRIVATE hdfs_ls_lib) + +install(TARGETS hdfs_ls RUNTIME DESTINATION bin) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.cc new file mode 100644 index 0000000000000..8367bbc71feee --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.cc @@ -0,0 +1,156 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include + +#include "hdfs-ls.h" +#include "tools_common.h" + +namespace hdfs::tools { +Ls::Ls(const int argc, char **argv) : HdfsTool(argc, argv) {} + +bool Ls::Initialize() { + auto add_options = opt_desc_.add_options(); + add_options("help,h", "List information about the files"); + add_options("recursive,R", "Operate on files and directories recursively"); + add_options("path", po::value(), + "The path for which we need to do ls"); + + // We allow only one positional argument to be passed to this tool. An + // exception is thrown if multiple arguments are passed. + pos_opt_desc_.add("path", 1); + + po::store(po::command_line_parser(argc_, argv_) + .options(opt_desc_) + .positional(pos_opt_desc_) + .run(), + opt_val_); + po::notify(opt_val_); + return true; +} + +std::string Ls::GetDescription() const { + std::stringstream desc; + desc << "Usage: hdfs_ls [OPTION] FILE" << std::endl + << std::endl + << "List information about the FILEs." << std::endl + << std::endl + << " -R list subdirectories recursively" << std::endl + << " -h display this help and exit" << std::endl + << std::endl + << "Examples:" << std::endl + << "hdfs_ls hdfs://localhost.localdomain:8020/dir" << std::endl + << "hdfs_ls -R /dir1/dir2" << std::endl; + return desc.str(); +} + +bool Ls::Do() { + if (!Initialize()) { + std::cerr << "Unable to initialize HDFS ls tool" << std::endl; + return false; + } + + if (!ValidateConstraints()) { + std::cout << GetDescription(); + return false; + } + + if (opt_val_.count("help") > 0) { + return HandleHelp(); + } + + if (opt_val_.count("path") > 0) { + const auto path = opt_val_["path"].as(); + const auto recursive = opt_val_.count("recursive") > 0; + return HandlePath(path, recursive); + } + + return false; +} + +bool Ls::HandleHelp() const { + std::cout << GetDescription(); + return true; +} + +bool Ls::HandlePath(const std::string &path, const bool recursive) const { + // Building a URI object from the given path + auto uri = hdfs::parse_path_or_exit(path); + + const auto fs = hdfs::doConnect(uri, true); + if (!fs) { + std::cerr << "Could not connect the file system. " << std::endl; + return false; + } + + const auto promise = std::make_shared>(); + auto future(promise->get_future()); + auto result = hdfs::Status::OK(); + + /* + * Keep requesting more until we get the entire listing. Set the promise + * when we have the entire listing to stop. + * + * Find and GetListing guarantee that the handler will only be called once at + * a time, so we do not need any locking here. They also guarantee that the + * handler will be only called once with has_more_results set to false. + */ + auto handler = [promise, + &result](const hdfs::Status &status, + const std::vector &stat_info, + const bool has_more_results) -> bool { + // Print result chunks as they arrive + if (!stat_info.empty()) { + for (const auto &info : stat_info) { + std::cout << info.str() << std::endl; + } + } + if (!status.ok() && result.ok()) { + // We make sure we set the result only on the first error + result = status; + } + if (!has_more_results) { + promise->set_value(); // Set promise + return false; // Request to stop sending results + } + return true; // Request more results + }; + + if (!recursive) { + // Asynchronous call to GetListing + fs->GetListing(uri.get_path(), handler); + } else { + // Asynchronous call to Find + fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), + handler); + } + + // Block until promise is set + future.get(); + if (!result.ok()) { + std::cerr << "Error: " << result.ToString() << std::endl; + return false; + } + return true; +} +} // namespace hdfs::tools diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.h new file mode 100644 index 0000000000000..c38731b67a5fe --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/hdfs-ls.h @@ -0,0 +1,92 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_LS +#define LIBHDFSPP_TOOLS_HDFS_LS + +#include + +#include + +#include "hdfs-tool.h" + +namespace hdfs::tools { +/** + * {@class Ls} is an {@class HdfsTool} that lists information about the files. + */ +class Ls : public HdfsTool { +public: + /** + * {@inheritdoc} + */ + Ls(int argc, char **argv); + + // Abiding to the Rule of 5 + Ls(const Ls &) = default; + Ls(Ls &&) = default; + Ls &operator=(const Ls &) = delete; + Ls &operator=(Ls &&) = delete; + ~Ls() override = default; + + /** + * {@inheritdoc} + */ + [[nodiscard]] std::string GetDescription() const override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Do() override; + +protected: + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Initialize() override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool ValidateConstraints() const override { return argc_ > 1; } + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool HandleHelp() const override; + + /** + * Handle the path argument that's passed to this tool. + * + * @param path The path to the directory for which we need to ls. + * @param recursive A boolean indicating whether ls needs to be + * performed recursively for the given path. + * + * @return A boolean indicating the result of this operation. + */ + [[nodiscard]] virtual bool HandlePath(const std::string &path, + bool recursive) const; + +private: + /** + * A boost data-structure containing the description of positional arguments + * passed to the command-line. + */ + po::positional_options_description pos_opt_desc_; +}; +} // namespace hdfs::tools +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/main.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/main.cc new file mode 100644 index 0000000000000..5d5312f492d45 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-ls/main.cc @@ -0,0 +1,52 @@ +/** + * 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. + */ + +#include +#include +#include + +#include + +#include "hdfs-ls.h" + +int main(int argc, char *argv[]) { + const auto result = std::atexit([]() -> void { + // Clean up static data on exit and prevent valgrind memory leaks + google::protobuf::ShutdownProtobufLibrary(); + }); + if (result != 0) { + std::cerr << "Error: Unable to schedule clean-up tasks for HDFS ls tool, " + "exiting" + << std::endl; + std::exit(EXIT_FAILURE); + } + + hdfs::tools::Ls ls(argc, argv); + auto success = false; + + try { + success = ls.Do(); + } catch (const std::exception &e) { + std::cerr << "Error: " << e.what() << std::endl; + } + + if (!success) { + std::exit(EXIT_FAILURE); + } + return 0; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_ls.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_ls.cc deleted file mode 100644 index e35e51da1d32c..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_ls.cc +++ /dev/null @@ -1,130 +0,0 @@ -/* - 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. -*/ - -#include -#include -#include -#include "tools_common.h" - -void usage(){ - std::cout << "Usage: hdfs_ls [OPTION] FILE" - << std::endl - << std::endl << "List information about the FILEs." - << std::endl - << std::endl << " -R list subdirectories recursively" - << std::endl << " -h display this help and exit" - << std::endl - << std::endl << "Examples:" - << std::endl << "hdfs_ls hdfs://localhost.localdomain:8020/dir" - << std::endl << "hdfs_ls -R /dir1/dir2" - << std::endl; -} - -int main(int argc, char *argv[]) { - //We should have at least 2 arguments - if (argc < 2) { - usage(); - exit(EXIT_FAILURE); - } - - bool recursive = false; - int input; - - //Using GetOpt to read in the values - opterr = 0; - while ((input = getopt(argc, argv, "Rh")) != -1) { - switch (input) - { - case 'R': - recursive = true; - break; - case 'h': - usage(); - exit(EXIT_SUCCESS); - case '?': - if (isprint(optopt)) - std::cerr << "Unknown option `-" << (char) optopt << "'." << std::endl; - else - std::cerr << "Unknown option character `" << (char) optopt << "'." << std::endl; - usage(); - exit(EXIT_FAILURE); - default: - exit(EXIT_FAILURE); - } - } - std::string uri_path = argv[optind]; - - //Building a URI object from the given uri_path - hdfs::URI uri = hdfs::parse_path_or_exit(uri_path); - - std::shared_ptr fs = hdfs::doConnect(uri, true); - if (!fs) { - std::cerr << "Could not connect the file system. " << std::endl; - exit(EXIT_FAILURE); - } - - std::shared_ptr> promise = std::make_shared>(); - std::future future(promise->get_future()); - hdfs::Status status = hdfs::Status::OK(); - - /** - * Keep requesting more until we get the entire listing. Set the promise - * when we have the entire listing to stop. - * - * Find and GetListing guarantee that the handler will only be called once at a time, - * so we do not need any locking here. They also guarantee that the handler will be - * only called once with has_more_results set to false. - */ - auto handler = [promise, &status] - (const hdfs::Status &s, const std::vector & si, bool has_more_results) -> bool { - //Print result chunks as they arrive - if(!si.empty()) { - for (hdfs::StatInfo const& s : si) { - std::cout << s.str() << std::endl; - } - } - if(!s.ok() && status.ok()){ - //We make sure we set 'status' only on the first error. - status = s; - } - if (!has_more_results) { - promise->set_value(); //set promise - return false; //request stop sending results - } - return true; //request more results - }; - - if(!recursive){ - //Asynchronous call to GetListing - fs->GetListing(uri.get_path(), handler); - } else { - //Asynchronous call to Find - fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handler); - } - - //block until promise is set - future.get(); - if(!status.ok()) { - std::cerr << "Error: " << status.ToString() << std::endl; - } - - // Clean up static data and prevent valgrind memory leaks - google::protobuf::ShutdownProtobufLibrary(); - return 0; -} From 26ba3846cc84fa3f4a4de37920ae5aae72584af1 Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Wed, 23 Mar 2022 11:01:35 +0800 Subject: [PATCH 056/145] Revert "HDFS-14617. Improve oiv tool to parse fsimage file in parallel with delimited format. (#2918). Contributed by Hongbing Wang." This reverts commit 88975496d8a076b8923999e9e9ecef13e3721e3d. --- .../OfflineImageViewerPB.java | 10 +- .../PBImageCorruptionDetector.java | 2 +- .../PBImageDelimitedTextWriter.java | 8 +- .../offlineImageViewer/PBImageTextWriter.java | 267 ++---------------- .../TestOfflineImageViewer.java | 59 +--- .../TestOfflineImageViewerForAcl.java | 2 +- 6 files changed, 42 insertions(+), 306 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java index 05e687ab97e43..dbcb452e166aa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java @@ -107,7 +107,6 @@ public class OfflineImageViewerPB { + " Delimited outputs. If not set, the processor\n" + " constructs the namespace in memory \n" + " before outputting text.\n" - + "-m,--multiThread Use multiThread to process sub-sections.\n" + "-h,--help Display usage information and exit\n"; /** @@ -133,7 +132,6 @@ private static Options buildOptions() { options.addOption("delimiter", true, ""); options.addOption("sp", false, ""); options.addOption("t", "temp", true, ""); - options.addOption("m", "multiThread", true, ""); return options; } @@ -187,7 +185,6 @@ public static int run(String[] args) throws Exception { String delimiter = cmd.getOptionValue("delimiter", PBImageTextWriter.DEFAULT_DELIMITER); String tempPath = cmd.getOptionValue("t", ""); - int threads = Integer.parseInt(cmd.getOptionValue("m", "1")); Configuration conf = new Configuration(); PrintStream out = null; @@ -230,14 +227,15 @@ public static int run(String[] args) throws Exception { boolean printStoragePolicy = cmd.hasOption("sp"); try (PBImageDelimitedTextWriter writer = new PBImageDelimitedTextWriter(out, delimiter, - tempPath, printStoragePolicy, threads, outputFile)) { - writer.visit(inputFile); + tempPath, printStoragePolicy); + RandomAccessFile r = new RandomAccessFile(inputFile, "r")) { + writer.visit(r); } break; case "DETECTCORRUPTION": try (PBImageCorruptionDetector detector = new PBImageCorruptionDetector(out, delimiter, tempPath)) { - detector.visit(inputFile); + detector.visit(new RandomAccessFile(inputFile, "r")); } break; default: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java index 17593867bd642..28c450701b846 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java @@ -337,7 +337,7 @@ public void afterOutput() throws IOException { if (parentId != -1) { entryBuilder.setParentId(parentId); } - printIfNotEmpty(serialOutStream(), entryBuilder.build()); + printIfNotEmpty(entryBuilder.build()); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java index 3e080ec8e65cd..45d42f0396b1a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java @@ -146,13 +146,7 @@ public String build() { PBImageDelimitedTextWriter(PrintStream out, String delimiter, String tempPath, boolean printStoragePolicy) throws IOException { - this(out, delimiter, tempPath, printStoragePolicy, 1, "-"); - } - - PBImageDelimitedTextWriter(PrintStream out, String delimiter, - String tempPath, boolean printStoragePolicy, int threads, - String parallelOut) throws IOException { - super(out, delimiter, tempPath, threads, parallelOut); + super(out, delimiter, tempPath); this.printStoragePolicy = printStoragePolicy; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java index 2dab44a036ac6..08fe7fb943c15 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java @@ -21,25 +21,17 @@ import java.io.Closeable; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.PrintStream; import java.io.RandomAccessFile; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.HashMap; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -463,22 +455,20 @@ public String getParentPath(long inode) throws IOException { return "/"; } long parent = getFromDirChildMap(inode); - byte[] bytes = dirMap.get(toBytes(parent)); - synchronized (this) { - if (!dirPathCache.containsKey(parent)) { - if (parent != INodeId.ROOT_INODE_ID && bytes == null) { - // The parent is an INodeReference, which is generated from snapshot. - // For delimited oiv tool, no need to print out metadata in snapshots. - throw PBImageTextWriter.createIgnoredSnapshotException(inode); - } - String parentName = toString(bytes); - String parentPath = - new Path(getParentPath(parent), - parentName.isEmpty() ? "/" : parentName).toString(); - dirPathCache.put(parent, parentPath); + if (!dirPathCache.containsKey(parent)) { + byte[] bytes = dirMap.get(toBytes(parent)); + if (parent != INodeId.ROOT_INODE_ID && bytes == null) { + // The parent is an INodeReference, which is generated from snapshot. + // For delimited oiv tool, no need to print out metadata in snapshots. + throw PBImageTextWriter.createIgnoredSnapshotException(inode); } - return dirPathCache.get(parent); + String parentName = toString(bytes); + String parentPath = + new Path(getParentPath(parent), + parentName.isEmpty() ? "/" : parentName).toString(); + dirPathCache.put(parent, parentPath); } + return dirPathCache.get(parent); } @Override @@ -503,12 +493,9 @@ public long getParentId(long id) throws IOException { } private SerialNumberManager.StringTable stringTable; - private final PrintStream out; + private PrintStream out; private MetadataMap metadataMap = null; private String delimiter; - private File filename; - private int numThreads; - private String parallelOutputFile; /** * Construct a PB FsImage writer to generate text file. @@ -516,8 +503,8 @@ public long getParentId(long id) throws IOException { * @param tempPath the path to store metadata. If it is empty, store metadata * in memory instead. */ - PBImageTextWriter(PrintStream out, String delimiter, String tempPath, - int numThreads, String parallelOutputFile) throws IOException { + PBImageTextWriter(PrintStream out, String delimiter, String tempPath) + throws IOException { this.out = out; this.delimiter = delimiter; if (tempPath.isEmpty()) { @@ -525,17 +512,6 @@ public long getParentId(long id) throws IOException { } else { metadataMap = new LevelDBMetadataMap(tempPath); } - this.numThreads = numThreads; - this.parallelOutputFile = parallelOutputFile; - } - - PBImageTextWriter(PrintStream out, String delimiter, String tempPath) - throws IOException { - this(out, delimiter, tempPath, 1, "-"); - } - - protected PrintStream serialOutStream() { - return out; } @Override @@ -586,9 +562,7 @@ void append(StringBuffer buffer, String field) { */ abstract protected void afterOutput() throws IOException; - public void visit(String filePath) throws IOException { - filename = new File(filePath); - RandomAccessFile file = new RandomAccessFile(filePath, "r"); + public void visit(RandomAccessFile file) throws IOException { Configuration conf = new Configuration(); if (!FSImageUtil.checkFileFormat(file)) { throw new IOException("Unrecognized FSImage"); @@ -668,122 +642,21 @@ long getParentId(long id) throws IOException { private void output(Configuration conf, FileSummary summary, FileInputStream fin, ArrayList sections) throws IOException { - ArrayList allINodeSubSections = - getINodeSubSections(sections); - if (numThreads > 1 && !parallelOutputFile.equals("-") && - allINodeSubSections.size() > 1) { - outputInParallel(conf, summary, allINodeSubSections); - } else { - LOG.info("Serial output due to threads num: {}, parallel output file: {}, " + - "subSections: {}.", numThreads, parallelOutputFile, allINodeSubSections.size()); - outputInSerial(conf, summary, fin, sections); - } - } - - private void outputInSerial(Configuration conf, FileSummary summary, - FileInputStream fin, ArrayList sections) - throws IOException { InputStream is; long startTime = Time.monotonicNow(); - serialOutStream().println(getHeader()); + out.println(getHeader()); for (FileSummary.Section section : sections) { if (SectionName.fromString(section.getName()) == SectionName.INODE) { fin.getChannel().position(section.getOffset()); is = FSImageUtil.wrapInputStreamForCompression(conf, summary.getCodec(), new BufferedInputStream(new LimitInputStream( fin, section.getLength()))); - INodeSection s = INodeSection.parseDelimitedFrom(is); - LOG.info("Found {} INodes in the INode section", s.getNumInodes()); - int count = outputINodes(is, serialOutStream()); - LOG.info("Outputted {} INodes.", count); + outputINodes(is); } } afterOutput(); long timeTaken = Time.monotonicNow() - startTime; - LOG.debug("Time to output inodes: {} ms", timeTaken); - } - - /** - * STEP1: Multi-threaded process sub-sections. - * Given n (n>1) threads to process k (k>=n) sections, - * output parsed results of each section to tmp file in order. - * STEP2: Merge tmp files. - */ - private void outputInParallel(Configuration conf, FileSummary summary, - ArrayList subSections) - throws IOException { - int nThreads = Integer.min(numThreads, subSections.size()); - LOG.info("Outputting in parallel with {} sub-sections using {} threads", - subSections.size(), nThreads); - final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); - CountDownLatch latch = new CountDownLatch(subSections.size()); - ExecutorService executorService = Executors.newFixedThreadPool(nThreads); - AtomicLong expectedINodes = new AtomicLong(0); - AtomicLong totalParsed = new AtomicLong(0); - String codec = summary.getCodec(); - String[] paths = new String[subSections.size()]; - - for (int i = 0; i < subSections.size(); i++) { - paths[i] = parallelOutputFile + ".tmp." + i; - int index = i; - executorService.submit(() -> { - LOG.info("Output iNodes of section-{}", index); - InputStream is = null; - try (PrintStream outStream = new PrintStream(paths[index], "UTF-8")) { - long startTime = Time.monotonicNow(); - is = getInputStreamForSection(subSections.get(index), codec, conf); - if (index == 0) { - // The first iNode section has a header which must be processed first - INodeSection s = INodeSection.parseDelimitedFrom(is); - expectedINodes.set(s.getNumInodes()); - } - totalParsed.addAndGet(outputINodes(is, outStream)); - long timeTaken = Time.monotonicNow() - startTime; - LOG.info("Time to output iNodes of section-{}: {} ms", index, timeTaken); - } catch (Exception e) { - exceptions.add(new IOException(e)); - } finally { - latch.countDown(); - try { - if (is != null) { - is.close(); - } - } catch (IOException ioe) { - LOG.warn("Failed to close the input stream, ignoring", ioe); - } - } - }); - } - - try { - latch.await(); - } catch (InterruptedException e) { - LOG.error("Interrupted waiting for countdown latch", e); - throw new IOException(e); - } - - executorService.shutdown(); - if (exceptions.size() != 0) { - LOG.error("Failed to output INode sub-sections, {} exception(s) occurred.", - exceptions.size()); - throw exceptions.get(0); - } - if (totalParsed.get() != expectedINodes.get()) { - throw new IOException("Expected to parse " + expectedINodes + " in parallel, " + - "but parsed " + totalParsed.get() + ". The image may be corrupt."); - } - LOG.info("Completed outputting all INode sub-sections to {} tmp files.", - subSections.size()); - - try (PrintStream ps = new PrintStream(parallelOutputFile, "UTF-8")) { - ps.println(getHeader()); - } - - // merge tmp files - long startTime = Time.monotonicNow(); - mergeFiles(paths, parallelOutputFile); - long timeTaken = Time.monotonicNow() - startTime; - LOG.info("Completed all stages. Time to merge files: {} ms", timeTaken); + LOG.debug("Time to output inodes: {}ms", timeTaken); } protected PermissionStatus getPermission(long perm) { @@ -890,27 +763,22 @@ protected void buildNamespace(InputStream in, List refIdList) LOG.info("Scanned {} INode directories to build namespace.", count); } - void printIfNotEmpty(PrintStream outStream, String line) { + void printIfNotEmpty(String line) { if (!line.isEmpty()) { - outStream.println(line); + out.println(line); } } - private int outputINodes(InputStream in, PrintStream outStream) - throws IOException { + private void outputINodes(InputStream in) throws IOException { + INodeSection s = INodeSection.parseDelimitedFrom(in); + LOG.info("Found {} INodes in the INode section", s.getNumInodes()); long ignored = 0; long ignoredSnapshots = 0; - // As the input stream is a LimitInputStream, the reading will stop when - // EOF is encountered at the end of the stream. - int count = 0; - while (true) { + for (int i = 0; i < s.getNumInodes(); ++i) { INode p = INode.parseDelimitedFrom(in); - if (p == null) { - break; - } try { String parentPath = metadataMap.getParentPath(p.getId()); - printIfNotEmpty(outStream, getEntry(parentPath, p)); + printIfNotEmpty(getEntry(parentPath, p)); } catch (IOException ioe) { ignored++; if (!(ioe instanceof IgnoreSnapshotException)) { @@ -922,16 +790,16 @@ private int outputINodes(InputStream in, PrintStream outStream) } } } - count++; - if (LOG.isDebugEnabled() && count % 100000 == 0) { - LOG.debug("Outputted {} INodes.", count); + + if (LOG.isDebugEnabled() && i % 100000 == 0) { + LOG.debug("Outputted {} INodes.", i); } } if (ignored > 0) { LOG.warn("Ignored {} nodes, including {} in snapshots. Please turn on" + " debug log for details", ignored, ignoredSnapshots); } - return count; + LOG.info("Outputted {} INodes.", s.getNumInodes()); } private static IgnoreSnapshotException createIgnoredSnapshotException( @@ -954,79 +822,4 @@ public int getStoragePolicy( } return HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED; } - - private ArrayList getINodeSubSections( - ArrayList sections) { - ArrayList subSections = new ArrayList<>(); - Iterator iter = sections.iterator(); - while (iter.hasNext()) { - FileSummary.Section s = iter.next(); - if (SectionName.fromString(s.getName()) == SectionName.INODE_SUB) { - subSections.add(s); - } - } - return subSections; - } - - /** - * Given a FSImage FileSummary.section, return a LimitInput stream set to - * the starting position of the section and limited to the section length. - * @param section The FileSummary.Section containing the offset and length - * @param compressionCodec The compression codec in use, if any - * @return An InputStream for the given section - * @throws IOException - */ - private InputStream getInputStreamForSection(FileSummary.Section section, - String compressionCodec, Configuration conf) - throws IOException { - // channel of RandomAccessFile is not thread safe, use File - FileInputStream fin = new FileInputStream(filename); - try { - FileChannel channel = fin.getChannel(); - channel.position(section.getOffset()); - InputStream in = new BufferedInputStream(new LimitInputStream(fin, - section.getLength())); - - in = FSImageUtil.wrapInputStreamForCompression(conf, - compressionCodec, in); - return in; - } catch (IOException e) { - fin.close(); - throw e; - } - } - - /** - * @param srcPaths Source files of contents to be merged - * @param resultPath Merged file path - * @throws IOException - */ - public static void mergeFiles(String[] srcPaths, String resultPath) - throws IOException { - if (srcPaths == null || srcPaths.length < 1) { - LOG.warn("no source files to merge."); - return; - } - - File[] files = new File[srcPaths.length]; - for (int i = 0; i < srcPaths.length; i++) { - files[i] = new File(srcPaths[i]); - } - - File resultFile = new File(resultPath); - try (FileChannel resultChannel = - new FileOutputStream(resultFile, true).getChannel()) { - for (File file : files) { - try (FileChannel src = new FileInputStream(file).getChannel()) { - resultChannel.transferFrom(src, resultChannel.size(), src.size()); - } - } - } - - for (File file : files) { - if (!file.delete() && file.exists()) { - LOG.warn("delete tmp file: {} returned false", file); - } - } - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java index 9878469c89e30..53031bcad90e3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java @@ -83,10 +83,8 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto; import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion; -import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.ErasureCodeConstants; import org.apache.hadoop.net.NetUtils; @@ -124,7 +122,6 @@ import static org.apache.hadoop.fs.permission.FsAction.ALL; import static org.apache.hadoop.fs.permission.FsAction.EXECUTE; import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE; -import static org.apache.hadoop.hdfs.MiniDFSCluster.HDFS_MINIDFS_BASEDIR; import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; import static org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter.ERASURE_CODING_SECTION_NAME; import static org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter.ERASURE_CODING_SECTION_POLICY; @@ -189,12 +186,6 @@ public static void createOriginalFSImage() throws IOException { conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL, "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT"); - // fsimage with sub-section conf - conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY, "true"); - conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY, "1"); - conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY, "4"); - conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_KEY, "4"); - cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); cluster.waitActive(); DistributedFileSystem hdfs = cluster.getFileSystem(); @@ -800,13 +791,6 @@ public void testPBDelimitedWriter() throws IOException, InterruptedException { new FileSystemTestHelper().getTestRootDir() + "/delimited.db"); } - @Test - public void testParallelPBDelimitedWriter() throws Exception { - testParallelPBDelimitedWriter(""); // Test in memory db. - testParallelPBDelimitedWriter(new FileSystemTestHelper().getTestRootDir() - + "/parallel-delimited.db"); - } - @Test public void testCorruptionOutputEntryBuilder() throws IOException { PBImageCorruptionDetector corrDetector = @@ -898,10 +882,11 @@ private void testPBDelimitedWriter(String db) final String DELIMITER = "\t"; ByteArrayOutputStream output = new ByteArrayOutputStream(); - try (PrintStream o = new PrintStream(output)) { + try (PrintStream o = new PrintStream(output); + RandomAccessFile r = new RandomAccessFile(originalFsimage, "r")) { PBImageDelimitedTextWriter v = new PBImageDelimitedTextWriter(o, DELIMITER, db); - v.visit(originalFsimage.getAbsolutePath()); + v.visit(r); } Set fileNames = new HashSet<>(); @@ -935,37 +920,6 @@ private void testPBDelimitedWriter(String db) assertEquals(writtenFiles.keySet(), fileNames); } - private void testParallelPBDelimitedWriter(String db) throws Exception{ - String delimiter = "\t"; - int numThreads = 4; - - File parallelDelimitedOut = new File(tempDir, "parallelDelimitedOut"); - if (OfflineImageViewerPB.run(new String[] {"-p", "Delimited", - "-i", originalFsimage.getAbsolutePath(), - "-o", parallelDelimitedOut.getAbsolutePath(), - "-delimiter", delimiter, - "-t", db, - "-m", String.valueOf(numThreads)}) != 0) { - throw new IOException("oiv returned failure outputting in parallel."); - } - MD5Hash parallelMd5 = MD5FileUtils.computeMd5ForFile(parallelDelimitedOut); - - File serialDelimitedOut = new File(tempDir, "serialDelimitedOut"); - if (db != "") { - db = db + "/../serial.db"; - } - if (OfflineImageViewerPB.run(new String[] {"-p", "Delimited", - "-i", originalFsimage.getAbsolutePath(), - "-o", serialDelimitedOut.getAbsolutePath(), - "-t", db, - "-delimiter", delimiter}) != 0) { - throw new IOException("oiv returned failure outputting in serial."); - } - MD5Hash serialMd5 = MD5FileUtils.computeMd5ForFile(serialDelimitedOut); - - assertEquals(parallelMd5, serialMd5); - } - private void testPBCorruptionDetector(String db) throws IOException, InterruptedException { final String delimiter = "\t"; @@ -974,7 +928,7 @@ private void testPBCorruptionDetector(String db) try (PrintStream o = new PrintStream(output)) { PBImageCorruptionDetector v = new PBImageCorruptionDetector(o, delimiter, db); - v.visit(originalFsimage.getAbsolutePath()); + v.visit(new RandomAccessFile(originalFsimage, "r")); } try ( @@ -1070,7 +1024,7 @@ private String testCorruptionDetectorRun(int runNumber, try (PrintStream o = new PrintStream(output)) { PBImageCorruptionDetector v = new PBImageCorruptionDetector(o, ",", db); - v.visit(corruptedImage.getAbsolutePath()); + v.visit(new RandomAccessFile(corruptedImage, "r")); } return output.toString(); } @@ -1258,9 +1212,6 @@ public void testReverseXmlWithoutSnapshotDiffSection() throws Throwable { public void testFileDistributionCalculatorForException() throws Exception { File fsimageFile = null; Configuration conf = new Configuration(); - // Avoid using the same cluster dir to cause the global originalFsimage - // file to be cleared. - conf.set(HDFS_MINIDFS_BASEDIR, GenericTestUtils.getRandomizedTempPath()); HashMap files = Maps.newHashMap(); // Create a initial fsimage file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java index b23ddf4afbcfb..4955846432a60 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java @@ -239,7 +239,7 @@ public void testPBDelimitedWriterForAcl() throws Exception { try (PrintStream o = new PrintStream(output)) { PBImageDelimitedTextWriter v = new PBImageDelimitedTextWriter(o, DELIMITER, ""); // run in memory. - v.visit(originalFsimage.getAbsolutePath()); + v.visit(new RandomAccessFile(originalFsimage, "r")); } try ( From ef8bff0df9f2b1eca99f655e2f74f40d3f098a2e Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Wed, 23 Mar 2022 11:32:12 +0800 Subject: [PATCH 057/145] HDFS-15987. Improve oiv tool to parse fsimage file in parallel with delimited format. (#2918). Contributed by Hongbing Wang. Signed-off-by: He Xiaoqiao --- .../OfflineImageViewerPB.java | 10 +- .../PBImageCorruptionDetector.java | 2 +- .../PBImageDelimitedTextWriter.java | 8 +- .../offlineImageViewer/PBImageTextWriter.java | 267 ++++++++++++++++-- .../TestOfflineImageViewer.java | 59 +++- .../TestOfflineImageViewerForAcl.java | 2 +- 6 files changed, 306 insertions(+), 42 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java index dbcb452e166aa..05e687ab97e43 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageViewerPB.java @@ -107,6 +107,7 @@ public class OfflineImageViewerPB { + " Delimited outputs. If not set, the processor\n" + " constructs the namespace in memory \n" + " before outputting text.\n" + + "-m,--multiThread Use multiThread to process sub-sections.\n" + "-h,--help Display usage information and exit\n"; /** @@ -132,6 +133,7 @@ private static Options buildOptions() { options.addOption("delimiter", true, ""); options.addOption("sp", false, ""); options.addOption("t", "temp", true, ""); + options.addOption("m", "multiThread", true, ""); return options; } @@ -185,6 +187,7 @@ public static int run(String[] args) throws Exception { String delimiter = cmd.getOptionValue("delimiter", PBImageTextWriter.DEFAULT_DELIMITER); String tempPath = cmd.getOptionValue("t", ""); + int threads = Integer.parseInt(cmd.getOptionValue("m", "1")); Configuration conf = new Configuration(); PrintStream out = null; @@ -227,15 +230,14 @@ public static int run(String[] args) throws Exception { boolean printStoragePolicy = cmd.hasOption("sp"); try (PBImageDelimitedTextWriter writer = new PBImageDelimitedTextWriter(out, delimiter, - tempPath, printStoragePolicy); - RandomAccessFile r = new RandomAccessFile(inputFile, "r")) { - writer.visit(r); + tempPath, printStoragePolicy, threads, outputFile)) { + writer.visit(inputFile); } break; case "DETECTCORRUPTION": try (PBImageCorruptionDetector detector = new PBImageCorruptionDetector(out, delimiter, tempPath)) { - detector.visit(new RandomAccessFile(inputFile, "r")); + detector.visit(inputFile); } break; default: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java index 28c450701b846..17593867bd642 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageCorruptionDetector.java @@ -337,7 +337,7 @@ public void afterOutput() throws IOException { if (parentId != -1) { entryBuilder.setParentId(parentId); } - printIfNotEmpty(entryBuilder.build()); + printIfNotEmpty(serialOutStream(), entryBuilder.build()); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java index 45d42f0396b1a..3e080ec8e65cd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageDelimitedTextWriter.java @@ -146,7 +146,13 @@ public String build() { PBImageDelimitedTextWriter(PrintStream out, String delimiter, String tempPath, boolean printStoragePolicy) throws IOException { - super(out, delimiter, tempPath); + this(out, delimiter, tempPath, printStoragePolicy, 1, "-"); + } + + PBImageDelimitedTextWriter(PrintStream out, String delimiter, + String tempPath, boolean printStoragePolicy, int threads, + String parallelOut) throws IOException { + super(out, delimiter, tempPath, threads, parallelOut); this.printStoragePolicy = printStoragePolicy; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java index 08fe7fb943c15..2dab44a036ac6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageTextWriter.java @@ -21,17 +21,25 @@ import java.io.Closeable; import java.io.File; import java.io.FileInputStream; +import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.PrintStream; import java.io.RandomAccessFile; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.HashMap; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -455,20 +463,22 @@ public String getParentPath(long inode) throws IOException { return "/"; } long parent = getFromDirChildMap(inode); - if (!dirPathCache.containsKey(parent)) { - byte[] bytes = dirMap.get(toBytes(parent)); - if (parent != INodeId.ROOT_INODE_ID && bytes == null) { - // The parent is an INodeReference, which is generated from snapshot. - // For delimited oiv tool, no need to print out metadata in snapshots. - throw PBImageTextWriter.createIgnoredSnapshotException(inode); + byte[] bytes = dirMap.get(toBytes(parent)); + synchronized (this) { + if (!dirPathCache.containsKey(parent)) { + if (parent != INodeId.ROOT_INODE_ID && bytes == null) { + // The parent is an INodeReference, which is generated from snapshot. + // For delimited oiv tool, no need to print out metadata in snapshots. + throw PBImageTextWriter.createIgnoredSnapshotException(inode); + } + String parentName = toString(bytes); + String parentPath = + new Path(getParentPath(parent), + parentName.isEmpty() ? "/" : parentName).toString(); + dirPathCache.put(parent, parentPath); } - String parentName = toString(bytes); - String parentPath = - new Path(getParentPath(parent), - parentName.isEmpty() ? "/" : parentName).toString(); - dirPathCache.put(parent, parentPath); + return dirPathCache.get(parent); } - return dirPathCache.get(parent); } @Override @@ -493,9 +503,12 @@ public long getParentId(long id) throws IOException { } private SerialNumberManager.StringTable stringTable; - private PrintStream out; + private final PrintStream out; private MetadataMap metadataMap = null; private String delimiter; + private File filename; + private int numThreads; + private String parallelOutputFile; /** * Construct a PB FsImage writer to generate text file. @@ -503,8 +516,8 @@ public long getParentId(long id) throws IOException { * @param tempPath the path to store metadata. If it is empty, store metadata * in memory instead. */ - PBImageTextWriter(PrintStream out, String delimiter, String tempPath) - throws IOException { + PBImageTextWriter(PrintStream out, String delimiter, String tempPath, + int numThreads, String parallelOutputFile) throws IOException { this.out = out; this.delimiter = delimiter; if (tempPath.isEmpty()) { @@ -512,6 +525,17 @@ public long getParentId(long id) throws IOException { } else { metadataMap = new LevelDBMetadataMap(tempPath); } + this.numThreads = numThreads; + this.parallelOutputFile = parallelOutputFile; + } + + PBImageTextWriter(PrintStream out, String delimiter, String tempPath) + throws IOException { + this(out, delimiter, tempPath, 1, "-"); + } + + protected PrintStream serialOutStream() { + return out; } @Override @@ -562,7 +586,9 @@ void append(StringBuffer buffer, String field) { */ abstract protected void afterOutput() throws IOException; - public void visit(RandomAccessFile file) throws IOException { + public void visit(String filePath) throws IOException { + filename = new File(filePath); + RandomAccessFile file = new RandomAccessFile(filePath, "r"); Configuration conf = new Configuration(); if (!FSImageUtil.checkFileFormat(file)) { throw new IOException("Unrecognized FSImage"); @@ -642,21 +668,122 @@ long getParentId(long id) throws IOException { private void output(Configuration conf, FileSummary summary, FileInputStream fin, ArrayList sections) throws IOException { + ArrayList allINodeSubSections = + getINodeSubSections(sections); + if (numThreads > 1 && !parallelOutputFile.equals("-") && + allINodeSubSections.size() > 1) { + outputInParallel(conf, summary, allINodeSubSections); + } else { + LOG.info("Serial output due to threads num: {}, parallel output file: {}, " + + "subSections: {}.", numThreads, parallelOutputFile, allINodeSubSections.size()); + outputInSerial(conf, summary, fin, sections); + } + } + + private void outputInSerial(Configuration conf, FileSummary summary, + FileInputStream fin, ArrayList sections) + throws IOException { InputStream is; long startTime = Time.monotonicNow(); - out.println(getHeader()); + serialOutStream().println(getHeader()); for (FileSummary.Section section : sections) { if (SectionName.fromString(section.getName()) == SectionName.INODE) { fin.getChannel().position(section.getOffset()); is = FSImageUtil.wrapInputStreamForCompression(conf, summary.getCodec(), new BufferedInputStream(new LimitInputStream( fin, section.getLength()))); - outputINodes(is); + INodeSection s = INodeSection.parseDelimitedFrom(is); + LOG.info("Found {} INodes in the INode section", s.getNumInodes()); + int count = outputINodes(is, serialOutStream()); + LOG.info("Outputted {} INodes.", count); } } afterOutput(); long timeTaken = Time.monotonicNow() - startTime; - LOG.debug("Time to output inodes: {}ms", timeTaken); + LOG.debug("Time to output inodes: {} ms", timeTaken); + } + + /** + * STEP1: Multi-threaded process sub-sections. + * Given n (n>1) threads to process k (k>=n) sections, + * output parsed results of each section to tmp file in order. + * STEP2: Merge tmp files. + */ + private void outputInParallel(Configuration conf, FileSummary summary, + ArrayList subSections) + throws IOException { + int nThreads = Integer.min(numThreads, subSections.size()); + LOG.info("Outputting in parallel with {} sub-sections using {} threads", + subSections.size(), nThreads); + final CopyOnWriteArrayList exceptions = new CopyOnWriteArrayList<>(); + CountDownLatch latch = new CountDownLatch(subSections.size()); + ExecutorService executorService = Executors.newFixedThreadPool(nThreads); + AtomicLong expectedINodes = new AtomicLong(0); + AtomicLong totalParsed = new AtomicLong(0); + String codec = summary.getCodec(); + String[] paths = new String[subSections.size()]; + + for (int i = 0; i < subSections.size(); i++) { + paths[i] = parallelOutputFile + ".tmp." + i; + int index = i; + executorService.submit(() -> { + LOG.info("Output iNodes of section-{}", index); + InputStream is = null; + try (PrintStream outStream = new PrintStream(paths[index], "UTF-8")) { + long startTime = Time.monotonicNow(); + is = getInputStreamForSection(subSections.get(index), codec, conf); + if (index == 0) { + // The first iNode section has a header which must be processed first + INodeSection s = INodeSection.parseDelimitedFrom(is); + expectedINodes.set(s.getNumInodes()); + } + totalParsed.addAndGet(outputINodes(is, outStream)); + long timeTaken = Time.monotonicNow() - startTime; + LOG.info("Time to output iNodes of section-{}: {} ms", index, timeTaken); + } catch (Exception e) { + exceptions.add(new IOException(e)); + } finally { + latch.countDown(); + try { + if (is != null) { + is.close(); + } + } catch (IOException ioe) { + LOG.warn("Failed to close the input stream, ignoring", ioe); + } + } + }); + } + + try { + latch.await(); + } catch (InterruptedException e) { + LOG.error("Interrupted waiting for countdown latch", e); + throw new IOException(e); + } + + executorService.shutdown(); + if (exceptions.size() != 0) { + LOG.error("Failed to output INode sub-sections, {} exception(s) occurred.", + exceptions.size()); + throw exceptions.get(0); + } + if (totalParsed.get() != expectedINodes.get()) { + throw new IOException("Expected to parse " + expectedINodes + " in parallel, " + + "but parsed " + totalParsed.get() + ". The image may be corrupt."); + } + LOG.info("Completed outputting all INode sub-sections to {} tmp files.", + subSections.size()); + + try (PrintStream ps = new PrintStream(parallelOutputFile, "UTF-8")) { + ps.println(getHeader()); + } + + // merge tmp files + long startTime = Time.monotonicNow(); + mergeFiles(paths, parallelOutputFile); + long timeTaken = Time.monotonicNow() - startTime; + LOG.info("Completed all stages. Time to merge files: {} ms", timeTaken); } protected PermissionStatus getPermission(long perm) { @@ -763,22 +890,27 @@ protected void buildNamespace(InputStream in, List refIdList) LOG.info("Scanned {} INode directories to build namespace.", count); } - void printIfNotEmpty(String line) { + void printIfNotEmpty(PrintStream outStream, String line) { if (!line.isEmpty()) { - out.println(line); + outStream.println(line); } } - private void outputINodes(InputStream in) throws IOException { - INodeSection s = INodeSection.parseDelimitedFrom(in); - LOG.info("Found {} INodes in the INode section", s.getNumInodes()); + private int outputINodes(InputStream in, PrintStream outStream) + throws IOException { long ignored = 0; long ignoredSnapshots = 0; - for (int i = 0; i < s.getNumInodes(); ++i) { + // As the input stream is a LimitInputStream, the reading will stop when + // EOF is encountered at the end of the stream. + int count = 0; + while (true) { INode p = INode.parseDelimitedFrom(in); + if (p == null) { + break; + } try { String parentPath = metadataMap.getParentPath(p.getId()); - printIfNotEmpty(getEntry(parentPath, p)); + printIfNotEmpty(outStream, getEntry(parentPath, p)); } catch (IOException ioe) { ignored++; if (!(ioe instanceof IgnoreSnapshotException)) { @@ -790,16 +922,16 @@ private void outputINodes(InputStream in) throws IOException { } } } - - if (LOG.isDebugEnabled() && i % 100000 == 0) { - LOG.debug("Outputted {} INodes.", i); + count++; + if (LOG.isDebugEnabled() && count % 100000 == 0) { + LOG.debug("Outputted {} INodes.", count); } } if (ignored > 0) { LOG.warn("Ignored {} nodes, including {} in snapshots. Please turn on" + " debug log for details", ignored, ignoredSnapshots); } - LOG.info("Outputted {} INodes.", s.getNumInodes()); + return count; } private static IgnoreSnapshotException createIgnoredSnapshotException( @@ -822,4 +954,79 @@ public int getStoragePolicy( } return HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED; } + + private ArrayList getINodeSubSections( + ArrayList sections) { + ArrayList subSections = new ArrayList<>(); + Iterator iter = sections.iterator(); + while (iter.hasNext()) { + FileSummary.Section s = iter.next(); + if (SectionName.fromString(s.getName()) == SectionName.INODE_SUB) { + subSections.add(s); + } + } + return subSections; + } + + /** + * Given a FSImage FileSummary.section, return a LimitInput stream set to + * the starting position of the section and limited to the section length. + * @param section The FileSummary.Section containing the offset and length + * @param compressionCodec The compression codec in use, if any + * @return An InputStream for the given section + * @throws IOException + */ + private InputStream getInputStreamForSection(FileSummary.Section section, + String compressionCodec, Configuration conf) + throws IOException { + // channel of RandomAccessFile is not thread safe, use File + FileInputStream fin = new FileInputStream(filename); + try { + FileChannel channel = fin.getChannel(); + channel.position(section.getOffset()); + InputStream in = new BufferedInputStream(new LimitInputStream(fin, + section.getLength())); + + in = FSImageUtil.wrapInputStreamForCompression(conf, + compressionCodec, in); + return in; + } catch (IOException e) { + fin.close(); + throw e; + } + } + + /** + * @param srcPaths Source files of contents to be merged + * @param resultPath Merged file path + * @throws IOException + */ + public static void mergeFiles(String[] srcPaths, String resultPath) + throws IOException { + if (srcPaths == null || srcPaths.length < 1) { + LOG.warn("no source files to merge."); + return; + } + + File[] files = new File[srcPaths.length]; + for (int i = 0; i < srcPaths.length; i++) { + files[i] = new File(srcPaths[i]); + } + + File resultFile = new File(resultPath); + try (FileChannel resultChannel = + new FileOutputStream(resultFile, true).getChannel()) { + for (File file : files) { + try (FileChannel src = new FileInputStream(file).getChannel()) { + resultChannel.transferFrom(src, resultChannel.size(), src.size()); + } + } + } + + for (File file : files) { + if (!file.delete() && file.exists()) { + LOG.warn("delete tmp file: {} returned false", file); + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java index 53031bcad90e3..9878469c89e30 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java @@ -83,8 +83,10 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto; import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion; +import org.apache.hadoop.hdfs.util.MD5FileUtils; import org.apache.hadoop.hdfs.web.WebHdfsFileSystem; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.ErasureCodeConstants; import org.apache.hadoop.net.NetUtils; @@ -122,6 +124,7 @@ import static org.apache.hadoop.fs.permission.FsAction.ALL; import static org.apache.hadoop.fs.permission.FsAction.EXECUTE; import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE; +import static org.apache.hadoop.hdfs.MiniDFSCluster.HDFS_MINIDFS_BASEDIR; import static org.apache.hadoop.hdfs.server.namenode.AclTestHelpers.aclEntry; import static org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter.ERASURE_CODING_SECTION_NAME; import static org.apache.hadoop.hdfs.tools.offlineImageViewer.PBImageXmlWriter.ERASURE_CODING_SECTION_POLICY; @@ -186,6 +189,12 @@ public static void createOriginalFSImage() throws IOException { conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true); conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL, "RULE:[2:$1@$0](JobTracker@.*FOO.COM)s/@.*//" + "DEFAULT"); + // fsimage with sub-section conf + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY, "true"); + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_INODE_THRESHOLD_KEY, "1"); + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_TARGET_SECTIONS_KEY, "4"); + conf.set(DFSConfigKeys.DFS_IMAGE_PARALLEL_THREADS_KEY, "4"); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); cluster.waitActive(); DistributedFileSystem hdfs = cluster.getFileSystem(); @@ -791,6 +800,13 @@ public void testPBDelimitedWriter() throws IOException, InterruptedException { new FileSystemTestHelper().getTestRootDir() + "/delimited.db"); } + @Test + public void testParallelPBDelimitedWriter() throws Exception { + testParallelPBDelimitedWriter(""); // Test in memory db. + testParallelPBDelimitedWriter(new FileSystemTestHelper().getTestRootDir() + + "/parallel-delimited.db"); + } + @Test public void testCorruptionOutputEntryBuilder() throws IOException { PBImageCorruptionDetector corrDetector = @@ -882,11 +898,10 @@ private void testPBDelimitedWriter(String db) final String DELIMITER = "\t"; ByteArrayOutputStream output = new ByteArrayOutputStream(); - try (PrintStream o = new PrintStream(output); - RandomAccessFile r = new RandomAccessFile(originalFsimage, "r")) { + try (PrintStream o = new PrintStream(output)) { PBImageDelimitedTextWriter v = new PBImageDelimitedTextWriter(o, DELIMITER, db); - v.visit(r); + v.visit(originalFsimage.getAbsolutePath()); } Set fileNames = new HashSet<>(); @@ -920,6 +935,37 @@ private void testPBDelimitedWriter(String db) assertEquals(writtenFiles.keySet(), fileNames); } + private void testParallelPBDelimitedWriter(String db) throws Exception{ + String delimiter = "\t"; + int numThreads = 4; + + File parallelDelimitedOut = new File(tempDir, "parallelDelimitedOut"); + if (OfflineImageViewerPB.run(new String[] {"-p", "Delimited", + "-i", originalFsimage.getAbsolutePath(), + "-o", parallelDelimitedOut.getAbsolutePath(), + "-delimiter", delimiter, + "-t", db, + "-m", String.valueOf(numThreads)}) != 0) { + throw new IOException("oiv returned failure outputting in parallel."); + } + MD5Hash parallelMd5 = MD5FileUtils.computeMd5ForFile(parallelDelimitedOut); + + File serialDelimitedOut = new File(tempDir, "serialDelimitedOut"); + if (db != "") { + db = db + "/../serial.db"; + } + if (OfflineImageViewerPB.run(new String[] {"-p", "Delimited", + "-i", originalFsimage.getAbsolutePath(), + "-o", serialDelimitedOut.getAbsolutePath(), + "-t", db, + "-delimiter", delimiter}) != 0) { + throw new IOException("oiv returned failure outputting in serial."); + } + MD5Hash serialMd5 = MD5FileUtils.computeMd5ForFile(serialDelimitedOut); + + assertEquals(parallelMd5, serialMd5); + } + private void testPBCorruptionDetector(String db) throws IOException, InterruptedException { final String delimiter = "\t"; @@ -928,7 +974,7 @@ private void testPBCorruptionDetector(String db) try (PrintStream o = new PrintStream(output)) { PBImageCorruptionDetector v = new PBImageCorruptionDetector(o, delimiter, db); - v.visit(new RandomAccessFile(originalFsimage, "r")); + v.visit(originalFsimage.getAbsolutePath()); } try ( @@ -1024,7 +1070,7 @@ private String testCorruptionDetectorRun(int runNumber, try (PrintStream o = new PrintStream(output)) { PBImageCorruptionDetector v = new PBImageCorruptionDetector(o, ",", db); - v.visit(new RandomAccessFile(corruptedImage, "r")); + v.visit(corruptedImage.getAbsolutePath()); } return output.toString(); } @@ -1212,6 +1258,9 @@ public void testReverseXmlWithoutSnapshotDiffSection() throws Throwable { public void testFileDistributionCalculatorForException() throws Exception { File fsimageFile = null; Configuration conf = new Configuration(); + // Avoid using the same cluster dir to cause the global originalFsimage + // file to be cleared. + conf.set(HDFS_MINIDFS_BASEDIR, GenericTestUtils.getRandomizedTempPath()); HashMap files = Maps.newHashMap(); // Create a initial fsimage file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java index 4955846432a60..b23ddf4afbcfb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java @@ -239,7 +239,7 @@ public void testPBDelimitedWriterForAcl() throws Exception { try (PrintStream o = new PrintStream(output)) { PBImageDelimitedTextWriter v = new PBImageDelimitedTextWriter(o, DELIMITER, ""); // run in memory. - v.visit(new RandomAccessFile(originalFsimage, "r")); + v.visit(originalFsimage.getAbsolutePath()); } try ( From 45ce1cce50c3ff65676d946e96bbc7846ad3131a Mon Sep 17 00:00:00 2001 From: qinyuren <1476659627@qq.com> Date: Wed, 23 Mar 2022 14:03:17 +0800 Subject: [PATCH 058/145] HDFS-16501. Print the exception when reporting a bad block (#4062) Reviewed-by: tomscut --- .../org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java index 5ab5b49e1a8aa..d8f1e23ec379b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java @@ -293,7 +293,7 @@ public void handle(ExtendedBlock block, IOException e) { volume, block); return; } - LOG.warn("Reporting bad {} on {}", block, volume); + LOG.warn("Reporting bad {} on {}", block, volume, e); scanner.datanode.handleBadBlock(block, e, true); } } From 921267ca3103763b4237195029cdcb06e8c2232d Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Wed, 23 Mar 2022 08:54:42 +0100 Subject: [PATCH 059/145] YARN-11084. Introduce new config to specify AM default node-label when not specified. Contributed by Junfan Zhang. --- .../hadoop/yarn/conf/YarnConfiguration.java | 3 ++ .../server/resourcemanager/RMAppManager.java | 15 +++++++-- .../resourcemanager/TestAppManager.java | 31 +++++++++++++++++++ .../src/site/markdown/NodeLabel.md | 8 +++++ 4 files changed, 54 insertions(+), 3 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index e61b5416e22a4..df01a12f0a395 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -4409,6 +4409,9 @@ public static boolean areNodeLabelsEnabled( private static final String RM_NODE_LABELS_PREFIX = RM_PREFIX + "node-labels."; + public static final String AM_DEFAULT_NODE_LABEL = + RM_NODE_LABELS_PREFIX + "am.default-node-label-expression"; + public static final String RM_NODE_LABELS_PROVIDER_CONFIG = RM_NODE_LABELS_PREFIX + "provider"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index 928eeb19160f1..5bedef4baa387 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -83,6 +83,9 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.SettableFuture; import org.apache.hadoop.yarn.util.StringHelper; +import static org.apache.commons.lang.StringUtils.isEmpty; +import static org.apache.commons.lang.StringUtils.isNotEmpty; + /** * This class manages the list of applications for the resource manager. */ @@ -106,6 +109,7 @@ public class RMAppManager implements EventHandler, private boolean timelineServiceV2Enabled; private boolean nodeLabelsEnabled; private Set exclusiveEnforcedPartitions; + private String amDefaultNodeLabel; private static final String USER_ID_PREFIX = "userid="; @@ -134,6 +138,8 @@ public RMAppManager(RMContext context, .areNodeLabelsEnabled(rmContext.getYarnConfiguration()); this.exclusiveEnforcedPartitions = YarnConfiguration .getExclusiveEnforcedPartitions(rmContext.getYarnConfiguration()); + this.amDefaultNodeLabel = conf + .get(YarnConfiguration.AM_DEFAULT_NODE_LABEL, null); } /** @@ -622,9 +628,12 @@ private List validateAndCreateResourceRequest( } // set label expression for AM ANY request if not set - if (null == anyReq.getNodeLabelExpression()) { - anyReq.setNodeLabelExpression(submissionContext - .getNodeLabelExpression()); + if (isEmpty(anyReq.getNodeLabelExpression())) { + if (isNotEmpty(amDefaultNodeLabel)) { + anyReq.setNodeLabelExpression(amDefaultNodeLabel); + } else { + anyReq.setNodeLabelExpression(submissionContext.getNodeLabelExpression()); + } } // Put ANY request at the front diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index 8c73a0d5bc15d..f9ecc85b0a567 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -729,6 +729,33 @@ public void testRMAppSubmitAMContainerResourceRequest() throws Exception { app.getAMResourceRequests()); } + @Test + public void testRMAppSubmitAMContainerWithNoLabelByRMDefaultAMNodeLabel() throws Exception { + List reqs = new ArrayList<>(); + ResourceRequest anyReq = ResourceRequest.newInstance( + Priority.newInstance(1), + ResourceRequest.ANY, Resources.createResource(1024), 1, false, null, + ExecutionTypeRequest.newInstance(ExecutionType.GUARANTEED)); + reqs.add(anyReq); + asContext.setAMContainerResourceRequests(cloneResourceRequests(reqs)); + asContext.setNodeLabelExpression("fixed"); + + Configuration conf = new Configuration(false); + String defaultAMNodeLabel = "core"; + conf.set(YarnConfiguration.AM_DEFAULT_NODE_LABEL, defaultAMNodeLabel); + + when(mockDefaultQueueInfo.getAccessibleNodeLabels()).thenReturn( + new HashSet() {{ add("core"); }}); + + TestRMAppManager newAppMonitor = createAppManager(rmContext, conf); + newAppMonitor.submitApplication(asContext, "test"); + + RMApp app = rmContext.getRMApps().get(appId); + waitUntilEventProcessed(); + Assert.assertEquals(defaultAMNodeLabel, + app.getAMResourceRequests().get(0).getNodeLabelExpression()); + } + @Test public void testRMAppSubmitResource() throws Exception { asContext.setResource(Resources.createResource(1024)); @@ -836,6 +863,10 @@ public void testRMAppSubmitAMContainerResourceRequestsTwoManyAny() private RMApp testRMAppSubmit() throws Exception { appMonitor.submitApplication(asContext, "test"); + return waitUntilEventProcessed(); + } + + private RMApp waitUntilEventProcessed() throws InterruptedException { RMApp app = rmContext.getRMApps().get(appId); Assert.assertNotNull("app is null", app); Assert.assertEquals("app id doesn't match", appId, app.getApplicationId()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md index 32545f3ac42d4..9c39e9950249e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/NodeLabel.md @@ -170,6 +170,14 @@ Applications can use following Java APIs to specify node label to request * `ResourceRequest.setNodeLabelExpression(..)` to set node label expression for individual resource requests. This can overwrite node label expression set in ApplicationSubmissionContext * Specify `setAMContainerResourceRequest.setNodeLabelExpression` in `ApplicationSubmissionContext` to indicate expected node label for application master container. +__Default AM node-label Configuration__ + +Property | Value +----- | ------ +yarn.resourcemanager.node-labels.am.default-node-label-expression | Overwrites default-node-label-expression only for the ApplicationMaster container. It is disabled by default. + + + Monitoring ---------- From 9edfe30a60f4c43405301927a8f4014d1dd3842d Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Wed, 23 Mar 2022 20:00:50 +0000 Subject: [PATCH 060/145] HADOOP-14661. Add S3 requester pays bucket support to S3A (#3962) Adds the option fs.s3a.requester.pays.enabled, which, if set to true, allows the client to access S3 buckets where the requester is billed for the IO. Contributed by Daniel Carl Jones --- .../org/apache/hadoop/fs/s3a/Constants.java | 9 ++ .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 8 ++ .../apache/hadoop/fs/s3a/S3AFileSystem.java | 1 + .../apache/hadoop/fs/s3a/S3ClientFactory.java | 4 +- .../fs/s3a/impl/RequestFactoryImpl.java | 22 ---- .../site/markdown/tools/hadoop-aws/index.md | 19 +++ .../site/markdown/tools/hadoop-aws/testing.md | 25 ++++ .../tools/hadoop-aws/troubleshooting_s3a.md | 14 +++ .../hadoop/fs/s3a/ITestS3ARequesterPays.java | 113 ++++++++++++++++++ .../hadoop/fs/s3a/S3ATestConstants.java | 11 ++ 10 files changed, 202 insertions(+), 24 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index dd7e425880962..cb3d72e566854 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -157,6 +157,15 @@ private Constants() { "fs.s3a.connection.ssl.enabled"; public static final boolean DEFAULT_SECURE_CONNECTIONS = true; + /** + * Configuration option for S3 Requester Pays feature: {@value}. + */ + public static final String ALLOW_REQUESTER_PAYS = "fs.s3a.requester.pays.enabled"; + /** + * Default configuration for {@value ALLOW_REQUESTER_PAYS}: {@value}. + */ + public static final boolean DEFAULT_ALLOW_REQUESTER_PAYS = false; + // use OpenSSL or JSEE for secure connections public static final String SSL_CHANNEL_MODE = "fs.s3a.ssl.channel.mode"; public static final DelegatingSSLSocketFactory.SSLChannelMode diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index c14558adf54b3..c374ef7397c97 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -54,6 +54,7 @@ import org.apache.hadoop.fs.s3a.statistics.impl.AwsStatisticsCollector; import org.apache.hadoop.fs.store.LogExactlyOnce; +import static com.amazonaws.services.s3.Headers.REQUESTER_PAYS_HEADER; import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION; import static org.apache.hadoop.fs.s3a.Constants.AWS_S3_CENTRAL_REGION; import static org.apache.hadoop.fs.s3a.Constants.EXPERIMENTAL_AWS_INTERNAL_THROTTLING; @@ -75,6 +76,8 @@ public class DefaultS3ClientFactory extends Configured private static final String S3_SERVICE_NAME = "s3"; + private static final String REQUESTER_PAYS_HEADER_VALUE = "requester"; + /** * Subclasses refer to this. */ @@ -118,6 +121,11 @@ public AmazonS3 createS3Client( parameters.getHeaders().forEach((h, v) -> awsConf.addHeader(h, v)); + if (parameters.isRequesterPays()) { + // All calls must acknowledge requester will pay via header. + awsConf.addHeader(REQUESTER_PAYS_HEADER, REQUESTER_PAYS_HEADER_VALUE); + } + // When EXPERIMENTAL_AWS_INTERNAL_THROTTLING is false // throttling is explicitly disabled on the S3 client so that // all failures are collected in S3A instrumentation, and its diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 4b450c4dcce5c..83c3a74f5b35f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -841,6 +841,7 @@ private void bindAWSClient(URI name, boolean dtEnabled) throws IOException { .withMetrics(statisticsContext.newStatisticsFromAwsSdk()) .withPathStyleAccess(conf.getBoolean(PATH_STYLE_ACCESS, false)) .withUserAgentSuffix(uaSuffix) + .withRequesterPays(conf.getBoolean(ALLOW_REQUESTER_PAYS, DEFAULT_ALLOW_REQUESTER_PAYS)) .withRequestHandlers(auditManager.createRequestHandlers()); s3 = ReflectionUtils.newInstance(s3ClientFactoryClass, conf) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java index 5ef99ed6f5c3c..34674c788901f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ClientFactory.java @@ -101,7 +101,7 @@ final class S3ClientCreationParameters { private boolean pathStyleAccess; /** - * This is in the settings awaiting wiring up and testing. + * Permit requests to requester pays buckets. */ private boolean requesterPays; @@ -168,7 +168,7 @@ public S3ClientCreationParameters withMetrics( } /** - * Requester pays option. Not yet wired up. + * Set requester pays option. * @param value new value * @return the builder */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index fa58323decd03..a73e7199380fc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -106,13 +106,6 @@ public class RequestFactoryImpl implements RequestFactory { */ private final long multipartPartCountLimit; - /** - * Requester Pays. - * This is to be wired up in a PR with its - * own tests and docs. - */ - private final boolean requesterPays; - /** * Callback to prepare requests. */ @@ -133,7 +126,6 @@ protected RequestFactoryImpl( this.cannedACL = builder.cannedACL; this.encryptionSecrets = builder.encryptionSecrets; this.multipartPartCountLimit = builder.multipartPartCountLimit; - this.requesterPays = builder.requesterPays; this.requestPreparer = builder.requestPreparer; this.contentEncoding = builder.contentEncoding; } @@ -615,9 +607,6 @@ public static final class RequestFactoryBuilder { */ private CannedAccessControlList cannedACL = null; - /** Requester Pays flag. */ - private boolean requesterPays = false; - /** Content Encoding. */ private String contentEncoding; @@ -684,17 +673,6 @@ public RequestFactoryBuilder withCannedACL( return this; } - /** - * Requester Pays flag. - * @param value new value - * @return the builder - */ - public RequestFactoryBuilder withRequesterPays( - final boolean value) { - requesterPays = value; - return this; - } - /** * Multipart limit. * @param value new value diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index aa4e13ff3dfe7..df08a969e9551 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1633,6 +1633,25 @@ Before using Access Points make sure you're not impacted by the following: considering endpoints, if you have any custom signers that use the host endpoint property make sure to update them if needed; +## Requester Pays buckets + +S3A supports buckets with +[Requester Pays](https://docs.aws.amazon.com/AmazonS3/latest/userguide/RequesterPaysBuckets.html) +enabled. When a bucket is configured with requester pays, the requester must cover +the per-request cost. + +For requests to be successful, the S3 client must acknowledge that they will pay +for these requests by setting a request flag, usually a header, on each request. + +To enable this feature within S3A, configure the `fs.s3a.requester.pays.enabled` property. + +```xml + + fs.s3a.requester.pays.enabled + true + +``` + ## How S3A writes data to S3 The original S3A client implemented file writes by diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 559687a3fdb3f..2641b870d2ea5 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -593,6 +593,31 @@ your `core-site.xml` file, so that trying to use S3 select fails fast with a meaningful error ("S3 Select not supported") rather than a generic Bad Request exception. +### Testing Requester Pays + +By default, the requester pays tests will look for a bucket that exists on Amazon S3 +in us-east-1. + +If the endpoint does support requester pays, you can specify an alternative object. +The test only requires an object of at least a few bytes in order +to check that lists and basic reads work. + +```xml + + test.fs.s3a.requester.pays.file + s3a://my-req-pays-enabled-bucket/on-another-endpoint.json + +``` + +If the endpoint does not support requester pays, you can also disable the tests by configuring +the test URI as a single space. + +```xml + + test.fs.s3a.requester.pays.file + + +``` ### Testing Session Credentials diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index 3019b8525dbfb..96e6e287deaaf 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -547,6 +547,20 @@ When trying to write or read SEE-KMS-encrypted data, the client gets a The caller does not have the permissions to access the key with which the data was encrypted. +### `AccessDeniedException` when using a "Requester Pays" enabled bucket + +When making cross-account requests to a requester pays enabled bucket, all calls must acknowledge via a header that the requester will be billed. + +If you don't enable this acknowledgement within S3A, then you will see a message similar to this: + +``` +java.nio.file.AccessDeniedException: s3a://my-bucket/my-object: getFileStatus on s3a://my-bucket/my-object: +com.amazonaws.services.s3.model.AmazonS3Exception: Forbidden (Service: Amazon S3; Status Code: 403; +Error Code: 403 Forbidden; Request ID: myshortreqid; S3 Extended Request ID: mylongreqid):403 Forbidden +``` + +To enable requester pays, set `fs.s3a.requester.pays.enabled` property to `true`. + ### "Unable to find a region via the region provider chain." when using session credentials. Region must be provided when requesting session credentials, or an exception will be thrown with the message: diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java new file mode 100644 index 0000000000000..c2e7684cad6da --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -0,0 +1,113 @@ +/* + * 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.hadoop.fs.s3a; + +import java.nio.file.AccessDeniedException; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticAssertions; +import org.apache.hadoop.fs.statistics.StreamStatisticNames; + +import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS; +import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Tests for Requester Pays feature. + */ +public class ITestS3ARequesterPays extends AbstractS3ATestBase { + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + ALLOW_REQUESTER_PAYS, + S3A_BUCKET_PROBE); + return conf; + } + + @Test + public void testRequesterPaysOptionSuccess() throws Throwable { + describe("Test requester pays enabled case by reading last then first byte"); + + Configuration conf = this.createConfiguration(); + conf.setBoolean(ALLOW_REQUESTER_PAYS, true); + // Enable bucket exists check, the first failure point people may encounter + conf.setInt(S3A_BUCKET_PROBE, 2); + + Path requesterPaysPath = getRequesterPaysPath(conf); + + try ( + FileSystem fs = requesterPaysPath.getFileSystem(conf); + FSDataInputStream inputStream = fs.open(requesterPaysPath); + ) { + long fileLength = fs.getFileStatus(requesterPaysPath).getLen(); + + inputStream.seek(fileLength - 1); + inputStream.readByte(); + + // Jump back to the start, triggering a new GetObject request. + inputStream.seek(0); + inputStream.readByte(); + + // Verify > 1 call was made, so we're sure it is correctly configured for each request + IOStatisticAssertions + .assertThatStatisticCounter(inputStream.getIOStatistics(), + StreamStatisticNames.STREAM_READ_OPENED) + .isGreaterThan(1); + + // Check list calls work without error + fs.listFiles(requesterPaysPath.getParent(), false); + } + } + + @Test + public void testRequesterPaysDisabledFails() throws Throwable { + describe("Verify expected failure for requester pays buckets when client has it disabled"); + + Configuration conf = this.createConfiguration(); + conf.setBoolean(ALLOW_REQUESTER_PAYS, false); + Path requesterPaysPath = getRequesterPaysPath(conf); + + try (FileSystem fs = requesterPaysPath.getFileSystem(conf)) { + intercept( + AccessDeniedException.class, + "403 Forbidden", + "Expected requester pays bucket to fail without header set", + () -> fs.open(requesterPaysPath).close() + ); + } + } + + private Path getRequesterPaysPath(Configuration conf) { + String requesterPaysFile = + conf.getTrimmed(KEY_REQUESTER_PAYS_FILE, DEFAULT_REQUESTER_PAYS_FILE); + S3ATestUtils.assume( + "Empty test property: " + KEY_REQUESTER_PAYS_FILE, + !requesterPaysFile.isEmpty() + ); + return new Path(requesterPaysFile); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index aca622a9e20b3..742c22ac5a51e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -97,6 +97,17 @@ public interface S3ATestConstants { */ String DEFAULT_CSVTEST_FILE = LANDSAT_BUCKET + "scene_list.gz"; + /** + * Configuration key for an existing object in a requester pays bucket: {@value}. + * If not set, defaults to {@value DEFAULT_REQUESTER_PAYS_FILE}. + */ + String KEY_REQUESTER_PAYS_FILE = TEST_FS_S3A + "requester.pays.file"; + + /** + * Default path for an S3 object inside a requester pays enabled bucket: {@value}. + */ + String DEFAULT_REQUESTER_PAYS_FILE = "s3a://usgs-landsat/collection02/catalog.json"; + /** * Name of the property to define the timeout for scale tests: {@value}. * Measured in seconds. From 077c6c62d6c1ed89e209449a5f9c5849b05e7dff Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 24 Mar 2022 06:16:13 +0100 Subject: [PATCH 061/145] YARN-10547. Decouple job parsing logic from SLSRunner. Contributed by Szilard Nemeth. --- .../apache/hadoop/yarn/sls/AMDefinition.java | 105 +++++ .../hadoop/yarn/sls/AMDefinitionFactory.java | 133 +++++++ .../hadoop/yarn/sls/AMDefinitionRumen.java | 167 ++++++++ .../hadoop/yarn/sls/AMDefinitionSLS.java | 186 +++++++++ .../hadoop/yarn/sls/AMDefinitionSynth.java | 146 +++++++ .../apache/hadoop/yarn/sls/JobDefinition.java | 87 ++++ .../org/apache/hadoop/yarn/sls/SLSRunner.java | 376 +++--------------- .../yarn/sls/TaskContainerDefinition.java | 248 ++++++++++++ .../yarn/sls/appmaster/AMSimulator.java | 39 +- .../yarn/sls/appmaster/DAGAMSimulator.java | 20 +- .../yarn/sls/appmaster/MRAMSimulator.java | 17 +- .../yarn/sls/appmaster/StreamAMSimulator.java | 23 +- .../sls/scheduler/ContainerSimulator.java | 37 +- .../hadoop/yarn/sls/utils/SLSUtils.java | 2 - .../hadoop/yarn/sls/TestDagAMSimulator.java | 16 +- .../yarn/sls/appmaster/TestAMSimulator.java | 95 ++++- 16 files changed, 1272 insertions(+), 425 deletions(-) create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinition.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionRumen.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSynth.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/JobDefinition.java create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinition.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinition.java new file mode 100644 index 0000000000000..1f9e351c4295f --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinition.java @@ -0,0 +1,105 @@ +/* + * 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.hadoop.yarn.sls; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; + +import java.util.List; + +public abstract class AMDefinition { + protected int jobCount; + protected String amType; + protected String user; + protected String queue; + protected long jobStartTime; + protected long jobFinishTime; + protected List taskContainers; + protected Resource amResource; + protected String labelExpression; + protected String oldAppId; + + public AMDefinition(AmDefinitionBuilder builder) { + this.jobStartTime = builder.jobStartTime; + this.jobFinishTime = builder.jobFinishTime; + this.amType = builder.amType; + this.taskContainers = builder.taskContainers; + this.labelExpression = builder.labelExpression; + this.user = builder.user; + this.amResource = builder.amResource; + this.queue = builder.queue; + this.jobCount = builder.jobCount; + this.oldAppId = builder.jobId; + } + + public String getAmType() { + return amType; + } + + public String getUser() { + return user; + } + + public String getOldAppId() { + return oldAppId; + } + + public long getJobStartTime() { + return jobStartTime; + } + + public long getJobFinishTime() { + return jobFinishTime; + } + + public List getTaskContainers() { + return taskContainers; + } + + public Resource getAmResource() { + return amResource; + } + + public String getLabelExpression() { + return labelExpression; + } + + public String getQueue() { + return queue; + } + + public int getJobCount() { + return jobCount; + } + + + public abstract static class AmDefinitionBuilder { + private static final String DEFAULT_USER = "default"; + + protected int jobCount = 1; + protected String amType = AMDefinitionFactory.DEFAULT_JOB_TYPE; + protected String user = DEFAULT_USER; + protected String queue; + protected String jobId; + protected long jobStartTime; + protected long jobFinishTime; + protected List taskContainers; + protected Resource amResource; + protected String labelExpression = null; + + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java new file mode 100644 index 0000000000000..2bbe7bb1ad159 --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java @@ -0,0 +1,133 @@ +/* + * 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.hadoop.yarn.sls; + +import java.util.Map; + +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.tools.rumen.LoggedJob; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; +import org.apache.hadoop.yarn.sls.synthetic.SynthJob; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public final class AMDefinitionFactory { + private static final Logger LOG = LoggerFactory.getLogger( + AMDefinitionFactory.class); + public final static String DEFAULT_JOB_TYPE = "mapreduce"; + + private AMDefinitionFactory() {} + + public static AMDefinitionSLS createFromSlsTrace(Map jsonJob, + SLSRunner slsRunner) throws YarnException { + AMDefinitionSLS amDefinition = AMDefinitionSLS.Builder.create(jsonJob) + .withAmType(SLSConfiguration.AM_TYPE) + .withAmResource(getAMContainerResourceSLS(jsonJob, slsRunner)) + .withTaskContainers( + AMDefinitionSLS.getTaskContainers(jsonJob, slsRunner)) + .withJobStartTime(SLSConfiguration.JOB_START_MS) + .withJobFinishTime(SLSConfiguration.JOB_END_MS) + .withLabelExpression(SLSConfiguration.JOB_LABEL_EXPR) + .withUser(SLSConfiguration.JOB_USER) + .withQueue(SLSConfiguration.JOB_QUEUE_NAME) + .withJobId(SLSConfiguration.JOB_ID) + .withJobCount(SLSConfiguration.JOB_COUNT) + .build(); + slsRunner.increaseQueueAppNum(amDefinition.getQueue()); + return amDefinition; + } + + public static AMDefinitionRumen createFromRumenTrace(LoggedJob job, + long baselineTimeMs, SLSRunner slsRunner) throws YarnException { + AMDefinitionRumen amDefinition = AMDefinitionRumen.Builder.create() + .withAmType(DEFAULT_JOB_TYPE) + .withAmResource(getAMContainerResourceSynthAndRumen(slsRunner)) + .withTaskContainers( + AMDefinitionRumen.getTaskContainers(job, slsRunner)) + .withJobStartTime(job.getSubmitTime()) + .withJobFinishTime(job.getFinishTime()) + .withBaseLineTimeMs(baselineTimeMs) + .withUser(job.getUser()) + .withQueue(job.getQueue().getValue()) + .withJobId(job.getJobID().toString()) + .build(); + slsRunner.increaseQueueAppNum(amDefinition.getQueue()); + return amDefinition; + } + + public static AMDefinitionSynth createFromSynth(SynthJob job, + SLSRunner slsRunner) throws YarnException { + AMDefinitionSynth amDefinition = + AMDefinitionSynth.Builder.create() + .withAmType(job.getType()) + .withAmResource(getAMContainerResourceSynthAndRumen(slsRunner)) + .withTaskContainers( + AMDefinitionSynth.getTaskContainers(job, slsRunner)) + .withUser(job.getUser()) + .withQueue(job.getQueueName()) + .withJobId(job.getJobID().toString()) + .withJobStartTime(job.getSubmissionTime()) + .withJobFinishTime(job.getSubmissionTime() + job.getDuration()) + .withBaseLineTimeMs(0) + .build(); + + slsRunner.increaseQueueAppNum(amDefinition.getQueue()); + return amDefinition; + } + + private static Resource getAMContainerResourceSLS(Map jsonJob, + Configured configured) { + Resource amContainerResource = + SLSConfiguration.getAMContainerResource(configured.getConf()); + if (jsonJob == null) { + return amContainerResource; + } + + ResourceInformation[] infors = ResourceUtils.getResourceTypesArray(); + for (ResourceInformation info : infors) { + String key = SLSConfiguration.JOB_AM_PREFIX + info.getName(); + if (jsonJob.containsKey(key)) { + long value = Long.parseLong(jsonJob.get(key).toString()); + amContainerResource.setResourceValue(info.getName(), value); + } + } + + return amContainerResource; + } + + private static Resource getAMContainerResourceSynthAndRumen( + Configured configured) { + return SLSConfiguration.getAMContainerResource(configured.getConf()); + } + + static void adjustTimeValuesToBaselineTime(AMDefinition amDef, + AMDefinition.AmDefinitionBuilder builder, long baselineTimeMs) { + builder.jobStartTime -= baselineTimeMs; + builder.jobFinishTime -= baselineTimeMs; + if (builder.jobStartTime < 0) { + LOG.warn("Warning: reset job {} start time to 0.", amDef.getOldAppId()); + builder.jobFinishTime = builder.jobFinishTime - builder.jobStartTime; + builder.jobStartTime = 0; + } + amDef.jobStartTime = builder.jobStartTime; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionRumen.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionRumen.java new file mode 100644 index 0000000000000..cc97a90ec00b7 --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionRumen.java @@ -0,0 +1,167 @@ +/* + * 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.hadoop.yarn.sls; + +import org.apache.hadoop.tools.rumen.LoggedJob; +import org.apache.hadoop.tools.rumen.LoggedTask; +import org.apache.hadoop.tools.rumen.LoggedTaskAttempt; +import org.apache.hadoop.tools.rumen.datatypes.UserName; +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.hadoop.yarn.sls.AMDefinitionFactory.adjustTimeValuesToBaselineTime; + +public class AMDefinitionRumen extends AMDefinition { + public final static int DEFAULT_MAPPER_PRIORITY = 20; + private final static int DEFAULT_REDUCER_PRIORITY = 10; + + public AMDefinitionRumen(AmDefinitionBuilder builder) { + super(builder); + } + + public static List getTaskContainers(LoggedJob job, + SLSRunner slsRunner) throws YarnException { + List containerList = new ArrayList<>(); + + TaskContainerDefinition.Builder builder = + TaskContainerDefinition.Builder.create() + .withCount(1) + .withResource(slsRunner.getDefaultContainerResource()) + .withExecutionType(ExecutionType.GUARANTEED) + .withAllocationId(-1) + .withRequestDelay(0); + + // mapper + for (LoggedTask mapTask : job.getMapTasks()) { + if (mapTask.getAttempts().size() == 0) { + throw new YarnException("Invalid map task, no attempt for a mapper!"); + } + LoggedTaskAttempt taskAttempt = + mapTask.getAttempts().get(mapTask.getAttempts().size() - 1); + TaskContainerDefinition containerDef = builder + .withHostname(taskAttempt.getHostName().getValue()) + .withDuration(taskAttempt.getFinishTime() - + taskAttempt.getStartTime()) + .withPriority(DEFAULT_MAPPER_PRIORITY) + .withType("map") + .build(); + containerList.add( + ContainerSimulator.createFromTaskContainerDefinition(containerDef)); + } + + // reducer + for (LoggedTask reduceTask : job.getReduceTasks()) { + if (reduceTask.getAttempts().size() == 0) { + throw new YarnException( + "Invalid reduce task, no attempt for a reducer!"); + } + LoggedTaskAttempt taskAttempt = + reduceTask.getAttempts().get(reduceTask.getAttempts().size() - 1); + TaskContainerDefinition containerDef = builder + .withHostname(taskAttempt.getHostName().getValue()) + .withDuration(taskAttempt.getFinishTime() - + taskAttempt.getStartTime()) + .withPriority(DEFAULT_REDUCER_PRIORITY) + .withType("reduce") + .build(); + containerList.add( + ContainerSimulator.createFromTaskContainerDefinition(containerDef)); + } + + return containerList; + } + + + public static final class Builder extends AmDefinitionBuilder { + private long baselineTimeMs; + + private Builder() { + } + + public static Builder create() { + return new Builder(); + } + + public Builder withAmType(String amType) { + this.amType = amType; + return this; + } + + public Builder withUser(UserName user) { + if (user != null) { + this.user = user.getValue(); + } + return this; + } + + public Builder withQueue(String queue) { + this.queue = queue; + return this; + } + + public Builder withJobId(String oldJobId) { + this.jobId = oldJobId; + return this; + } + + public Builder withJobStartTime(long time) { + this.jobStartTime = time; + return this; + } + + public Builder withJobFinishTime(long time) { + this.jobFinishTime = time; + return this; + } + + public Builder withBaseLineTimeMs(long baselineTimeMs) { + this.baselineTimeMs = baselineTimeMs; + return this; + } + + public Builder withLabelExpression(String expr) { + this.labelExpression = expr; + return this; + } + + public AMDefinitionRumen.Builder withTaskContainers( + List taskContainers) { + this.taskContainers = taskContainers; + return this; + } + + public AMDefinitionRumen.Builder withAmResource(Resource amResource) { + this.amResource = amResource; + return this; + } + + public AMDefinitionRumen build() { + AMDefinitionRumen amDef = new AMDefinitionRumen(this); + + if (baselineTimeMs == 0) { + baselineTimeMs = jobStartTime; + } + adjustTimeValuesToBaselineTime(amDef, this, baselineTimeMs); + return amDef; + } + + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java new file mode 100644 index 0000000000000..7439ddf8ceef7 --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java @@ -0,0 +1,186 @@ +/* + * 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.hadoop.yarn.sls; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; +import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class AMDefinitionSLS extends AMDefinition { + public AMDefinitionSLS(AmDefinitionBuilder builder) { + super(builder); + } + + public String getQueue() { + return queue; + } + + public static List getTaskContainers(Map jsonJob, + SLSRunner slsRunner) throws YarnException { + List> tasks = (List) jsonJob.get(SLSConfiguration.JOB_TASKS); + if (tasks == null || tasks.size() == 0) { + throw new YarnException("No task for the job!"); + } + + List containers = new ArrayList<>(); + for (Map jsonTask : tasks) { + TaskContainerDefinition containerDef = + TaskContainerDefinition.Builder.create() + .withCount(jsonTask, SLSConfiguration.COUNT) + .withHostname((String) jsonTask.get(SLSConfiguration.TASK_HOST)) + .withDuration(jsonTask, SLSConfiguration.TASK_DURATION_MS) + .withDurationLegacy(jsonTask, SLSConfiguration.DURATION_MS) + .withTaskStart(jsonTask, SLSConfiguration.TASK_START_MS) + .withTaskFinish(jsonTask, SLSConfiguration.TASK_END_MS) + .withResource(getResourceForContainer(jsonTask, slsRunner)) + .withPriority(jsonTask, SLSConfiguration.TASK_PRIORITY) + .withType(jsonTask, SLSConfiguration.TASK_TYPE) + .withExecutionType(jsonTask, SLSConfiguration.TASK_EXECUTION_TYPE) + .withAllocationId(jsonTask, SLSConfiguration.TASK_ALLOCATION_ID) + .withRequestDelay(jsonTask, SLSConfiguration.TASK_REQUEST_DELAY) + .build(); + + for (int i = 0; i < containerDef.getCount(); i++) { + containers.add(ContainerSimulator. + createFromTaskContainerDefinition(containerDef)); + } + } + return containers; + } + + private static Resource getResourceForContainer(Map jsonTask, + SLSRunner slsRunner) { + Resource res = slsRunner.getDefaultContainerResource(); + ResourceInformation[] infors = ResourceUtils.getResourceTypesArray(); + for (ResourceInformation info : infors) { + if (jsonTask.containsKey(SLSConfiguration.TASK_PREFIX + info.getName())) { + long value = Long.parseLong( + jsonTask.get(SLSConfiguration.TASK_PREFIX + info.getName()) + .toString()); + res.setResourceValue(info.getName(), value); + } + } + return res; + } + + public static final class Builder extends AmDefinitionBuilder { + private final Map jsonJob; + + private Builder(Map jsonJob) { + this.jsonJob = jsonJob; + } + + public static Builder create(Map jsonJob) { + return new Builder(jsonJob); + } + + public Builder withAmType(String key) { + if (jsonJob.containsKey(key)) { + String amType = (String) jsonJob.get(key); + if (amType != null) { + this.amType = amType; + } + } + return this; + } + + public Builder withUser(String key) { + if (jsonJob.containsKey(key)) { + String user = (String) jsonJob.get(key); + if (user != null) { + this.user = user; + } + } + return this; + } + + public Builder withQueue(String key) { + if (jsonJob.containsKey(key)) { + this.queue = jsonJob.get(key).toString(); + } + return this; + } + + public Builder withJobId(String key) { + if (jsonJob.containsKey(key)) { + this.jobId = (String) jsonJob.get(key); + } + return this; + } + + public Builder withJobCount(String key) { + if (jsonJob.containsKey(key)) { + jobCount = Integer.parseInt(jsonJob.get(key).toString()); + jobCount = Math.max(jobCount, 1); + } + return this; + } + + public Builder withJobStartTime(String key) { + if (jsonJob.containsKey(key)) { + this.jobStartTime = Long.parseLong(jsonJob.get(key).toString()); + } + return this; + } + + public Builder withJobFinishTime(String key) { + if (jsonJob.containsKey(key)) { + this.jobFinishTime = Long.parseLong(jsonJob.get(key).toString()); + } + return this; + } + + public Builder withLabelExpression(String key) { + if (jsonJob.containsKey(key)) { + this.labelExpression = jsonJob.get(key).toString(); + } + return this; + } + + public AMDefinitionSLS.Builder withTaskContainers( + List taskContainers) { + this.taskContainers = taskContainers; + return this; + } + + public AMDefinitionSLS.Builder withAmResource(Resource amResource) { + this.amResource = amResource; + return this; + } + + public AMDefinitionSLS build() { + AMDefinitionSLS amDef = new AMDefinitionSLS(this); + // Job id is generated automatically if this job configuration allows + // multiple job instances + if (jobCount > 1) { + amDef.oldAppId = null; + } else { + amDef.oldAppId = jobId; + } + amDef.jobCount = jobCount; + return amDef; + } + } + +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSynth.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSynth.java new file mode 100644 index 0000000000000..db736f06f7212 --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSynth.java @@ -0,0 +1,146 @@ +/* + * 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.hadoop.yarn.sls; + +import static org.apache.hadoop.yarn.sls.AMDefinitionFactory.adjustTimeValuesToBaselineTime; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; +import org.apache.hadoop.yarn.sls.synthetic.SynthJob; + +public class AMDefinitionSynth extends AMDefinition { + public AMDefinitionSynth(AmDefinitionBuilder builder) { + super(builder); + } + + public static List getTaskContainers( + SynthJob job, SLSRunner slsRunner) throws YarnException { + List containerList = new ArrayList<>(); + ArrayList keyAsArray = new ArrayList<>( + slsRunner.getNmMap().keySet()); + Random rand = new Random(slsRunner.getStjp().getSeed()); + + for (SynthJob.SynthTask task : job.getTasks()) { + RMNode node = getRandomNode(slsRunner, keyAsArray, rand); + TaskContainerDefinition containerDef = + TaskContainerDefinition.Builder.create() + .withCount(1) + .withHostname("/" + node.getRackName() + "/" + node.getHostName()) + .withDuration(task.getTime()) + .withResource(Resource + .newInstance((int) task.getMemory(), (int) task.getVcores())) + .withPriority(task.getPriority()) + .withType(task.getType()) + .withExecutionType(task.getExecutionType()) + .withAllocationId(-1) + .withRequestDelay(0) + .build(); + containerList.add( + ContainerSimulator.createFromTaskContainerDefinition(containerDef)); + } + + return containerList; + } + + private static RMNode getRandomNode(SLSRunner slsRunner, + ArrayList keyAsArray, Random rand) { + int randomIndex = rand.nextInt(keyAsArray.size()); + return slsRunner.getNmMap().get(keyAsArray.get(randomIndex)).getNode(); + } + + public static final class Builder extends AmDefinitionBuilder { + private long baselineTimeMs; + + private Builder() { + } + + public static Builder create() { + return new Builder(); + } + + public Builder withAmType(String amType) { + this.amType = amType; + return this; + } + + public Builder withUser(String user) { + if (user != null) { + this.user = user; + } + return this; + } + + public Builder withQueue(String queue) { + this.queue = queue; + return this; + } + + public Builder withJobId(String oldJobId) { + this.jobId = oldJobId; + return this; + } + + public Builder withJobStartTime(long time) { + this.jobStartTime = time; + return this; + } + + public Builder withJobFinishTime(long time) { + this.jobFinishTime = time; + return this; + } + + public Builder withBaseLineTimeMs(long baselineTimeMs) { + this.baselineTimeMs = baselineTimeMs; + return this; + } + + public AMDefinitionSynth.Builder withLabelExpression(String expr) { + this.labelExpression = expr; + return this; + } + + public AMDefinitionSynth.Builder withTaskContainers( + List taskContainers) { + this.taskContainers = taskContainers; + return this; + } + + public AMDefinitionSynth.Builder withAmResource(Resource amResource) { + this.amResource = amResource; + return this; + } + + public AMDefinitionSynth build() { + AMDefinitionSynth amDef = new AMDefinitionSynth(this); + + if (baselineTimeMs == 0) { + baselineTimeMs = jobStartTime; + } + adjustTimeValuesToBaselineTime(amDef, this, baselineTimeMs); + return amDef; + } + } + +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/JobDefinition.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/JobDefinition.java new file mode 100644 index 0000000000000..4a39d3710c92a --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/JobDefinition.java @@ -0,0 +1,87 @@ +/* + * 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.hadoop.yarn.sls; + +import org.apache.hadoop.yarn.api.records.ReservationId; +import java.util.Map; + +public class JobDefinition { + private AMDefinition amDefinition; + private ReservationId reservationId; + private long deadline; + private Map params; + + public AMDefinition getAmDefinition() { + return amDefinition; + } + + public ReservationId getReservationId() { + return reservationId; + } + + public long getDeadline() { + return deadline; + } + + //Currently unused + public Map getParams() { + return params; + } + + public static final class Builder { + private AMDefinition amDefinition; + private ReservationId reservationId; + private long deadline; + private Map params; + + private Builder() { + } + + public static Builder create() { + return new Builder(); + } + + public Builder withAmDefinition(AMDefinition amDefinition) { + this.amDefinition = amDefinition; + return this; + } + + public Builder withReservationId(ReservationId reservationId) { + this.reservationId = reservationId; + return this; + } + + public Builder withDeadline(long deadline) { + this.deadline = deadline; + return this; + } + + public Builder withParams(Map params) { + this.params = params; + return this; + } + + public JobDefinition build() { + JobDefinition jobDef = new JobDefinition(); + jobDef.params = this.params; + jobDef.amDefinition = this.amDefinition; + jobDef.reservationId = this.reservationId; + jobDef.deadline = this.deadline; + return jobDef; + } + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index a8d2aa6584bcb..83834e8f9c989 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -23,12 +23,10 @@ import java.io.InputStreamReader; import java.io.Reader; import java.security.Security; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Random; import java.util.Set; @@ -59,13 +57,10 @@ import org.apache.hadoop.net.TableMapping; import org.apache.hadoop.tools.rumen.JobTraceReader; import org.apache.hadoop.tools.rumen.LoggedJob; -import org.apache.hadoop.tools.rumen.LoggedTask; -import org.apache.hadoop.tools.rumen.LoggedTaskAttempt; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.api.records.NodeState; @@ -89,7 +84,6 @@ import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics; import org.apache.hadoop.yarn.sls.scheduler.TaskRunner; import org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler; -import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper; import org.apache.hadoop.yarn.sls.synthetic.SynthJob; import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer; @@ -138,13 +132,8 @@ public class SLSRunner extends Configured implements Tool { // logger public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class); - private final static int DEFAULT_MAPPER_PRIORITY = 20; - private final static int DEFAULT_REDUCER_PRIORITY = 10; - private static boolean exitAtTheFinish = false; - private static final String DEFAULT_USER = "default"; - /** * The type of trace in input. */ @@ -472,7 +461,10 @@ private void startAMFromSLSTrace(String inputTrace) throws IOException { while (jobIter.hasNext()) { try { - createAMForJob(jobIter.next()); + Map jsonJob = jobIter.next(); + AMDefinitionSLS amDef = AMDefinitionFactory.createFromSlsTrace( + jsonJob, this); + startAMs(amDef); } catch (Exception e) { LOG.error("Failed to create an AM: {}", e.getMessage()); } @@ -480,150 +472,29 @@ private void startAMFromSLSTrace(String inputTrace) throws IOException { } } - private void createAMForJob(Map jsonJob) throws YarnException { - long jobStartTime = Long.parseLong( - jsonJob.get(SLSConfiguration.JOB_START_MS).toString()); - - long jobFinishTime = 0; - if (jsonJob.containsKey(SLSConfiguration.JOB_END_MS)) { - jobFinishTime = Long.parseLong( - jsonJob.get(SLSConfiguration.JOB_END_MS).toString()); - } - - String jobLabelExpr = null; - if (jsonJob.containsKey(SLSConfiguration.JOB_LABEL_EXPR)) { - jobLabelExpr = jsonJob.get(SLSConfiguration.JOB_LABEL_EXPR).toString(); - } - - String user = (String) jsonJob.get(SLSConfiguration.JOB_USER); - if (user == null) { - user = "default"; - } - - String queue = jsonJob.get(SLSConfiguration.JOB_QUEUE_NAME).toString(); - increaseQueueAppNum(queue); - - String amType = (String)jsonJob.get(SLSConfiguration.AM_TYPE); - if (amType == null) { - amType = SLSUtils.DEFAULT_JOB_TYPE; - } - - int jobCount = 1; - if (jsonJob.containsKey(SLSConfiguration.JOB_COUNT)) { - jobCount = Integer.parseInt( - jsonJob.get(SLSConfiguration.JOB_COUNT).toString()); - } - jobCount = Math.max(jobCount, 1); - - String oldAppId = (String)jsonJob.get(SLSConfiguration.JOB_ID); - // Job id is generated automatically if this job configuration allows - // multiple job instances - if(jobCount > 1) { - oldAppId = null; - } - - for (int i = 0; i < jobCount; i++) { - runNewAM(amType, user, queue, oldAppId, jobStartTime, jobFinishTime, - getTaskContainers(jsonJob), getAMContainerResource(jsonJob), - jobLabelExpr); - } - } - - private List getTaskContainers(Map jsonJob) - throws YarnException { - List containers = new ArrayList<>(); - List tasks = (List) jsonJob.get(SLSConfiguration.JOB_TASKS); - if (tasks == null || tasks.size() == 0) { - throw new YarnException("No task for the job!"); - } - - for (Object o : tasks) { - Map jsonTask = (Map) o; - - String hostname = (String) jsonTask.get(SLSConfiguration.TASK_HOST); - - long duration = 0; - if (jsonTask.containsKey(SLSConfiguration.TASK_DURATION_MS)) { - duration = Integer.parseInt( - jsonTask.get(SLSConfiguration.TASK_DURATION_MS).toString()); - } else if (jsonTask.containsKey(SLSConfiguration.DURATION_MS)) { - // Also support "duration.ms" for backward compatibility - duration = Integer.parseInt( - jsonTask.get(SLSConfiguration.DURATION_MS).toString()); - } else if (jsonTask.containsKey(SLSConfiguration.TASK_START_MS) && - jsonTask.containsKey(SLSConfiguration.TASK_END_MS)) { - long taskStart = Long.parseLong( - jsonTask.get(SLSConfiguration.TASK_START_MS).toString()); - long taskFinish = Long.parseLong( - jsonTask.get(SLSConfiguration.TASK_END_MS).toString()); - duration = taskFinish - taskStart; - } - if (duration <= 0) { - throw new YarnException("Duration of a task shouldn't be less or equal" - + " to 0!"); - } - - Resource res = getResourceForContainer(jsonTask); - - int priority = DEFAULT_MAPPER_PRIORITY; - if (jsonTask.containsKey(SLSConfiguration.TASK_PRIORITY)) { - priority = Integer.parseInt( - jsonTask.get(SLSConfiguration.TASK_PRIORITY).toString()); - } - - String type = "map"; - if (jsonTask.containsKey(SLSConfiguration.TASK_TYPE)) { - type = jsonTask.get(SLSConfiguration.TASK_TYPE).toString(); - } - - int count = 1; - if (jsonTask.containsKey(SLSConfiguration.COUNT)) { - count = Integer.parseInt( - jsonTask.get(SLSConfiguration.COUNT).toString()); - } - count = Math.max(count, 1); - - ExecutionType executionType = ExecutionType.GUARANTEED; - if (jsonTask.containsKey(SLSConfiguration.TASK_EXECUTION_TYPE)) { - executionType = ExecutionType.valueOf( - jsonTask.get(SLSConfiguration.TASK_EXECUTION_TYPE).toString()); - } - long allocationId = -1; - if (jsonTask.containsKey(SLSConfiguration.TASK_ALLOCATION_ID)) { - allocationId = Long.parseLong( - jsonTask.get(SLSConfiguration.TASK_ALLOCATION_ID).toString()); - } - - long requestDelay = 0; - if (jsonTask.containsKey(SLSConfiguration.TASK_REQUEST_DELAY)) { - requestDelay = Long.parseLong( - jsonTask.get(SLSConfiguration.TASK_REQUEST_DELAY).toString()); - } - requestDelay = Math.max(requestDelay, 0); - - for (int i = 0; i < count; i++) { - containers.add( - new ContainerSimulator(res, duration, hostname, priority, type, - executionType, allocationId, requestDelay)); - } + private void startAMs(AMDefinition amDef) { + for (int i = 0; i < amDef.getJobCount(); i++) { + JobDefinition jobDef = JobDefinition.Builder.create() + .withAmDefinition(amDef) + .withDeadline(-1) + .withReservationId(null) + .withParams(null) + .build(); + runNewAM(jobDef); } - - return containers; } - private Resource getResourceForContainer(Map jsonTask) { - Resource res = getDefaultContainerResource(); - ResourceInformation[] infors = ResourceUtils.getResourceTypesArray(); - for (ResourceInformation info : infors) { - if (jsonTask.containsKey(SLSConfiguration.TASK_PREFIX + info.getName())) { - long value = Long.parseLong( - jsonTask.get(SLSConfiguration.TASK_PREFIX + info.getName()) - .toString()); - res.setResourceValue(info.getName(), value); - } + private void startAMs(AMDefinition amDef, ReservationId reservationId, + Map params, long deadline) { + for (int i = 0; i < amDef.getJobCount(); i++) { + JobDefinition jobDef = JobDefinition.Builder.create() + .withAmDefinition(amDef) + .withReservationId(reservationId) + .withParams(params) + .withDeadline(deadline) + .build(); + runNewAM(jobDef); } - - return res; } /** @@ -642,76 +513,19 @@ private void startAMFromRumenTrace(String inputTrace, long baselineTimeMS) while (job != null) { try { - createAMForJob(job, baselineTimeMS); + AMDefinitionRumen amDef = + AMDefinitionFactory.createFromRumenTrace(job, baselineTimeMS, + this); + startAMs(amDef); } catch (Exception e) { LOG.error("Failed to create an AM", e); } - job = reader.getNext(); } } } - private void createAMForJob(LoggedJob job, long baselineTimeMs) - throws YarnException { - String user = job.getUser() == null ? "default" : - job.getUser().getValue(); - String jobQueue = job.getQueue().getValue(); - String oldJobId = job.getJobID().toString(); - long jobStartTimeMS = job.getSubmitTime(); - long jobFinishTimeMS = job.getFinishTime(); - if (baselineTimeMs == 0) { - baselineTimeMs = job.getSubmitTime(); - } - jobStartTimeMS -= baselineTimeMs; - jobFinishTimeMS -= baselineTimeMs; - if (jobStartTimeMS < 0) { - LOG.warn("Warning: reset job {} start time to 0.", oldJobId); - jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS; - jobStartTimeMS = 0; - } - - increaseQueueAppNum(jobQueue); - - List containerList = new ArrayList<>(); - // mapper - for (LoggedTask mapTask : job.getMapTasks()) { - if (mapTask.getAttempts().size() == 0) { - throw new YarnException("Invalid map task, no attempt for a mapper!"); - } - LoggedTaskAttempt taskAttempt = - mapTask.getAttempts().get(mapTask.getAttempts().size() - 1); - String hostname = taskAttempt.getHostName().getValue(); - long containerLifeTime = taskAttempt.getFinishTime() - - taskAttempt.getStartTime(); - containerList.add( - new ContainerSimulator(getDefaultContainerResource(), - containerLifeTime, hostname, DEFAULT_MAPPER_PRIORITY, "map")); - } - - // reducer - for (LoggedTask reduceTask : job.getReduceTasks()) { - if (reduceTask.getAttempts().size() == 0) { - throw new YarnException( - "Invalid reduce task, no attempt for a reducer!"); - } - LoggedTaskAttempt taskAttempt = - reduceTask.getAttempts().get(reduceTask.getAttempts().size() - 1); - String hostname = taskAttempt.getHostName().getValue(); - long containerLifeTime = taskAttempt.getFinishTime() - - taskAttempt.getStartTime(); - containerList.add( - new ContainerSimulator(getDefaultContainerResource(), - containerLifeTime, hostname, DEFAULT_REDUCER_PRIORITY, "reduce")); - } - - // Only supports the default job type currently - runNewAM(SLSUtils.DEFAULT_JOB_TYPE, user, jobQueue, oldJobId, - jobStartTimeMS, jobFinishTimeMS, containerList, - getAMContainerResource(null)); - } - - private Resource getDefaultContainerResource() { + Resource getDefaultContainerResource() { int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB, SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT); int containerVCores = getConf().getInt(SLSConfiguration.CONTAINER_VCORES, @@ -726,94 +540,26 @@ private Resource getDefaultContainerResource() { private void startAMFromSynthGenerator() throws YarnException, IOException { Configuration localConf = new Configuration(); localConf.set("fs.defaultFS", "file:///"); - long baselineTimeMS = 0; - // if we use the nodeFile this could have been not initialized yet. if (stjp == null) { stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0])); } - SynthJob job = null; + SynthJob job; // we use stjp, a reference to the job producer instantiated during node // creation while ((job = (SynthJob) stjp.getNextJob()) != null) { - // only support MapReduce currently - String user = job.getUser() == null ? DEFAULT_USER : - job.getUser(); - String jobQueue = job.getQueueName(); - String oldJobId = job.getJobID().toString(); - long jobStartTimeMS = job.getSubmissionTime(); - - // CARLO: Finish time is only used for logging, omit for now - long jobFinishTimeMS = jobStartTimeMS + job.getDuration(); - - if (baselineTimeMS == 0) { - baselineTimeMS = jobStartTimeMS; - } - jobStartTimeMS -= baselineTimeMS; - jobFinishTimeMS -= baselineTimeMS; - if (jobStartTimeMS < 0) { - LOG.warn("Warning: reset job {} start time to 0.", oldJobId); - jobFinishTimeMS = jobFinishTimeMS - jobStartTimeMS; - jobStartTimeMS = 0; - } - - increaseQueueAppNum(jobQueue); - - List containerList = - new ArrayList(); - ArrayList keyAsArray = new ArrayList(nmMap.keySet()); - Random rand = new Random(stjp.getSeed()); - - for (SynthJob.SynthTask task : job.getTasks()) { - RMNode node = nmMap.get(keyAsArray.get(rand.nextInt(keyAsArray.size()))) - .getNode(); - String hostname = "/" + node.getRackName() + "/" + node.getHostName(); - long containerLifeTime = task.getTime(); - Resource containerResource = Resource - .newInstance((int) task.getMemory(), (int) task.getVcores()); - containerList.add( - new ContainerSimulator(containerResource, containerLifeTime, - hostname, task.getPriority(), task.getType(), - task.getExecutionType())); - } - - ReservationId reservationId = null; - - if(job.hasDeadline()){ + if (job.hasDeadline()) { reservationId = ReservationId - .newInstance(this.rm.getStartTime(), AM_ID); - } - - runNewAM(job.getType(), user, jobQueue, oldJobId, - jobStartTimeMS, jobFinishTimeMS, containerList, reservationId, - job.getDeadline(), getAMContainerResource(null), null, - job.getParams()); - } - } - - private Resource getAMContainerResource(Map jsonJob) { - Resource amContainerResource = - SLSConfiguration.getAMContainerResource(getConf()); - - if (jsonJob == null) { - return amContainerResource; - } - - ResourceInformation[] infors = ResourceUtils.getResourceTypesArray(); - for (ResourceInformation info : infors) { - String key = SLSConfiguration.JOB_AM_PREFIX + info.getName(); - if (jsonJob.containsKey(key)) { - long value = Long.parseLong(jsonJob.get(key).toString()); - amContainerResource.setResourceValue(info.getName(), value); + .newInstance(rm.getStartTime(), AM_ID); } + AMDefinitionSynth amDef = AMDefinitionFactory.createFromSynth(job, this); + startAMs(amDef, reservationId, job.getParams(), job.getDeadline()); } - - return amContainerResource; } - private void increaseQueueAppNum(String queue) throws YarnException { + void increaseQueueAppNum(String queue) throws YarnException { SchedulerWrapper wrapper = (SchedulerWrapper)rm.getResourceScheduler(); String queueName = wrapper.getRealQueueName(queue); Integer appNum = queueAppNumMap.get(queueName); @@ -830,32 +576,16 @@ private void increaseQueueAppNum(String queue) throws YarnException { } } - private void runNewAM(String jobType, String user, - String jobQueue, String oldJobId, long jobStartTimeMS, - long jobFinishTimeMS, List containerList, - Resource amContainerResource) { - runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS, - jobFinishTimeMS, containerList, null, -1, - amContainerResource, null, null); - } - - private void runNewAM(String jobType, String user, - String jobQueue, String oldJobId, long jobStartTimeMS, - long jobFinishTimeMS, List containerList, - Resource amContainerResource, String labelExpr) { - runNewAM(jobType, user, jobQueue, oldJobId, jobStartTimeMS, - jobFinishTimeMS, containerList, null, -1, - amContainerResource, labelExpr, null); + private AMSimulator createAmSimulator(String jobType) { + return (AMSimulator) ReflectionUtils.newInstance( + amClassMap.get(jobType), new Configuration()); } - @SuppressWarnings("checkstyle:parameternumber") - private void runNewAM(String jobType, String user, - String jobQueue, String oldJobId, long jobStartTimeMS, - long jobFinishTimeMS, List containerList, - ReservationId reservationId, long deadline, Resource amContainerResource, - String labelExpr, Map params) { - AMSimulator amSim = (AMSimulator) ReflectionUtils.newInstance( - amClassMap.get(jobType), new Configuration()); + private void runNewAM(JobDefinition jobDef) { + AMDefinition amDef = jobDef.getAmDefinition(); + String oldJobId = amDef.getOldAppId(); + AMSimulator amSim = + createAmSimulator(amDef.getAmType()); if (amSim != null) { int heartbeatInterval = getConf().getInt( @@ -867,19 +597,17 @@ private void runNewAM(String jobType, String user, oldJobId = Integer.toString(AM_ID); } AM_ID++; - amSim.init(heartbeatInterval, containerList, rm, this, jobStartTimeMS, - jobFinishTimeMS, user, jobQueue, isTracked, oldJobId, - runner.getStartTimeMS(), amContainerResource, labelExpr, params, - appIdAMSim); - if(reservationId != null) { + amSim.init(amDef, rm, this, isTracked, runner.getStartTimeMS(), heartbeatInterval, appIdAMSim); + if (jobDef.getReservationId() != null) { // if we have a ReservationId, delegate reservation creation to // AMSim (reservation shape is impl specific) UTCClock clock = new UTCClock(); - amSim.initReservation(reservationId, deadline, clock.getTime()); + amSim.initReservation(jobDef.getReservationId(), jobDef.getDeadline(), + clock.getTime()); } runner.schedule(amSim); - maxRuntime = Math.max(maxRuntime, jobFinishTimeMS); - numTasks += containerList.size(); + maxRuntime = Math.max(maxRuntime, amDef.getJobFinishTime()); + numTasks += amDef.getTaskContainers().size(); amMap.put(oldJobId, amSim); } } @@ -1121,4 +849,12 @@ public int hashCode() { return result; } } + + public ResourceManager getRm() { + return rm; + } + + public SynthTraceJobProducer getStjp() { + return stjp; + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java new file mode 100644 index 0000000000000..1b0cd9003b63a --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java @@ -0,0 +1,248 @@ +/* + * 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.hadoop.yarn.sls; + +import org.apache.hadoop.yarn.api.records.ExecutionType; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.exceptions.YarnException; +import java.util.Map; + +import static org.apache.hadoop.yarn.sls.AMDefinitionRumen.DEFAULT_MAPPER_PRIORITY; + +public class TaskContainerDefinition { + private long duration; + private Resource resource; + private int priority; + private String type; + private int count; + private ExecutionType executionType; + private long allocationId = -1; + private long requestDelay = 0; + private String hostname; + + public long getDuration() { + return duration; + } + + public Resource getResource() { + return resource; + } + + public int getPriority() { + return priority; + } + + public String getType() { + return type; + } + + public int getCount() { + return count; + } + + public ExecutionType getExecutionType() { + return executionType; + } + + public long getAllocationId() { + return allocationId; + } + + public long getRequestDelay() { + return requestDelay; + } + + public String getHostname() { + return hostname; + } + + public static final class Builder { + private long duration = -1; + private long durationLegacy = -1; + private long taskStart = -1; + private long taskFinish = -1; + private Resource resource; + private int priority = DEFAULT_MAPPER_PRIORITY; + private String type = "map"; + private int count = 1; + private ExecutionType executionType = ExecutionType.GUARANTEED; + private long allocationId = -1; + private long requestDelay = 0; + private String hostname; + + public static Builder create() { + return new Builder(); + } + + public Builder withDuration(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.duration = Integer.parseInt(jsonTask.get(key).toString()); + } + return this; + } + + public Builder withDuration(long duration) { + this.duration = duration; + return this; + } + + /** + * Also support "duration.ms" for backward compatibility. + * @param jsonTask the json representation of the task. + * @param key The json key. + * @return the builder + */ + public Builder withDurationLegacy(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.durationLegacy = Integer.parseInt(jsonTask.get(key).toString()); + } + return this; + } + + public Builder withTaskStart(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.taskStart = Long.parseLong(jsonTask.get(key).toString()); + } + return this; + } + + public Builder withTaskFinish(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.taskFinish = Long.parseLong(jsonTask.get(key).toString()); + } + return this; + } + + public Builder withResource(Resource resource) { + this.resource = resource; + return this; + } + + public Builder withPriority(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.priority = Integer.parseInt(jsonTask.get(key).toString()); + } + return this; + } + + public Builder withPriority(int priority) { + this.priority = priority; + return this; + } + + public Builder withType(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.type = jsonTask.get(key).toString(); + } + return this; + } + + public Builder withType(String type) { + this.type = type; + return this; + } + + public Builder withCount(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + count = Integer.parseInt(jsonTask.get(key).toString()); + count = Math.max(count, 1); + } + return this; + } + + public Builder withCount(int count) { + this.count = count; + return this; + } + + public Builder withExecutionType(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.executionType = ExecutionType.valueOf( + jsonTask.get(key).toString()); + } + return this; + } + + public Builder withExecutionType(ExecutionType executionType) { + this.executionType = executionType; + return this; + } + + public Builder withAllocationId(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + this.allocationId = Long.parseLong(jsonTask.get(key).toString()); + } + return this; + } + + public Builder withAllocationId(long allocationId) { + this.allocationId = allocationId; + return this; + } + + public Builder withRequestDelay(Map jsonTask, String key) { + if (jsonTask.containsKey(key)) { + requestDelay = Long.parseLong(jsonTask.get(key).toString()); + requestDelay = Math.max(requestDelay, 0); + } + return this; + } + + public Builder withRequestDelay(long requestDelay) { + this.requestDelay = requestDelay; + return this; + } + + public Builder withHostname(String hostname) { + this.hostname = hostname; + return this; + } + + public TaskContainerDefinition build() throws YarnException { + TaskContainerDefinition taskContainerDef = + new TaskContainerDefinition(); + taskContainerDef.duration = validateAndGetDuration(this); + taskContainerDef.resource = this.resource; + taskContainerDef.type = this.type; + taskContainerDef.requestDelay = this.requestDelay; + taskContainerDef.priority = this.priority; + taskContainerDef.count = this.count; + taskContainerDef.allocationId = this.allocationId; + taskContainerDef.executionType = this.executionType; + taskContainerDef.hostname = this.hostname; + return taskContainerDef; + } + + private long validateAndGetDuration(Builder builder) throws YarnException { + long duration = 0; + + if (builder.duration != -1) { + duration = builder.duration; + } else if (builder.durationLegacy != -1) { + duration = builder.durationLegacy; + } else if (builder.taskStart != -1 && builder.taskFinish != -1) { + duration = builder.taskFinish - builder.taskStart; + } + + if (duration <= 0) { + throw new YarnException("Duration of a task shouldn't be less or equal" + + " to 0!"); + } + return duration; + } + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java index 922f9a2b97a6c..0a87a6c207058 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/AMSimulator.java @@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.sls.AMDefinition; import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; @@ -128,27 +129,25 @@ public AMSimulator() { this.responseQueue = new LinkedBlockingQueue<>(); } - @SuppressWarnings("checkstyle:parameternumber") - public void init(int heartbeatInterval, - List containerList, ResourceManager resourceManager, - SLSRunner slsRunnner, long startTime, long finishTime, String simUser, - String simQueue, boolean tracked, String oldApp, long baseTimeMS, - Resource amResource, String nodeLabelExpr, Map params, - Map appIdAMSim) { - super.init(startTime, startTime + 1000000L * heartbeatInterval, - heartbeatInterval); - this.user = simUser; - this.rm = resourceManager; - this.se = slsRunnner; - this.queue = simQueue; - this.oldAppId = oldApp; + public void init(AMDefinition amDef, ResourceManager rm, SLSRunner slsRunner, + boolean tracked, long baselineTimeMS, long heartbeatInterval, + Map appIdToAMSim) { + long startTime = amDef.getJobStartTime(); + long endTime = startTime + 1000000L * heartbeatInterval; + super.init(startTime, endTime, heartbeatInterval); + + this.user = amDef.getUser(); + this.queue = amDef.getQueue(); + this.oldAppId = amDef.getOldAppId(); + this.amContainerResource = amDef.getAmResource(); + this.nodeLabelExpression = amDef.getLabelExpression(); + this.traceStartTimeMS = amDef.getJobStartTime(); + this.traceFinishTimeMS = amDef.getJobFinishTime(); + this.rm = rm; + this.se = slsRunner; this.isTracked = tracked; - this.baselineTimeMS = baseTimeMS; - this.traceStartTimeMS = startTime; - this.traceFinishTimeMS = finishTime; - this.amContainerResource = amResource; - this.nodeLabelExpression = nodeLabelExpr; - this.appIdToAMSim = appIdAMSim; + this.baselineTimeMS = baselineTimeMS; + this.appIdToAMSim = appIdToAMSim; } /** diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/DAGAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/DAGAMSimulator.java index 83467e0e5cf46..418408db9fbdc 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/DAGAMSimulator.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/DAGAMSimulator.java @@ -32,10 +32,10 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ReservationId; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.sls.AMDefinition; import org.apache.hadoop.yarn.sls.SLSRunner; import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; import org.slf4j.Logger; @@ -93,19 +93,15 @@ public class DAGAMSimulator extends AMSimulator { LoggerFactory.getLogger(DAGAMSimulator.class); @SuppressWarnings("checkstyle:parameternumber") - public void init(int heartbeatInterval, - List containerList, ResourceManager resourceManager, - SLSRunner slsRunnner, long startTime, long finishTime, String simUser, - String simQueue, boolean tracked, String oldApp, long baseTimeMS, - Resource amResource, String nodeLabelExpr, Map params, - Map appIdAMSim) { - super.init(heartbeatInterval, containerList, resourceManager, slsRunnner, - startTime, finishTime, simUser, simQueue, tracked, oldApp, baseTimeMS, - amResource, nodeLabelExpr, params, appIdAMSim); + public void init(AMDefinition amDef, ResourceManager rm, SLSRunner slsRunner, + boolean tracked, long baselineTimeMS, long heartbeatInterval, + Map appIdToAMSim) { + super.init(amDef, rm, slsRunner, tracked, baselineTimeMS, heartbeatInterval, + appIdToAMSim); super.amtype = "dag"; - allContainers.addAll(containerList); - pendingContainers.addAll(containerList); + allContainers.addAll(amDef.getTaskContainers()); + pendingContainers.addAll(amDef.getTaskContainers()); totalContainers = allContainers.size(); LOG.info("Added new job with {} containers", allContainers.size()); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java index 184fdca2e575c..976c0229b86f3 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/MRAMSimulator.java @@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.sls.AMDefinition; import org.apache.hadoop.yarn.sls.ReservationClientUtil; import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; import org.apache.hadoop.yarn.sls.SLSRunner; @@ -123,19 +124,15 @@ scheduled when all maps have finished (not support slow-start currently). LoggerFactory.getLogger(MRAMSimulator.class); @SuppressWarnings("checkstyle:parameternumber") - public void init(int heartbeatInterval, - List containerList, ResourceManager rm, SLSRunner se, - long traceStartTime, long traceFinishTime, String user, String queue, - boolean isTracked, String oldAppId, long baselineStartTimeMS, - Resource amContainerResource, String nodeLabelExpr, - Map params, Map appIdAMSim) { - super.init(heartbeatInterval, containerList, rm, se, traceStartTime, - traceFinishTime, user, queue, isTracked, oldAppId, baselineStartTimeMS, - amContainerResource, nodeLabelExpr, params, appIdAMSim); + public void init(AMDefinition amDef, ResourceManager rm, SLSRunner slsRunner, + boolean tracked, long baselineTimeMS, long heartbeatInterval, + Map appIdToAMSim) { + super.init(amDef, rm, slsRunner, tracked, baselineTimeMS, + heartbeatInterval, appIdToAMSim); amtype = "mapreduce"; // get map/reduce tasks - for (ContainerSimulator cs : containerList) { + for (ContainerSimulator cs : amDef.getTaskContainers()) { if (cs.getType().equals("map")) { cs.setPriority(PRIORITY_MAP); allMaps.add(cs); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java index 7e3545191f20e..09297afd4d0e8 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/appmaster/StreamAMSimulator.java @@ -30,11 +30,11 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ReservationId; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.sls.AMDefinition; import org.apache.hadoop.yarn.sls.SLSRunner; import org.apache.hadoop.yarn.sls.scheduler.ContainerSimulator; import org.slf4j.Logger; @@ -93,21 +93,14 @@ public class StreamAMSimulator extends AMSimulator { LoggerFactory.getLogger(StreamAMSimulator.class); @SuppressWarnings("checkstyle:parameternumber") - public void init(int heartbeatInterval, - List containerList, ResourceManager rm, SLSRunner se, - long traceStartTime, long traceFinishTime, String user, String queue, - boolean isTracked, String oldAppId, long baselineStartTimeMS, - Resource amContainerResource, String nodeLabelExpr, - Map params, Map appIdAMSim) { - super.init(heartbeatInterval, containerList, rm, se, traceStartTime, - traceFinishTime, user, queue, isTracked, oldAppId, baselineStartTimeMS, - amContainerResource, nodeLabelExpr, params, appIdAMSim); + public void init(AMDefinition amDef, ResourceManager rm, SLSRunner slsRunner, + boolean tracked, long baselineTimeMS, long heartbeatInterval, + Map appIdToAMSim) { + super.init(amDef, rm, slsRunner, tracked, baselineTimeMS, + heartbeatInterval, appIdToAMSim); amtype = "stream"; - - allStreams.addAll(containerList); - - duration = traceFinishTime - traceStartTime; - + allStreams.addAll(amDef.getTaskContainers()); + duration = amDef.getJobFinishTime() - amDef.getJobStartTime(); LOG.info("Added new job with {} streams, running for {}", allStreams.size(), duration); } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ContainerSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ContainerSimulator.java index e83ee91d8e13e..8f119943570e2 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ContainerSimulator.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/ContainerSimulator.java @@ -26,54 +26,41 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.sls.TaskContainerDefinition; @Private @Unstable public class ContainerSimulator implements Delayed { - // id private ContainerId id; - // resource allocated private Resource resource; - // end time private long endTime; // life time (ms) private long lifeTime; // time(ms) after which container would be requested by AM private long requestDelay; - // host name private String hostname; - // priority private int priority; - // type private String type; - // execution type private ExecutionType executionType = ExecutionType.GUARANTEED; - // allocation id private long allocationId; /** - * invoked when AM schedules containers to allocate. + * Invoked when AM schedules containers to allocate. + * @param def The task's definition object. + * @return ContainerSimulator object */ - public ContainerSimulator(Resource resource, long lifeTime, - String hostname, int priority, String type) { - this(resource, lifeTime, hostname, priority, type, - ExecutionType.GUARANTEED); + public static ContainerSimulator createFromTaskContainerDefinition( + TaskContainerDefinition def) { + return new ContainerSimulator(def.getResource(), def.getDuration(), + def.getHostname(), def.getPriority(), def.getType(), + def.getExecutionType(), def.getAllocationId(), def.getRequestDelay()); } /** - * invoked when AM schedules containers to allocate. - */ - public ContainerSimulator(Resource resource, long lifeTime, - String hostname, int priority, String type, ExecutionType executionType) { - this(resource, lifeTime, hostname, priority, type, - executionType, -1, 0); - } - - /** - * invoked when AM schedules containers to allocate. + * Invoked when AM schedules containers to allocate. */ @SuppressWarnings("checkstyle:parameternumber") - public ContainerSimulator(Resource resource, long lifeTime, + private ContainerSimulator(Resource resource, long lifeTime, String hostname, int priority, String type, ExecutionType executionType, long allocationId, long requestDelay) { this.resource = resource; @@ -87,7 +74,7 @@ public ContainerSimulator(Resource resource, long lifeTime, } /** - * invoke when NM schedules containers to run. + * Invoked when NM schedules containers to run. */ public ContainerSimulator(ContainerId id, Resource resource, long endTime, long lifeTime, long allocationId) { diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java index 256dcf4629188..e529d1841a671 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java @@ -57,8 +57,6 @@ @Private @Unstable public class SLSUtils { - public final static String DEFAULT_JOB_TYPE = "mapreduce"; - private static final String LABEL_FORMAT_ERR_MSG = "Input format for adding node-labels is not correct, it should be " + "labelName1[(exclusive=true/false)],labelName2[] .."; diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestDagAMSimulator.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestDagAMSimulator.java index 8ac7fff75cb82..e458b860e4d4e 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestDagAMSimulator.java +++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestDagAMSimulator.java @@ -26,6 +26,8 @@ import java.util.List; import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Tests for DagAMSimulator. @@ -74,7 +76,17 @@ public void testGetToBeScheduledContainers() throws Exception { private ContainerSimulator createContainerSim(long allocationId, long requestDelay) { - return new ContainerSimulator(null, 1000, "*", 1, "Map", - null, allocationId, requestDelay); + TaskContainerDefinition taskContainerDef = + mock(TaskContainerDefinition.class); + when(taskContainerDef.getResource()).thenReturn(null); + when(taskContainerDef.getDuration()).thenReturn(1000L); + when(taskContainerDef.getHostname()).thenReturn("*"); + when(taskContainerDef.getPriority()).thenReturn(1); + when(taskContainerDef.getType()).thenReturn("Map"); + when(taskContainerDef.getExecutionType()).thenReturn(null); + when(taskContainerDef.getAllocationId()).thenReturn(allocationId); + when(taskContainerDef.getRequestDelay()).thenReturn(requestDelay); + return ContainerSimulator.createFromTaskContainerDefinition( + taskContainerDef); } } diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java index 50ac700d9c666..f5db1684c7164 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java +++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/appmaster/TestAMSimulator.java @@ -20,6 +20,7 @@ import com.codahale.metrics.MetricRegistry; import java.util.HashMap; import org.apache.commons.io.FileUtils; +import org.apache.hadoop.tools.rumen.datatypes.UserName; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ExecutionType; import org.apache.hadoop.yarn.api.records.NodeId; @@ -33,6 +34,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; +import org.apache.hadoop.yarn.sls.AMDefinitionRumen; +import org.apache.hadoop.yarn.sls.TaskContainerDefinition; import org.apache.hadoop.yarn.sls.SLSRunner; import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator; @@ -57,6 +60,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentMap; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @RunWith(Parameterized.class) @@ -157,9 +161,20 @@ public void testAMSimulator() throws Exception { String queue = "default"; List containers = new ArrayList<>(); HashMap map = new HashMap<>(); - app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true, - appId, 0, SLSConfiguration.getAMContainerResource(conf), null, null, - map); + + UserName mockUser = mock(UserName.class); + when(mockUser.getValue()).thenReturn("user1"); + AMDefinitionRumen amDef = + AMDefinitionRumen.Builder.create() + .withUser(mockUser) + .withQueue(queue) + .withJobId(appId) + .withJobStartTime(0) + .withJobFinishTime(1000000L) + .withAmResource(SLSConfiguration.getAMContainerResource(conf)) + .withTaskContainers(containers) + .build(); + app.init(amDef, rm, null, true, 0, 1000, map); app.firstStep(); verifySchedulerMetrics(appId); @@ -184,9 +199,21 @@ public void testAMSimulatorWithNodeLabels() throws Exception { String queue = "default"; List containers = new ArrayList<>(); HashMap map = new HashMap<>(); - app.init(1000, containers, rm, null, 0, 1000000L, "user1", queue, true, - appId, 0, SLSConfiguration.getAMContainerResource(conf), "label1", - null, map); + + UserName mockUser = mock(UserName.class); + when(mockUser.getValue()).thenReturn("user1"); + AMDefinitionRumen amDef = + AMDefinitionRumen.Builder.create() + .withUser(mockUser) + .withQueue(queue) + .withJobId(appId) + .withJobStartTime(0) + .withJobFinishTime(1000000L) + .withAmResource(SLSConfiguration.getAMContainerResource(conf)) + .withTaskContainers(containers) + .withLabelExpression("label1") + .build(); + app.init(amDef, rm, null, true, 0, 1000, map); app.firstStep(); verifySchedulerMetrics(appId); @@ -201,7 +228,7 @@ public void testAMSimulatorWithNodeLabels() throws Exception { } @Test - public void testPackageRequests() { + public void testPackageRequests() throws YarnException { MockAMSimulator app = new MockAMSimulator(); List containerSimulators = new ArrayList<>(); Resource resource = Resources.createResource(1024); @@ -209,12 +236,25 @@ public void testPackageRequests() { ExecutionType execType = ExecutionType.GUARANTEED; String type = "map"; - ContainerSimulator s1 = new ContainerSimulator(resource, 100, - "/default-rack/h1", priority, type, execType); - ContainerSimulator s2 = new ContainerSimulator(resource, 100, - "/default-rack/h1", priority, type, execType); - ContainerSimulator s3 = new ContainerSimulator(resource, 100, - "/default-rack/h2", priority, type, execType); + TaskContainerDefinition.Builder builder = + TaskContainerDefinition.Builder.create() + .withResource(resource) + .withDuration(100) + .withPriority(1) + .withType(type) + .withExecutionType(execType) + .withAllocationId(-1) + .withRequestDelay(0); + + ContainerSimulator s1 = ContainerSimulator + .createFromTaskContainerDefinition( + builder.withHostname("/default-rack/h1").build()); + ContainerSimulator s2 = ContainerSimulator + .createFromTaskContainerDefinition( + builder.withHostname("/default-rack/h1").build()); + ContainerSimulator s3 = ContainerSimulator + .createFromTaskContainerDefinition( + builder.withHostname("/default-rack/h2").build()); containerSimulators.add(s1); containerSimulators.add(s2); @@ -250,12 +290,15 @@ public void testPackageRequests() { Assert.assertEquals(2, nodeRequestCount); containerSimulators.clear(); - s1 = new ContainerSimulator(resource, 100, - "/default-rack/h1", priority, type, execType, 1, 0); - s2 = new ContainerSimulator(resource, 100, - "/default-rack/h1", priority, type, execType, 2, 0); - s3 = new ContainerSimulator(resource, 100, - "/default-rack/h2", priority, type, execType, 1, 0); + s1 = ContainerSimulator.createFromTaskContainerDefinition( + createDefaultTaskContainerDefMock(resource, priority, execType, type, + "/default-rack/h1", 1)); + s2 = ContainerSimulator.createFromTaskContainerDefinition( + createDefaultTaskContainerDefMock(resource, priority, execType, type, + "/default-rack/h1", 2)); + s3 = ContainerSimulator.createFromTaskContainerDefinition( + createDefaultTaskContainerDefMock(resource, priority, execType, type, + "/default-rack/h2", 1)); containerSimulators.add(s1); containerSimulators.add(s2); @@ -317,6 +360,20 @@ public void testAMSimulatorRanNodesCleared() throws Exception { Assert.assertFalse(nm.getNode().getRunningApps().contains(app.appId)); Assert.assertTrue(nm.getNode().getRunningApps().isEmpty()); } + private TaskContainerDefinition createDefaultTaskContainerDefMock( + Resource resource, int priority, ExecutionType execType, String type, + String hostname, long allocationId) { + TaskContainerDefinition taskContainerDef = + mock(TaskContainerDefinition.class); + when(taskContainerDef.getResource()).thenReturn(resource); + when(taskContainerDef.getDuration()).thenReturn(100L); + when(taskContainerDef.getPriority()).thenReturn(priority); + when(taskContainerDef.getType()).thenReturn(type); + when(taskContainerDef.getExecutionType()).thenReturn(execType); + when(taskContainerDef.getHostname()).thenReturn(hostname); + when(taskContainerDef.getAllocationId()).thenReturn(allocationId); + return taskContainerDef; + } @After public void tearDown() { From 526142447abdee02b86820d884b577b23b769663 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Thu, 24 Mar 2022 16:24:33 +0100 Subject: [PATCH 062/145] YARN-10552. Eliminate code duplication in SLSCapacityScheduler and SLSFairScheduler. Contributed by Szilard Nemeth. --- .../sls/scheduler/SLSCapacityScheduler.java | 328 ++--------------- .../yarn/sls/scheduler/SLSFairScheduler.java | 291 +-------------- .../sls/scheduler/SLSSchedulerCommons.java | 343 ++++++++++++++++++ 3 files changed, 386 insertions(+), 576 deletions(-) create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java index e0cb15138f451..b6fe5c0f96588 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java @@ -17,12 +17,7 @@ */ package org.apache.hadoop.yarn.sls.scheduler; -import java.io.IOException; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -30,119 +25,51 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.SchedulingRequest; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; -import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; -import org.apache.hadoop.yarn.sls.SLSRunner; -import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; -import org.apache.hadoop.yarn.util.resource.Resources; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.codahale.metrics.Timer; @Private @Unstable public class SLSCapacityScheduler extends CapacityScheduler implements SchedulerWrapper,Configurable { - private Configuration conf; - - private Map appQueueMap = - new ConcurrentHashMap(); - - private Map preemptionContainerMap = - new ConcurrentHashMap(); - - // metrics - private SchedulerMetrics schedulerMetrics; - private boolean metricsON; - private Tracker tracker; - - // logger - private static final Logger LOG = LoggerFactory.getLogger(SLSCapacityScheduler.class); - public Tracker getTracker() { - return tracker; - } + private final SLSSchedulerCommons schedulerCommons; + private Configuration conf; public SLSCapacityScheduler() { - tracker = new Tracker(); + schedulerCommons = new SLSSchedulerCommons(this); } @Override public void setConf(Configuration conf) { this.conf = conf; super.setConf(conf); - metricsON = conf.getBoolean(SLSConfiguration.METRICS_SWITCH, true); - if (metricsON) { - try { - schedulerMetrics = SchedulerMetrics.getInstance(conf, - CapacityScheduler.class); - schedulerMetrics.init(this, conf); - } catch (Exception e) { - LOG.error("Caught exception while initializing schedulerMetrics", e); - } - } + schedulerCommons.initMetrics(CapacityScheduler.class, conf); } @Override public Allocation allocate(ApplicationAttemptId attemptId, List resourceRequests, List schedulingRequests, List containerIds, - List strings, List strings2, ContainerUpdates updateRequests) { - if (metricsON) { - final Timer.Context context = schedulerMetrics.getSchedulerAllocateTimer() - .time(); - Allocation allocation = null; - try { - allocation = super - .allocate(attemptId, resourceRequests, schedulingRequests, - containerIds, strings, - strings2, updateRequests); - return allocation; - } catch (Exception e) { - LOG.error("Caught exception from allocate", e); - throw e; - } finally { - context.stop(); - schedulerMetrics.increaseSchedulerAllocationCounter(); - try { - updateQueueWithAllocateRequest(allocation, attemptId, - resourceRequests, containerIds); - } catch (IOException e) { - LOG.error("Caught exception while executing finally block", e); - } - } - } else { - return super.allocate(attemptId, resourceRequests, schedulingRequests, - containerIds, strings, - strings2, updateRequests); - } + List blacklistAdditions, List blacklistRemovals, + ContainerUpdates updateRequests) { + return schedulerCommons.allocate(attemptId, resourceRequests, schedulingRequests, + containerIds, blacklistAdditions, blacklistRemovals, updateRequests); } @Override public boolean tryCommit(Resource cluster, ResourceCommitRequest r, boolean updatePending) { - if (metricsON) { + if (schedulerCommons.isMetricsON()) { boolean isSuccess = false; long startTimeNs = System.nanoTime(); try { @@ -151,13 +78,13 @@ public boolean tryCommit(Resource cluster, ResourceCommitRequest r, } finally { long elapsedNs = System.nanoTime() - startTimeNs; if (isSuccess) { - schedulerMetrics.getSchedulerCommitSuccessTimer() + getSchedulerMetrics().getSchedulerCommitSuccessTimer() .update(elapsedNs, TimeUnit.NANOSECONDS); - schedulerMetrics.increaseSchedulerCommitSuccessCounter(); + getSchedulerMetrics().increaseSchedulerCommitSuccessCounter(); } else { - schedulerMetrics.getSchedulerCommitFailureTimer() + getSchedulerMetrics().getSchedulerCommitFailureTimer() .update(elapsedNs, TimeUnit.NANOSECONDS); - schedulerMetrics.increaseSchedulerCommitFailureCounter(); + getSchedulerMetrics().increaseSchedulerCommitFailureCounter(); } } } else { @@ -167,222 +94,26 @@ public boolean tryCommit(Resource cluster, ResourceCommitRequest r, @Override public void handle(SchedulerEvent schedulerEvent) { - if (!metricsON) { - super.handle(schedulerEvent); - return; - } - - if (!schedulerMetrics.isRunning()) { - schedulerMetrics.setRunning(true); - } - - Timer.Context handlerTimer = null; - Timer.Context operationTimer = null; - - NodeUpdateSchedulerEventWrapper eventWrapper; - try { - if (schedulerEvent.getType() == SchedulerEventType.NODE_UPDATE - && schedulerEvent instanceof NodeUpdateSchedulerEvent) { - eventWrapper = new NodeUpdateSchedulerEventWrapper( - (NodeUpdateSchedulerEvent)schedulerEvent); - schedulerEvent = eventWrapper; - updateQueueWithNodeUpdate(eventWrapper); - } else if (schedulerEvent.getType() == - SchedulerEventType.APP_ATTEMPT_REMOVED - && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { - // check if having AM Container, update resource usage information - AppAttemptRemovedSchedulerEvent appRemoveEvent = - (AppAttemptRemovedSchedulerEvent) schedulerEvent; - ApplicationAttemptId appAttemptId = - appRemoveEvent.getApplicationAttemptID(); - String queue = appQueueMap.get(appAttemptId); - SchedulerAppReport app = super.getSchedulerAppInfo(appAttemptId); - if (!app.getLiveContainers().isEmpty()) { // have 0 or 1 - // should have one container which is AM container - RMContainer rmc = app.getLiveContainers().iterator().next(); - schedulerMetrics.updateQueueMetricsByRelease( - rmc.getContainer().getResource(), queue); - } - } - - handlerTimer = schedulerMetrics.getSchedulerHandleTimer().time(); - operationTimer = schedulerMetrics.getSchedulerHandleTimer( - schedulerEvent.getType()).time(); - - super.handle(schedulerEvent); - } finally { - if (handlerTimer != null) { - handlerTimer.stop(); - } - if (operationTimer != null) { - operationTimer.stop(); - } - schedulerMetrics.increaseSchedulerHandleCounter(schedulerEvent.getType()); - - if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED - && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { - SLSRunner.decreaseRemainingApps(); - AppAttemptRemovedSchedulerEvent appRemoveEvent = - (AppAttemptRemovedSchedulerEvent) schedulerEvent; - appQueueMap.remove(appRemoveEvent.getApplicationAttemptID()); - if (SLSRunner.getRemainingApps() == 0) { - try { - getSchedulerMetrics().tearDown(); - SLSRunner.exitSLSRunner(); - } catch (Exception e) { - LOG.error("Scheduler Metrics failed to tear down.", e); - } - } - } else if (schedulerEvent.getType() == - SchedulerEventType.APP_ATTEMPT_ADDED - && schedulerEvent instanceof AppAttemptAddedSchedulerEvent) { - AppAttemptAddedSchedulerEvent appAddEvent = - (AppAttemptAddedSchedulerEvent) schedulerEvent; - SchedulerApplication app = - applications.get(appAddEvent.getApplicationAttemptId() - .getApplicationId()); - appQueueMap.put(appAddEvent.getApplicationAttemptId(), app.getQueue() - .getQueueName()); - } - } - } - - private void updateQueueWithNodeUpdate( - NodeUpdateSchedulerEventWrapper eventWrapper) { - RMNodeWrapper node = (RMNodeWrapper) eventWrapper.getRMNode(); - List containerList = node.getContainerUpdates(); - for (UpdatedContainerInfo info : containerList) { - for (ContainerStatus status : info.getCompletedContainers()) { - ContainerId containerId = status.getContainerId(); - SchedulerAppReport app = super.getSchedulerAppInfo( - containerId.getApplicationAttemptId()); - - if (app == null) { - // this happens for the AM container - // The app have already removed when the NM sends the release - // information. - continue; - } - - String queue = appQueueMap.get(containerId.getApplicationAttemptId()); - int releasedMemory = 0, releasedVCores = 0; - if (status.getExitStatus() == ContainerExitStatus.SUCCESS) { - for (RMContainer rmc : app.getLiveContainers()) { - if (rmc.getContainerId() == containerId) { - releasedMemory += rmc.getContainer().getResource().getMemorySize(); - releasedVCores += rmc.getContainer() - .getResource().getVirtualCores(); - break; - } - } - } else if (status.getExitStatus() == ContainerExitStatus.ABORTED) { - if (preemptionContainerMap.containsKey(containerId)) { - Resource preResource = preemptionContainerMap.get(containerId); - releasedMemory += preResource.getMemorySize(); - releasedVCores += preResource.getVirtualCores(); - preemptionContainerMap.remove(containerId); - } - } - // update queue counters - schedulerMetrics.updateQueueMetricsByRelease( - Resource.newInstance(releasedMemory, releasedVCores), queue); - } - } - } - - private void updateQueueWithAllocateRequest(Allocation allocation, - ApplicationAttemptId attemptId, - List resourceRequests, - List containerIds) throws IOException { - // update queue information - Resource pendingResource = Resources.createResource(0, 0); - Resource allocatedResource = Resources.createResource(0, 0); - String queueName = appQueueMap.get(attemptId); - // container requested - for (ResourceRequest request : resourceRequests) { - if (request.getResourceName().equals(ResourceRequest.ANY)) { - Resources.addTo(pendingResource, - Resources.multiply(request.getCapability(), - request.getNumContainers())); - } - } - // container allocated - for (Container container : allocation.getContainers()) { - Resources.addTo(allocatedResource, container.getResource()); - Resources.subtractFrom(pendingResource, container.getResource()); - } - // container released from AM - SchedulerAppReport report = super.getSchedulerAppInfo(attemptId); - for (ContainerId containerId : containerIds) { - Container container = null; - for (RMContainer c : report.getLiveContainers()) { - if (c.getContainerId().equals(containerId)) { - container = c.getContainer(); - break; - } - } - if (container != null) { - // released allocated containers - Resources.subtractFrom(allocatedResource, container.getResource()); - } else { - for (RMContainer c : report.getReservedContainers()) { - if (c.getContainerId().equals(containerId)) { - container = c.getContainer(); - break; - } - } - if (container != null) { - // released reserved containers - Resources.subtractFrom(pendingResource, container.getResource()); - } - } - } - // containers released/preemption from scheduler - Set preemptionContainers = new HashSet(); - if (allocation.getContainerPreemptions() != null) { - preemptionContainers.addAll(allocation.getContainerPreemptions()); - } - if (allocation.getStrictContainerPreemptions() != null) { - preemptionContainers.addAll(allocation.getStrictContainerPreemptions()); - } - if (! preemptionContainers.isEmpty()) { - for (ContainerId containerId : preemptionContainers) { - if (! preemptionContainerMap.containsKey(containerId)) { - Container container = null; - for (RMContainer c : report.getLiveContainers()) { - if (c.getContainerId().equals(containerId)) { - container = c.getContainer(); - break; - } - } - if (container != null) { - preemptionContainerMap.put(containerId, container.getResource()); - } - } - - } - } - - // update metrics - schedulerMetrics.updateQueueMetrics(pendingResource, allocatedResource, - queueName); + schedulerCommons.handle(schedulerEvent); } @Override public void serviceStop() throws Exception { - try { - if (metricsON) { - schedulerMetrics.tearDown(); - } - } catch (Exception e) { - LOG.error("Caught exception while stopping service", e); - } + schedulerCommons.stopMetrics(); super.serviceStop(); } + public String getRealQueueName(String queue) throws YarnException { + if (getQueue(queue) == null) { + throw new YarnException("Can't find the queue by the given name: " + queue + + "! Please check if queue " + queue + " is in the allocation file."); + } + return getQueue(queue).getQueuePath(); + } + public SchedulerMetrics getSchedulerMetrics() { - return schedulerMetrics; + return schedulerCommons.getSchedulerMetrics(); } @Override @@ -390,11 +121,8 @@ public Configuration getConf() { return conf; } - public String getRealQueueName(String queue) throws YarnException { - if (getQueue(queue) == null) { - throw new YarnException("Can't find the queue by the given name: " + queue - + "! Please check if queue " + queue + " is in the allocation file."); - } - return getQueue(queue).getQueuePath(); + public Tracker getTracker() { + return schedulerCommons.getTracker(); } -} \ No newline at end of file +} + diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java index 84549bc492205..b164316486378 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java @@ -17,84 +17,35 @@ */ package org.apache.hadoop.yarn.sls.scheduler; -import com.codahale.metrics.Timer; import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ContainerStatus; -import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.SchedulingRequest; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; -import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; -import org.apache.hadoop.yarn.sls.SLSRunner; -import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; -import org.apache.hadoop.yarn.util.resource.Resources; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; @Private @Unstable public class SLSFairScheduler extends FairScheduler implements SchedulerWrapper, Configurable { - private SchedulerMetrics schedulerMetrics; - private boolean metricsON; - private Tracker tracker; - - private Map preemptionContainerMap = - new ConcurrentHashMap<>(); - - // logger - private static final Logger LOG = - LoggerFactory.getLogger(SLSFairScheduler.class); - - public SchedulerMetrics getSchedulerMetrics() { - return schedulerMetrics; - } - - public Tracker getTracker() { - return tracker; - } + private final SLSSchedulerCommons schedulerCommons; public SLSFairScheduler() { - tracker = new Tracker(); + schedulerCommons = new SLSSchedulerCommons(this); } @Override public void setConf(Configuration conf) { super.setConfig(conf); - - metricsON = conf.getBoolean(SLSConfiguration.METRICS_SWITCH, true); - if (metricsON) { - try { - schedulerMetrics = SchedulerMetrics.getInstance(conf, - FairScheduler.class); - schedulerMetrics.init(this, conf); - } catch (Exception e) { - LOG.error("Caught exception while initializing schedulerMetrics", e); - } - } + schedulerCommons.initMetrics(FairScheduler.class, conf); } @Override @@ -103,237 +54,18 @@ public Allocation allocate(ApplicationAttemptId attemptId, List schedulingRequests, List containerIds, List blacklistAdditions, List blacklistRemovals, ContainerUpdates updateRequests) { - if (metricsON) { - final Timer.Context context = schedulerMetrics.getSchedulerAllocateTimer() - .time(); - Allocation allocation = null; - try { - allocation = super.allocate(attemptId, resourceRequests, - schedulingRequests, containerIds, - blacklistAdditions, blacklistRemovals, updateRequests); - return allocation; - } catch (Exception e) { - LOG.error("Caught exception from allocate", e); - throw e; - } finally { - context.stop(); - schedulerMetrics.increaseSchedulerAllocationCounter(); - try { - updateQueueWithAllocateRequest(allocation, attemptId, - resourceRequests, containerIds); - } catch (IOException e) { - LOG.error("Caught exception while executing finally block", e); - } - } - } else { - return super.allocate(attemptId, resourceRequests, schedulingRequests, - containerIds, - blacklistAdditions, blacklistRemovals, updateRequests); - } + return schedulerCommons.allocate(attemptId, resourceRequests, schedulingRequests, + containerIds, blacklistAdditions, blacklistRemovals, updateRequests); } @Override public void handle(SchedulerEvent schedulerEvent) { - // metrics off - if (!metricsON) { - super.handle(schedulerEvent); - return; - } - - // metrics on - if(!schedulerMetrics.isRunning()) { - schedulerMetrics.setRunning(true); - } - - Timer.Context handlerTimer = null; - Timer.Context operationTimer = null; - - NodeUpdateSchedulerEventWrapper eventWrapper; - try { - if (schedulerEvent.getType() == SchedulerEventType.NODE_UPDATE - && schedulerEvent instanceof NodeUpdateSchedulerEvent) { - eventWrapper = new NodeUpdateSchedulerEventWrapper( - (NodeUpdateSchedulerEvent)schedulerEvent); - schedulerEvent = eventWrapper; - updateQueueWithNodeUpdate(eventWrapper); - } else if ( - schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED - && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { - // check if having AM Container, update resource usage information - AppAttemptRemovedSchedulerEvent appRemoveEvent = - (AppAttemptRemovedSchedulerEvent) schedulerEvent; - ApplicationAttemptId appAttemptId = - appRemoveEvent.getApplicationAttemptID(); - String queueName = getSchedulerApp(appAttemptId).getQueue().getName(); - SchedulerAppReport app = getSchedulerAppInfo(appAttemptId); - if (!app.getLiveContainers().isEmpty()) { // have 0 or 1 - // should have one container which is AM container - RMContainer rmc = app.getLiveContainers().iterator().next(); - schedulerMetrics.updateQueueMetricsByRelease( - rmc.getContainer().getResource(), queueName); - } - } - - handlerTimer = schedulerMetrics.getSchedulerHandleTimer().time(); - operationTimer = schedulerMetrics.getSchedulerHandleTimer( - schedulerEvent.getType()).time(); - - super.handle(schedulerEvent); - } finally { - if (handlerTimer != null) { - handlerTimer.stop(); - } - if (operationTimer != null) { - operationTimer.stop(); - } - schedulerMetrics.increaseSchedulerHandleCounter(schedulerEvent.getType()); - - if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED - && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { - SLSRunner.decreaseRemainingApps(); - if (SLSRunner.getRemainingApps() == 0) { - try { - getSchedulerMetrics().tearDown(); - SLSRunner.exitSLSRunner(); - } catch (Exception e) { - LOG.error("Scheduler Metrics failed to tear down.", e); - } - } - } - } - } - - private void updateQueueWithNodeUpdate( - NodeUpdateSchedulerEventWrapper eventWrapper) { - RMNodeWrapper node = (RMNodeWrapper) eventWrapper.getRMNode(); - List containerList = node.getContainerUpdates(); - for (UpdatedContainerInfo info : containerList) { - for (ContainerStatus status : info.getCompletedContainers()) { - ContainerId containerId = status.getContainerId(); - SchedulerAppReport app = super.getSchedulerAppInfo( - containerId.getApplicationAttemptId()); - - if (app == null) { - // this happens for the AM container - // The app have already removed when the NM sends the release - // information. - continue; - } - - int releasedMemory = 0, releasedVCores = 0; - if (status.getExitStatus() == ContainerExitStatus.SUCCESS) { - for (RMContainer rmc : app.getLiveContainers()) { - if (rmc.getContainerId() == containerId) { - Resource resource = rmc.getContainer().getResource(); - releasedMemory += resource.getMemorySize(); - releasedVCores += resource.getVirtualCores(); - break; - } - } - } else if (status.getExitStatus() == ContainerExitStatus.ABORTED) { - if (preemptionContainerMap.containsKey(containerId)) { - Resource preResource = preemptionContainerMap.get(containerId); - releasedMemory += preResource.getMemorySize(); - releasedVCores += preResource.getVirtualCores(); - preemptionContainerMap.remove(containerId); - } - } - // update queue counters - String queue = getSchedulerApp(containerId.getApplicationAttemptId()). - getQueueName(); - schedulerMetrics.updateQueueMetricsByRelease( - Resource.newInstance(releasedMemory, releasedVCores), queue); - } - } - } - - private void updateQueueWithAllocateRequest(Allocation allocation, - ApplicationAttemptId attemptId, - List resourceRequests, - List containerIds) throws IOException { - // update queue information - Resource pendingResource = Resources.createResource(0, 0); - Resource allocatedResource = Resources.createResource(0, 0); - // container requested - for (ResourceRequest request : resourceRequests) { - if (request.getResourceName().equals(ResourceRequest.ANY)) { - Resources.addTo(pendingResource, - Resources.multiply(request.getCapability(), - request.getNumContainers())); - } - } - // container allocated - for (Container container : allocation.getContainers()) { - Resources.addTo(allocatedResource, container.getResource()); - Resources.subtractFrom(pendingResource, container.getResource()); - } - // container released from AM - SchedulerAppReport report = super.getSchedulerAppInfo(attemptId); - for (ContainerId containerId : containerIds) { - Container container = null; - for (RMContainer c : report.getLiveContainers()) { - if (c.getContainerId().equals(containerId)) { - container = c.getContainer(); - break; - } - } - if (container != null) { - // released allocated containers - Resources.subtractFrom(allocatedResource, container.getResource()); - } else { - for (RMContainer c : report.getReservedContainers()) { - if (c.getContainerId().equals(containerId)) { - container = c.getContainer(); - break; - } - } - if (container != null) { - // released reserved containers - Resources.subtractFrom(pendingResource, container.getResource()); - } - } - } - // containers released/preemption from scheduler - Set preemptionContainers = new HashSet(); - if (allocation.getContainerPreemptions() != null) { - preemptionContainers.addAll(allocation.getContainerPreemptions()); - } - if (allocation.getStrictContainerPreemptions() != null) { - preemptionContainers.addAll(allocation.getStrictContainerPreemptions()); - } - if (!preemptionContainers.isEmpty()) { - for (ContainerId containerId : preemptionContainers) { - if (!preemptionContainerMap.containsKey(containerId)) { - Container container = null; - for (RMContainer c : report.getLiveContainers()) { - if (c.getContainerId().equals(containerId)) { - container = c.getContainer(); - break; - } - } - if (container != null) { - preemptionContainerMap.put(containerId, container.getResource()); - } - } - - } - } - - // update metrics - String queueName = getSchedulerApp(attemptId).getQueueName(); - schedulerMetrics.updateQueueMetrics(pendingResource, allocatedResource, - queueName); + schedulerCommons.handle(schedulerEvent); } @Override public void serviceStop() throws Exception { - try { - if (metricsON) { - schedulerMetrics.tearDown(); - } - } catch (Exception e) { - LOG.error("Caught exception while stopping service", e); - } + schedulerCommons.stopMetrics(); super.serviceStop(); } @@ -344,5 +76,12 @@ public String getRealQueueName(String queue) throws YarnException { } return getQueueManager().getQueue(queue).getQueueName(); } -} + public SchedulerMetrics getSchedulerMetrics() { + return schedulerCommons.getSchedulerMetrics(); + } + + public Tracker getTracker() { + return schedulerCommons.getTracker(); + } +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java new file mode 100644 index 0000000000000..92aa96069c848 --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java @@ -0,0 +1,343 @@ +/* + * 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.hadoop.yarn.sls.scheduler; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerExitStatus; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ContainerStatus; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.SchedulingRequest; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppReport; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType; +import org.apache.hadoop.yarn.sls.SLSRunner; +import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class SLSSchedulerCommons { + private static final Logger LOG = LoggerFactory.getLogger(SLSSchedulerCommons.class); + + private AbstractYarnScheduler scheduler; + private boolean metricsON; + private SchedulerMetrics schedulerMetrics; + private Map preemptionContainerMap = + new ConcurrentHashMap<>(); + + private Map appQueueMap = + new ConcurrentHashMap<>(); + private Tracker tracker; + + public SLSSchedulerCommons(AbstractYarnScheduler scheduler) { + this.scheduler = scheduler; + this.tracker = new Tracker(); + } + + public void initMetrics(Class schedulerClass, Configuration conf) { + metricsON = conf.getBoolean(SLSConfiguration.METRICS_SWITCH, true); + if (metricsON) { + try { + schedulerMetrics = SchedulerMetrics.getInstance(conf, schedulerClass); + schedulerMetrics.init(scheduler, conf); + } catch (Exception e) { + LOG.error("Caught exception while initializing schedulerMetrics", e); + } + } + } + + void stopMetrics() { + try { + if (metricsON) { + schedulerMetrics.tearDown(); + } + } catch (Exception e) { + LOG.error("Caught exception while stopping service", e); + } + } + + public Allocation allocate(ApplicationAttemptId attemptId, + List resourceRequests, + List schedulingRequests, + List containerIds, + List blacklistAdditions, + List blacklistRemovals, + ContainerUpdates updateRequests) { + if (metricsON) { + final Timer.Context context = schedulerMetrics.getSchedulerAllocateTimer() + .time(); + Allocation allocation = null; + try { + allocation = scheduler.allocate(attemptId, resourceRequests, + schedulingRequests, containerIds, + blacklistAdditions, blacklistRemovals, updateRequests); + return allocation; + } catch (Exception e) { + LOG.error("Caught exception from allocate", e); + throw e; + } finally { + context.stop(); + schedulerMetrics.increaseSchedulerAllocationCounter(); + try { + updateQueueWithAllocateRequest(allocation, attemptId, + resourceRequests, containerIds); + } catch (IOException e) { + LOG.error("Caught exception while executing finally block", e); + } + } + } else { + return scheduler.allocate(attemptId, resourceRequests, schedulingRequests, + containerIds, + blacklistAdditions, blacklistRemovals, updateRequests); + } + } + + private void updateQueueWithAllocateRequest(Allocation allocation, + ApplicationAttemptId attemptId, + List resourceRequests, + List containerIds) throws IOException { + // update queue information + Resource pendingResource = Resources.createResource(0, 0); + Resource allocatedResource = Resources.createResource(0, 0); + String queueName = appQueueMap.get(attemptId); + // container requested + for (ResourceRequest request : resourceRequests) { + if (request.getResourceName().equals(ResourceRequest.ANY)) { + Resources.addTo(pendingResource, + Resources.multiply(request.getCapability(), + request.getNumContainers())); + } + } + // container allocated + for (Container container : allocation.getContainers()) { + Resources.addTo(allocatedResource, container.getResource()); + Resources.subtractFrom(pendingResource, container.getResource()); + } + // container released from AM + SchedulerAppReport report = scheduler.getSchedulerAppInfo(attemptId); + for (ContainerId containerId : containerIds) { + Container container = null; + for (RMContainer c : report.getLiveContainers()) { + if (c.getContainerId().equals(containerId)) { + container = c.getContainer(); + break; + } + } + if (container != null) { + // released allocated containers + Resources.subtractFrom(allocatedResource, container.getResource()); + } else { + for (RMContainer c : report.getReservedContainers()) { + if (c.getContainerId().equals(containerId)) { + container = c.getContainer(); + break; + } + } + if (container != null) { + // released reserved containers + Resources.subtractFrom(pendingResource, container.getResource()); + } + } + } + // containers released/preemption from scheduler + Set preemptionContainers = new HashSet(); + if (allocation.getContainerPreemptions() != null) { + preemptionContainers.addAll(allocation.getContainerPreemptions()); + } + if (allocation.getStrictContainerPreemptions() != null) { + preemptionContainers.addAll(allocation.getStrictContainerPreemptions()); + } + if (!preemptionContainers.isEmpty()) { + for (ContainerId containerId : preemptionContainers) { + if (!preemptionContainerMap.containsKey(containerId)) { + Container container = null; + for (RMContainer c : report.getLiveContainers()) { + if (c.getContainerId().equals(containerId)) { + container = c.getContainer(); + break; + } + } + if (container != null) { + preemptionContainerMap.put(containerId, container.getResource()); + } + } + + } + } + + // update metrics + schedulerMetrics.updateQueueMetrics(pendingResource, allocatedResource, + queueName); + } + + public void handle(SchedulerEvent schedulerEvent) { + if (!metricsON) { + scheduler.handle(schedulerEvent); + return; + } + + if (!schedulerMetrics.isRunning()) { + schedulerMetrics.setRunning(true); + } + + Timer.Context handlerTimer = null; + Timer.Context operationTimer = null; + + NodeUpdateSchedulerEventWrapper eventWrapper; + try { + if (schedulerEvent.getType() == SchedulerEventType.NODE_UPDATE + && schedulerEvent instanceof NodeUpdateSchedulerEvent) { + eventWrapper = new NodeUpdateSchedulerEventWrapper( + (NodeUpdateSchedulerEvent) schedulerEvent); + schedulerEvent = eventWrapper; + updateQueueWithNodeUpdate(eventWrapper); + } else if ( + schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED + && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { + // check if having AM Container, update resource usage information + AppAttemptRemovedSchedulerEvent appRemoveEvent = + (AppAttemptRemovedSchedulerEvent) schedulerEvent; + ApplicationAttemptId appAttemptId = + appRemoveEvent.getApplicationAttemptID(); + String queue = appQueueMap.get(appAttemptId); + SchedulerAppReport app = scheduler.getSchedulerAppInfo(appAttemptId); + if (!app.getLiveContainers().isEmpty()) { // have 0 or 1 + // should have one container which is AM container + RMContainer rmc = app.getLiveContainers().iterator().next(); + schedulerMetrics.updateQueueMetricsByRelease( + rmc.getContainer().getResource(), queue); + } + } + + handlerTimer = schedulerMetrics.getSchedulerHandleTimer().time(); + operationTimer = schedulerMetrics.getSchedulerHandleTimer( + schedulerEvent.getType()).time(); + + scheduler.handle(schedulerEvent); + } finally { + if (handlerTimer != null) { + handlerTimer.stop(); + } + if (operationTimer != null) { + operationTimer.stop(); + } + schedulerMetrics.increaseSchedulerHandleCounter(schedulerEvent.getType()); + + if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED + && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { + SLSRunner.decreaseRemainingApps(); + AppAttemptRemovedSchedulerEvent appRemoveEvent = + (AppAttemptRemovedSchedulerEvent) schedulerEvent; + appQueueMap.remove(appRemoveEvent.getApplicationAttemptID()); + if (SLSRunner.getRemainingApps() == 0) { + try { + schedulerMetrics.tearDown(); + SLSRunner.exitSLSRunner(); + } catch (Exception e) { + LOG.error("Scheduler Metrics failed to tear down.", e); + } + } + } else if (schedulerEvent.getType() == + SchedulerEventType.APP_ATTEMPT_ADDED + && schedulerEvent instanceof AppAttemptAddedSchedulerEvent) { + AppAttemptAddedSchedulerEvent appAddEvent = + (AppAttemptAddedSchedulerEvent) schedulerEvent; + SchedulerApplication app = + (SchedulerApplication) scheduler.getSchedulerApplications() + .get(appAddEvent.getApplicationAttemptId().getApplicationId()); + appQueueMap.put(appAddEvent.getApplicationAttemptId(), app.getQueue() + .getQueueName()); + } + } + } + + private void updateQueueWithNodeUpdate( + NodeUpdateSchedulerEventWrapper eventWrapper) { + RMNodeWrapper node = (RMNodeWrapper) eventWrapper.getRMNode(); + List containerList = node.getContainerUpdates(); + for (UpdatedContainerInfo info : containerList) { + for (ContainerStatus status : info.getCompletedContainers()) { + ContainerId containerId = status.getContainerId(); + SchedulerAppReport app = scheduler.getSchedulerAppInfo( + containerId.getApplicationAttemptId()); + + if (app == null) { + // this happens for the AM container + // The app have already removed when the NM sends the release + // information. + continue; + } + + int releasedMemory = 0, releasedVCores = 0; + if (status.getExitStatus() == ContainerExitStatus.SUCCESS) { + for (RMContainer rmc : app.getLiveContainers()) { + if (rmc.getContainerId() == containerId) { + Resource resource = rmc.getContainer().getResource(); + releasedMemory += resource.getMemorySize(); + releasedVCores += resource.getVirtualCores(); + break; + } + } + } else if (status.getExitStatus() == ContainerExitStatus.ABORTED) { + if (preemptionContainerMap.containsKey(containerId)) { + Resource preResource = preemptionContainerMap.get(containerId); + releasedMemory += preResource.getMemorySize(); + releasedVCores += preResource.getVirtualCores(); + preemptionContainerMap.remove(containerId); + } + } + // update queue counters + String queue = appQueueMap.get(containerId.getApplicationAttemptId()); + schedulerMetrics.updateQueueMetricsByRelease( + Resource.newInstance(releasedMemory, releasedVCores), queue); + } + } + } + + public SchedulerMetrics getSchedulerMetrics() { + return schedulerMetrics; + } + + public boolean isMetricsON() { + return metricsON; + } + + public Tracker getTracker() { + return tracker; + } +} From ffa0eab48867ebbb84a3d015a941b0c08fe0b61a Mon Sep 17 00:00:00 2001 From: Benjamin Teke Date: Fri, 25 Mar 2022 12:01:44 +0100 Subject: [PATCH 063/145] YARN-11094. Follow up changes for YARN-10547. Contributed by Szilard Nemeth --- .../hadoop/yarn/sls/AMDefinitionFactory.java | 6 +-- .../hadoop/yarn/sls/AMDefinitionSLS.java | 35 ++++++++-------- .../org/apache/hadoop/yarn/sls/SLSRunner.java | 35 ++++++++-------- .../yarn/sls/TaskContainerDefinition.java | 41 +++++++++---------- .../sls/scheduler/SLSSchedulerCommons.java | 13 +++--- 5 files changed, 61 insertions(+), 69 deletions(-) diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java index 2bbe7bb1ad159..61975f04a375b 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionFactory.java @@ -37,7 +37,7 @@ public final class AMDefinitionFactory { private AMDefinitionFactory() {} - public static AMDefinitionSLS createFromSlsTrace(Map jsonJob, + public static AMDefinitionSLS createFromSlsTrace(Map jsonJob, SLSRunner slsRunner) throws YarnException { AMDefinitionSLS amDefinition = AMDefinitionSLS.Builder.create(jsonJob) .withAmType(SLSConfiguration.AM_TYPE) @@ -94,7 +94,7 @@ public static AMDefinitionSynth createFromSynth(SynthJob job, return amDefinition; } - private static Resource getAMContainerResourceSLS(Map jsonJob, + private static Resource getAMContainerResourceSLS(Map jsonJob, Configured configured) { Resource amContainerResource = SLSConfiguration.getAMContainerResource(configured.getConf()); @@ -106,7 +106,7 @@ private static Resource getAMContainerResourceSLS(Map jsonJob, for (ResourceInformation info : infors) { String key = SLSConfiguration.JOB_AM_PREFIX + info.getName(); if (jsonJob.containsKey(key)) { - long value = Long.parseLong(jsonJob.get(key).toString()); + long value = Long.parseLong(jsonJob.get(key)); amContainerResource.setResourceValue(info.getName(), value); } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java index 7439ddf8ceef7..a84c9240cb6a3 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMDefinitionSLS.java @@ -36,19 +36,19 @@ public String getQueue() { return queue; } - public static List getTaskContainers(Map jsonJob, + public static List getTaskContainers(Map jsonJob, SLSRunner slsRunner) throws YarnException { - List> tasks = (List) jsonJob.get(SLSConfiguration.JOB_TASKS); + List> tasks = (List) jsonJob.get(SLSConfiguration.JOB_TASKS); if (tasks == null || tasks.size() == 0) { throw new YarnException("No task for the job!"); } List containers = new ArrayList<>(); - for (Map jsonTask : tasks) { + for (Map jsonTask : tasks) { TaskContainerDefinition containerDef = TaskContainerDefinition.Builder.create() .withCount(jsonTask, SLSConfiguration.COUNT) - .withHostname((String) jsonTask.get(SLSConfiguration.TASK_HOST)) + .withHostname(jsonTask.get(SLSConfiguration.TASK_HOST)) .withDuration(jsonTask, SLSConfiguration.TASK_DURATION_MS) .withDurationLegacy(jsonTask, SLSConfiguration.DURATION_MS) .withTaskStart(jsonTask, SLSConfiguration.TASK_START_MS) @@ -69,15 +69,14 @@ public static List getTaskContainers(Map jsonJob, return containers; } - private static Resource getResourceForContainer(Map jsonTask, + private static Resource getResourceForContainer(Map jsonTask, SLSRunner slsRunner) { Resource res = slsRunner.getDefaultContainerResource(); ResourceInformation[] infors = ResourceUtils.getResourceTypesArray(); for (ResourceInformation info : infors) { if (jsonTask.containsKey(SLSConfiguration.TASK_PREFIX + info.getName())) { long value = Long.parseLong( - jsonTask.get(SLSConfiguration.TASK_PREFIX + info.getName()) - .toString()); + jsonTask.get(SLSConfiguration.TASK_PREFIX + info.getName())); res.setResourceValue(info.getName(), value); } } @@ -85,19 +84,19 @@ private static Resource getResourceForContainer(Map jsonTask, } public static final class Builder extends AmDefinitionBuilder { - private final Map jsonJob; + private final Map jsonJob; - private Builder(Map jsonJob) { + private Builder(Map jsonJob) { this.jsonJob = jsonJob; } - public static Builder create(Map jsonJob) { + public static Builder create(Map jsonJob) { return new Builder(jsonJob); } public Builder withAmType(String key) { if (jsonJob.containsKey(key)) { - String amType = (String) jsonJob.get(key); + String amType = jsonJob.get(key); if (amType != null) { this.amType = amType; } @@ -107,7 +106,7 @@ public Builder withAmType(String key) { public Builder withUser(String key) { if (jsonJob.containsKey(key)) { - String user = (String) jsonJob.get(key); + String user = jsonJob.get(key); if (user != null) { this.user = user; } @@ -117,21 +116,21 @@ public Builder withUser(String key) { public Builder withQueue(String key) { if (jsonJob.containsKey(key)) { - this.queue = jsonJob.get(key).toString(); + this.queue = jsonJob.get(key); } return this; } public Builder withJobId(String key) { if (jsonJob.containsKey(key)) { - this.jobId = (String) jsonJob.get(key); + this.jobId = jsonJob.get(key); } return this; } public Builder withJobCount(String key) { if (jsonJob.containsKey(key)) { - jobCount = Integer.parseInt(jsonJob.get(key).toString()); + jobCount = Integer.parseInt(jsonJob.get(key)); jobCount = Math.max(jobCount, 1); } return this; @@ -139,21 +138,21 @@ public Builder withJobCount(String key) { public Builder withJobStartTime(String key) { if (jsonJob.containsKey(key)) { - this.jobStartTime = Long.parseLong(jsonJob.get(key).toString()); + this.jobStartTime = Long.parseLong(jsonJob.get(key)); } return this; } public Builder withJobFinishTime(String key) { if (jsonJob.containsKey(key)) { - this.jobFinishTime = Long.parseLong(jsonJob.get(key).toString()); + this.jobFinishTime = Long.parseLong(jsonJob.get(key)); } return this; } public Builder withLabelExpression(String key) { if (jsonJob.containsKey(key)) { - this.labelExpression = jsonJob.get(key).toString(); + this.labelExpression = jsonJob.get(key); } return this; } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index 83834e8f9c989..260a6001372f4 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -22,12 +22,14 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.Reader; +import java.nio.charset.StandardCharsets; import java.security.Security; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.Collections; @@ -37,6 +39,7 @@ import java.util.concurrent.TimeUnit; import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.cli.CommandLine; @@ -126,8 +129,7 @@ public class SLSRunner extends Configured implements Tool { private long maxRuntime; private String tableMapping; - private final static Map simulateInfoMap = - new HashMap(); + private final static Map simulateInfoMap = new HashMap<>(); // logger public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class); @@ -227,7 +229,7 @@ public static Map getSimulateInfoMap() { public void setSimulationParams(TraceType inType, String[] inTraces, String nodes, String outDir, Set trackApps, - boolean printsimulation) throws IOException, ClassNotFoundException { + boolean printsimulation) { this.inputType = inType; this.inputTraces = inTraces.clone(); @@ -420,7 +422,6 @@ private void waitForNodesRunning() throws InterruptedException { System.currentTimeMillis() - startTimeMS); } - @SuppressWarnings("unchecked") private void startAM() throws YarnException, IOException { switch (inputType) { case SLS: @@ -449,21 +450,21 @@ private void startAM() throws YarnException, IOException { /** * Parse workload from a SLS trace file. */ - @SuppressWarnings("unchecked") private void startAMFromSLSTrace(String inputTrace) throws IOException { JsonFactory jsonF = new JsonFactory(); ObjectMapper mapper = new ObjectMapper(); try (Reader input = new InputStreamReader( - new FileInputStream(inputTrace), "UTF-8")) { - Iterator jobIter = mapper.readValues( - jsonF.createParser(input), Map.class); + new FileInputStream(inputTrace), StandardCharsets.UTF_8)) { + JavaType type = mapper.getTypeFactory(). + constructMapType(Map.class, String.class, String.class); + Iterator> jobIter = mapper.readValues( + jsonF.createParser(input), type); while (jobIter.hasNext()) { try { - Map jsonJob = jobIter.next(); - AMDefinitionSLS amDef = AMDefinitionFactory.createFromSlsTrace( - jsonJob, this); + Map jsonJob = jobIter.next(); + AMDefinitionSLS amDef = AMDefinitionFactory.createFromSlsTrace(jsonJob, this); startAMs(amDef); } catch (Exception e) { LOG.error("Failed to create an AM: {}", e.getMessage()); @@ -500,7 +501,6 @@ private void startAMs(AMDefinition amDef, ReservationId reservationId, /** * Parse workload from a rumen trace file. */ - @SuppressWarnings("unchecked") private void startAMFromRumenTrace(String inputTrace, long baselineTimeMS) throws IOException { Configuration conf = new Configuration(); @@ -536,7 +536,6 @@ Resource getDefaultContainerResource() { /** * parse workload information from synth-generator trace files. */ - @SuppressWarnings("unchecked") private void startAMFromSynthGenerator() throws YarnException, IOException { Configuration localConf = new Configuration(); localConf.set("fs.defaultFS", "file:///"); @@ -729,17 +728,17 @@ public int run(final String[] argv) throws IOException, InterruptedException, throw new YarnException("Cannot create output directory"); } - Set trackedJobSet = new HashSet(); + Set trackedJobSet = new HashSet<>(); if (cmd.hasOption("trackjobs")) { String trackjobs = cmd.getOptionValue("trackjobs"); - String jobIds[] = trackjobs.split(","); + String[] jobIds = trackjobs.split(","); trackedJobSet.addAll(Arrays.asList(jobIds)); } String tempNodeFile = cmd.hasOption("nodes") ? cmd.getOptionValue("nodes") : ""; - TraceType tempTraceType = TraceType.SLS; + TraceType tempTraceType; switch (traceType) { case "SLS": tempTraceType = TraceType.SLS; @@ -834,9 +833,7 @@ public boolean equals(Object o) { NodeDetails that = (NodeDetails) o; return StringUtils.equals(hostname, that.hostname) && ( - nodeResource == null ? - that.nodeResource == null : - nodeResource.equals(that.nodeResource)) && SetUtils + Objects.equals(nodeResource, that.nodeResource)) && SetUtils .isEqualSet(labels, that.labels); } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java index 1b0cd9003b63a..04bea3ab2cd83 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/TaskContainerDefinition.java @@ -88,9 +88,9 @@ public static Builder create() { return new Builder(); } - public Builder withDuration(Map jsonTask, String key) { + public Builder withDuration(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.duration = Integer.parseInt(jsonTask.get(key).toString()); + this.duration = Integer.parseInt(jsonTask.get(key)); } return this; } @@ -106,23 +106,23 @@ public Builder withDuration(long duration) { * @param key The json key. * @return the builder */ - public Builder withDurationLegacy(Map jsonTask, String key) { + public Builder withDurationLegacy(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.durationLegacy = Integer.parseInt(jsonTask.get(key).toString()); + this.durationLegacy = Integer.parseInt(jsonTask.get(key)); } return this; } - public Builder withTaskStart(Map jsonTask, String key) { + public Builder withTaskStart(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.taskStart = Long.parseLong(jsonTask.get(key).toString()); + this.taskStart = Long.parseLong(jsonTask.get(key)); } return this; } - public Builder withTaskFinish(Map jsonTask, String key) { + public Builder withTaskFinish(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.taskFinish = Long.parseLong(jsonTask.get(key).toString()); + this.taskFinish = Long.parseLong(jsonTask.get(key)); } return this; } @@ -132,9 +132,9 @@ public Builder withResource(Resource resource) { return this; } - public Builder withPriority(Map jsonTask, String key) { + public Builder withPriority(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.priority = Integer.parseInt(jsonTask.get(key).toString()); + this.priority = Integer.parseInt(jsonTask.get(key)); } return this; } @@ -144,9 +144,9 @@ public Builder withPriority(int priority) { return this; } - public Builder withType(Map jsonTask, String key) { + public Builder withType(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.type = jsonTask.get(key).toString(); + this.type = jsonTask.get(key); } return this; } @@ -156,9 +156,9 @@ public Builder withType(String type) { return this; } - public Builder withCount(Map jsonTask, String key) { + public Builder withCount(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - count = Integer.parseInt(jsonTask.get(key).toString()); + count = Integer.parseInt(jsonTask.get(key)); count = Math.max(count, 1); } return this; @@ -169,10 +169,9 @@ public Builder withCount(int count) { return this; } - public Builder withExecutionType(Map jsonTask, String key) { + public Builder withExecutionType(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.executionType = ExecutionType.valueOf( - jsonTask.get(key).toString()); + this.executionType = ExecutionType.valueOf(jsonTask.get(key)); } return this; } @@ -182,9 +181,9 @@ public Builder withExecutionType(ExecutionType executionType) { return this; } - public Builder withAllocationId(Map jsonTask, String key) { + public Builder withAllocationId(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - this.allocationId = Long.parseLong(jsonTask.get(key).toString()); + this.allocationId = Long.parseLong(jsonTask.get(key)); } return this; } @@ -194,9 +193,9 @@ public Builder withAllocationId(long allocationId) { return this; } - public Builder withRequestDelay(Map jsonTask, String key) { + public Builder withRequestDelay(Map jsonTask, String key) { if (jsonTask.containsKey(key)) { - requestDelay = Long.parseLong(jsonTask.get(key).toString()); + requestDelay = Long.parseLong(jsonTask.get(key)); requestDelay = Math.max(requestDelay, 0); } return this; diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java index 92aa96069c848..7132fc9e62002 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java @@ -54,22 +54,19 @@ public class SLSSchedulerCommons { private static final Logger LOG = LoggerFactory.getLogger(SLSSchedulerCommons.class); - private AbstractYarnScheduler scheduler; + private final AbstractYarnScheduler scheduler; private boolean metricsON; private SchedulerMetrics schedulerMetrics; - private Map preemptionContainerMap = - new ConcurrentHashMap<>(); - - private Map appQueueMap = - new ConcurrentHashMap<>(); - private Tracker tracker; + private final Map preemptionContainerMap = new ConcurrentHashMap<>(); + private final Map appQueueMap = new ConcurrentHashMap<>(); + private final Tracker tracker; public SLSSchedulerCommons(AbstractYarnScheduler scheduler) { this.scheduler = scheduler; this.tracker = new Tracker(); } - public void initMetrics(Class schedulerClass, Configuration conf) { + public void initMetrics(Class> schedulerClass, Configuration conf) { metricsON = conf.getBoolean(SLSConfiguration.METRICS_SWITCH, true); if (metricsON) { try { From 565e848d88c7fbade3ddbd908e2544abb55ec171 Mon Sep 17 00:00:00 2001 From: litao Date: Fri, 25 Mar 2022 19:19:01 +0800 Subject: [PATCH 064/145] HDFS-16434. Add opname to read/write lock for remaining operations (#3915) --- .../DelegationTokenSecretManager.java | 4 +-- .../server/blockmanagement/BlockManager.java | 30 +++++++++---------- .../blockmanagement/BlockManagerSafeMode.java | 2 +- .../CacheReplicationMonitor.java | 2 +- .../DatanodeAdminBackoffMonitor.java | 18 +++++------ .../DatanodeAdminDefaultMonitor.java | 4 +-- .../blockmanagement/DatanodeManager.java | 4 +-- .../blockmanagement/HeartbeatManager.java | 4 +-- .../hdfs/server/namenode/BackupImage.java | 2 +- .../hdfs/server/namenode/Checkpointer.java | 2 +- .../namenode/EncryptionZoneManager.java | 6 ++-- .../namenode/FSDirEncryptionZoneOp.java | 4 +-- .../hdfs/server/namenode/FSNamesystem.java | 2 ++ .../server/namenode/FsImageValidation.java | 2 +- .../hadoop/hdfs/server/namenode/NameNode.java | 12 ++++---- .../server/namenode/ReencryptionHandler.java | 2 +- .../server/namenode/SecondaryNameNode.java | 2 +- .../server/namenode/ha/EditLogTailer.java | 2 +- .../namenode/snapshot/SnapshotDeletionGc.java | 2 +- .../org/apache/hadoop/hdfs/util/RwLock.java | 12 ++++++++ 20 files changed, 66 insertions(+), 52 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java index f97c2f2c80b71..b89998d989586 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java @@ -383,7 +383,7 @@ protected void logUpdateMasterKey(DelegationKey key) namesystem.logUpdateMasterKey(key); } } finally { - namesystem.readUnlock(); + namesystem.readUnlock("logUpdateMasterKey"); } } catch (InterruptedException ie) { // AbstractDelegationTokenManager may crash if an exception is thrown. @@ -412,7 +412,7 @@ protected void logExpireToken(final DelegationTokenIdentifier dtId) namesystem.logExpireDelegationToken(dtId); } } finally { - namesystem.readUnlock(); + namesystem.readUnlock("logExpireToken"); } } catch (InterruptedException ie) { // AbstractDelegationTokenManager may crash if an exception is thrown. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 2b647704e3f9b..494c2f01c99ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -2017,7 +2017,7 @@ int computeBlockReconstructionWork(int blocksToProcess) { blocksToReconstruct = neededReconstruction .chooseLowRedundancyBlocks(blocksToProcess, reset); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("computeBlockReconstructionWork"); } return computeReconstructionWorkForBlocks(blocksToReconstruct); } @@ -2051,7 +2051,7 @@ int computeReconstructionWorkForBlocks( } } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("computeReconstructionWorkForBlocks"); } // Step 2: choose target nodes for each reconstruction task @@ -2092,7 +2092,7 @@ int computeReconstructionWorkForBlocks( } } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("computeReconstructionWorkForBlocks"); } if (blockLog.isDebugEnabled()) { @@ -2577,7 +2577,7 @@ void processPendingReconstructions() { } } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("processPendingReconstructions"); } /* If we know the target datanodes where the replication timedout, * we could invoke decBlocksScheduled() on it. Its ok for now. @@ -2826,7 +2826,7 @@ public boolean processReport(final DatanodeID nodeID, storageInfo.receivedBlockReport(); } finally { endTime = Time.monotonicNow(); - namesystem.writeUnlock(); + namesystem.writeUnlock("processReport"); } if(blockLog.isDebugEnabled()) { @@ -2870,7 +2870,7 @@ public void removeBRLeaseIfNeeded(final DatanodeID nodeID, context.getTotalRpcs(), Long.toHexString(context.getReportId())); } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("removeBRLeaseIfNeeded"); } } @@ -2908,7 +2908,7 @@ void rescanPostponedMisreplicatedBlocks() { postponedMisreplicatedBlocks.addAll(rescannedMisreplicatedBlocks); rescannedMisreplicatedBlocks.clear(); long endSize = postponedMisreplicatedBlocks.size(); - namesystem.writeUnlock(); + namesystem.writeUnlock("rescanPostponedMisreplicatedBlocks"); LOG.info("Rescan of postponedMisreplicatedBlocks completed in {}" + " msecs. {} blocks are left. {} blocks were removed.", (Time.monotonicNow() - startTime), endSize, (startSize - endSize)); @@ -3775,7 +3775,7 @@ private void processMisReplicatesAsync() throws InterruptedException { break; } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("processMisReplicatesAsync"); // Make sure it is out of the write lock for sufficiently long time. Thread.sleep(sleepDuration); } @@ -3830,7 +3830,7 @@ public int processMisReplicatedBlocks(List blocks) { "Re-scanned block {}, result is {}", blk, r); } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("processMisReplicatedBlocks"); } } } catch (InterruptedException ex) { @@ -4553,7 +4553,7 @@ void processExtraRedundancyBlocksOnInService( // testPlacementWithLocalRackNodesDecommissioned, it is not protected by // lock, only when called by DatanodeManager.refreshNodes have writeLock if (namesystem.hasWriteLock()) { - namesystem.writeUnlock(); + namesystem.writeUnlock("processExtraRedundancyBlocksOnInService"); try { Thread.sleep(1); } catch (InterruptedException e) { @@ -4685,7 +4685,7 @@ private void updateNeededReconstructions(final BlockInfo block, repl.outOfServiceReplicas(), oldExpectedReplicas); } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("updateNeededReconstructions"); } } @@ -4742,7 +4742,7 @@ private int invalidateWorkForOneNode(DatanodeInfo dn) { return 0; } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("invalidateWorkForOneNode"); } blockLog.debug("BLOCK* {}: ask {} to delete {}", getClass().getSimpleName(), dn, toInvalidate); @@ -4974,7 +4974,7 @@ private void remove(long time) { } } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("markedDeleteBlockScrubberThread"); } } } @@ -5092,7 +5092,7 @@ int computeDatanodeWork() { this.updateState(); this.scheduledReplicationBlocksCount = workFound; } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("computeDatanodeWork"); } workFound += this.computeInvalidateWork(nodesToProcess); return workFound; @@ -5332,7 +5332,7 @@ private void processQueue() { action = queue.poll(); } while (action != null); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("processQueue"); metrics.addBlockOpsBatched(processed - 1); } } catch (InterruptedException e) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java index de26929c616b6..4349ba01401d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerSafeMode.java @@ -670,7 +670,7 @@ public void run() { break; } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("leaveSafeMode"); } try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java index 4cc404f55c5d6..1e5f952040d53 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java @@ -302,7 +302,7 @@ private void rescan() throws InterruptedException { rescanCachedBlockMap(); blockManager.getDatanodeManager().resetLastCachingDirectiveSentTime(); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("cacheReplicationMonitorRescan"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java index eb65b3843a9c4..9b38f2353fbd7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java @@ -215,7 +215,7 @@ public void run() { processPendingNodes(); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("DatanodeAdminMonitorV2Thread"); } // After processing the above, various parts of the check() method will // take and drop the read / write lock as needed. Aside from the @@ -345,12 +345,12 @@ private void processMaintenanceNodes() { // which added the node to the cancelled list. Therefore expired // maintenance nodes do not need to be added to the toRemove list. dnAdmin.stopMaintenance(dn); - namesystem.writeUnlock(); + namesystem.writeUnlock("processMaintenanceNodes"); namesystem.writeLock(); } } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("processMaintenanceNodes"); } } @@ -409,7 +409,7 @@ private void processCompletedNodes(List toRemove) { } } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("processCompletedNodes"); } } @@ -531,7 +531,7 @@ private void moveBlocksToPending() { // replication if (blocksProcessed >= blocksPerLock) { blocksProcessed = 0; - namesystem.writeUnlock(); + namesystem.writeUnlock("moveBlocksToPending"); namesystem.writeLock(); } blocksProcessed++; @@ -553,7 +553,7 @@ private void moveBlocksToPending() { } } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("moveBlocksToPending"); } LOG.debug("{} blocks are now pending replication", pendingCount); } @@ -637,7 +637,7 @@ private void scanDatanodeStorage(DatanodeDescriptor dn, try { storage = dn.getStorageInfos(); } finally { - namesystem.readUnlock(); + namesystem.readUnlock("scanDatanodeStorage"); } for (DatanodeStorageInfo s : storage) { @@ -667,7 +667,7 @@ private void scanDatanodeStorage(DatanodeDescriptor dn, numBlocksChecked++; } } finally { - namesystem.readUnlock(); + namesystem.readUnlock("scanDatanodeStorage"); } } } @@ -722,7 +722,7 @@ private void processPendingReplication() { suspectBlocks.getOutOfServiceBlockCount()); } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("processPendingReplication"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java index 21e3eb0322d3d..c1d62e35c0375 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java @@ -158,7 +158,7 @@ public void run() { LOG.warn("DatanodeAdminMonitor caught exception when processing node.", e); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("DatanodeAdminMonitorThread"); } if (numBlocksChecked + numNodesChecked > 0) { LOG.info("Checked {} blocks and {} nodes this tick. {} nodes are now " + @@ -373,7 +373,7 @@ private void processBlocksInternal( // lock. // Yielding is required in case of block number is greater than the // configured per-iteration-limit. - namesystem.writeUnlock(); + namesystem.writeUnlock("processBlocksInternal"); try { LOG.debug("Yielded lock during decommission/maintenance check"); Thread.sleep(0, 500); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index cb601e94f822c..092ef6502a6f2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -852,7 +852,7 @@ public void removeDatanode(final DatanodeID node) + node + " does not exist"); } } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("removeDatanode"); } } @@ -1296,7 +1296,7 @@ public void refreshNodes(final Configuration conf) throws IOException { refreshDatanodes(); countSoftwareVersions(); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("refreshNodes"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java index 372cb237ca1e9..b923ba3a65590 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java @@ -503,7 +503,7 @@ void heartbeatCheck() { try { dm.removeDeadDatanode(dead, !dead.isMaintenance()); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("removeDeadDatanode"); } } if (failedStorage != null) { @@ -512,7 +512,7 @@ void heartbeatCheck() { try { blockManager.removeBlocksAssociatedTo(failedStorage); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("removeBlocksAssociatedTo"); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java index 504df6068ef3c..7bde21dcb69e9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java @@ -222,7 +222,7 @@ private synchronized void applyEdits(long firstTxId, int numTxns, byte[] data) try { getNamesystem().dir.updateCountForQuota(); } finally { - getNamesystem().writeUnlock(); + getNamesystem().writeUnlock("applyEdits"); } } finally { backupInputStream.clear(); 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 d18d448ab2500..ab657fb672490 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 @@ -250,7 +250,7 @@ void doCheckpoint() throws IOException { sig.mostRecentCheckpointTxId); bnImage.reloadFromImageFile(file, backupNode.getNamesystem()); } finally { - backupNode.namesystem.writeUnlock(); + backupNode.namesystem.writeUnlock("doCheckpointByBackupNode"); } } rollForwardByApplyingLogs(manifest, bnImage, backupNode.getNamesystem()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java index 62a81f4064eec..7bf5879971615 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java @@ -192,7 +192,7 @@ public void pauseForTestingAfterNthCheckpoint(final String zone, try { iip = dir.resolvePath(pc, zone, DirOp.READ); } finally { - dir.getFSNamesystem().readUnlock(); + dir.getFSNamesystem().readUnlock("pauseForTestingAfterNthCheckpoint"); } reencryptionHandler .pauseForTestingAfterNthCheckpoint(iip.getLastINode().getId(), count); @@ -224,7 +224,7 @@ public ZoneReencryptionStatus getZoneStatus(final String zone) return getReencryptionStatus().getZoneStatus(inode.getId()); } finally { dir.readUnlock(); - dir.getFSNamesystem().readUnlock(); + dir.getFSNamesystem().readUnlock("getZoneStatus"); } } @@ -285,7 +285,7 @@ void stopReencryptThread() { try { reencryptionHandler.stopThreads(); } finally { - dir.getFSNamesystem().writeUnlock(); + dir.getFSNamesystem().writeUnlock("stopReencryptThread"); } if (reencryptHandlerExecutor != null) { reencryptHandlerExecutor.shutdownNow(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java index 2971af1829809..2110a408b0877 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java @@ -656,7 +656,7 @@ static EncryptionKeyInfo getEncryptionKeyInfo(FSNamesystem fsn, Preconditions.checkNotNull(ezKeyName); // Generate EDEK while not holding the fsn lock. - fsn.writeUnlock(); + fsn.writeUnlock("getEncryptionKeyInfo"); try { EncryptionFaultInjector.getInstance().startFileBeforeGenerateKey(); return new EncryptionKeyInfo(protocolVersion, suite, ezKeyName, @@ -733,7 +733,7 @@ static String getKeyNameForZone(final FSDirectory dir, dir.ezManager.checkEncryptionZoneRoot(iip.getLastINode(), zone); return dir.ezManager.getKeyName(iip); } finally { - dir.getFSNamesystem().readUnlock(); + dir.getFSNamesystem().readUnlock("getKeyNameForZone"); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index ef59b9f75ea06..389bd6455c92f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -1798,6 +1798,7 @@ public void readUnlock() { this.fsLock.readUnlock(); } + @Override public void readUnlock(String opName) { this.fsLock.readUnlock(opName); } @@ -1822,6 +1823,7 @@ public void writeUnlock() { this.fsLock.writeUnlock(); } + @Override public void writeUnlock(String opName) { this.fsLock.writeUnlock(opName); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java index 3b8c33165395b..3325222267d5d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FsImageValidation.java @@ -243,7 +243,7 @@ public void run() { loader.load(fsImageFile, false); } finally { namesystem.getFSDirectory().writeUnlock(); - namesystem.writeUnlock(); + namesystem.writeUnlock("loadImage"); } } t.cancel(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index ef0eef8510cd2..50f3b0bb2e617 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -2090,7 +2090,7 @@ public void writeLock() { @Override public void writeUnlock() { namesystem.unlockRetryCache(); - namesystem.writeUnlock(); + namesystem.writeUnlock("HAState"); } /** Check if an operation of given category is allowed */ @@ -2254,7 +2254,7 @@ private String reconfReplicationParameters(final String newVal, throw new ReconfigurationException(property, newVal, getConf().get( property), e); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("reconfReplicationParameters"); } } @@ -2291,7 +2291,7 @@ private String reconfHeartbeatInterval(final DatanodeManager datanodeManager, throw new ReconfigurationException(property, newVal, getConf().get( property), nfe); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("reconfHeartbeatInterval"); LOG.info("RECONFIGURE* changed heartbeatInterval to " + datanodeManager.getHeartbeatInterval()); } @@ -2315,7 +2315,7 @@ private String reconfHeartbeatRecheckInterval( throw new ReconfigurationException(property, newVal, getConf().get( property), nfe); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("reconfHeartbeatRecheckInterval"); LOG.info("RECONFIGURE* changed heartbeatRecheckInterval to " + datanodeManager.getHeartbeatRecheckInterval()); } @@ -2434,7 +2434,7 @@ String reconfigureSlowNodesParameters(final DatanodeManager datanodeManager, throw new ReconfigurationException(property, newVal, getConf().get( property), e); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("reconfigureSlowNodesParameters"); } } @@ -2454,7 +2454,7 @@ private String reconfigureBlockInvalidateLimit(final DatanodeManager datanodeMan } catch (NumberFormatException e) { throw new ReconfigurationException(property, newVal, getConf().get(property), e); } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("reconfigureBlockInvalidateLimit"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java index 1a60879a970fc..2a7002b5cd2f5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java @@ -350,7 +350,7 @@ public void run() { getReencryptionStatus().markZoneStarted(zoneId); resetSubmissionTracker(zoneId); } finally { - dir.getFSNamesystem().readUnlock(); + dir.getFSNamesystem().readUnlock("reEncryptThread"); } try { 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 74730522d863a..a0c8e9840f4b3 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 @@ -1098,7 +1098,7 @@ void doMerge( try { dstImage.reloadFromImageFile(file, dstNamesystem); } finally { - dstNamesystem.writeUnlock(); + dstNamesystem.writeUnlock("reloadFromImageFile"); } dstNamesystem.imageLoadComplete(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java index ec1169b85a830..25596dce9f51d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java @@ -385,7 +385,7 @@ public long doTailEdits() throws IOException, InterruptedException { lastLoadedTxnId = image.getLastAppliedTxId(); return editsLoaded; } finally { - namesystem.writeUnlock(); + namesystem.writeUnlock("doTailEdits"); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDeletionGc.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDeletionGc.java index d57da2204aa38..c62455c724feb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDeletionGc.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotDeletionGc.java @@ -79,7 +79,7 @@ private void gcDeletedSnapshot(String name) { LOG.error("Failed to chooseDeletedSnapshot", e); throw e; } finally { - namesystem.readUnlock(); + namesystem.readUnlock("gcDeletedSnapshot"); } if (deleted == null) { LOG.trace("{}: no snapshots are marked as deleted.", name); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java index deaeaa43247fd..05c1a06abda23 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java @@ -28,6 +28,12 @@ public interface RwLock { /** Release read lock. */ public void readUnlock(); + /** + * Release read lock with operation name. + * @param opName Option name. + */ + public void readUnlock(String opName); + /** Check if the current thread holds read lock. */ public boolean hasReadLock(); @@ -40,6 +46,12 @@ public interface RwLock { /** Release write lock. */ public void writeUnlock(); + /** + * Release write lock with operation name. + * @param opName Option name. + */ + public void writeUnlock(String opName); + /** Check if the current thread holds write lock. */ public boolean hasWriteLock(); } From 08a77a765ba635da1cc44f36b103116605a517ee Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Fri, 25 Mar 2022 18:48:56 +0100 Subject: [PATCH 065/145] YARN-10548. Decouple AM runner logic from SLSRunner. Contributed by Szilard Nemeth. --- .../org/apache/hadoop/yarn/sls/AMRunner.java | 297 ++++++++++++++++++ .../org/apache/hadoop/yarn/sls/SLSRunner.java | 250 +++------------ .../sls/resourcemanager/MockAMLauncher.java | 17 +- 3 files changed, 354 insertions(+), 210 deletions(-) create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java new file mode 100644 index 0000000000000..da95c687ee71c --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java @@ -0,0 +1,297 @@ +/* + * 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.hadoop.yarn.sls; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.tools.rumen.JobTraceReader; +import org.apache.hadoop.tools.rumen.LoggedJob; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ReservationId; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.sls.SLSRunner.TraceType; +import org.apache.hadoop.yarn.sls.appmaster.AMSimulator; +import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; +import org.apache.hadoop.yarn.sls.scheduler.TaskRunner; +import org.apache.hadoop.yarn.sls.synthetic.SynthJob; +import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer; +import org.apache.hadoop.yarn.util.UTCClock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.Reader; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + + +public class AMRunner { + private static final Logger LOG = LoggerFactory.getLogger(AMRunner.class); + static int REMAINING_APPS = 0; + + private final Configuration conf; + private int AM_ID; + private Map amMap; + private Map appIdAMSim; + private Set trackedApps; + private Map amClassMap; + private TraceType inputType; + private String[] inputTraces; + private SynthTraceJobProducer stjp; + private TaskRunner runner; + private SLSRunner slsRunner; + private int numAMs, numTasks; + private long maxRuntime; + private ResourceManager rm; + + public AMRunner(TaskRunner runner, SLSRunner slsRunner) { + this.runner = runner; + this.slsRunner = slsRunner; + this.conf = slsRunner.getConf(); + } + + + public void init(Configuration conf) throws ClassNotFoundException { + amMap = new ConcurrentHashMap<>(); + amClassMap = new HashMap<>(); + appIdAMSim = new ConcurrentHashMap<>(); + // map + for (Map.Entry e : conf) { + String key = e.getKey().toString(); + if (key.startsWith(SLSConfiguration.AM_TYPE_PREFIX)) { + String amType = key.substring(SLSConfiguration.AM_TYPE_PREFIX.length()); + amClassMap.put(amType, Class.forName(conf.get(key))); + } + } + } + + public void startAM() throws YarnException, IOException { + switch (inputType) { + case SLS: + for (String inputTrace : inputTraces) { + startAMFromSLSTrace(inputTrace); + } + break; + case RUMEN: + long baselineTimeMS = 0; + for (String inputTrace : inputTraces) { + startAMFromRumenTrace(inputTrace, baselineTimeMS); + } + break; + case SYNTH: + startAMFromSynthGenerator(); + break; + default: + throw new YarnException("Input configuration not recognized, " + + "trace type should be SLS, RUMEN, or SYNTH"); + } + + numAMs = amMap.size(); + REMAINING_APPS = numAMs; + } + + /** + * Parse workload from a SLS trace file. + */ + private void startAMFromSLSTrace(String inputTrace) throws IOException { + JsonFactory jsonF = new JsonFactory(); + ObjectMapper mapper = new ObjectMapper(); + + try (Reader input = new InputStreamReader( + new FileInputStream(inputTrace), StandardCharsets.UTF_8)) { + JavaType type = mapper.getTypeFactory(). + constructMapType(Map.class, String.class, String.class); + Iterator> jobIter = mapper.readValues( + jsonF.createParser(input), type); + + while (jobIter.hasNext()) { + try { + Map jsonJob = jobIter.next(); + AMDefinitionSLS amDef = AMDefinitionFactory.createFromSlsTrace(jsonJob, slsRunner); + startAMs(amDef); + } catch (Exception e) { + LOG.error("Failed to create an AM: {}", e.getMessage()); + } + } + } + } + + /** + * parse workload information from synth-generator trace files. + */ + private void startAMFromSynthGenerator() throws YarnException, IOException { + Configuration localConf = new Configuration(); + localConf.set("fs.defaultFS", "file:///"); + // if we use the nodeFile this could have been not initialized yet. + if (stjp == null) { + stjp = new SynthTraceJobProducer(conf, new Path(inputTraces[0])); + } + + SynthJob job; + // we use stjp, a reference to the job producer instantiated during node + // creation + while ((job = (SynthJob) stjp.getNextJob()) != null) { + ReservationId reservationId = null; + if (job.hasDeadline()) { + reservationId = ReservationId + .newInstance(rm.getStartTime(), AM_ID); + } + AMDefinitionSynth amDef = AMDefinitionFactory.createFromSynth(job, slsRunner); + startAMs(amDef, reservationId, job.getParams(), job.getDeadline()); + } + } + + /** + * Parse workload from a rumen trace file. + */ + private void startAMFromRumenTrace(String inputTrace, long baselineTimeMS) + throws IOException { + Configuration conf = new Configuration(); + conf.set("fs.defaultFS", "file:///"); + File fin = new File(inputTrace); + + try (JobTraceReader reader = new JobTraceReader( + new Path(fin.getAbsolutePath()), conf)) { + LoggedJob job = reader.getNext(); + + while (job != null) { + try { + AMDefinitionRumen amDef = + AMDefinitionFactory.createFromRumenTrace(job, baselineTimeMS, + slsRunner); + startAMs(amDef); + } catch (Exception e) { + LOG.error("Failed to create an AM", e); + } + job = reader.getNext(); + } + } + } + + private void startAMs(AMDefinition amDef) { + for (int i = 0; i < amDef.getJobCount(); i++) { + JobDefinition jobDef = JobDefinition.Builder.create() + .withAmDefinition(amDef) + .withDeadline(-1) + .withReservationId(null) + .withParams(null) + .build(); + runNewAM(jobDef); + } + } + + private void startAMs(AMDefinition amDef, + ReservationId reservationId, + Map params, long deadline) { + for (int i = 0; i < amDef.getJobCount(); i++) { + JobDefinition jobDef = JobDefinition.Builder.create() + .withAmDefinition(amDef) + .withReservationId(reservationId) + .withParams(params) + .withDeadline(deadline) + .build(); + runNewAM(jobDef); + } + } + + private void runNewAM(JobDefinition jobDef) { + AMDefinition amDef = jobDef.getAmDefinition(); + String oldJobId = amDef.getOldAppId(); + AMSimulator amSim = + createAmSimulator(amDef.getAmType()); + + if (amSim != null) { + int heartbeatInterval = conf.getInt( + SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS, + SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT); + boolean isTracked = trackedApps.contains(oldJobId); + + if (oldJobId == null) { + oldJobId = Integer.toString(AM_ID); + } + AM_ID++; + amSim.init(amDef, rm, slsRunner, isTracked, runner.getStartTimeMS(), heartbeatInterval, appIdAMSim); + if (jobDef.getReservationId() != null) { + // if we have a ReservationId, delegate reservation creation to + // AMSim (reservation shape is impl specific) + UTCClock clock = new UTCClock(); + amSim.initReservation(jobDef.getReservationId(), jobDef.getDeadline(), clock.getTime()); + } + runner.schedule(amSim); + maxRuntime = Math.max(maxRuntime, amDef.getJobFinishTime()); + numTasks += amDef.getTaskContainers().size(); + amMap.put(oldJobId, amSim); + } + } + + private AMSimulator createAmSimulator(String jobType) { + return (AMSimulator) ReflectionUtils.newInstance( + amClassMap.get(jobType), new Configuration()); + } + + public AMSimulator getAMSimulator(ApplicationId appId) { + return appIdAMSim.get(appId); + } + + public void setInputType(TraceType inputType) { + this.inputType = inputType; + } + + public void setInputTraces(String[] inputTraces) { + this.inputTraces = inputTraces; + } + + public void setResourceManager(ResourceManager rm) { + this.rm = rm; + } + + public Set getTrackedApps() { + return trackedApps; + } + + public void setTrackedApps(Set trackApps) { + this.trackedApps = trackApps; + } + + public int getNumAMs() { + return numAMs; + } + + public int getNumTasks() { + return numTasks; + } + + public long getMaxRuntime() { + return maxRuntime; + } + + public Map getAmMap() { + return amMap; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index 260a6001372f4..48ad610310214 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -58,16 +58,12 @@ import org.apache.hadoop.metrics2.source.JvmMetrics; import org.apache.hadoop.net.DNSToSwitchMapping; import org.apache.hadoop.net.TableMapping; -import org.apache.hadoop.tools.rumen.JobTraceReader; -import org.apache.hadoop.tools.rumen.LoggedJob; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.api.records.NodeState; -import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -84,19 +80,32 @@ import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator; import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher; import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler; -import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics; -import org.apache.hadoop.yarn.sls.scheduler.TaskRunner; import org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler; +import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics; import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper; -import org.apache.hadoop.yarn.sls.synthetic.SynthJob; +import org.apache.hadoop.yarn.sls.scheduler.TaskRunner; import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer; import org.apache.hadoop.yarn.sls.utils.SLSUtils; -import org.apache.hadoop.yarn.util.UTCClock; import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.Resources; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.IOException; +import java.security.Security; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + @Private @Unstable public class SLSRunner extends Configured implements Tool { @@ -112,21 +121,12 @@ public class SLSRunner extends Configured implements Tool { private Resource nodeManagerResource; private String nodeFile; - // AM simulator - private int AM_ID; - private Map amMap; - private Map appIdAMSim; - private Set trackedApps; - private Map amClassMap; - private static int remainingApps = 0; - // metrics private String metricsOutputDir; private boolean printSimulation; // other simulation information - private int numNMs, numRacks, numAMs, numTasks; - private long maxRuntime; + private int numNMs, numRacks; private String tableMapping; private final static Map simulateInfoMap = new HashMap<>(); @@ -135,6 +135,7 @@ public class SLSRunner extends Configured implements Tool { public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class); private static boolean exitAtTheFinish = false; + private AMRunner amRunner; /** * The type of trace in input. @@ -151,7 +152,7 @@ public enum TraceType { private SynthTraceJobProducer stjp; public static int getRemainingApps() { - return remainingApps; + return AMRunner.REMAINING_APPS; } public SLSRunner() throws ClassNotFoundException { @@ -176,9 +177,7 @@ public void setConf(Configuration conf) { private void init(Configuration tempConf) throws ClassNotFoundException { nmMap = new ConcurrentHashMap<>(); queueAppNumMap = new HashMap<>(); - amMap = new ConcurrentHashMap<>(); - amClassMap = new HashMap<>(); - appIdAMSim = new ConcurrentHashMap<>(); + amRunner = new AMRunner(runner, this); // runner configuration setConf(tempConf); @@ -186,15 +185,8 @@ private void init(Configuration tempConf) throws ClassNotFoundException { poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE, SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT); SLSRunner.runner.setQueueSize(poolSize); - // map - for (Map.Entry e : tempConf) { - String key = e.getKey().toString(); - if (key.startsWith(SLSConfiguration.AM_TYPE_PREFIX)) { - String amType = key.substring(SLSConfiguration.AM_TYPE_PREFIX.length()); - amClassMap.put(amType, Class.forName(tempConf.get(key))); - } - } + amRunner.init(tempConf); nodeManagerResource = getNodeManagerResource(); } @@ -227,14 +219,25 @@ public static Map getSimulateInfoMap() { return Collections.unmodifiableMap(simulateInfoMap); } + /** + * This is invoked before start. + * @param inType + * @param inTraces + * @param nodes + * @param outDir + * @param trackApps + * @param printsimulation + */ public void setSimulationParams(TraceType inType, String[] inTraces, String nodes, String outDir, Set trackApps, boolean printsimulation) { this.inputType = inType; this.inputTraces = inTraces.clone(); + this.amRunner.setInputType(this.inputType); + this.amRunner.setInputTraces(this.inputTraces); + this.amRunner.setTrackedApps(trackApps); this.nodeFile = nodes; - this.trackedApps = trackApps; this.printSimulation = printsimulation; metricsOutputDir = outDir; tableMapping = outDir + "/tableMapping.csv"; @@ -247,15 +250,16 @@ public void start() throws IOException, ClassNotFoundException, YarnException, // start resource manager startRM(); + amRunner.setResourceManager(rm); // start node managers startNM(); // start application masters - startAM(); + amRunner.startAM(); // set queue & tracked apps information ((SchedulerWrapper) rm.getResourceScheduler()).getTracker() .setQueueSet(this.queueAppNumMap.keySet()); ((SchedulerWrapper) rm.getResourceScheduler()).getTracker() - .setTrackedAppSet(this.trackedApps); + .setTrackedAppSet(amRunner.getTrackedApps()); // print out simulation info printSimulationInfo(); // blocked until all nodes RUNNING @@ -310,7 +314,7 @@ private void startRM() throws ClassNotFoundException, YarnException { rm = new ResourceManager() { @Override protected ApplicationMasterLauncher createAMLauncher() { - return new MockAMLauncher(se, this.rmContext, appIdAMSim); + return new MockAMLauncher(se, this.rmContext); } }; @@ -422,109 +426,6 @@ private void waitForNodesRunning() throws InterruptedException { System.currentTimeMillis() - startTimeMS); } - private void startAM() throws YarnException, IOException { - switch (inputType) { - case SLS: - for (String inputTrace : inputTraces) { - startAMFromSLSTrace(inputTrace); - } - break; - case RUMEN: - long baselineTimeMS = 0; - for (String inputTrace : inputTraces) { - startAMFromRumenTrace(inputTrace, baselineTimeMS); - } - break; - case SYNTH: - startAMFromSynthGenerator(); - break; - default: - throw new YarnException("Input configuration not recognized, " - + "trace type should be SLS, RUMEN, or SYNTH"); - } - - numAMs = amMap.size(); - remainingApps = numAMs; - } - - /** - * Parse workload from a SLS trace file. - */ - private void startAMFromSLSTrace(String inputTrace) throws IOException { - JsonFactory jsonF = new JsonFactory(); - ObjectMapper mapper = new ObjectMapper(); - - try (Reader input = new InputStreamReader( - new FileInputStream(inputTrace), StandardCharsets.UTF_8)) { - JavaType type = mapper.getTypeFactory(). - constructMapType(Map.class, String.class, String.class); - Iterator> jobIter = mapper.readValues( - jsonF.createParser(input), type); - - while (jobIter.hasNext()) { - try { - Map jsonJob = jobIter.next(); - AMDefinitionSLS amDef = AMDefinitionFactory.createFromSlsTrace(jsonJob, this); - startAMs(amDef); - } catch (Exception e) { - LOG.error("Failed to create an AM: {}", e.getMessage()); - } - } - } - } - - private void startAMs(AMDefinition amDef) { - for (int i = 0; i < amDef.getJobCount(); i++) { - JobDefinition jobDef = JobDefinition.Builder.create() - .withAmDefinition(amDef) - .withDeadline(-1) - .withReservationId(null) - .withParams(null) - .build(); - runNewAM(jobDef); - } - } - - private void startAMs(AMDefinition amDef, ReservationId reservationId, - Map params, long deadline) { - for (int i = 0; i < amDef.getJobCount(); i++) { - JobDefinition jobDef = JobDefinition.Builder.create() - .withAmDefinition(amDef) - .withReservationId(reservationId) - .withParams(params) - .withDeadline(deadline) - .build(); - runNewAM(jobDef); - } - } - - /** - * Parse workload from a rumen trace file. - */ - private void startAMFromRumenTrace(String inputTrace, long baselineTimeMS) - throws IOException { - Configuration conf = new Configuration(); - conf.set("fs.defaultFS", "file:///"); - File fin = new File(inputTrace); - - try (JobTraceReader reader = new JobTraceReader( - new Path(fin.getAbsolutePath()), conf)) { - LoggedJob job = reader.getNext(); - - while (job != null) { - try { - AMDefinitionRumen amDef = - AMDefinitionFactory.createFromRumenTrace(job, baselineTimeMS, - this); - startAMs(amDef); - } catch (Exception e) { - LOG.error("Failed to create an AM", e); - } - job = reader.getNext(); - } - } - } - Resource getDefaultContainerResource() { int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB, SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT); @@ -533,31 +434,6 @@ Resource getDefaultContainerResource() { return Resources.createResource(containerMemory, containerVCores); } - /** - * parse workload information from synth-generator trace files. - */ - private void startAMFromSynthGenerator() throws YarnException, IOException { - Configuration localConf = new Configuration(); - localConf.set("fs.defaultFS", "file:///"); - // if we use the nodeFile this could have been not initialized yet. - if (stjp == null) { - stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0])); - } - - SynthJob job; - // we use stjp, a reference to the job producer instantiated during node - // creation - while ((job = (SynthJob) stjp.getNextJob()) != null) { - ReservationId reservationId = null; - if (job.hasDeadline()) { - reservationId = ReservationId - .newInstance(rm.getStartTime(), AM_ID); - } - AMDefinitionSynth amDef = AMDefinitionFactory.createFromSynth(job, this); - startAMs(amDef, reservationId, job.getParams(), job.getDeadline()); - } - } - void increaseQueueAppNum(String queue) throws YarnException { SchedulerWrapper wrapper = (SchedulerWrapper)rm.getResourceScheduler(); String queueName = wrapper.getRealQueueName(queue); @@ -575,43 +451,12 @@ void increaseQueueAppNum(String queue) throws YarnException { } } - private AMSimulator createAmSimulator(String jobType) { - return (AMSimulator) ReflectionUtils.newInstance( - amClassMap.get(jobType), new Configuration()); - } - - private void runNewAM(JobDefinition jobDef) { - AMDefinition amDef = jobDef.getAmDefinition(); - String oldJobId = amDef.getOldAppId(); - AMSimulator amSim = - createAmSimulator(amDef.getAmType()); - - if (amSim != null) { - int heartbeatInterval = getConf().getInt( - SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS, - SLSConfiguration.AM_HEARTBEAT_INTERVAL_MS_DEFAULT); - boolean isTracked = trackedApps.contains(oldJobId); - - if (oldJobId == null) { - oldJobId = Integer.toString(AM_ID); - } - AM_ID++; - amSim.init(amDef, rm, this, isTracked, runner.getStartTimeMS(), heartbeatInterval, appIdAMSim); - if (jobDef.getReservationId() != null) { - // if we have a ReservationId, delegate reservation creation to - // AMSim (reservation shape is impl specific) - UTCClock clock = new UTCClock(); - amSim.initReservation(jobDef.getReservationId(), jobDef.getDeadline(), - clock.getTime()); - } - runner.schedule(amSim); - maxRuntime = Math.max(maxRuntime, amDef.getJobFinishTime()); - numTasks += amDef.getTaskContainers().size(); - amMap.put(oldJobId, amSim); - } - } - private void printSimulationInfo() { + final int numAMs = amRunner.getNumAMs(); + final int numTasks = amRunner.getNumTasks(); + final long maxRuntime = amRunner.getMaxRuntime(); + Map amMap = amRunner.getAmMap(); + if (printSimulation) { // node LOG.info("------------------------------------"); @@ -663,7 +508,10 @@ public Map getNmMap() { } public static void decreaseRemainingApps() { - remainingApps--; + AMRunner.REMAINING_APPS--; + if (AMRunner.REMAINING_APPS == 0) { + exitSLSRunner(); + } } public static void exitSLSRunner() { @@ -854,4 +702,8 @@ public ResourceManager getRm() { public SynthTraceJobProducer getStjp() { return stjp; } + + public AMSimulator getAMSimulatorByAppId(ApplicationId appId) { + return amRunner.getAMSimulator(appId); + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java index d28407669cbe3..e46dea521c551 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java @@ -44,15 +44,11 @@ public class MockAMLauncher extends ApplicationMasterLauncher private static final Logger LOG = LoggerFactory.getLogger( MockAMLauncher.class); - private Map appIdAMSim; + private SLSRunner slsRunner; - SLSRunner se; - - public MockAMLauncher(SLSRunner se, RMContext rmContext, - Map appIdAMSim) { + public MockAMLauncher(SLSRunner slsRunner, RMContext rmContext) { super(rmContext); - this.appIdAMSim = appIdAMSim; - this.se = se; + this.slsRunner = slsRunner; } @Override @@ -79,12 +75,11 @@ private void setupAMRMToken(RMAppAttempt appAttempt) { } @Override - @SuppressWarnings("unchecked") public void handle(AMLauncherEvent event) { ApplicationId appId = event.getAppAttempt().getAppAttemptId().getApplicationId(); // find AMSimulator - AMSimulator ams = appIdAMSim.get(appId); + AMSimulator ams = slsRunner.getAMSimulatorByAppId(appId); if (ams == null) { throw new YarnRuntimeException( "Didn't find any AMSimulator for applicationId=" + appId); @@ -103,7 +98,7 @@ public void handle(AMLauncherEvent event) { event.getAppAttempt().getMasterContainer()); LOG.info("Notify AM launcher launched:" + amContainer.getId()); - se.getNmMap().get(amContainer.getNodeId()) + slsRunner.getNmMap().get(amContainer.getNodeId()) .addNewContainer(amContainer, -1, appId); ams.getRanNodes().add(amContainer.getNodeId()); return; @@ -111,7 +106,7 @@ public void handle(AMLauncherEvent event) { throw new YarnRuntimeException(e); } case CLEANUP: - se.getNmMap().get(amContainer.getNodeId()) + slsRunner.getNmMap().get(amContainer.getNodeId()) .cleanupContainer(amContainer.getId()); break; default: From da09d68056d4e6a9490ddc6d9ae816b65217e117 Mon Sep 17 00:00:00 2001 From: Tamas Domok Date: Thu, 27 Jan 2022 09:57:52 +0100 Subject: [PATCH 066/145] YARN-11069. Dynamic Queue ACL handling in Legacy and Flexible Auto Created Queues. Contributed by Tamas Domok --- .../yarn/security/PrivilegedEntity.java | 5 + .../server/resourcemanager/RMAppManager.java | 70 +- .../scheduler/capacity/AbstractCSQueue.java | 8 +- .../scheduler/capacity/AbstractLeafQueue.java | 15 + .../CapacitySchedulerConfiguration.java | 84 + .../CapacitySchedulerQueueManager.java | 43 +- .../scheduler/capacity/ParentQueue.java | 12 + .../webapp/dao/CapacitySchedulerInfo.java | 2 +- .../dao/CapacitySchedulerQueueInfo.java | 10 +- .../resourcemanager/TestAppManager.java | 701 ++++- ...WebServicesCapacitySchedDynamicConfig.java | 51 +- ...sponse-AbsoluteModeLegacyAutoCreation.json | 24 +- .../webapp/scheduler-response-WeightMode.json | 531 +++- ...WeightModeWithAutoCreatedQueues-After.json | 2397 +++++++++++++++-- ...eightModeWithAutoCreatedQueues-Before.json | 449 ++- .../src/site/markdown/CapacityScheduler.md | 8 +- 16 files changed, 4045 insertions(+), 365 deletions(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java index 580bdf490a3f2..3539c8f0bb595 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/PrivilegedEntity.java @@ -45,6 +45,11 @@ public PrivilegedEntity(EntityType type, String name) { this.name = name; } + public PrivilegedEntity(String name) { + this.type = EntityType.QUEUE; + this.name = name; + } + public EntityType getType() { return type; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java index 5bedef4baa387..f847152c47d76 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java @@ -28,6 +28,10 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.security.ConfiguredYarnAuthorizer; +import org.apache.hadoop.yarn.security.Permission; +import org.apache.hadoop.yarn.security.PrivilegedEntity; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -473,32 +477,33 @@ private RMAppImpl createAndPopulateNewRMApp( if (scheduler instanceof CapacityScheduler) { String queueName = placementContext == null ? submissionContext.getQueue() : placementContext.getFullQueuePath(); - - String appName = submissionContext.getApplicationName(); - CSQueue csqueue = ((CapacityScheduler) scheduler).getQueue(queueName); - - if (csqueue == null && placementContext != null) { - //could be an auto created queue through queue mapping. Validate - // parent queue exists and has valid acls - String parentQueueName = placementContext.getParentQueue(); - csqueue = ((CapacityScheduler) scheduler).getQueue(parentQueueName); + CapacityScheduler cs = (CapacityScheduler) scheduler; + CSQueue csqueue = cs.getQueue(queueName); + PrivilegedEntity privilegedEntity = new PrivilegedEntity( + csqueue == null ? queueName : csqueue.getQueuePath()); + + YarnAuthorizationProvider dynamicAuthorizer = null; + if (csqueue == null) { + List permissions = + cs.getCapacitySchedulerQueueManager().getPermissionsForDynamicQueue( + new QueuePath(queueName), cs.getConfiguration()); + if (!permissions.isEmpty()) { + dynamicAuthorizer = new ConfiguredYarnAuthorizer(); + dynamicAuthorizer.setPermission(permissions, userUgi); + } } - if (csqueue != null - && !authorizer.checkPermission( - new AccessRequest(csqueue.getPrivilegedEntity(), userUgi, - SchedulerUtils.toAccessType(QueueACL.SUBMIT_APPLICATIONS), - applicationId.toString(), appName, Server.getRemoteAddress(), - null)) - && !authorizer.checkPermission( - new AccessRequest(csqueue.getPrivilegedEntity(), userUgi, - SchedulerUtils.toAccessType(QueueACL.ADMINISTER_QUEUE), - applicationId.toString(), appName, Server.getRemoteAddress(), - null))) { - throw RPCUtil.getRemoteException(new AccessControlException( - "User " + user + " does not have permission to submit " - + applicationId + " to queue " - + submissionContext.getQueue())); + if (csqueue != null || dynamicAuthorizer != null) { + String appName = submissionContext.getApplicationName(); + if (!checkPermission(createAccessRequest(privilegedEntity, userUgi, applicationId, + appName, QueueACL.SUBMIT_APPLICATIONS), dynamicAuthorizer) && + !checkPermission(createAccessRequest(privilegedEntity, userUgi, applicationId, + appName, QueueACL.ADMINISTER_QUEUE), dynamicAuthorizer)) { + throw RPCUtil.getRemoteException(new AccessControlException( + "User " + user + " does not have permission to submit " + + applicationId + " to queue " + + submissionContext.getQueue())); + } } } if (scheduler instanceof FairScheduler) { @@ -572,6 +577,23 @@ private RMAppImpl createAndPopulateNewRMApp( return application; } + private boolean checkPermission(AccessRequest accessRequest, + YarnAuthorizationProvider dynamicAuthorizer) { + return authorizer.checkPermission(accessRequest) || + (dynamicAuthorizer != null && dynamicAuthorizer.checkPermission(accessRequest)); + } + + private static AccessRequest createAccessRequest(PrivilegedEntity privilegedEntity, + UserGroupInformation userUgi, + ApplicationId applicationId, + String appName, + QueueACL submitApplications) { + return new AccessRequest(privilegedEntity, userUgi, + SchedulerUtils.toAccessType(submitApplications), + applicationId.toString(), appName, Server.getRemoteAddress(), + null); + } + private List validateAndCreateResourceRequest( ApplicationSubmissionContext submissionContext, boolean isRecovery) throws InvalidResourceRequestException { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index 87fa9dac34a8d..6caca65ffb0e2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -341,8 +341,11 @@ protected void setupQueueConfigs(Resource clusterResource) throws writeLock.lock(); try { CapacitySchedulerConfiguration configuration = queueContext.getConfiguration(); + this.acls = configuration.getAcls(getQueuePath()); + if (isDynamicQueue() || this instanceof AbstractAutoCreatedLeafQueue) { setDynamicQueueProperties(); + setDynamicQueueACLProperties(); } // Collect and set the Node label configuration @@ -369,8 +372,6 @@ protected void setupQueueConfigs(Resource clusterResource) throws authorizer = YarnAuthorizationProvider.getInstance(configuration); - this.acls = configuration.getAcls(getQueuePath()); - this.userWeights = getUserWeightsFromHierarchy(); this.reservationsContinueLooking = @@ -426,6 +427,9 @@ protected void setDynamicQueueProperties() { } } + protected void setDynamicQueueACLProperties() { + } + private UserWeights getUserWeightsFromHierarchy() { UserWeights unionInheritedWeights = UserWeights.createEmpty(); CSQueue parentQ = parent; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java index e194800cd1b86..3d6863335de89 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java @@ -86,6 +86,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedLeafQueue; + public class AbstractLeafQueue extends AbstractCSQueue { private static final Logger LOG = LoggerFactory.getLogger(AbstractLeafQueue.class); @@ -1697,6 +1699,19 @@ protected void setDynamicQueueProperties() { super.setDynamicQueueProperties(); } + @Override + protected void setDynamicQueueACLProperties() { + super.setDynamicQueueACLProperties(); + + if (parent instanceof AbstractManagedParentQueue) { + acls.putAll(queueContext.getConfiguration().getACLsForLegacyAutoCreatedLeafQueue( + parent.getQueuePath())); + } else if (parent instanceof ParentQueue) { + acls.putAll(getACLsForFlexibleAutoCreatedLeafQueue( + ((ParentQueue) parent).getAutoCreatedQueueTemplate())); + } + } + private void updateSchedulerHealthForCompletedContainer( RMContainer rmContainer, ContainerStatus containerStatus) { // Update SchedulerHealth for released / preempted container diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java index a9e8423ab2bc5..51616da14b60b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerConfiguration.java @@ -941,6 +941,90 @@ private static String getAclKey(AccessType acl) { return "acl_" + StringUtils.toLowerCase(acl.toString()); } + /** + * Creates a mapping of queue ACLs for a Legacy Auto Created Leaf Queue. + * + * @param parentQueuePath the parent's queue path + * @return A mapping of the queue ACLs. + */ + public Map getACLsForLegacyAutoCreatedLeafQueue( + String parentQueuePath) { + final String prefix = + getQueuePrefix(getAutoCreatedQueueTemplateConfPrefix( + parentQueuePath)); + + Map properties = new HashMap<>(); + for (QueueACL acl : QueueACL.values()) { + final String key = getAclKey(acl); + final String value = get(prefix + key); + if (value != null) { + properties.put(key, get(prefix + key)); + } + } + return getACLsFromProperties(properties); + } + + /** + * Creates a mapping of queue ACLs for a Flexible Auto Created Parent Queue. + * The .parent-template is preferred to .template ACLs. + * + * @param aqc The AQC templates to use. + * @return A mapping of the queue ACLs. + */ + public static Map getACLsForFlexibleAutoCreatedParentQueue( + AutoCreatedQueueTemplate aqc) { + return getACLsFromProperties(aqc.getParentOnlyProperties(), + aqc.getTemplateProperties()); + } + + /** + * Creates a mapping of queue ACLs for a Flexible Auto Created Leaf Queue. + * The .leaf-template is preferred to .template ACLs. + * + * @param aqc The AQC templates to use. + * @return A mapping of the queue ACLs. + */ + public static Map getACLsForFlexibleAutoCreatedLeafQueue( + AutoCreatedQueueTemplate aqc) { + return getACLsFromProperties(aqc.getLeafOnlyProperties(), + aqc.getTemplateProperties()); + } + + /** + * Transforms the string ACL properties to AccessType and AccessControlList mapping. + * + * @param properties The ACL properties. + * @return A mapping of the queue ACLs. + */ + private static Map getACLsFromProperties( + Map properties) { + return getACLsFromProperties(properties, new HashMap<>()); + } + + /** + * Transforms the string ACL properties to AccessType and AccessControlList mapping. + * + * @param properties The ACL properties. + * @param fallbackProperties The fallback properties to use. + * @return A mapping of the queue ACLs. + */ + private static Map getACLsFromProperties( + Map properties, Map fallbackProperties) { + Map acls = new HashMap<>(); + for (QueueACL acl : QueueACL.values()) { + String aclStr = properties.get(getAclKey(acl)); + if (aclStr == null) { + aclStr = fallbackProperties.get(getAclKey(acl)); + if (aclStr == null) { + aclStr = NONE_ACL; + } + } + acls.put(SchedulerUtils.toAccessType(acl), + new AccessControlList(aclStr)); + } + return acls; + } + @Override public Map getReservationAcls(String queue) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java index 29e5de09d4bd8..ba6849cb7807b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueManager.java @@ -23,13 +23,13 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.yarn.security.PrivilegedEntity; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -52,6 +52,9 @@ import org.apache.hadoop.classification.VisibleForTesting; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedLeafQueue; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedParentQueue; + /** * * Context of the Queues in Capacity Scheduler. @@ -596,6 +599,44 @@ public List determineMissingParents( return parentsToCreate; } + public List getPermissionsForDynamicQueue( + QueuePath queuePath, + CapacitySchedulerConfiguration csConf) { + List permissions = new ArrayList<>(); + + try { + PrivilegedEntity privilegedEntity = new PrivilegedEntity(queuePath.getFullPath()); + + CSQueue parentQueue = getQueueByFullName(queuePath.getParent()); + if (parentQueue == null) { + for (String missingParent : determineMissingParents(queuePath)) { + String parentOfMissingParent = new QueuePath(missingParent).getParent(); + permissions.add(new Permission(new PrivilegedEntity(missingParent), + getACLsForFlexibleAutoCreatedParentQueue( + new AutoCreatedQueueTemplate(csConf, + new QueuePath(parentOfMissingParent))))); + } + } + + if (parentQueue instanceof AbstractManagedParentQueue) { + // An AbstractManagedParentQueue must have been found for Legacy AQC + permissions.add(new Permission(privilegedEntity, + csConf.getACLsForLegacyAutoCreatedLeafQueue(queuePath.getParent()))); + } else { + // Every other case must be a Flexible Leaf Queue + permissions.add(new Permission(privilegedEntity, + getACLsForFlexibleAutoCreatedLeafQueue( + new AutoCreatedQueueTemplate(csConf, new QueuePath(queuePath.getParent()))))); + } + + } catch (SchedulerDynamicEditException e) { + LOG.debug("Could not determine missing parents for queue {} reason {}", + queuePath.getFullPath(), e.getMessage()); + } + + return permissions; + } + /** * Get {@code ConfiguredNodeLabels} which contains the configured node labels * for all queues. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java index 671ed21c4312f..0949d512a79f1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/ParentQueue.java @@ -78,6 +78,8 @@ import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.Resources; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getACLsForFlexibleAutoCreatedParentQueue; + @Private @Evolving public class ParentQueue extends AbstractCSQueue { @@ -188,6 +190,16 @@ protected void setupQueueConfigs(Resource clusterResource) } } + @Override + protected void setDynamicQueueACLProperties() { + super.setDynamicQueueACLProperties(); + + if (parent instanceof ParentQueue) { + acls.putAll(getACLsForFlexibleAutoCreatedParentQueue( + ((ParentQueue) parent).getAutoCreatedQueueTemplate())); + } + } + private static float PRECISION = 0.0005f; // 0.05% precision // Check weight configuration, throw exception when configuration is invalid diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java index 3d410ecddafad..83394b15a37d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerInfo.java @@ -97,7 +97,7 @@ public CapacitySchedulerInfo(CSQueue parent, CapacityScheduler cs) { CapacitySchedulerConfiguration conf = cs.getConfiguration(); queueAcls = new QueueAclsInfo(); - queueAcls.addAll(getSortedQueueAclInfoList(queueName, conf)); + queueAcls.addAll(getSortedQueueAclInfoList(parent, queueName, conf)); queuePriority = parent.getPriority().getPriority(); if (parent instanceof ParentQueue) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java index 78b53922aafe8..bda69c1d0543e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerQueueInfo.java @@ -157,7 +157,7 @@ public class CapacitySchedulerQueueInfo { CapacitySchedulerConfiguration conf = cs.getConfiguration(); queueAcls = new QueueAclsInfo(); - queueAcls.addAll(getSortedQueueAclInfoList(queuePath, conf)); + queueAcls.addAll(getSortedQueueAclInfoList(q, queuePath, conf)); queuePriority = q.getPriority().getPriority(); if (q instanceof ParentQueue) { @@ -183,11 +183,11 @@ public class CapacitySchedulerQueueInfo { leafQueueTemplate = new LeafQueueTemplateInfo(conf, queuePath); } - public static ArrayList getSortedQueueAclInfoList(String queuePath, - CapacitySchedulerConfiguration conf) { + public static ArrayList getSortedQueueAclInfoList( + CSQueue queue, String queuePath, CapacitySchedulerConfiguration conf) { ArrayList queueAclsInfo = new ArrayList<>(); - for (Map.Entry e : conf - .getAcls(queuePath).entrySet()) { + for (Map.Entry e : + ((AbstractCSQueue) queue).getACLs().entrySet()) { QueueAclInfo queueAcl = new QueueAclInfo(e.getKey().toString(), e.getValue().getAclString()); queueAclsInfo.add(queueAcl); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index f9ecc85b0a567..cc5e8af176285 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -68,9 +68,13 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedLeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ManagedParentQueue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.QueuePath; import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.resourcemanager.timelineservice.RMTimelineCollectorManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; @@ -108,7 +112,11 @@ import java.util.TreeSet; import java.util.concurrent.ConcurrentMap; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_LEAF_TEMPLATE_PREFIX; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_PARENT_TEMPLATE_PREFIX; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.AutoCreatedQueueTemplate.AUTO_QUEUE_TEMPLATE_PREFIX; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.getQueuePrefix; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; @@ -141,6 +149,7 @@ public class TestAppManager extends AppManagerTestBase{ private ResourceScheduler scheduler; private static final String USER_ID_PREFIX = "userid="; + private static final String ROOT_PARENT = PREFIX + "root.parent."; public synchronized RMAppEventType getAppEventType() { return appEventType; @@ -307,11 +316,7 @@ private TestRMAppManager createAppManager(RMContext context, Configuration confi @Test public void testQueueSubmitWithACLsEnabledWithQueueMapping() throws YarnException { - YarnConfiguration conf = new YarnConfiguration(new Configuration(false)); - conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true"); - conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, - ResourceScheduler.class); - + YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration(conf, false); csConf.set(PREFIX + "root.queues", "default,test"); @@ -331,35 +336,29 @@ public void testQueueSubmitWithACLsEnabledWithQueueMapping() csConf.set(PREFIX + "root.test.acl_submit_applications", "test"); csConf.set(PREFIX + "root.test.acl_administer_queue", "test"); - asContext.setQueue("oldQueue"); - MockRM newMockRM = new MockRM(csConf); RMContext newMockRMContext = newMockRM.getRMContext(); - newMockRMContext.setQueuePlacementManager(createMockPlacementManager("test", "test", null)); + newMockRMContext.setQueuePlacementManager( + createMockPlacementManager("test", "root.test", null)); TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); - newAppMonitor.submitApplication(asContext, "test"); - RMApp app = newMockRMContext.getRMApps().get(appId); - Assert.assertNotNull("app should not be null", app); - Assert.assertEquals("the queue should be placed on 'test' queue", "test", app.getQueue()); - - try { - asContext.setApplicationId(appId = MockApps.newAppID(2)); - newAppMonitor.submitApplication(asContext, "test1"); - Assert.fail("should fail since test1 does not have permission to submit to queue"); - } catch(YarnException e) { - assertTrue(e.getCause() instanceof AccessControlException); - } + ApplicationSubmissionContext submission = createAppSubmissionContext(MockApps.newAppID(1)); + submission.setQueue("oldQueue"); + verifyAppSubmission(submission, + newAppMonitor, + newMockRMContext, + "test", + "root.test"); + + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(2)), + "test1"); } @Test - public void testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue() + public void testQueueSubmitWithACLsEnabledWithQueueMappingForLegacyAutoCreatedQueue() throws IOException, YarnException { - YarnConfiguration conf = new YarnConfiguration(); - conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true"); - conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, - ResourceScheduler.class); - + YarnConfiguration conf = createYarnACLEnabledConfiguration(); CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( conf, false); csConf.set(PREFIX + "root.queues", "default,managedparent"); @@ -383,8 +382,6 @@ public void testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue() csConf.setAutoCreatedLeafQueueConfigCapacity("root.managedparent", 30f); csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.managedparent", 100f); - asContext.setQueue("oldQueue"); - MockRM newMockRM = new MockRM(csConf); CapacityScheduler cs = ((CapacityScheduler) newMockRM.getResourceScheduler()); @@ -395,25 +392,653 @@ public void testQueueSubmitWithACLsEnabledWithQueueMappingForAutoCreatedQueue() RMContext newMockRMContext = newMockRM.getRMContext(); newMockRMContext.setQueuePlacementManager(createMockPlacementManager( - "user1|user2", "user1", "managedparent")); + "user1|user2", "user1", "root.managedparent")); TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); - newAppMonitor.submitApplication(asContext, "user1"); - RMApp app = newMockRMContext.getRMApps().get(appId); - Assert.assertNotNull("app should not be null", app); - Assert.assertEquals("the queue should be placed on 'managedparent.user1' queue", - "managedparent.user1", - app.getQueue()); + ApplicationSubmissionContext submission = createAppSubmissionContext(MockApps.newAppID(1)); + submission.setQueue("oldQueue"); + verifyAppSubmission(submission, + newAppMonitor, + newMockRMContext, + "user1", + "root.managedparent.user1"); + + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(2)), + "user2"); + } + + @Test + public void testLegacyAutoCreatedQueuesWithACLTemplates() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( + conf, false); + csConf.set(PREFIX + "root.queues", "parent"); + csConf.set(PREFIX + "root.acl_submit_applications", " "); + csConf.set(PREFIX + "root.acl_administer_queue", " "); + + csConf.setCapacity("root.parent", 100.0f); + csConf.set(PREFIX + "root.parent.acl_administer_queue", "user1,user4"); + csConf.set(PREFIX + "root.parent.acl_submit_applications", "user1,user4"); + + csConf.setAutoCreateChildQueueEnabled("root.parent", true); + csConf.setAutoCreatedLeafQueueConfigCapacity("root.parent", 50f); + csConf.setAutoCreatedLeafQueueConfigMaxCapacity("root.parent", 100f); + csConf.set(getQueuePrefix(csConf.getAutoCreatedQueueTemplateConfPrefix("root.parent")) + + "acl_administer_queue", "user2,user4"); + csConf.set(getQueuePrefix(csConf.getAutoCreatedQueueTemplateConfPrefix("root.parent")) + + "acl_submit_applications", "user2,user4"); + + MockRM newMockRM = new MockRM(csConf); + + RMContext newMockRMContext = newMockRM.getRMContext(); + TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); + + // user1 has permission on root.parent so a queue would be created + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user1", "user1", "root.parent")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)), + newAppMonitor, + newMockRMContext, + "user1", + "root.parent.user1"); + + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user1|user2|user3|user4", "user2", "root.parent")); + + // user2 has permission (due to ACL templates) + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)), + newAppMonitor, + newMockRMContext, + "user2", + "root.parent.user2"); + + // user3 doesn't have permission + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(3)), + "user3"); + + // user4 has permission on root.parent + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(4)), + newAppMonitor, + newMockRMContext, + "user4", + "root.parent.user2"); + + // create the root.parent.user2 manually + CapacityScheduler cs = + ((CapacityScheduler) newMockRM.getResourceScheduler()); + cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2")); + AutoCreatedLeafQueue autoCreatedLeafQueue = (AutoCreatedLeafQueue) cs.getQueue("user2"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + ManagedParentQueue parentQueue = (ManagedParentQueue) cs.getQueue("parent"); + assertEquals(parentQueue, autoCreatedLeafQueue.getParent()); + // reinitialize to load the ACLs for the queue + cs.reinitialize(csConf, newMockRMContext); + + // template ACLs do work after reinitialize + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)), + newAppMonitor, + newMockRMContext, + "user2", + "root.parent.user2"); + + // user3 doesn't have permission for root.parent.user2 queue + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(6)), + "user3"); + + // user1 doesn't have permission for root.parent.user2 queue, but it has for root.parent + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(7)), + newAppMonitor, + newMockRMContext, + "user1", + "root.parent.user2"); + } + + @Test + public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicParentQueue() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); + + csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue", + "user2"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications", + "user2"); + + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + "user3"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications", + "user3"); + + MockRM newMockRM = new MockRM(csConf); + + RMContext newMockRMContext = newMockRM.getRMContext(); + TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); + + // user1 has permission on root.parent so a queue would be created + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user1", "user1", "root.parent")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)), + newAppMonitor, + newMockRMContext, + "user1", + "root.parent.user1"); + + // user2 doesn't have permission to create a dynamic leaf queue (parent only template) + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user2", "user2", "root.parent")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(2)), + "user2"); + + // user3 has permission on root.parent.user2.user3 due to ACL templates + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.parent.user2")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(3)), + newAppMonitor, + newMockRMContext, + "user3", + "root.parent.user2.user3"); + + // user4 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user4", "user4", "root.parent.user2")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(4)), + "user4"); + + // create the root.parent.user2.user3 manually + CapacityScheduler cs = + ((CapacityScheduler) newMockRM.getResourceScheduler()); + cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3")); + + ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent"); + assertEquals(parentQueue, autoCreatedParentQueue.getParent()); + + LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); + + // reinitialize to load the ACLs for the queue + cs.reinitialize(csConf, newMockRMContext); + + // template ACLs do work after reinitialize + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.parent.user2")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)), + newAppMonitor, + newMockRMContext, + "user3", + "root.parent.user2.user3"); + + // user4 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user4", "user4", "root.parent.user2")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(6)), + "user4"); + } + + @Test + public void testFlexibleAutoCreatedQueuesWithMixedCommonLeafACLTemplatesAndDynamicParentQueue() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); + + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + "user2"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + "user2"); + + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + "user3"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications", + "user3"); + + testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario(conf, csConf); + } + + @Test + public void testFlexibleAutoCreatedQueuesWithMixedCommonCommonACLTemplatesAndDynamicParentQueue() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); + + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + "user2"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + "user2"); + + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + "user3"); + csConf.set(ROOT_PARENT + "*." + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + "user3"); + + testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario(conf, csConf); + } + + private void testFlexibleAQCDWithMixedTemplatesDynamicParentACLScenario( + YarnConfiguration conf, CapacitySchedulerConfiguration csConf) + throws YarnException, IOException { + MockRM newMockRM = new MockRM(csConf); + + RMContext newMockRMContext = newMockRM.getRMContext(); + TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); + + // user1 has permission on root.parent so a queue would be created + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user1", "user1", "root.parent")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)), + newAppMonitor, + newMockRMContext, + "user1", + "root.parent.user1"); + + // user2 has permission on root.parent a dynamic leaf queue would be created + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user2", "user2", "root.parent")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)), + newAppMonitor, + newMockRMContext, + "user2", + "root.parent.user2"); + + // user3 has permission on root.parent.user2.user3 a dynamic parent and leaf queue + // would be created + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.parent.user2")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(3)), + newAppMonitor, + newMockRMContext, + "user3", + "root.parent.user2.user3"); + + // user4 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user4", "user4", "root.parent.user2")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(4)), + "user4"); + + // create the root.parent.user2.user3 manually + CapacityScheduler cs = + ((CapacityScheduler) newMockRM.getResourceScheduler()); + cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2.user3")); + + ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + ParentQueue parentQueue = (ParentQueue) cs.getQueue("parent"); + assertEquals(parentQueue, autoCreatedParentQueue.getParent()); + + LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); + + // reinitialize to load the ACLs for the queue + cs.reinitialize(csConf, newMockRMContext); + + // template ACLs do work after reinitialize + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.parent.user2")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)), + newAppMonitor, + newMockRMContext, + "user3", + "root.parent.user2.user3"); + + // user4 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user4", "user4", "root.parent.user2")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(6)), + "user4"); + } + + @Test + public void testFlexibleAutoCreatedQueuesWithACLTemplatesALeafOnly() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); + + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_administer_queue", + "user2"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_TEMPLATE_PREFIX + "acl_submit_applications", + "user2"); + + testFlexibleAQCLeafOnly(conf, csConf); + } + + @Test + public void testFlexibleAutoCreatedQueuesWithSpecialisedACLTemplatesALeafOnly() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + CapacitySchedulerConfiguration csConf = createFlexibleAQCBaseACLConfiguration(conf); + + csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + "user2"); + csConf.set(ROOT_PARENT + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_submit_applications", + "user2"); + + testFlexibleAQCLeafOnly(conf, csConf); + } + + private void testFlexibleAQCLeafOnly( + YarnConfiguration conf, + CapacitySchedulerConfiguration csConf) + throws YarnException, IOException { + MockRM newMockRM = new MockRM(csConf); + RMContext newMockRMContext = newMockRM.getRMContext(); + TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); + + // user1 has permission on root.parent so a queue would be created + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user1", "user1", "root.parent")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)), + newAppMonitor, + newMockRMContext, + "user1", + "root.parent.user1"); + + // user2 has permission on root.parent.user2 due to ACL templates + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user2", "user2", "root.parent")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)), + newAppMonitor, + newMockRMContext, + "user2", + "root.parent.user2"); + + // user3 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.parent")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(3)), + "user3"); + + // create the root.parent.user2 manually + CapacityScheduler cs = + ((CapacityScheduler) newMockRM.getResourceScheduler()); + cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.parent.user2")); + ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("parent"); + LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user2"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); + + // reinitialize to load the ACLs for the queue + cs.reinitialize(csConf, newMockRMContext); + + // template ACLs do work after reinitialize + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user2", "user2", "root.parent")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(4)), + newAppMonitor, + newMockRMContext, + "user2", + "root.parent.user2"); + + // user3 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.parent")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(5)), + "user3"); + } + + @Test + public void testFlexibleAutoCreatedQueuesWithSpecializedACLTemplatesAndDynamicRootParentQueue() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + + CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( + conf, false); + csConf.set(PREFIX + "root.queues", ""); + csConf.set(PREFIX + "root.acl_submit_applications", "user1"); + csConf.set(PREFIX + "root.acl_administer_queue", "admin1"); + + csConf.setAutoQueueCreationV2Enabled("root", true); + + csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue", + "user2"); + csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications", + "user2"); + + csConf.set(PREFIX + "root." + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(PREFIX + "root." + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "acl_administer_queue", + "user3"); + csConf.set(PREFIX + "root." + "*." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + + "acl_submit_applications", + "user3"); + + MockRM newMockRM = new MockRM(csConf); + + RMContext newMockRMContext = newMockRM.getRMContext(); + TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); + + // user1 has permission on root so a queue would be created + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user1", "user1", "root")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)), + newAppMonitor, + newMockRMContext, + "user1", + "root.user1"); + + // user2 doesn't have permission to create a dynamic leaf queue (parent only template) + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user2", "user2", "root")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(2)), + "user2"); + + // user3 has permission on root.user2.user3 due to ACL templates + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.user2")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(3)), + newAppMonitor, + newMockRMContext, + "user3", + "root.user2.user3"); + + // user4 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user4", "user4", "root.user2")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(4)), + "user4"); + + // create the root.user2.user3 manually + CapacityScheduler cs = + ((CapacityScheduler) newMockRM.getResourceScheduler()); + cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3")); + + ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + ParentQueue parentQueue = (ParentQueue) cs.getQueue("root"); + assertEquals(parentQueue, autoCreatedParentQueue.getParent()); + + LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("user3"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertEquals(autoCreatedParentQueue, autoCreatedLeafQueue.getParent()); + + // reinitialize to load the ACLs for the queue + cs.reinitialize(csConf, newMockRMContext); + + // template ACLs do work after reinitialize + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "user3", "root.user2")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(5)), + newAppMonitor, + newMockRMContext, + "user3", + "root.user2.user3"); + + // user4 doesn't have permission + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user4", "user4", "root.user2")); + verifyAppSubmissionFailure(newAppMonitor, + createAppSubmissionContext(MockApps.newAppID(6)), + "user4"); + } + + @Test + public void testFlexibleAutoCreatedQueuesMultiLevelDynamicParentACL() + throws IOException, YarnException { + YarnConfiguration conf = createYarnACLEnabledConfiguration(); + + CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( + conf, false); + csConf.set(PREFIX + "root.queues", ""); + csConf.set(PREFIX + "root.acl_submit_applications", "user1"); + csConf.set(PREFIX + "root.acl_administer_queue", "admin1"); + + csConf.setAutoQueueCreationV2Enabled("root", true); + + csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_administer_queue", + "user2"); + csConf.set(PREFIX + "root." + AUTO_QUEUE_PARENT_TEMPLATE_PREFIX + "acl_submit_applications", + "user2"); + + csConf.set(PREFIX + "root." + "user2.user3." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + "capacity", + "1w"); + csConf.set(PREFIX + "root." + "user2.user3." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + + "acl_administer_queue", + "user3"); + csConf.set(PREFIX + "root." + "user2.user3." + AUTO_QUEUE_LEAF_TEMPLATE_PREFIX + + "acl_submit_applications", + "user3"); + csConf.setMaximumAutoCreatedQueueDepth(4); + + MockRM newMockRM = new MockRM(csConf); + + RMContext newMockRMContext = newMockRM.getRMContext(); + TestRMAppManager newAppMonitor = createAppManager(newMockRMContext, conf); + + // user3 has permission on root.user2.user3.queue due to ACL templates + newMockRMContext.setQueuePlacementManager(createMockPlacementManager( + "user3", "queue", "root.user2.user3")); + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(1)), + newAppMonitor, + newMockRMContext, + "user3", + "root.user2.user3.queue"); + + // create the root.user2.user3.queue manually + CapacityScheduler cs = + ((CapacityScheduler) newMockRM.getResourceScheduler()); + cs.getCapacitySchedulerQueueManager().createQueue(new QueuePath("root.user2.user3.queue")); + + ParentQueue autoCreatedParentQueue = (ParentQueue) cs.getQueue("user2"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue); + ParentQueue parentQueue = (ParentQueue) cs.getQueue("root"); + assertEquals(parentQueue, autoCreatedParentQueue.getParent()); + + ParentQueue autoCreatedParentQueue2 = (ParentQueue) cs.getQueue("user3"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedParentQueue2); + assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent()); + + LeafQueue autoCreatedLeafQueue = (LeafQueue) cs.getQueue("queue"); + Assert.assertNotNull("Auto Creation of Queue failed", autoCreatedLeafQueue); + assertEquals(autoCreatedParentQueue, autoCreatedParentQueue2.getParent()); + + // reinitialize to load the ACLs for the queue + cs.reinitialize(csConf, newMockRMContext); + + // template ACLs do work after reinitialize + verifyAppSubmission(createAppSubmissionContext(MockApps.newAppID(2)), + newAppMonitor, + newMockRMContext, + "user3", + "root.user2.user3.queue"); + } + + private YarnConfiguration createYarnACLEnabledConfiguration() { + YarnConfiguration conf = new YarnConfiguration(new Configuration(false)); + conf.set(YarnConfiguration.YARN_ACL_ENABLE, "true"); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + return conf; + } + + private CapacitySchedulerConfiguration createFlexibleAQCBaseACLConfiguration( + YarnConfiguration conf) { + CapacitySchedulerConfiguration csConf = new CapacitySchedulerConfiguration( + conf, false); + csConf.set(PREFIX + "root.queues", "parent"); + csConf.set(PREFIX + "root.acl_submit_applications", " "); + csConf.set(PREFIX + "root.acl_administer_queue", " "); + + csConf.setCapacity("root.parent", "1w"); + csConf.set(PREFIX + "root.parent.acl_administer_queue", "user1"); + csConf.set(PREFIX + "root.parent.acl_submit_applications", "user1"); + + csConf.setAutoQueueCreationV2Enabled("root.parent", true); + return csConf; + } + + private static void verifyAppSubmissionFailure(TestRMAppManager appManager, + ApplicationSubmissionContext submission, + String user) { try { - asContext.setApplicationId(appId = MockApps.newAppID(2)); - newAppMonitor.submitApplication(asContext, "user2"); - Assert.fail("should fail since user2 does not have permission to submit to queue"); + appManager.submitApplication(submission, user); + Assert.fail( + String.format("should fail since %s does not have permission to submit to queue", user)); } catch (YarnException e) { assertTrue(e.getCause() instanceof AccessControlException); } } + private static void verifyAppSubmission(ApplicationSubmissionContext submission, + TestRMAppManager appManager, + RMContext rmContext, + String user, + String expectedQueue) throws YarnException { + appManager.submitApplication(submission, user); + RMApp app = rmContext.getRMApps().get(submission.getApplicationId()); + Assert.assertNotNull("app should not be null", app); + Assert.assertEquals(String.format("the queue should be placed on '%s' queue", expectedQueue), + expectedQueue, + app.getQueue()); + } + + private static ApplicationSubmissionContext createAppSubmissionContext(ApplicationId id) { + RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null); + ApplicationSubmissionContext appSubmission = + recordFactory.newRecordInstance(ApplicationSubmissionContext.class); + appSubmission.setApplicationId(id); + appSubmission.setAMContainerSpec(mockContainerLaunchContext(recordFactory)); + appSubmission.setResource(mockResource()); + appSubmission.setPriority(Priority.newInstance(0)); + appSubmission.setQueue("default"); + return appSubmission; + } + @After public void tearDown() { setAppEventType(RMAppEventType.KILL); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java index 4cc50c4d57c1b..f2336e79d2ace 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySchedDynamicConfig.java @@ -162,14 +162,9 @@ public void testSchedulerResponseWeightModeWithAutoCreatedQueues() createQueue("root.auto2"); createQueue("root.auto3"); createQueue("root.autoParent1.auto4"); + createQueue("root.autoParent2.auto5"); + createQueue("root.parent.autoParent3.auto6"); - /* - * root default test1 test2 autoParent1 auto1 auto2 auto3 auto4 - * weight: 1 10 4 6 1 1 1 1 1 - * normalizedWeight: 1 0.41 0.16 0.25 1 0.04 0.04 0.04 0.04 - * autoCreationEligibility: flexible off off off flexible off off off off - * queueType: parent leaf leaf leaf parent leaf leaf leaf leaf - */ assertJsonResponse(sendRequest(), "webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json"); } @@ -227,6 +222,10 @@ public static Configuration createAbsoluteConfigLegacyAutoCreation() { conf.put("yarn.scheduler.capacity.root.managed.state", "RUNNING"); conf.put("yarn.scheduler.capacity.root.managed." + "auto-create-child-queue.enabled", "true"); + conf.put("yarn.scheduler.capacity.root.managed.leaf-queue-template.acl_submit_applications", + "user"); + conf.put("yarn.scheduler.capacity.root.managed.leaf-queue-template.acl_administer_queue", + "admin"); return createConfiguration(conf); } @@ -256,17 +255,45 @@ public static Configuration createWeightConfigWithAutoQueueCreationEnabled() { private static Configuration createWeightConfigInternal(boolean enableAqc) { Map conf = new HashMap<>(); - conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2"); + conf.put("yarn.scheduler.capacity.root.queues", "default, test1, test2, parent"); conf.put("yarn.scheduler.capacity.root.capacity", "1w"); conf.put("yarn.scheduler.capacity.root.default.capacity", "10w"); - conf.put("yarn.scheduler.capacity.root.test1.capacity", "4w"); - conf.put("yarn.scheduler.capacity.root.test2.capacity", "6w"); + conf.put("yarn.scheduler.capacity.root.test1.capacity", "5w"); + conf.put("yarn.scheduler.capacity.root.test2.capacity", "10w"); + conf.put("yarn.scheduler.capacity.root.parent.capacity", "20w"); conf.put("yarn.scheduler.capacity.root.test1.state", "RUNNING"); conf.put("yarn.scheduler.capacity.root.test2.state", "RUNNING"); if (enableAqc) { - conf.put("yarn.scheduler.capacity.root.auto-queue-creation-v2.enabled", - "true"); + final String root = "yarn.scheduler.capacity.root."; + conf.put(root + "auto-queue-creation-v2.enabled", "true"); + + conf.put(root + "auto-queue-creation-v2.parent-template.acl_submit_applications", + "parentUser1"); + conf.put(root + "auto-queue-creation-v2.parent-template.acl_administer_queue", + "parentAdmin1"); + + conf.put(root + "autoParent1.auto-queue-creation-v2.leaf-template.acl_submit_applications", + "user1"); + conf.put(root + "autoParent1.auto-queue-creation-v2.leaf-template.acl_administer_queue", + "admin1"); + + conf.put(root + "*.auto-queue-creation-v2.leaf-template.acl_submit_applications", + "wildUser1"); + conf.put(root + "*.auto-queue-creation-v2.leaf-template.acl_administer_queue", + "wildAdmin1"); + + + conf.put(root + "parent.auto-queue-creation-v2.enabled", "true"); + conf.put(root + "parent.auto-queue-creation-v2.parent-template.acl_submit_applications", + "parentUser2"); + conf.put(root + "parent.auto-queue-creation-v2.parent-template.acl_administer_queue", + "parentAdmin2"); + + conf.put(root + "parent.*.auto-queue-creation-v2.leaf-template.acl_submit_applications", + "wildUser2"); + conf.put(root + "parent.*.auto-queue-creation-v2.leaf-template.acl_administer_queue", + "wildAdmin2"); } return createConfiguration(conf); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json index 7580005398051..719aa811e88be 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-AbsoluteModeLegacyAutoCreation.json @@ -885,7 +885,7 @@ "queueAcls": {"queueAcl": [ { "accessType": "ADMINISTER_QUEUE", - "accessControlList": " " + "accessControlList": "admin " }, { "accessType": "APPLICATION_MAX_PRIORITY", @@ -893,7 +893,7 @@ }, { "accessType": "SUBMIT_APP", - "accessControlList": " " + "accessControlList": "user " } ]}, "queuePriority": 0, @@ -1296,10 +1296,20 @@ "queuePriority": 0, "orderingPolicyInfo": "utilization", "autoCreateChildQueueEnabled": true, - "leafQueueTemplate": {"property": [{ - "name": "leaf-queue-template.capacity", - "value": "[memory=2048,vcores=2]" - }]}, + "leafQueueTemplate": {"property": [ + { + "name": "leaf-queue-template.acl_administer_queue", + "value": "admin" + }, + { + "name": "leaf-queue-template.capacity", + "value": "[memory=2048,vcores=2]" + }, + { + "name": "leaf-queue-template.acl_submit_applications", + "value": "user" + } + ]}, "mode": "absolute", "queueType": "parent", "creationMethod": "static", @@ -1579,4 +1589,4 @@ "autoQueueTemplateProperties": {}, "autoQueueParentTemplateProperties": {}, "autoQueueLeafTemplateProperties": {} -}}} +}}} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json index b1894c58ad7f0..1f0d07f8cd956 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightMode.json @@ -16,11 +16,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 30.000002, + "absoluteCapacity": 22.222223, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, - "weight": 6, - "normalizedWeight": 0.3, + "weight": 10, + "normalizedWeight": 0.22222222, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "test2", @@ -60,12 +60,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 30.000002, + "absoluteCapacity": 22.222223, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, - "weight": 6, - "normalizedWeight": 0.3, + "weight": 10, + "normalizedWeight": 0.22222222, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -410,8 +410,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 3000, - "maxApplicationsPerUser": 3000, + "maxApplications": 2222, + "maxApplicationsPerUser": 2222, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -501,11 +501,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 20, + "absoluteCapacity": 11.111112, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, - "weight": 4, - "normalizedWeight": 0.2, + "weight": 5, + "normalizedWeight": 0.11111111, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "test1", @@ -545,12 +545,497 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 20, + "absoluteCapacity": 11.111112, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 10, + "weight": 5, + "normalizedWeight": 0.11111111, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "configuredMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "reserved": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "pending": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amLimit": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "userAmLimit": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "minEffectiveCapacity": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "maxEffectiveCapacity": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "maximumAllocation": { + "memory": 8192, + "vCores": 4, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 8192 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 4 + } + ]} + }, + "queueAcls": {"queueAcl": [ + { + "accessType": "ADMINISTER_QUEUE", + "accessControlList": " " + }, + { + "accessType": "APPLICATION_MAX_PRIORITY", + "accessControlList": "*" + }, + { + "accessType": "SUBMIT_APP", + "accessControlList": " " + } + ]}, + "queuePriority": 0, + "orderingPolicyInfo": "fifo", + "autoCreateChildQueueEnabled": false, + "leafQueueTemplate": {}, + "mode": "weight", + "queueType": "leaf", + "creationMethod": "static", + "autoCreationEligibility": "off", + "autoQueueTemplateProperties": {}, + "autoQueueParentTemplateProperties": {}, + "autoQueueLeafTemplateProperties": {}, + "numActiveApplications": 0, + "numPendingApplications": 0, + "numContainers": 0, + "maxApplications": 1111, + "maxApplicationsPerUser": 1111, + "userLimit": 100, + "users": {}, + "userLimitFactor": 1, + "configuredMaxAMResourceLimit": 0.1, + "AMResourceLimit": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "usedAMResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "userAMResourceLimit": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "preemptionDisabled": true, + "intraQueuePreemptionDisabled": true, + "defaultPriority": 0, + "isAutoCreatedLeafQueue": false, + "maxApplicationLifetime": -1, + "defaultApplicationLifetime": -1 + }, + { + "type": "capacitySchedulerLeafQueueInfo", + "queuePath": "root.parent", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 44.444447, + "absoluteMaxCapacity": 100, + "absoluteUsedCapacity": 0, + "weight": 20, + "normalizedWeight": 0.44444445, + "numApplications": 0, + "maxParallelApps": 2147483647, + "queueName": "parent", + "isAbsoluteResource": false, + "state": "RUNNING", + "resourcesUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 44.444447, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, - "weight": 4, - "normalizedWeight": 0.2, + "weight": 20, + "normalizedWeight": 0.44444445, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -895,8 +1380,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 2000, - "maxApplicationsPerUser": 2000, + "maxApplications": 4444, + "maxApplicationsPerUser": 4444, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -986,11 +1471,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 50, + "absoluteCapacity": 22.222223, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, "weight": 10, - "normalizedWeight": 0.5, + "normalizedWeight": 0.22222222, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "default", @@ -1030,12 +1515,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 50, + "absoluteCapacity": 22.222223, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, "weight": 10, - "normalizedWeight": 0.5, + "normalizedWeight": 0.22222222, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -1380,8 +1865,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 5000, - "maxApplicationsPerUser": 5000, + "maxApplications": 2222, + "maxApplicationsPerUser": 2222, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -1736,4 +2221,4 @@ "autoQueueTemplateProperties": {}, "autoQueueParentTemplateProperties": {}, "autoQueueLeafTemplateProperties": {} -}}} +}}} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json index 188b72a7a405d..a9c30d1dd1e99 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-After.json @@ -16,11 +16,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 25, + "absoluteCapacity": 20, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, - "weight": 6, - "normalizedWeight": 0.25, + "weight": 10, + "normalizedWeight": 0.2, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "test2", @@ -60,12 +60,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 25, + "absoluteCapacity": 20, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, - "weight": 6, - "normalizedWeight": 0.25, + "weight": 10, + "normalizedWeight": 0.2, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -115,8 +115,8 @@ ]} }, "effectiveMinResource": { - "memory": 307200, - "vCores": 300, + "memory": 245760, + "vCores": 240, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -125,7 +125,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 307200 + "value": 245760 }, { "attributes": {}, @@ -134,7 +134,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 300 + "value": 240 } ]} }, @@ -311,8 +311,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 307200, - "vCores": 300, + "memory": 245760, + "vCores": 240, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -321,7 +321,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 307200 + "value": 245760 }, { "attributes": {}, @@ -330,7 +330,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 300 + "value": 240 } ]} }, @@ -410,8 +410,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 2500, - "maxApplicationsPerUser": 2500, + "maxApplications": 2000, + "maxApplicationsPerUser": 2000, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -501,11 +501,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 16.666668, + "absoluteCapacity": 10, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, - "weight": 4, - "normalizedWeight": 0.16666667, + "weight": 5, + "normalizedWeight": 0.1, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "test1", @@ -545,12 +545,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 16.666668, + "absoluteCapacity": 10, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, - "weight": 4, - "normalizedWeight": 0.16666667, + "weight": 5, + "normalizedWeight": 0.1, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -600,8 +600,8 @@ ]} }, "effectiveMinResource": { - "memory": 204800, - "vCores": 200, + "memory": 122880, + "vCores": 120, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -610,7 +610,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 204800 + "value": 122880 }, { "attributes": {}, @@ -619,7 +619,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 200 + "value": 120 } ]} }, @@ -796,8 +796,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 204800, - "vCores": 200, + "memory": 122880, + "vCores": 120, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -806,7 +806,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 204800 + "value": 122880 }, { "attributes": {}, @@ -815,7 +815,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 200 + "value": 120 } ]} }, @@ -895,8 +895,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 1666, - "maxApplicationsPerUser": 1666, + "maxApplications": 1000, + "maxApplicationsPerUser": 1000, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -986,11 +986,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 41.666664, + "absoluteCapacity": 20, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, "weight": 10, - "normalizedWeight": 0.41666666, + "normalizedWeight": 0.2, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "default", @@ -1030,12 +1030,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 41.666664, + "absoluteCapacity": 20, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, "weight": 10, - "normalizedWeight": 0.41666666, + "normalizedWeight": 0.2, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -1085,8 +1085,8 @@ ]} }, "effectiveMinResource": { - "memory": 511999, - "vCores": 499, + "memory": 245760, + "vCores": 240, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -1095,7 +1095,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 511999 + "value": 245760 }, { "attributes": {}, @@ -1104,7 +1104,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 499 + "value": 240 } ]} }, @@ -1281,8 +1281,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 511999, - "vCores": 499, + "memory": 245760, + "vCores": 240, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -1291,7 +1291,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 511999 + "value": 245760 }, { "attributes": {}, @@ -1300,7 +1300,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 499 + "value": 240 } ]} }, @@ -1380,8 +1380,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 4166, - "maxApplicationsPerUser": 4166, + "maxApplications": 2000, + "maxApplicationsPerUser": 2000, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -1471,11 +1471,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 2, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, "weight": 1, - "normalizedWeight": 0.041666668, + "normalizedWeight": 0.02, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "auto1", @@ -1515,12 +1515,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 2, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 100, "weight": 1, - "normalizedWeight": 0.041666668, + "normalizedWeight": 0.02, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -1570,8 +1570,8 @@ ]} }, "effectiveMinResource": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -1580,7 +1580,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -1589,7 +1589,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -1766,8 +1766,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -1776,7 +1776,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -1785,7 +1785,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -1865,8 +1865,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 416, - "maxApplicationsPerUser": 416, + "maxApplications": 200, + "maxApplicationsPerUser": 200, "userLimit": 100, "users": {}, "userLimitFactor": -1, @@ -1956,11 +1956,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 2, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, "weight": 1, - "normalizedWeight": 0.041666668, + "normalizedWeight": 0.02, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "auto2", @@ -2000,12 +2000,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 2, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 100, "weight": 1, - "normalizedWeight": 0.041666668, + "normalizedWeight": 0.02, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -2055,8 +2055,8 @@ ]} }, "effectiveMinResource": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -2065,7 +2065,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -2074,7 +2074,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -2251,8 +2251,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -2261,7 +2261,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -2270,7 +2270,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -2350,8 +2350,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 416, - "maxApplicationsPerUser": 416, + "maxApplications": 200, + "maxApplicationsPerUser": 200, "userLimit": 100, "users": {}, "userLimitFactor": -1, @@ -2441,11 +2441,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 2, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, "weight": 1, - "normalizedWeight": 0.041666668, + "normalizedWeight": 0.02, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "auto3", @@ -2485,12 +2485,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 2, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 100, "weight": 1, - "normalizedWeight": 0.041666668, + "normalizedWeight": 0.02, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -2540,8 +2540,8 @@ ]} }, "effectiveMinResource": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -2550,7 +2550,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -2559,7 +2559,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -2736,8 +2736,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -2746,7 +2746,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -2755,7 +2755,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -2835,8 +2835,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 416, - "maxApplicationsPerUser": 416, + "maxApplications": 200, + "maxApplicationsPerUser": 200, "userLimit": 100, "users": {}, "userLimitFactor": -1, @@ -2921,84 +2921,59 @@ "defaultApplicationLifetime": -1 }, { - "queuePath": "root.autoParent1", + "queuePath": "root.parent", "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 40, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, - "weight": 1, - "normalizedWeight": 0.041666668, + "weight": 20, + "normalizedWeight": 0.4, "numApplications": 0, "maxParallelApps": 2147483647, - "queueName": "autoParent1", + "queueName": "parent", "isAbsoluteResource": false, "state": "RUNNING", "queues": {"queue": [{ - "type": "capacitySchedulerLeafQueueInfo", - "queuePath": "root.autoParent1.auto4", + "queuePath": "root.parent.autoParent3", "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 40, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, "weight": 1, "normalizedWeight": 1, "numApplications": 0, "maxParallelApps": 2147483647, - "queueName": "auto4", + "queueName": "autoParent3", "isAbsoluteResource": false, "state": "RUNNING", - "resourcesUsed": { - "memory": 0, - "vCores": 0, - "resourceInformations": {"resourceInformation": [ - { - "attributes": {}, - "maximumAllocation": 9223372036854775807, - "minimumAllocation": 0, - "name": "memory-mb", - "resourceType": "COUNTABLE", - "units": "Mi", - "value": 0 - }, - { - "attributes": {}, - "maximumAllocation": 9223372036854775807, - "minimumAllocation": 0, - "name": "vcores", - "resourceType": "COUNTABLE", - "units": "", - "value": 0 - } - ]} - }, - "hideReservationQueues": false, - "nodeLabels": ["*"], - "allocatedContainers": 0, - "reservedContainers": 0, - "pendingContainers": 0, - "capacities": {"queueCapacitiesByPartition": [{ - "partitionName": "", + "queues": {"queue": [{ + "type": "capacitySchedulerLeafQueueInfo", + "queuePath": "root.parent.autoParent3.auto6", "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, - "absoluteUsedCapacity": 0, + "absoluteCapacity": 40, "absoluteMaxCapacity": 100, - "maxAMLimitPercentage": 100, + "absoluteUsedCapacity": 0, "weight": 1, "normalizedWeight": 1, - "configuredMinResource": { + "numApplications": 0, + "maxParallelApps": 2147483647, + "queueName": "auto6", + "isAbsoluteResource": false, + "state": "RUNNING", + "resourcesUsed": { "memory": 0, "vCores": 0, "resourceInformations": {"resourceInformation": [ { "attributes": {}, - "maximumAllocation": 8192, - "minimumAllocation": 1024, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", @@ -3006,8 +2981,8 @@ }, { "attributes": {}, - "maximumAllocation": 4, - "minimumAllocation": 1, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, "name": "vcores", "resourceType": "COUNTABLE", "units": "", @@ -3015,33 +2990,293 @@ } ]} }, - "configuredMaxResource": { - "memory": 0, - "vCores": 0, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 40, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 100, + "weight": 1, + "normalizedWeight": 1, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "configuredMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 491520, + "vCores": 480, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 491520 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 480 + } + ]} + }, + "effectiveMaxResource": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "reserved": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "pending": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amLimit": { + "memory": 1228800, + "vCores": 1, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1 + } + ]} + }, + "userAmLimit": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "minEffectiveCapacity": { + "memory": 491520, + "vCores": 480, "resourceInformations": {"resourceInformation": [ { "attributes": {}, - "maximumAllocation": 8192, - "minimumAllocation": 1024, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 0 + "value": 491520 }, { "attributes": {}, - "maximumAllocation": 4, - "minimumAllocation": 1, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 0 + "value": 480 } ]} }, - "effectiveMinResource": { - "memory": 51200, - "vCores": 50, + "maxEffectiveCapacity": { + "memory": 1228800, + "vCores": 1200, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3050,7 +3285,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 1228800 }, { "attributes": {}, @@ -3059,13 +3294,71 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 1200 } ]} }, - "effectiveMaxResource": { + "maximumAllocation": { + "memory": 8192, + "vCores": 4, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 8192 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 4 + } + ]} + }, + "queueAcls": {"queueAcl": [ + { + "accessType": "ADMINISTER_QUEUE", + "accessControlList": "wildAdmin2 " + }, + { + "accessType": "APPLICATION_MAX_PRIORITY", + "accessControlList": "*" + }, + { + "accessType": "SUBMIT_APP", + "accessControlList": "wildUser2 " + } + ]}, + "queuePriority": 0, + "orderingPolicyInfo": "fifo", + "autoCreateChildQueueEnabled": false, + "leafQueueTemplate": {}, + "mode": "weight", + "queueType": "leaf", + "creationMethod": "dynamicFlexible", + "autoCreationEligibility": "off", + "autoQueueTemplateProperties": {}, + "autoQueueParentTemplateProperties": {}, + "autoQueueLeafTemplateProperties": {}, + "numActiveApplications": 0, + "numPendingApplications": 0, + "numContainers": 0, + "maxApplications": 4000, + "maxApplicationsPerUser": 4000, + "userLimit": 100, + "users": {}, + "userLimitFactor": -1, + "configuredMaxAMResourceLimit": 1, + "AMResourceLimit": { "memory": 1228800, - "vCores": 1200, + "vCores": 1, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3083,14 +3376,11 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 1200 + "value": 1 } ]} - } - }]}, - "resources": {"resourceUsagesByPartition": [{ - "partitionName": "", - "used": { + }, + "usedAMResource": { "memory": 0, "vCores": 0, "resourceInformations": {"resourceInformation": [ @@ -3114,9 +3404,9 @@ } ]} }, - "reserved": { - "memory": 0, - "vCores": 0, + "userAMResourceLimit": { + "memory": 1228800, + "vCores": 1, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3125,7 +3415,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 0 + "value": 1228800 }, { "attributes": {}, @@ -3134,13 +3424,108 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", + "value": 1 + } + ]} + }, + "preemptionDisabled": true, + "intraQueuePreemptionDisabled": true, + "defaultPriority": 0, + "isAutoCreatedLeafQueue": false, + "maxApplicationLifetime": -1, + "defaultApplicationLifetime": -1 + }]}, + "resourcesUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 40, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 0, + "weight": 1, + "normalizedWeight": 1, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", "value": 0 } ]} }, - "pending": { + "configuredMaxResource": { "memory": 0, "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 491520, + "vCores": 480, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3149,7 +3534,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 0 + "value": 491520 }, { "attributes": {}, @@ -3158,11 +3543,38 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 0 + "value": 480 } ]} }, - "amUsed": { + "effectiveMaxResource": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { "memory": 0, "vCores": 0, "resourceInformations": {"resourceInformation": [ @@ -3186,9 +3598,9 @@ } ]} }, - "amLimit": { - "memory": 1228800, - "vCores": 1, + "reserved": { + "memory": 0, + "vCores": 0, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3197,7 +3609,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 1228800 + "value": 0 }, { "attributes": {}, @@ -3206,11 +3618,11 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 1 + "value": 0 } ]} }, - "userAmLimit": { + "pending": { "memory": 0, "vCores": 0, "resourceInformations": {"resourceInformation": [ @@ -3236,8 +3648,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 51200, - "vCores": 50, + "memory": 491520, + "vCores": 480, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3246,7 +3658,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 491520 }, { "attributes": {}, @@ -3255,7 +3667,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 480 } ]} }, @@ -3310,7 +3722,7 @@ "queueAcls": {"queueAcl": [ { "accessType": "ADMINISTER_QUEUE", - "accessControlList": " " + "accessControlList": "parentAdmin2 " }, { "accessType": "APPLICATION_MAX_PRIORITY", @@ -3318,7 +3730,1579 @@ }, { "accessType": "SUBMIT_APP", - "accessControlList": " " + "accessControlList": "parentUser2 " + } + ]}, + "queuePriority": 0, + "orderingPolicyInfo": "utilization", + "autoCreateChildQueueEnabled": false, + "leafQueueTemplate": {}, + "mode": "weight", + "queueType": "parent", + "creationMethod": "dynamicFlexible", + "autoCreationEligibility": "flexible", + "autoQueueTemplateProperties": {}, + "autoQueueParentTemplateProperties": {}, + "autoQueueLeafTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "wildAdmin2" + }, + { + "name": "acl_submit_applications", + "value": "wildUser2" + } + ]} + }]}, + "resourcesUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 40, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 0, + "weight": 20, + "normalizedWeight": 0.4, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "configuredMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 491520, + "vCores": 480, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 491520 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 480 + } + ]} + }, + "effectiveMaxResource": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "reserved": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "pending": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "minEffectiveCapacity": { + "memory": 491520, + "vCores": 480, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 491520 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 480 + } + ]} + }, + "maxEffectiveCapacity": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + }, + "maximumAllocation": { + "memory": 8192, + "vCores": 4, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 8192 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 4 + } + ]} + }, + "queueAcls": {"queueAcl": [ + { + "accessType": "ADMINISTER_QUEUE", + "accessControlList": " " + }, + { + "accessType": "APPLICATION_MAX_PRIORITY", + "accessControlList": "*" + }, + { + "accessType": "SUBMIT_APP", + "accessControlList": " " + } + ]}, + "queuePriority": 0, + "orderingPolicyInfo": "utilization", + "autoCreateChildQueueEnabled": false, + "leafQueueTemplate": {}, + "mode": "weight", + "queueType": "parent", + "creationMethod": "static", + "autoCreationEligibility": "flexible", + "autoQueueTemplateProperties": {}, + "autoQueueParentTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "parentAdmin2" + }, + { + "name": "acl_submit_applications", + "value": "parentUser2" + } + ]}, + "autoQueueLeafTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "wildAdmin1" + }, + { + "name": "acl_submit_applications", + "value": "wildUser1" + } + ]} + }, + { + "queuePath": "root.autoParent1", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 2, + "absoluteMaxCapacity": 100, + "absoluteUsedCapacity": 0, + "weight": 1, + "normalizedWeight": 0.02, + "numApplications": 0, + "maxParallelApps": 2147483647, + "queueName": "autoParent1", + "isAbsoluteResource": false, + "state": "RUNNING", + "queues": {"queue": [{ + "type": "capacitySchedulerLeafQueueInfo", + "queuePath": "root.autoParent1.auto4", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 2, + "absoluteMaxCapacity": 100, + "absoluteUsedCapacity": 0, + "weight": 1, + "normalizedWeight": 1, + "numApplications": 0, + "maxParallelApps": 2147483647, + "queueName": "auto4", + "isAbsoluteResource": false, + "state": "RUNNING", + "resourcesUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 2, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 100, + "weight": 1, + "normalizedWeight": 1, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "configuredMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 24575, + "vCores": 23, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 24575 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 23 + } + ]} + }, + "effectiveMaxResource": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "reserved": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "pending": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amLimit": { + "memory": 1228800, + "vCores": 1, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1 + } + ]} + }, + "userAmLimit": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "minEffectiveCapacity": { + "memory": 24575, + "vCores": 23, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 24575 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 23 + } + ]} + }, + "maxEffectiveCapacity": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + }, + "maximumAllocation": { + "memory": 8192, + "vCores": 4, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 8192 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 4 + } + ]} + }, + "queueAcls": {"queueAcl": [ + { + "accessType": "ADMINISTER_QUEUE", + "accessControlList": "admin1 " + }, + { + "accessType": "APPLICATION_MAX_PRIORITY", + "accessControlList": "*" + }, + { + "accessType": "SUBMIT_APP", + "accessControlList": "user1 " + } + ]}, + "queuePriority": 0, + "orderingPolicyInfo": "fifo", + "autoCreateChildQueueEnabled": false, + "leafQueueTemplate": {}, + "mode": "weight", + "queueType": "leaf", + "creationMethod": "dynamicFlexible", + "autoCreationEligibility": "off", + "autoQueueTemplateProperties": {}, + "autoQueueParentTemplateProperties": {}, + "autoQueueLeafTemplateProperties": {}, + "numActiveApplications": 0, + "numPendingApplications": 0, + "numContainers": 0, + "maxApplications": 300, + "maxApplicationsPerUser": 300, + "userLimit": 100, + "users": {}, + "userLimitFactor": -1, + "configuredMaxAMResourceLimit": 1, + "AMResourceLimit": { + "memory": 1228800, + "vCores": 1, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1 + } + ]} + }, + "usedAMResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "userAMResourceLimit": { + "memory": 1228800, + "vCores": 1, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1 + } + ]} + }, + "preemptionDisabled": true, + "intraQueuePreemptionDisabled": true, + "defaultPriority": 0, + "isAutoCreatedLeafQueue": false, + "maxApplicationLifetime": -1, + "defaultApplicationLifetime": -1 + }]}, + "resourcesUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 2, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 0, + "weight": 1, + "normalizedWeight": 0.02, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "configuredMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 24575, + "vCores": 23, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 24575 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 23 + } + ]} + }, + "effectiveMaxResource": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "reserved": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "pending": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "minEffectiveCapacity": { + "memory": 24575, + "vCores": 23, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 24575 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 23 + } + ]} + }, + "maxEffectiveCapacity": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + }, + "maximumAllocation": { + "memory": 8192, + "vCores": 4, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 8192 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 4 + } + ]} + }, + "queueAcls": {"queueAcl": [ + { + "accessType": "ADMINISTER_QUEUE", + "accessControlList": "parentAdmin1 " + }, + { + "accessType": "APPLICATION_MAX_PRIORITY", + "accessControlList": "*" + }, + { + "accessType": "SUBMIT_APP", + "accessControlList": "parentUser1 " + } + ]}, + "queuePriority": 0, + "orderingPolicyInfo": "utilization", + "autoCreateChildQueueEnabled": false, + "leafQueueTemplate": {}, + "mode": "weight", + "queueType": "parent", + "creationMethod": "dynamicFlexible", + "autoCreationEligibility": "flexible", + "autoQueueTemplateProperties": {"property": [{ + "name": "maximum-applications", + "value": "300" + }]}, + "autoQueueParentTemplateProperties": {}, + "autoQueueLeafTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "admin1" + }, + { + "name": "acl_submit_applications", + "value": "user1" + } + ]} + }, + { + "queuePath": "root.autoParent2", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 2, + "absoluteMaxCapacity": 100, + "absoluteUsedCapacity": 0, + "weight": 1, + "normalizedWeight": 0.02, + "numApplications": 0, + "maxParallelApps": 2147483647, + "queueName": "autoParent2", + "isAbsoluteResource": false, + "state": "RUNNING", + "queues": {"queue": [{ + "type": "capacitySchedulerLeafQueueInfo", + "queuePath": "root.autoParent2.auto5", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 2, + "absoluteMaxCapacity": 100, + "absoluteUsedCapacity": 0, + "weight": 1, + "normalizedWeight": 1, + "numApplications": 0, + "maxParallelApps": 2147483647, + "queueName": "auto5", + "isAbsoluteResource": false, + "state": "RUNNING", + "resourcesUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 2, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 100, + "weight": 1, + "normalizedWeight": 1, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "configuredMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 24575, + "vCores": 23, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 24575 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 23 + } + ]} + }, + "effectiveMaxResource": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "reserved": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "pending": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "amLimit": { + "memory": 1228800, + "vCores": 1, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1 + } + ]} + }, + "userAmLimit": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "minEffectiveCapacity": { + "memory": 24575, + "vCores": 23, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 24575 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 23 + } + ]} + }, + "maxEffectiveCapacity": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + }, + "maximumAllocation": { + "memory": 8192, + "vCores": 4, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 8192 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 4 + } + ]} + }, + "queueAcls": {"queueAcl": [ + { + "accessType": "ADMINISTER_QUEUE", + "accessControlList": "wildAdmin1 " + }, + { + "accessType": "APPLICATION_MAX_PRIORITY", + "accessControlList": "*" + }, + { + "accessType": "SUBMIT_APP", + "accessControlList": "wildUser1 " } ]}, "queuePriority": 0, @@ -3335,8 +5319,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 300, - "maxApplicationsPerUser": 300, + "maxApplications": 200, + "maxApplicationsPerUser": 200, "userLimit": 100, "users": {}, "userLimitFactor": -1, @@ -3454,12 +5438,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 4.166667, + "absoluteCapacity": 2, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 0, "weight": 1, - "normalizedWeight": 0.041666668, + "normalizedWeight": 0.02, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -3509,8 +5493,8 @@ ]} }, "effectiveMinResource": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3519,7 +5503,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -3528,7 +5512,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -3633,8 +5617,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 51200, - "vCores": 50, + "memory": 24575, + "vCores": 23, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -3643,7 +5627,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 51200 + "value": 24575 }, { "attributes": {}, @@ -3652,7 +5636,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 50 + "value": 23 } ]} }, @@ -3707,7 +5691,7 @@ "queueAcls": {"queueAcl": [ { "accessType": "ADMINISTER_QUEUE", - "accessControlList": " " + "accessControlList": "parentAdmin1 " }, { "accessType": "APPLICATION_MAX_PRIORITY", @@ -3715,7 +5699,7 @@ }, { "accessType": "SUBMIT_APP", - "accessControlList": " " + "accessControlList": "parentUser1 " } ]}, "queuePriority": 0, @@ -3726,12 +5710,18 @@ "queueType": "parent", "creationMethod": "dynamicFlexible", "autoCreationEligibility": "flexible", - "autoQueueTemplateProperties": {"property": [{ - "name": "maximum-applications", - "value": "300" - }]}, + "autoQueueTemplateProperties": {}, "autoQueueParentTemplateProperties": {}, - "autoQueueLeafTemplateProperties": {} + "autoQueueLeafTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "wildAdmin1" + }, + { + "name": "acl_submit_applications", + "value": "wildUser1" + } + ]} } ]}, "capacities": {"queueCapacitiesByPartition": [{ @@ -4002,6 +5992,15 @@ "creationMethod": "static", "autoCreationEligibility": "flexible", "autoQueueTemplateProperties": {}, - "autoQueueParentTemplateProperties": {}, + "autoQueueParentTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "parentAdmin1" + }, + { + "name": "acl_submit_applications", + "value": "parentUser1" + } + ]}, "autoQueueLeafTemplateProperties": {} -}}} +}}} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json index ee4cd14a1b3fa..84fd17399e8d9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/resources/webapp/scheduler-response-WeightModeWithAutoCreatedQueues-Before.json @@ -16,11 +16,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 30.000002, + "absoluteCapacity": 22.222223, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, - "weight": 6, - "normalizedWeight": 0.3, + "weight": 10, + "normalizedWeight": 0.22222222, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "test2", @@ -60,12 +60,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 30.000002, + "absoluteCapacity": 22.222223, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, - "weight": 6, - "normalizedWeight": 0.3, + "weight": 10, + "normalizedWeight": 0.22222222, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -115,8 +115,8 @@ ]} }, "effectiveMinResource": { - "memory": 368640, - "vCores": 360, + "memory": 273066, + "vCores": 266, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -125,7 +125,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 368640 + "value": 273066 }, { "attributes": {}, @@ -134,7 +134,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 360 + "value": 266 } ]} }, @@ -311,8 +311,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 368640, - "vCores": 360, + "memory": 273066, + "vCores": 266, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -321,7 +321,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 368640 + "value": 273066 }, { "attributes": {}, @@ -330,7 +330,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 360 + "value": 266 } ]} }, @@ -410,8 +410,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 3000, - "maxApplicationsPerUser": 3000, + "maxApplications": 2222, + "maxApplicationsPerUser": 2222, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -501,11 +501,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 20, + "absoluteCapacity": 11.111112, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, - "weight": 4, - "normalizedWeight": 0.2, + "weight": 5, + "normalizedWeight": 0.11111111, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "test1", @@ -545,12 +545,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 20, + "absoluteCapacity": 11.111112, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, - "weight": 4, - "normalizedWeight": 0.2, + "weight": 5, + "normalizedWeight": 0.11111111, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -600,8 +600,8 @@ ]} }, "effectiveMinResource": { - "memory": 245760, - "vCores": 240, + "memory": 136533, + "vCores": 133, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -610,7 +610,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 245760 + "value": 136533 }, { "attributes": {}, @@ -619,7 +619,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 240 + "value": 133 } ]} }, @@ -796,8 +796,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 245760, - "vCores": 240, + "memory": 136533, + "vCores": 133, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -806,7 +806,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 245760 + "value": 136533 }, { "attributes": {}, @@ -815,7 +815,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 240 + "value": 133 } ]} }, @@ -895,8 +895,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 2000, - "maxApplicationsPerUser": 2000, + "maxApplications": 1111, + "maxApplicationsPerUser": 1111, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -986,11 +986,11 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 50, + "absoluteCapacity": 22.222223, "absoluteMaxCapacity": 100, "absoluteUsedCapacity": 0, "weight": 10, - "normalizedWeight": 0.5, + "normalizedWeight": 0.22222222, "numApplications": 0, "maxParallelApps": 2147483647, "queueName": "default", @@ -1030,12 +1030,12 @@ "capacity": 0, "usedCapacity": 0, "maxCapacity": 100, - "absoluteCapacity": 50, + "absoluteCapacity": 22.222223, "absoluteUsedCapacity": 0, "absoluteMaxCapacity": 100, "maxAMLimitPercentage": 10, "weight": 10, - "normalizedWeight": 0.5, + "normalizedWeight": 0.22222222, "configuredMinResource": { "memory": 0, "vCores": 0, @@ -1085,8 +1085,8 @@ ]} }, "effectiveMinResource": { - "memory": 614400, - "vCores": 600, + "memory": 273066, + "vCores": 266, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -1095,7 +1095,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 614400 + "value": 273066 }, { "attributes": {}, @@ -1104,7 +1104,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 600 + "value": 266 } ]} }, @@ -1281,8 +1281,8 @@ } }]}, "minEffectiveCapacity": { - "memory": 614400, - "vCores": 600, + "memory": 273066, + "vCores": 266, "resourceInformations": {"resourceInformation": [ { "attributes": {}, @@ -1291,7 +1291,7 @@ "name": "memory-mb", "resourceType": "COUNTABLE", "units": "Mi", - "value": 614400 + "value": 273066 }, { "attributes": {}, @@ -1300,7 +1300,7 @@ "name": "vcores", "resourceType": "COUNTABLE", "units": "", - "value": 600 + "value": 266 } ]} }, @@ -1380,8 +1380,8 @@ "numActiveApplications": 0, "numPendingApplications": 0, "numContainers": 0, - "maxApplications": 5000, - "maxApplicationsPerUser": 5000, + "maxApplications": 2222, + "maxApplicationsPerUser": 2222, "userLimit": 100, "users": {}, "userLimitFactor": 1, @@ -1464,6 +1464,350 @@ "isAutoCreatedLeafQueue": false, "maxApplicationLifetime": -1, "defaultApplicationLifetime": -1 + }, + { + "queuePath": "root.parent", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 44.444447, + "absoluteMaxCapacity": 100, + "absoluteUsedCapacity": 0, + "weight": 20, + "normalizedWeight": 0.44444445, + "numApplications": 0, + "maxParallelApps": 2147483647, + "queueName": "parent", + "isAbsoluteResource": false, + "state": "RUNNING", + "queues": {}, + "resourcesUsed": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "hideReservationQueues": false, + "nodeLabels": ["*"], + "allocatedContainers": 0, + "reservedContainers": 0, + "pendingContainers": 0, + "capacities": {"queueCapacitiesByPartition": [{ + "partitionName": "", + "capacity": 0, + "usedCapacity": 0, + "maxCapacity": 100, + "absoluteCapacity": 44.444447, + "absoluteUsedCapacity": 0, + "absoluteMaxCapacity": 100, + "maxAMLimitPercentage": 0, + "weight": 20, + "normalizedWeight": 0.44444445, + "configuredMinResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "configuredMaxResource": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 8192, + "minimumAllocation": 1024, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 4, + "minimumAllocation": 1, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "effectiveMinResource": { + "memory": 546133, + "vCores": 533, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 546133 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 533 + } + ]} + }, + "effectiveMaxResource": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + } + }]}, + "resources": {"resourceUsagesByPartition": [{ + "partitionName": "", + "used": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "reserved": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + }, + "pending": { + "memory": 0, + "vCores": 0, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 0 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 0 + } + ]} + } + }]}, + "minEffectiveCapacity": { + "memory": 546133, + "vCores": 533, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 546133 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 533 + } + ]} + }, + "maxEffectiveCapacity": { + "memory": 1228800, + "vCores": 1200, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 1228800 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 1200 + } + ]} + }, + "maximumAllocation": { + "memory": 8192, + "vCores": 4, + "resourceInformations": {"resourceInformation": [ + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "memory-mb", + "resourceType": "COUNTABLE", + "units": "Mi", + "value": 8192 + }, + { + "attributes": {}, + "maximumAllocation": 9223372036854775807, + "minimumAllocation": 0, + "name": "vcores", + "resourceType": "COUNTABLE", + "units": "", + "value": 4 + } + ]} + }, + "queueAcls": {"queueAcl": [ + { + "accessType": "ADMINISTER_QUEUE", + "accessControlList": " " + }, + { + "accessType": "APPLICATION_MAX_PRIORITY", + "accessControlList": "*" + }, + { + "accessType": "SUBMIT_APP", + "accessControlList": " " + } + ]}, + "queuePriority": 0, + "orderingPolicyInfo": "utilization", + "autoCreateChildQueueEnabled": false, + "leafQueueTemplate": {}, + "mode": "weight", + "queueType": "parent", + "creationMethod": "static", + "autoCreationEligibility": "flexible", + "autoQueueTemplateProperties": {}, + "autoQueueParentTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "parentAdmin2" + }, + { + "name": "acl_submit_applications", + "value": "parentUser2" + } + ]}, + "autoQueueLeafTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "wildAdmin1" + }, + { + "name": "acl_submit_applications", + "value": "wildUser1" + } + ]} } ]}, "capacities": {"queueCapacitiesByPartition": [{ @@ -1734,6 +2078,15 @@ "creationMethod": "static", "autoCreationEligibility": "flexible", "autoQueueTemplateProperties": {}, - "autoQueueParentTemplateProperties": {}, + "autoQueueParentTemplateProperties": {"property": [ + { + "name": "acl_administer_queue", + "value": "parentAdmin1" + }, + { + "name": "acl_submit_applications", + "value": "parentUser1" + } + ]}, "autoQueueLeafTemplateProperties": {} -}}} +}}} \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md index 9173cb0743968..3a694e3f2485e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/CapacityScheduler.md @@ -642,9 +642,7 @@ support other pre-configured queues to co-exist along with auto-created queues. The parent queue which has been enabled for auto leaf queue creation,supports the configuration of template parameters for automatic configuration of the auto-created leaf queues. The auto-created queues support all of the - leaf queue configuration parameters except for **Queue ACL**, **Absolute - Resource** configurations. Queue ACLs are - currently inherited from the parent queue i.e they are not configurable on the leaf queue template + leaf queue configuration parameters except for **Absolute Resource** configurations. | Property | Description | |:---- |:---- | @@ -730,11 +728,11 @@ The `Flexible Dynamic Queue Auto-Creation and Management` feature allows a **Par * Configuring **flexible** `Auto-Created Leaf Queues` with `CapacityScheduler` -The parent queue which has the flexible auto queue creation enabled supports the configuration of dynamically created leaf and parent queues through template parameters. The auto-created queues support all of the leaf queue configuration parameters except for **Queue ACL**, **Absolute Resource** configurations. Queue ACLs are currently inherited from the parent queue i.e they are not configurable on the leaf queue template +The parent queue which has the flexible auto queue creation enabled supports the configuration of dynamically created leaf and parent queues through template parameters. The auto-created queues support all of the leaf queue configuration parameters except for **Absolute Resource** configurations. | Property | Description | |:---- |:---- | -| `yarn.scheduler.capacity..auto-queue-creation-v2.template.` | *Optional* parameter: Specifies a queue property (like capacity, maximum-capacity, user-limit-factor, maximum-am-resource-percent ... - Refer **Queue Properties** section) inherited by the auto-created **parent** and **leaf** queues. | +| `yarn.scheduler.capacity..auto-queue-creation-v2.template.` | *Optional* parameter: Specifies a queue property (like capacity, maximum-capacity, user-limit-factor, maximum-am-resource-percent ... - Refer **Queue Properties** section) inherited by the auto-created **parent** and **leaf** queues. Dynamic Queue ACLs set here can be overwritten by the parent-template for dynamic parent queues and with the leaf-template for dynamic leaf queues. | | `yarn.scheduler.capacity..auto-queue-creation-v2.leaf-template.` | *Optional* parameter: Specifies a queue property inherited by auto-created **leaf** queues. | | `yarn.scheduler.capacity..auto-queue-creation-v2.parent-template.` | *Optional* parameter: Specifies a queue property inherited by auto-created **parent** queues. | From 61e809b245c5a2dc0e4db6638093932b0b34ca5a Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Sat, 26 Mar 2022 12:31:16 +0100 Subject: [PATCH 067/145] HADOOP-13386. Upgrade Avro to 1.9.2 (#3990) Signed-off-by: Akira Ajisaka --- LICENSE-binary | 2 +- .../org/apache/hadoop/io/AvroTestUtil.java | 2 +- .../apache/hadoop/io/TestEnumSetWritable.java | 2 +- .../dev-support/findbugs-exclude.xml | 8 +++++ hadoop-project/pom.xml | 2 +- .../pom.xml | 18 ---------- .../hadoop-yarn/hadoop-yarn-common/pom.xml | 18 ---------- .../pom.xml | 36 +++++-------------- .../hadoop-yarn-server-common/pom.xml | 20 +++++++++++ .../hadoop-yarn-server-nodemanager/pom.xml | 18 ---------- .../pom.xml | 18 ---------- .../pom.xml | 6 ++++ 12 files changed, 47 insertions(+), 103 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index de4e1cb75b356..de8df181fc3d1 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -283,7 +283,7 @@ log4j:log4j:1.2.17 net.java.dev.jna:jna:5.2.0 net.minidev:accessors-smart:1.2 net.minidev:json-smart:2.4.7 -org.apache.avro:avro:1.7.7 +org.apache.avro:avro:1.9.2 org.apache.commons:commons-collections4:4.2 org.apache.commons:commons-compress:1.21 org.apache.commons:commons-configuration2:2.1.1 diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java index ec76ea008077d..9c9b75fa76e6c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/AvroTestUtil.java @@ -41,7 +41,7 @@ public static void testReflect(Object value, Type type, String schema) // check that schema matches expected Schema s = ReflectData.get().getSchema(type); - assertEquals(Schema.parse(schema), s); + assertEquals(new Schema.Parser().parse(schema), s); // check that value is serialized correctly ReflectDatumWriter writer = new ReflectDatumWriter(s); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestEnumSetWritable.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestEnumSetWritable.java index 5e71601742f3c..11459261f5b74 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestEnumSetWritable.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestEnumSetWritable.java @@ -119,7 +119,7 @@ public void testSerializeAndDeserializeNull() throws IOException { public void testAvroReflect() throws Exception { String schema = "{\"type\":\"array\",\"items\":{\"type\":\"enum\"," + "\"name\":\"TestEnumSet\"," - + "\"namespace\":\"org.apache.hadoop.io.TestEnumSetWritable$\"," + + "\"namespace\":\"org.apache.hadoop.io.TestEnumSetWritable\"," + "\"symbols\":[\"CREATE\",\"OVERWRITE\",\"APPEND\"]}," + "\"java-class\":\"org.apache.hadoop.io.EnumSetWritable\"}"; Type type = diff --git a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml index 4e459b652b29c..8631ebba575c2 100644 --- a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml +++ b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml @@ -291,6 +291,14 @@ + + + + + + diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 80e07f0ac1d7a..93330d479e0b0 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -63,7 +63,7 @@ file:///dev/urandom - 1.7.7 + 1.9.2 1.19 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/pom.xml index bae498cfb4a6b..a36169d645b85 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/pom.xml @@ -95,24 +95,6 @@ com.sun.jersey jersey-json ${jersey.version} - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-jaxrs - - - org.codehaus.jackson - jackson-xc - - diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml index efebb0853d707..9c88950690c51 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml @@ -158,24 +158,6 @@ com.sun.jersey jersey-json - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-jaxrs - - - org.codehaus.jackson - jackson-xc - - com.sun.jersey.contribs diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml index 9c87f44a6d73d..9d2ade2f1f5b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/pom.xml @@ -93,30 +93,20 @@ com.sun.jersey - jersey-json - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-jaxrs - - - org.codehaus.jackson - jackson-xc - - + jersey-core + + + com.sun.jersey + jersey-client com.sun.jersey.contribs jersey-guice + + com.sun.jersey + jersey-json + org.apache.hadoop @@ -142,14 +132,6 @@ org.codehaus.jettison jettison - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-client - org.apache.hadoop.thirdparty hadoop-shaded-guava diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml index 3bf70bf7560e5..2d903b0be42dc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/pom.xml @@ -90,6 +90,26 @@ com.google.protobuf protobuf-java + + org.codehaus.jackson + jackson-core-asl + runtime + + + org.codehaus.jackson + jackson-mapper-asl + runtime + + + org.codehaus.jackson + jackson-jaxrs + runtime + + + org.codehaus.jackson + jackson-xc + runtime + junit junit diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml index b51d51cffde95..79bb6cfa22de4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/pom.xml @@ -154,24 +154,6 @@ com.sun.jersey jersey-json - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-jaxrs - - - org.codehaus.jackson - jackson-xc - - com.sun.jersey.contribs diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml index 1a0f4c00f77b3..f17ef707ef4b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/pom.xml @@ -109,24 +109,6 @@ com.sun.jersey jersey-json - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - org.codehaus.jackson - jackson-jaxrs - - - org.codehaus.jackson - jackson-xc - - com.sun.jersey.contribs diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml index b2f2d3a966c4a..e0151afd0c666 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/pom.xml @@ -188,6 +188,12 @@ test + + org.xerial.snappy + snappy-java + runtime + + com.sun.jersey jersey-client From adbaf48082d366bd5f9f0b1baca6b54e76a9d84e Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Sat, 26 Mar 2022 21:43:10 +0100 Subject: [PATCH 068/145] YARN-11100. Fix StackOverflowError in SLS scheduler event handling. Contributed by Szilard Nemeth. --- .../org/apache/hadoop/yarn/sls/SLSRunner.java | 5 +++-- .../sls/scheduler/SLSCapacityScheduler.java | 16 +++++++++++++++- .../yarn/sls/scheduler/SLSFairScheduler.java | 15 +++++++++++++++ .../sls/scheduler/SLSSchedulerCommons.java | 10 ++++++---- .../yarn/sls/scheduler/SchedulerWrapper.java | 19 +++++++++++++++++++ 5 files changed, 58 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index 48ad610310214..2110e3c196ca3 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -175,11 +175,12 @@ public void setConf(Configuration conf) { } private void init(Configuration tempConf) throws ClassNotFoundException { + // runner configuration + setConf(tempConf); + nmMap = new ConcurrentHashMap<>(); queueAppNumMap = new HashMap<>(); amRunner = new AMRunner(runner, this); - // runner configuration - setConf(tempConf); // runner poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE, diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java index b6fe5c0f96588..39170b18727af 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java @@ -39,7 +39,7 @@ @Private @Unstable public class SLSCapacityScheduler extends CapacityScheduler implements - SchedulerWrapper,Configurable { + SchedulerWrapper, Configurable { private final SLSSchedulerCommons schedulerCommons; private Configuration conf; @@ -65,6 +65,15 @@ public Allocation allocate(ApplicationAttemptId attemptId, containerIds, blacklistAdditions, blacklistRemovals, updateRequests); } + @Override + public Allocation allocatePropagated(ApplicationAttemptId attemptId, + List resourceRequests, + List schedulingRequests, + List containerIds, List blacklistAdditions, + List blacklistRemovals, ContainerUpdates updateRequests) { + return super.allocate(attemptId, resourceRequests, schedulingRequests, + containerIds, blacklistAdditions, blacklistRemovals, updateRequests); + } @Override public boolean tryCommit(Resource cluster, ResourceCommitRequest r, @@ -97,6 +106,11 @@ public void handle(SchedulerEvent schedulerEvent) { schedulerCommons.handle(schedulerEvent); } + @Override + public void propagatedHandle(SchedulerEvent schedulerEvent) { + super.handle(schedulerEvent); + } + @Override public void serviceStop() throws Exception { schedulerCommons.stopMetrics(); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java index b164316486378..2835b413cabe0 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java @@ -63,6 +63,21 @@ public void handle(SchedulerEvent schedulerEvent) { schedulerCommons.handle(schedulerEvent); } + @Override + public void propagatedHandle(SchedulerEvent schedulerEvent) { + super.handle(schedulerEvent); + } + + @Override + public Allocation allocatePropagated(ApplicationAttemptId attemptId, + List resourceRequests, + List schedulingRequests, + List containerIds, List blacklistAdditions, + List blacklistRemovals, ContainerUpdates updateRequests) { + return super.allocate(attemptId, resourceRequests, schedulingRequests, + containerIds, blacklistAdditions, blacklistRemovals, updateRequests); + } + @Override public void serviceStop() throws Exception { schedulerCommons.stopMetrics(); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java index 7132fc9e62002..d83fe5c6d0e69 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java @@ -100,7 +100,8 @@ public Allocation allocate(ApplicationAttemptId attemptId, .time(); Allocation allocation = null; try { - allocation = scheduler.allocate(attemptId, resourceRequests, + allocation = ((SchedulerWrapper)scheduler).allocatePropagated( + attemptId, resourceRequests, schedulingRequests, containerIds, blacklistAdditions, blacklistRemovals, updateRequests); return allocation; @@ -118,7 +119,8 @@ public Allocation allocate(ApplicationAttemptId attemptId, } } } else { - return scheduler.allocate(attemptId, resourceRequests, schedulingRequests, + return ((SchedulerWrapper)scheduler).allocatePropagated( + attemptId, resourceRequests, schedulingRequests, containerIds, blacklistAdditions, blacklistRemovals, updateRequests); } @@ -204,7 +206,7 @@ private void updateQueueWithAllocateRequest(Allocation allocation, public void handle(SchedulerEvent schedulerEvent) { if (!metricsON) { - scheduler.handle(schedulerEvent); + ((SchedulerWrapper)scheduler).propagatedHandle(schedulerEvent); return; } @@ -245,7 +247,7 @@ public void handle(SchedulerEvent schedulerEvent) { operationTimer = schedulerMetrics.getSchedulerHandleTimer( schedulerEvent.getType()).time(); - scheduler.handle(schedulerEvent); + ((SchedulerWrapper)scheduler).propagatedHandle(schedulerEvent); } finally { if (handlerTimer != null) { handlerTimer.stop(); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java index 7112b1a6fd57b..5ee088d792a2f 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java @@ -19,7 +19,16 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Unstable; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ResourceRequest; +import org.apache.hadoop.yarn.api.records.SchedulingRequest; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; + +import java.util.List; @Private @Unstable @@ -29,4 +38,14 @@ public interface SchedulerWrapper { Tracker getTracker(); String getRealQueueName(String queue) throws YarnException; + + void propagatedHandle(SchedulerEvent schedulerEvent); + + Allocation allocatePropagated(ApplicationAttemptId attemptId, + List resourceRequests, + List schedulingRequests, + List containerIds, + List blacklistAdditions, + List blacklistRemovals, + ContainerUpdates updateRequests); } From 046a6204b4a895b98ccd41dde1c9524a6bb0ea31 Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Sun, 27 Mar 2022 21:23:48 +0800 Subject: [PATCH 069/145] HDFS-16355. Improve the description of dfs.block.scanner.volume.bytes.per.second (#3724) Co-authored-by: gf13871 Signed-off-by: Akira Ajisaka --- .../src/main/resources/hdfs-default.xml | 2 +- .../hdfs/server/datanode/TestBlockScanner.java | 16 +++++++++++----- 2 files changed, 12 insertions(+), 6 deletions(-) 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 d49800c34849e..d45f8eb5b7ec5 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -1651,7 +1651,7 @@ dfs.block.scanner.volume.bytes.per.second 1048576 - If this is 0, the DataNode's block scanner will be disabled. If this + If this is configured less than or equal to zero, the DataNode's block scanner will be disabled. If this is positive, this is the number of bytes per second that the DataNode's block scanner will try to scan from each volume. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java index e1473fb7e9803..4251c15718545 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockScanner.java @@ -282,11 +282,17 @@ public void testVolumeIteratorWithCaching() throws Exception { public void testDisableVolumeScanner() throws Exception { Configuration conf = new Configuration(); disableBlockScanner(conf); - TestContext ctx = new TestContext(conf, 1); - try { - Assert.assertFalse(ctx.datanode.getBlockScanner().isEnabled()); - } finally { - ctx.close(); + try(TestContext ctx = new TestContext(conf, 1)) { + assertFalse(ctx.datanode.getBlockScanner().isEnabled()); + } + } + + @Test(timeout=60000) + public void testDisableVolumeScanner2() throws Exception { + Configuration conf = new Configuration(); + conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND, -1L); + try(TestContext ctx = new TestContext(conf, 1)) { + assertFalse(ctx.datanode.getBlockScanner().isEnabled()); } } From 10876333acd4194da3cb545c4f7a2b2cc88be959 Mon Sep 17 00:00:00 2001 From: Masatake Iwasaki Date: Mon, 28 Mar 2022 08:02:10 +0000 Subject: [PATCH 070/145] Make upstream aware of 3.2.3 release. --- .../markdown/release/3.2.3/CHANGELOG.3.2.3.md | 386 ++++++++++++++++++ .../release/3.2.3/RELEASENOTES.3.2.3.md | 71 ++++ 2 files changed, 457 insertions(+) create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/CHANGELOG.3.2.3.md create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/RELEASENOTES.3.2.3.md diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/CHANGELOG.3.2.3.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/CHANGELOG.3.2.3.md new file mode 100644 index 0000000000000..34928bf54e50d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/CHANGELOG.3.2.3.md @@ -0,0 +1,386 @@ + + +# Apache Hadoop Changelog + +## Release 3.2.3 - 2022-03-02 + + + +### NEW FEATURES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-15691](https://issues.apache.org/jira/browse/HADOOP-15691) | Add PathCapabilities to FS and FC to complement StreamCapabilities | Major | . | Steve Loughran | Steve Loughran | +| [HDFS-15711](https://issues.apache.org/jira/browse/HDFS-15711) | Add Metrics to HttpFS Server | Major | httpfs | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15759](https://issues.apache.org/jira/browse/HDFS-15759) | EC: Verify EC reconstruction correctness on DataNode | Major | datanode, ec, erasure-coding | Toshihiko Uchida | Toshihiko Uchida | +| [HDFS-16337](https://issues.apache.org/jira/browse/HDFS-16337) | Show start time of Datanode on Web | Minor | . | tomscut | tomscut | + + +### IMPROVEMENTS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-16052](https://issues.apache.org/jira/browse/HADOOP-16052) | Remove Subversion and Forrest from Dockerfile | Minor | build | Akira Ajisaka | Xieming Li | +| [YARN-9783](https://issues.apache.org/jira/browse/YARN-9783) | Remove low-level zookeeper test to be able to build Hadoop against zookeeper 3.5.5 | Major | test | Mate Szalay-Beko | Mate Szalay-Beko | +| [HADOOP-16717](https://issues.apache.org/jira/browse/HADOOP-16717) | Remove GenericsUtil isLog4jLogger dependency on Log4jLoggerAdapter | Major | . | David Mollitor | Xieming Li | +| [YARN-10036](https://issues.apache.org/jira/browse/YARN-10036) | Install yarnpkg and upgrade nodejs in Dockerfile | Major | buid, yarn-ui-v2 | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-16811](https://issues.apache.org/jira/browse/HADOOP-16811) | Use JUnit TemporaryFolder Rule in TestFileUtils | Minor | common, test | David Mollitor | David Mollitor | +| [HDFS-15075](https://issues.apache.org/jira/browse/HDFS-15075) | Remove process command timing from BPServiceActor | Major | . | Íñigo Goiri | Xiaoqiao He | +| [HADOOP-16054](https://issues.apache.org/jira/browse/HADOOP-16054) | Update Dockerfile to use Bionic | Major | build, test | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15574](https://issues.apache.org/jira/browse/HDFS-15574) | Remove unnecessary sort of block list in DirectoryScanner | Major | . | Stephen O'Donnell | Stephen O'Donnell | +| [HDFS-15583](https://issues.apache.org/jira/browse/HDFS-15583) | Backport DirectoryScanner improvements HDFS-14476, HDFS-14751 and HDFS-15048 to branch 3.2 and 3.1 | Major | datanode | Stephen O'Donnell | Stephen O'Donnell | +| [HDFS-15567](https://issues.apache.org/jira/browse/HDFS-15567) | [SBN Read] HDFS should expose msync() API to allow downstream applications call it explicitly. | Major | ha, hdfs-client | Konstantin Shvachko | Konstantin Shvachko | +| [HDFS-15633](https://issues.apache.org/jira/browse/HDFS-15633) | Avoid redundant RPC calls for getDiskStatus | Major | dfsclient | Ayush Saxena | Ayush Saxena | +| [YARN-10450](https://issues.apache.org/jira/browse/YARN-10450) | Add cpu and memory utilization per node and cluster-wide metrics | Minor | yarn | Jim Brennan | Jim Brennan | +| [HDFS-15652](https://issues.apache.org/jira/browse/HDFS-15652) | Make block size from NNThroughputBenchmark configurable | Minor | benchmarks | Hui Fei | Hui Fei | +| [YARN-10475](https://issues.apache.org/jira/browse/YARN-10475) | Scale RM-NM heartbeat interval based on node utilization | Minor | yarn | Jim Brennan | Jim Brennan | +| [HDFS-15665](https://issues.apache.org/jira/browse/HDFS-15665) | Balancer logging improvement | Major | balancer & mover | Konstantin Shvachko | Konstantin Shvachko | +| [HADOOP-17342](https://issues.apache.org/jira/browse/HADOOP-17342) | Creating a token identifier should not do kerberos name resolution | Major | common | Jim Brennan | Jim Brennan | +| [YARN-10479](https://issues.apache.org/jira/browse/YARN-10479) | RMProxy should retry on SocketTimeout Exceptions | Major | yarn | Jim Brennan | Jim Brennan | +| [HDFS-15623](https://issues.apache.org/jira/browse/HDFS-15623) | Respect configured values of rpc.engine | Major | hdfs | Hector Sandoval Chaverri | Hector Sandoval Chaverri | +| [HDFS-14395](https://issues.apache.org/jira/browse/HDFS-14395) | Remove WARN Logging From Interrupts in DataStreamer | Minor | hdfs-client | David Mollitor | David Mollitor | +| [HADOOP-17367](https://issues.apache.org/jira/browse/HADOOP-17367) | Add InetAddress api to ProxyUsers.authorize | Major | performance, security | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15694](https://issues.apache.org/jira/browse/HDFS-15694) | Avoid calling UpdateHeartBeatState inside DataNodeDescriptor | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15703](https://issues.apache.org/jira/browse/HDFS-15703) | Don't generate edits for set operations that are no-op | Major | namenode | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17392](https://issues.apache.org/jira/browse/HADOOP-17392) | Remote exception messages should not include the exception class | Major | ipc | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15706](https://issues.apache.org/jira/browse/HDFS-15706) | HttpFS: Log more information on request failures | Major | httpfs | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17389](https://issues.apache.org/jira/browse/HADOOP-17389) | KMS should log full UGI principal | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15720](https://issues.apache.org/jira/browse/HDFS-15720) | namenode audit async logger should add some log4j config | Minor | hdfs | Max Xie | | +| [HDFS-15704](https://issues.apache.org/jira/browse/HDFS-15704) | Mitigate lease monitor's rapid infinite loop | Major | namenode | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15751](https://issues.apache.org/jira/browse/HDFS-15751) | Add documentation for msync() API to filesystem.md | Major | documentation | Konstantin Shvachko | Konstantin Shvachko | +| [YARN-10538](https://issues.apache.org/jira/browse/YARN-10538) | Add recommissioning nodes to the list of updated nodes returned to the AM | Major | . | Srinivas S T | Srinivas S T | +| [YARN-4589](https://issues.apache.org/jira/browse/YARN-4589) | Diagnostics for localization timeouts is lacking | Major | . | Chang Li | Chang Li | +| [YARN-10562](https://issues.apache.org/jira/browse/YARN-10562) | Follow up changes for YARN-9833 | Major | yarn | Jim Brennan | Jim Brennan | +| [HDFS-15783](https://issues.apache.org/jira/browse/HDFS-15783) | Speed up BlockPlacementPolicyRackFaultTolerant#verifyBlockPlacement | Major | block placement | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17478](https://issues.apache.org/jira/browse/HADOOP-17478) | Improve the description of hadoop.http.authentication.signature.secret.file | Minor | documentation | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15789](https://issues.apache.org/jira/browse/HDFS-15789) | Lease renewal does not require namesystem lock | Major | hdfs | Jim Brennan | Jim Brennan | +| [HADOOP-17501](https://issues.apache.org/jira/browse/HADOOP-17501) | Fix logging typo in ShutdownHookManager | Major | common | Konstantin Shvachko | Fengnan Li | +| [HADOOP-17354](https://issues.apache.org/jira/browse/HADOOP-17354) | Move Jenkinsfile outside of the root directory | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15799](https://issues.apache.org/jira/browse/HDFS-15799) | Make DisallowedDatanodeException terse | Minor | hdfs | Richard | Richard | +| [HDFS-15813](https://issues.apache.org/jira/browse/HDFS-15813) | DataStreamer: keep sending heartbeat packets while streaming | Major | hdfs | Jim Brennan | Jim Brennan | +| [MAPREDUCE-7319](https://issues.apache.org/jira/browse/MAPREDUCE-7319) | Log list of mappers at trace level in ShuffleHandler audit log | Minor | yarn | Jim Brennan | Jim Brennan | +| [HDFS-15821](https://issues.apache.org/jira/browse/HDFS-15821) | Add metrics for in-service datanodes | Minor | . | Zehao Chen | Zehao Chen | +| [YARN-10626](https://issues.apache.org/jira/browse/YARN-10626) | Log resource allocation in NM log at container start time | Major | . | Eric Badger | Eric Badger | +| [HDFS-15815](https://issues.apache.org/jira/browse/HDFS-15815) | if required storageType are unavailable, log the failed reason during choosing Datanode | Minor | block placement | Yang Yun | Yang Yun | +| [HDFS-15826](https://issues.apache.org/jira/browse/HDFS-15826) | Solve the problem of incorrect progress of delegation tokens when loading FsImage | Major | . | JiangHua Zhu | JiangHua Zhu | +| [HDFS-15734](https://issues.apache.org/jira/browse/HDFS-15734) | [READ] DirectoryScanner#scan need not check StorageType.PROVIDED | Minor | datanode | Yuxuan Wang | Yuxuan Wang | +| [HADOOP-17538](https://issues.apache.org/jira/browse/HADOOP-17538) | Add kms-default.xml and httpfs-default.xml to site index | Minor | documentation | Masatake Iwasaki | Masatake Iwasaki | +| [YARN-10613](https://issues.apache.org/jira/browse/YARN-10613) | Config to allow Intra- and Inter-queue preemption to enable/disable conservativeDRF | Minor | capacity scheduler, scheduler preemption | Eric Payne | Eric Payne | +| [YARN-10653](https://issues.apache.org/jira/browse/YARN-10653) | Fixed the findbugs issues introduced by YARN-10647. | Major | . | Qi Zhu | Qi Zhu | +| [MAPREDUCE-7324](https://issues.apache.org/jira/browse/MAPREDUCE-7324) | ClientHSSecurityInfo class is in wrong META-INF file | Major | . | Eric Badger | Eric Badger | +| [HADOOP-17546](https://issues.apache.org/jira/browse/HADOOP-17546) | Update Description of hadoop-http-auth-signature-secret in HttpAuthentication.md | Minor | . | Ravuri Sushma sree | Ravuri Sushma sree | +| [YARN-10664](https://issues.apache.org/jira/browse/YARN-10664) | Allow parameter expansion in NM\_ADMIN\_USER\_ENV | Major | yarn | Jim Brennan | Jim Brennan | +| [HADOOP-17570](https://issues.apache.org/jira/browse/HADOOP-17570) | Apply YETUS-1102 to re-enable GitHub comments | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17594](https://issues.apache.org/jira/browse/HADOOP-17594) | DistCp: Expose the JobId for applications executing through run method | Major | . | Ayush Saxena | Ayush Saxena | +| [HDFS-15911](https://issues.apache.org/jira/browse/HDFS-15911) | Provide blocks moved count in Balancer iteration result | Major | balancer & mover | Viraj Jasani | Viraj Jasani | +| [HDFS-15919](https://issues.apache.org/jira/browse/HDFS-15919) | BlockPoolManager should log stack trace if unable to get Namenode addresses | Major | datanode | Stephen O'Donnell | Stephen O'Donnell | +| [HADOOP-16870](https://issues.apache.org/jira/browse/HADOOP-16870) | Use spotbugs-maven-plugin instead of findbugs-maven-plugin | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15932](https://issues.apache.org/jira/browse/HDFS-15932) | Improve the balancer error message when process exits abnormally. | Major | . | Renukaprasad C | Renukaprasad C | +| [HDFS-15931](https://issues.apache.org/jira/browse/HDFS-15931) | Fix non-static inner classes for better memory management | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-15942](https://issues.apache.org/jira/browse/HDFS-15942) | Increase Quota initialization threads | Major | namenode | Stephen O'Donnell | Stephen O'Donnell | +| [HDFS-15937](https://issues.apache.org/jira/browse/HDFS-15937) | Reduce memory used during datanode layout upgrade | Major | datanode | Stephen O'Donnell | Stephen O'Donnell | +| [HADOOP-17569](https://issues.apache.org/jira/browse/HADOOP-17569) | Building native code fails on Fedora 33 | Major | build, common | Kengo Seki | Masatake Iwasaki | +| [HADOOP-17633](https://issues.apache.org/jira/browse/HADOOP-17633) | Bump json-smart to 2.4.2 and nimbus-jose-jwt to 9.8 due to CVEs | Major | auth, build | helen huang | Viraj Jasani | +| [HADOOP-16822](https://issues.apache.org/jira/browse/HADOOP-16822) | Provide source artifacts for hadoop-client-api | Major | . | Karel Kolman | Karel Kolman | +| [HADOOP-17680](https://issues.apache.org/jira/browse/HADOOP-17680) | Allow ProtobufRpcEngine to be extensible | Major | common | Hector Sandoval Chaverri | Hector Sandoval Chaverri | +| [YARN-10123](https://issues.apache.org/jira/browse/YARN-10123) | Error message around yarn app -stop/start can be improved to highlight that an implementation at framework level is needed for the stop/start functionality to work | Minor | client, documentation | Siddharth Ahuja | Siddharth Ahuja | +| [HADOOP-17756](https://issues.apache.org/jira/browse/HADOOP-17756) | Increase precommit job timeout from 20 hours to 24 hours. | Major | build | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-16073](https://issues.apache.org/jira/browse/HDFS-16073) | Remove redundant RPC requests for getFileLinkInfo in ClientNamenodeProtocolTranslatorPB | Minor | . | lei w | lei w | +| [HDFS-16074](https://issues.apache.org/jira/browse/HDFS-16074) | Remove an expensive debug string concatenation | Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HDFS-15150](https://issues.apache.org/jira/browse/HDFS-15150) | Introduce read write lock to Datanode | Major | datanode | Stephen O'Donnell | Stephen O'Donnell | +| [YARN-10834](https://issues.apache.org/jira/browse/YARN-10834) | Intra-queue preemption: apps that don't use defined custom resource won't be preempted. | Major | . | Eric Payne | Eric Payne | +| [HADOOP-17749](https://issues.apache.org/jira/browse/HADOOP-17749) | Remove lock contention in SelectorPool of SocketIOWithTimeout | Major | common | Xuesen Liang | Xuesen Liang | +| [HADOOP-17775](https://issues.apache.org/jira/browse/HADOOP-17775) | Remove JavaScript package from Docker environment | Major | build | Masatake Iwasaki | Masatake Iwasaki | +| [HADOOP-17794](https://issues.apache.org/jira/browse/HADOOP-17794) | Add a sample configuration to use ZKDelegationTokenSecretManager in Hadoop KMS | Major | documentation, kms, security | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-12665](https://issues.apache.org/jira/browse/HADOOP-12665) | Document hadoop.security.token.service.use\_ip | Major | documentation | Arpit Agarwal | Akira Ajisaka | +| [YARN-10456](https://issues.apache.org/jira/browse/YARN-10456) | RM PartitionQueueMetrics records are named QueueMetrics in Simon metrics registry | Major | resourcemanager | Eric Payne | Eric Payne | +| [HDFS-15650](https://issues.apache.org/jira/browse/HDFS-15650) | Make the socket timeout for computing checksum of striped blocks configurable | Minor | datanode, ec, erasure-coding | Yushi Hayasaka | Yushi Hayasaka | +| [YARN-10858](https://issues.apache.org/jira/browse/YARN-10858) | [UI2] YARN-10826 breaks Queue view | Major | yarn-ui-v2 | Andras Gyori | Masatake Iwasaki | +| [YARN-10860](https://issues.apache.org/jira/browse/YARN-10860) | Make max container per heartbeat configs refreshable | Major | . | Eric Badger | Eric Badger | +| [HADOOP-17813](https://issues.apache.org/jira/browse/HADOOP-17813) | Checkstyle - Allow line length: 100 | Major | . | Akira Ajisaka | Viraj Jasani | +| [HADOOP-17819](https://issues.apache.org/jira/browse/HADOOP-17819) | Add extensions to ProtobufRpcEngine RequestHeaderProto | Major | common | Hector Sandoval Chaverri | Hector Sandoval Chaverri | +| [HDFS-16153](https://issues.apache.org/jira/browse/HDFS-16153) | Avoid evaluation of LOG.debug statement in QuorumJournalManager | Trivial | . | wangzhaohui | wangzhaohui | +| [HDFS-16154](https://issues.apache.org/jira/browse/HDFS-16154) | TestMiniJournalCluster failing intermittently because of not reseting UserGroupInformation completely | Minor | . | wangzhaohui | wangzhaohui | +| [HADOOP-17849](https://issues.apache.org/jira/browse/HADOOP-17849) | Exclude spotbugs-annotations from transitive dependencies on branch-3.2 | Major | . | Masatake Iwasaki | Masatake Iwasaki | +| [HDFS-16173](https://issues.apache.org/jira/browse/HDFS-16173) | Improve CopyCommands#Put#executor queue configurability | Major | fs | JiangHua Zhu | JiangHua Zhu | +| [HDFS-15160](https://issues.apache.org/jira/browse/HDFS-15160) | ReplicaMap, Disk Balancer, Directory Scanner and various FsDatasetImpl methods should use datanode readlock | Major | datanode | Stephen O'Donnell | Stephen O'Donnell | +| [HDFS-14997](https://issues.apache.org/jira/browse/HDFS-14997) | BPServiceActor processes commands from NameNode asynchronously | Major | datanode | Xiaoqiao He | Xiaoqiao He | +| [HDFS-16241](https://issues.apache.org/jira/browse/HDFS-16241) | Standby close reconstruction thread | Major | . | zhanghuazong | zhanghuazong | +| [HDFS-16286](https://issues.apache.org/jira/browse/HDFS-16286) | Debug tool to verify the correctness of erasure coding on file | Minor | erasure-coding, tools | daimin | daimin | +| [HADOOP-17998](https://issues.apache.org/jira/browse/HADOOP-17998) | Allow get command to run with multi threads. | Major | fs | Chengwei Wang | Chengwei Wang | +| [HADOOP-18023](https://issues.apache.org/jira/browse/HADOOP-18023) | Allow cp command to run with multi threads. | Major | fs | Chengwei Wang | Chengwei Wang | +| [HADOOP-17643](https://issues.apache.org/jira/browse/HADOOP-17643) | WASB : Make metadata checks case insensitive | Major | . | Anoop Sam John | Anoop Sam John | +| [HDFS-16386](https://issues.apache.org/jira/browse/HDFS-16386) | Reduce DataNode load when FsDatasetAsyncDiskService is working | Major | datanode | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16430](https://issues.apache.org/jira/browse/HDFS-16430) | Validate maximum blocks in EC group when adding an EC policy | Minor | ec, erasure-coding | daimin | daimin | +| [HDFS-16403](https://issues.apache.org/jira/browse/HDFS-16403) | Improve FUSE IO performance by supporting FUSE parameter max\_background | Minor | fuse-dfs | daimin | daimin | +| [HADOOP-18093](https://issues.apache.org/jira/browse/HADOOP-18093) | Better exception handling for testFileStatusOnMountLink() in ViewFsBaseTest.java | Trivial | . | Xing Lin | Xing Lin | +| [HADOOP-18155](https://issues.apache.org/jira/browse/HADOOP-18155) | Refactor tests in TestFileUtil | Trivial | common | Gautham Banasandra | Gautham Banasandra | + + +### BUG FIXES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-15939](https://issues.apache.org/jira/browse/HADOOP-15939) | Filter overlapping objenesis class in hadoop-client-minicluster | Minor | build | Xiaoyu Yao | Xiaoyu Yao | +| [YARN-8936](https://issues.apache.org/jira/browse/YARN-8936) | Bump up Atsv2 hbase versions | Major | . | Rohith Sharma K S | Vrushali C | +| [HDFS-14189](https://issues.apache.org/jira/browse/HDFS-14189) | Fix intermittent failure of TestNameNodeMetrics | Major | . | Ayush Saxena | Ayush Saxena | +| [YARN-9246](https://issues.apache.org/jira/browse/YARN-9246) | NPE when executing a command yarn node -status or -states without additional arguments | Minor | client | Masahiro Tanaka | Masahiro Tanaka | +| [YARN-7266](https://issues.apache.org/jira/browse/YARN-7266) | Timeline Server event handler threads locked | Major | ATSv2, timelineserver | Venkata Puneet Ravuri | Prabhu Joseph | +| [YARN-9990](https://issues.apache.org/jira/browse/YARN-9990) | Testcase fails with "Insufficient configured threads: required=16 \< max=10" | Major | . | Prabhu Joseph | Prabhu Joseph | +| [YARN-10020](https://issues.apache.org/jira/browse/YARN-10020) | Fix build instruction of hadoop-yarn-ui | Minor | yarn-ui-v2 | Masatake Iwasaki | Masatake Iwasaki | +| [YARN-10037](https://issues.apache.org/jira/browse/YARN-10037) | Upgrade build tools for YARN Web UI v2 | Major | build, security, yarn-ui-v2 | Akira Ajisaka | Masatake Iwasaki | +| [HDFS-15187](https://issues.apache.org/jira/browse/HDFS-15187) | CORRUPT replica mismatch between namenodes after failover | Critical | . | Ayush Saxena | Ayush Saxena | +| [HDFS-15200](https://issues.apache.org/jira/browse/HDFS-15200) | Delete Corrupt Replica Immediately Irrespective of Replicas On Stale Storage | Critical | . | Ayush Saxena | Ayush Saxena | +| [HDFS-15113](https://issues.apache.org/jira/browse/HDFS-15113) | Missing IBR when NameNode restart if open processCommand async feature | Blocker | datanode | Xiaoqiao He | Xiaoqiao He | +| [HDFS-15210](https://issues.apache.org/jira/browse/HDFS-15210) | EC : File write hanged when DN is shutdown by admin command. | Major | ec | Surendra Singh Lilhore | Surendra Singh Lilhore | +| [HADOOP-16768](https://issues.apache.org/jira/browse/HADOOP-16768) | SnappyCompressor test cases wrongly assume that the compressed data is always smaller than the input data | Major | io, test | zhao bo | Akira Ajisaka | +| [HDFS-11041](https://issues.apache.org/jira/browse/HDFS-11041) | Unable to unregister FsDatasetState MBean if DataNode is shutdown twice | Trivial | datanode | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HADOOP-17068](https://issues.apache.org/jira/browse/HADOOP-17068) | client fails forever when namenode ipaddr changed | Major | hdfs-client | Sean Chow | Sean Chow | +| [HDFS-15378](https://issues.apache.org/jira/browse/HDFS-15378) | TestReconstructStripedFile#testErasureCodingWorkerXmitsWeight is failing on trunk | Major | . | Hemanth Boyina | Hemanth Boyina | +| [YARN-10331](https://issues.apache.org/jira/browse/YARN-10331) | Upgrade node.js to 10.21.0 | Critical | build, yarn-ui-v2 | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17119](https://issues.apache.org/jira/browse/HADOOP-17119) | Jetty upgrade to 9.4.x causes MR app fail with IOException | Major | . | Bilwa S T | Bilwa S T | +| [HADOOP-17138](https://issues.apache.org/jira/browse/HADOOP-17138) | Fix spotbugs warnings surfaced after upgrade to 4.0.6 | Minor | . | Masatake Iwasaki | Masatake Iwasaki | +| [HDFS-15439](https://issues.apache.org/jira/browse/HDFS-15439) | Setting dfs.mover.retry.max.attempts to negative value will retry forever. | Major | balancer & mover | AMC-team | AMC-team | +| [YARN-10430](https://issues.apache.org/jira/browse/YARN-10430) | Log improvements in NodeStatusUpdaterImpl | Minor | nodemanager | Bilwa S T | Bilwa S T | +| [HDFS-15438](https://issues.apache.org/jira/browse/HDFS-15438) | Setting dfs.disk.balancer.max.disk.errors = 0 will fail the block copy | Major | balancer & mover | AMC-team | AMC-team | +| [YARN-10438](https://issues.apache.org/jira/browse/YARN-10438) | Handle null containerId in ClientRMService#getContainerReport() | Major | resourcemanager | Raghvendra Singh | Shubham Gupta | +| [HDFS-15628](https://issues.apache.org/jira/browse/HDFS-15628) | HttpFS server throws NPE if a file is a symlink | Major | fs, httpfs | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15627](https://issues.apache.org/jira/browse/HDFS-15627) | Audit log deletes before collecting blocks | Major | logging, namenode | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17309](https://issues.apache.org/jira/browse/HADOOP-17309) | Javadoc warnings and errors are ignored in the precommit jobs | Major | build, documentation | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17310](https://issues.apache.org/jira/browse/HADOOP-17310) | Touch command with -c option is broken | Major | . | Ayush Saxena | Ayush Saxena | +| [HDFS-15639](https://issues.apache.org/jira/browse/HDFS-15639) | [JDK 11] Fix Javadoc errors in hadoop-hdfs-client | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-15622](https://issues.apache.org/jira/browse/HDFS-15622) | Deleted blocks linger in the replications queue | Major | hdfs | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15641](https://issues.apache.org/jira/browse/HDFS-15641) | DataNode could meet deadlock if invoke refreshNameNode | Critical | . | Hongbing Wang | Hongbing Wang | +| [MAPREDUCE-7302](https://issues.apache.org/jira/browse/MAPREDUCE-7302) | Upgrading to JUnit 4.13 causes testcase TestFetcher.testCorruptedIFile() to fail | Major | test | Peter Bacsko | Peter Bacsko | +| [HDFS-15644](https://issues.apache.org/jira/browse/HDFS-15644) | Failed volumes can cause DNs to stop block reporting | Major | block placement, datanode | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17236](https://issues.apache.org/jira/browse/HADOOP-17236) | Bump up snakeyaml to 1.26 to mitigate CVE-2017-18640 | Major | . | Brahma Reddy Battula | Brahma Reddy Battula | +| [YARN-10467](https://issues.apache.org/jira/browse/YARN-10467) | ContainerIdPBImpl objects can be leaked in RMNodeImpl.completedContainers | Major | resourcemanager | Haibo Chen | Haibo Chen | +| [HADOOP-17329](https://issues.apache.org/jira/browse/HADOOP-17329) | mvn site commands fails due to MetricsSystemImpl changes | Major | . | Xiaoqiao He | Xiaoqiao He | +| [HDFS-15651](https://issues.apache.org/jira/browse/HDFS-15651) | Client could not obtain block when DN CommandProcessingThread exit | Major | . | Yiqun Lin | Mingxiang Li | +| [HADOOP-17340](https://issues.apache.org/jira/browse/HADOOP-17340) | TestLdapGroupsMapping failing -string mismatch in exception validation | Major | test | Steve Loughran | Steve Loughran | +| [HADOOP-17352](https://issues.apache.org/jira/browse/HADOOP-17352) | Update PATCH\_NAMING\_RULE in the personality file | Minor | build | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15485](https://issues.apache.org/jira/browse/HDFS-15485) | Fix outdated properties of JournalNode when performing rollback | Minor | . | Deegue | Deegue | +| [HADOOP-17358](https://issues.apache.org/jira/browse/HADOOP-17358) | Improve excessive reloading of Configurations | Major | conf | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15538](https://issues.apache.org/jira/browse/HDFS-15538) | Fix the documentation for dfs.namenode.replication.max-streams in hdfs-default.xml | Major | . | Xieming Li | Xieming Li | +| [HADOOP-17362](https://issues.apache.org/jira/browse/HADOOP-17362) | Doing hadoop ls on Har file triggers too many RPC calls | Major | fs | Ahmed Hussein | Ahmed Hussein | +| [YARN-10485](https://issues.apache.org/jira/browse/YARN-10485) | TimelineConnector swallows InterruptedException | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17360](https://issues.apache.org/jira/browse/HADOOP-17360) | Log the remote address for authentication success | Minor | ipc | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17346](https://issues.apache.org/jira/browse/HADOOP-17346) | Fair call queue is defeated by abusive service principals | Major | common, ipc | Ahmed Hussein | Ahmed Hussein | +| [YARN-10470](https://issues.apache.org/jira/browse/YARN-10470) | When building new web ui with root user, the bower install should support it. | Major | build, yarn-ui-v2 | Qi Zhu | Qi Zhu | +| [YARN-10498](https://issues.apache.org/jira/browse/YARN-10498) | Fix Yarn CapacityScheduler Markdown document | Trivial | documentation | zhaoshengjie | zhaoshengjie | +| [HDFS-15695](https://issues.apache.org/jira/browse/HDFS-15695) | NN should not let the balancer run in safemode | Major | namenode | Ahmed Hussein | Ahmed Hussein | +| [YARN-10511](https://issues.apache.org/jira/browse/YARN-10511) | Update yarn.nodemanager.env-whitelist value in docs | Minor | documentation | Andrea Scarpino | Andrea Scarpino | +| [HDFS-15707](https://issues.apache.org/jira/browse/HDFS-15707) | NNTop counts don't add up as expected | Major | hdfs, metrics, namenode | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15709](https://issues.apache.org/jira/browse/HDFS-15709) | EC: Socket file descriptor leak in StripedBlockChecksumReconstructor | Major | datanode, ec, erasure-coding | Yushi Hayasaka | Yushi Hayasaka | +| [YARN-10491](https://issues.apache.org/jira/browse/YARN-10491) | Fix deprecation warnings in SLSWebApp.java | Minor | build | Akira Ajisaka | Ankit Kumar | +| [HADOOP-13571](https://issues.apache.org/jira/browse/HADOOP-13571) | ServerSocketUtil.getPort() should use loopback address, not 0.0.0.0 | Major | . | Eric Badger | Eric Badger | +| [HDFS-15725](https://issues.apache.org/jira/browse/HDFS-15725) | Lease Recovery never completes for a committed block which the DNs never finalize | Major | namenode | Stephen O'Donnell | Stephen O'Donnell | +| [HDFS-15170](https://issues.apache.org/jira/browse/HDFS-15170) | EC: Block gets marked as CORRUPT in case of failover and pipeline recovery | Critical | erasure-coding | Ayush Saxena | Ayush Saxena | +| [HDFS-15719](https://issues.apache.org/jira/browse/HDFS-15719) | [Hadoop 3] Both NameNodes can crash simultaneously due to the short JN socket timeout | Critical | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [YARN-10560](https://issues.apache.org/jira/browse/YARN-10560) | Upgrade node.js to 10.23.1 and yarn to 1.22.5 in Web UI v2 | Major | webapp, yarn-ui-v2 | Akira Ajisaka | Akira Ajisaka | +| [YARN-10528](https://issues.apache.org/jira/browse/YARN-10528) | maxAMShare should only be accepted for leaf queues, not parent queues | Major | . | Siddharth Ahuja | Siddharth Ahuja | +| [HADOOP-17438](https://issues.apache.org/jira/browse/HADOOP-17438) | Increase docker memory limit in Jenkins | Major | build, scripts, test, yetus | Ahmed Hussein | Ahmed Hussein | +| [MAPREDUCE-7310](https://issues.apache.org/jira/browse/MAPREDUCE-7310) | Clear the fileMap in JHEventHandlerForSigtermTest | Minor | test | Zhengxi Li | Zhengxi Li | +| [HADOOP-16947](https://issues.apache.org/jira/browse/HADOOP-16947) | Stale record should be remove when MutableRollingAverages generating aggregate data. | Major | . | Haibin Huang | Haibin Huang | +| [HDFS-15632](https://issues.apache.org/jira/browse/HDFS-15632) | AbstractContractDeleteTest should set recursive parameter to true for recursive test cases. | Major | . | Konstantin Shvachko | Anton Kutuzov | +| [HDFS-10498](https://issues.apache.org/jira/browse/HDFS-10498) | Intermittent test failure org.apache.hadoop.hdfs.server.namenode.snapshot.TestSnapshotFileLength.testSnapshotfileLength | Major | hdfs, snapshots | Hanisha Koneru | Jim Brennan | +| [HADOOP-17506](https://issues.apache.org/jira/browse/HADOOP-17506) | Fix typo in BUILDING.txt | Trivial | documentation | Gautham Banasandra | Gautham Banasandra | +| [HDFS-15795](https://issues.apache.org/jira/browse/HDFS-15795) | EC: Wrong checksum when reconstruction was failed by exception | Major | datanode, ec, erasure-coding | Yushi Hayasaka | Yushi Hayasaka | +| [HDFS-15779](https://issues.apache.org/jira/browse/HDFS-15779) | EC: fix NPE caused by StripedWriter.clearBuffers during reconstruct block | Major | . | Hongbing Wang | Hongbing Wang | +| [HDFS-15798](https://issues.apache.org/jira/browse/HDFS-15798) | EC: Reconstruct task failed, and It would be XmitsInProgress of DN has negative number | Major | . | Haiyang Hu | Haiyang Hu | +| [YARN-10428](https://issues.apache.org/jira/browse/YARN-10428) | Zombie applications in the YARN queue using FAIR + sizebasedweight | Critical | capacityscheduler | Guang Yang | Andras Gyori | +| [YARN-10607](https://issues.apache.org/jira/browse/YARN-10607) | User environment is unable to prepend PATH when mapreduce.admin.user.env also sets PATH | Major | . | Eric Badger | Eric Badger | +| [HADOOP-17516](https://issues.apache.org/jira/browse/HADOOP-17516) | Upgrade ant to 1.10.9 | Major | . | Akira Ajisaka | Akira Ajisaka | +| [YARN-10500](https://issues.apache.org/jira/browse/YARN-10500) | TestDelegationTokenRenewer fails intermittently | Major | test | Akira Ajisaka | Masatake Iwasaki | +| [HADOOP-17534](https://issues.apache.org/jira/browse/HADOOP-17534) | Upgrade Jackson databind to 2.10.5.1 | Major | build | Adam Roberts | Akira Ajisaka | +| [MAPREDUCE-7323](https://issues.apache.org/jira/browse/MAPREDUCE-7323) | Remove job\_history\_summary.py | Major | . | Akira Ajisaka | Akira Ajisaka | +| [YARN-10647](https://issues.apache.org/jira/browse/YARN-10647) | Fix TestRMNodeLabelsManager failed after YARN-10501. | Major | . | Qi Zhu | Qi Zhu | +| [HADOOP-17510](https://issues.apache.org/jira/browse/HADOOP-17510) | Hadoop prints sensitive Cookie information. | Major | . | Renukaprasad C | Renukaprasad C | +| [HDFS-15422](https://issues.apache.org/jira/browse/HDFS-15422) | Reported IBR is partially replaced with stored info when queuing. | Critical | namenode | Kihwal Lee | Stephen O'Donnell | +| [YARN-10651](https://issues.apache.org/jira/browse/YARN-10651) | CapacityScheduler crashed with NPE in AbstractYarnScheduler.updateNodeResource() | Major | . | Haibo Chen | Haibo Chen | +| [MAPREDUCE-7320](https://issues.apache.org/jira/browse/MAPREDUCE-7320) | ClusterMapReduceTestCase does not clean directories | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HDFS-14013](https://issues.apache.org/jira/browse/HDFS-14013) | Skip any credentials stored in HDFS when starting ZKFC | Major | hdfs | Krzysztof Adamski | Stephen O'Donnell | +| [HDFS-15849](https://issues.apache.org/jira/browse/HDFS-15849) | ExpiredHeartbeats metric should be of Type.COUNTER | Major | metrics | Konstantin Shvachko | Qi Zhu | +| [YARN-10672](https://issues.apache.org/jira/browse/YARN-10672) | All testcases in TestReservations are flaky | Major | . | Szilard Nemeth | Szilard Nemeth | +| [HADOOP-17557](https://issues.apache.org/jira/browse/HADOOP-17557) | skip-dir option is not processed by Yetus | Major | build, precommit, yetus | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15875](https://issues.apache.org/jira/browse/HDFS-15875) | Check whether file is being truncated before truncate | Major | . | Hui Fei | Hui Fei | +| [HADOOP-17582](https://issues.apache.org/jira/browse/HADOOP-17582) | Replace GitHub App Token with GitHub OAuth token | Major | build | Akira Ajisaka | Akira Ajisaka | +| [YARN-10687](https://issues.apache.org/jira/browse/YARN-10687) | Add option to disable/enable free disk space checking and percentage checking for full and not-full disks | Major | nodemanager | Qi Zhu | Qi Zhu | +| [HADOOP-17586](https://issues.apache.org/jira/browse/HADOOP-17586) | Upgrade org.codehaus.woodstox:stax2-api to 4.2.1 | Major | . | Ayush Saxena | Ayush Saxena | +| [HADOOP-17585](https://issues.apache.org/jira/browse/HADOOP-17585) | Correct timestamp format in the docs for the touch command | Major | . | Stephen O'Donnell | Stephen O'Donnell | +| [YARN-10588](https://issues.apache.org/jira/browse/YARN-10588) | Percentage of queue and cluster is zero in WebUI | Major | . | Bilwa S T | Bilwa S T | +| [MAPREDUCE-7322](https://issues.apache.org/jira/browse/MAPREDUCE-7322) | revisiting TestMRIntermediateDataEncryption | Major | job submission, security, test | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17592](https://issues.apache.org/jira/browse/HADOOP-17592) | Fix the wrong CIDR range example in Proxy User documentation | Minor | documentation | Kwangsun Noh | Kwangsun Noh | +| [YARN-10706](https://issues.apache.org/jira/browse/YARN-10706) | Upgrade com.github.eirslett:frontend-maven-plugin to 1.11.2 | Major | buid | Mingliang Liu | Mingliang Liu | +| [MAPREDUCE-7325](https://issues.apache.org/jira/browse/MAPREDUCE-7325) | Intermediate data encryption is broken in LocalJobRunner | Major | job submission, security | Ahmed Hussein | Ahmed Hussein | +| [YARN-10697](https://issues.apache.org/jira/browse/YARN-10697) | Resources are displayed in bytes in UI for schedulers other than capacity | Major | . | Bilwa S T | Bilwa S T | +| [HADOOP-17602](https://issues.apache.org/jira/browse/HADOOP-17602) | Upgrade JUnit to 4.13.1 | Major | build, security, test | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15900](https://issues.apache.org/jira/browse/HDFS-15900) | RBF: empty blockpool id on dfsrouter caused by UNAVAILABLE NameNode | Major | rbf | Harunobu Daikoku | Harunobu Daikoku | +| [YARN-10501](https://issues.apache.org/jira/browse/YARN-10501) | Can't remove all node labels after add node label without nodemanager port | Critical | yarn | caozhiqiang | caozhiqiang | +| [YARN-10716](https://issues.apache.org/jira/browse/YARN-10716) | Fix typo in ContainerRuntime | Trivial | documentation | Wanqiang Ji | xishuhai | +| [HDFS-15950](https://issues.apache.org/jira/browse/HDFS-15950) | Remove unused hdfs.proto import | Major | hdfs-client | Gautham Banasandra | Gautham Banasandra | +| [HDFS-15949](https://issues.apache.org/jira/browse/HDFS-15949) | Fix integer overflow | Major | libhdfs++ | Gautham Banasandra | Gautham Banasandra | +| [HDFS-15948](https://issues.apache.org/jira/browse/HDFS-15948) | Fix test4tests for libhdfspp | Critical | build, libhdfs++ | Gautham Banasandra | Gautham Banasandra | +| [HADOOP-17608](https://issues.apache.org/jira/browse/HADOOP-17608) | Fix TestKMS failure | Major | kms | Akira Ajisaka | Akira Ajisaka | +| [YARN-10460](https://issues.apache.org/jira/browse/YARN-10460) | Upgrading to JUnit 4.13 causes tests in TestNodeStatusUpdater to fail | Major | nodemanager, test | Peter Bacsko | Peter Bacsko | +| [HADOOP-17641](https://issues.apache.org/jira/browse/HADOOP-17641) | ITestWasbUriAndConfiguration.testCanonicalServiceName() failing now mockaccount exists | Minor | fs/azure, test | Steve Loughran | Steve Loughran | +| [HADOOP-17655](https://issues.apache.org/jira/browse/HADOOP-17655) | Upgrade Jetty to 9.4.40 | Blocker | . | Akira Ajisaka | Akira Ajisaka | +| [YARN-10749](https://issues.apache.org/jira/browse/YARN-10749) | Can't remove all node labels after add node label without nodemanager port, broken by YARN-10647 | Major | . | D M Murali Krishna Reddy | D M Murali Krishna Reddy | +| [HDFS-15621](https://issues.apache.org/jira/browse/HDFS-15621) | Datanode DirectoryScanner uses excessive memory | Major | datanode | Stephen O'Donnell | Stephen O'Donnell | +| [YARN-10756](https://issues.apache.org/jira/browse/YARN-10756) | Remove additional junit 4.11 dependency from javadoc | Major | build, test, timelineservice | ANANDA G B | Akira Ajisaka | +| [YARN-10555](https://issues.apache.org/jira/browse/YARN-10555) | Missing access check before getAppAttempts | Critical | webapp | lujie | lujie | +| [HADOOP-17703](https://issues.apache.org/jira/browse/HADOOP-17703) | checkcompatibility.py errors out when specifying annotations | Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HADOOP-14922](https://issues.apache.org/jira/browse/HADOOP-14922) | Build of Mapreduce Native Task module fails with unknown opcode "bswap" | Major | . | Anup Halarnkar | Anup Halarnkar | +| [HADOOP-17718](https://issues.apache.org/jira/browse/HADOOP-17718) | Explicitly set locale in the Dockerfile | Blocker | build | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HADOOP-17700](https://issues.apache.org/jira/browse/HADOOP-17700) | ExitUtil#halt info log should log HaltException | Major | . | Viraj Jasani | Viraj Jasani | +| [YARN-10770](https://issues.apache.org/jira/browse/YARN-10770) | container-executor permission is wrong in SecureContainer.md | Major | documentation | Akira Ajisaka | Siddharth Ahuja | +| [HDFS-15915](https://issues.apache.org/jira/browse/HDFS-15915) | Race condition with async edits logging due to updating txId outside of the namesystem log | Major | hdfs, namenode | Konstantin Shvachko | Konstantin Shvachko | +| [HDFS-16040](https://issues.apache.org/jira/browse/HDFS-16040) | RpcQueueTime metric counts requeued calls as unique events. | Major | hdfs | Simbarashe Dzinamarira | Simbarashe Dzinamarira | +| [YARN-10809](https://issues.apache.org/jira/browse/YARN-10809) | testWithHbaseConfAtHdfsFileSystem consistently failing | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-16055](https://issues.apache.org/jira/browse/HDFS-16055) | Quota is not preserved in snapshot INode | Major | hdfs | Siyao Meng | Siyao Meng | +| [HDFS-16068](https://issues.apache.org/jira/browse/HDFS-16068) | WebHdfsFileSystem has a possible connection leak in connection with HttpFS | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [YARN-10767](https://issues.apache.org/jira/browse/YARN-10767) | Yarn Logs Command retrying on Standby RM for 30 times | Major | . | D M Murali Krishna Reddy | D M Murali Krishna Reddy | +| [HDFS-15618](https://issues.apache.org/jira/browse/HDFS-15618) | Improve datanode shutdown latency | Major | datanode | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17760](https://issues.apache.org/jira/browse/HADOOP-17760) | Delete hadoop.ssl.enabled and dfs.https.enable from docs and core-default.xml | Major | documentation | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-13671](https://issues.apache.org/jira/browse/HDFS-13671) | Namenode deletes large dir slowly caused by FoldedTreeSet#removeAndGet | Major | . | Yiqun Lin | Haibin Huang | +| [HDFS-16061](https://issues.apache.org/jira/browse/HDFS-16061) | DFTestUtil.waitReplication can produce false positives | Major | hdfs | Ahmed Hussein | Ahmed Hussein | +| [HDFS-14575](https://issues.apache.org/jira/browse/HDFS-14575) | LeaseRenewer#daemon threads leak in DFSClient | Major | . | Tao Yang | Renukaprasad C | +| [YARN-10826](https://issues.apache.org/jira/browse/YARN-10826) | [UI2] Upgrade Node.js to at least v12.22.1 | Major | yarn-ui-v2 | Akira Ajisaka | Masatake Iwasaki | +| [YARN-10828](https://issues.apache.org/jira/browse/YARN-10828) | Backport YARN-9789 to branch-3.2 | Major | . | Tarun Parimi | Tarun Parimi | +| [HADOOP-17769](https://issues.apache.org/jira/browse/HADOOP-17769) | Upgrade JUnit to 4.13.2 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [YARN-10824](https://issues.apache.org/jira/browse/YARN-10824) | Title not set for JHS and NM webpages | Major | . | Rajshree Mishra | Bilwa S T | +| [HDFS-16092](https://issues.apache.org/jira/browse/HDFS-16092) | Avoid creating LayoutFlags redundant objects | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-16108](https://issues.apache.org/jira/browse/HDFS-16108) | Incorrect log placeholders used in JournalNodeSyncer | Minor | . | Viraj Jasani | Viraj Jasani | +| [MAPREDUCE-7353](https://issues.apache.org/jira/browse/MAPREDUCE-7353) | Mapreduce job fails when NM is stopped | Major | . | Bilwa S T | Bilwa S T | +| [HDFS-16121](https://issues.apache.org/jira/browse/HDFS-16121) | Iterative snapshot diff report can generate duplicate records for creates, deletes and Renames | Major | snapshots | Srinivasu Majeti | Shashikant Banerjee | +| [HDFS-15796](https://issues.apache.org/jira/browse/HDFS-15796) | ConcurrentModificationException error happens on NameNode occasionally | Critical | hdfs | Daniel Ma | Daniel Ma | +| [HADOOP-17793](https://issues.apache.org/jira/browse/HADOOP-17793) | Better token validation | Major | . | Artem Smotrakov | Artem Smotrakov | +| [HDFS-16042](https://issues.apache.org/jira/browse/HDFS-16042) | DatanodeAdminMonitor scan should be delay based | Major | datanode | Ahmed Hussein | Ahmed Hussein | +| [HDFS-16127](https://issues.apache.org/jira/browse/HDFS-16127) | Improper pipeline close recovery causes a permanent write failure or data loss. | Major | . | Kihwal Lee | Kihwal Lee | +| [HADOOP-17028](https://issues.apache.org/jira/browse/HADOOP-17028) | ViewFS should initialize target filesystems lazily | Major | client-mounts, fs, viewfs | Uma Maheswara Rao G | Abhishek Das | +| [HDFS-12920](https://issues.apache.org/jira/browse/HDFS-12920) | HDFS default value change (with adding time unit) breaks old version MR tarball work with Hadoop 3.x | Critical | configuration, hdfs | Junping Du | Akira Ajisaka | +| [YARN-10813](https://issues.apache.org/jira/browse/YARN-10813) | Set default capacity of root for node labels | Major | . | Andras Gyori | Andras Gyori | +| [YARN-9551](https://issues.apache.org/jira/browse/YARN-9551) | TestTimelineClientV2Impl.testSyncCall fails intermittently | Minor | ATSv2, test | Prabhu Joseph | Andras Gyori | +| [HDFS-15175](https://issues.apache.org/jira/browse/HDFS-15175) | Multiple CloseOp shared block instance causes the standby namenode to crash when rolling editlog | Critical | . | Yicong Cai | Wan Chang | +| [YARN-10789](https://issues.apache.org/jira/browse/YARN-10789) | RM HA startup can fail due to race conditions in ZKConfigurationStore | Major | . | Tarun Parimi | Tarun Parimi | +| [YARN-6221](https://issues.apache.org/jira/browse/YARN-6221) | Entities missing from ATS when summary log file info got returned to the ATS before the domain log | Critical | yarn | Sushmitha Sreenivasan | Xiaomin Zhang | +| [MAPREDUCE-7258](https://issues.apache.org/jira/browse/MAPREDUCE-7258) | HistoryServerRest.html#Task\_Counters\_API, modify the jobTaskCounters's itemName from "taskcounterGroup" to "taskCounterGroup". | Minor | documentation | jenny | jenny | +| [YARN-8990](https://issues.apache.org/jira/browse/YARN-8990) | Fix fair scheduler race condition in app submit and queue cleanup | Blocker | fairscheduler | Wilfred Spiegelenburg | Wilfred Spiegelenburg | +| [YARN-8992](https://issues.apache.org/jira/browse/YARN-8992) | Fair scheduler can delete a dynamic queue while an application attempt is being added to the queue | Major | fairscheduler | Haibo Chen | Wilfred Spiegelenburg | +| [HADOOP-17370](https://issues.apache.org/jira/browse/HADOOP-17370) | Upgrade commons-compress to 1.21 | Major | common | Dongjoon Hyun | Akira Ajisaka | +| [HADOOP-17844](https://issues.apache.org/jira/browse/HADOOP-17844) | Upgrade JSON smart to 2.4.7 | Major | . | Renukaprasad C | Renukaprasad C | +| [HADOOP-17850](https://issues.apache.org/jira/browse/HADOOP-17850) | Upgrade ZooKeeper to 3.4.14 in branch-3.2 | Major | . | Akira Ajisaka | Masatake Iwasaki | +| [HDFS-16177](https://issues.apache.org/jira/browse/HDFS-16177) | Bug fix for Util#receiveFile | Minor | . | tomscut | tomscut | +| [YARN-10814](https://issues.apache.org/jira/browse/YARN-10814) | YARN shouldn't start with empty hadoop.http.authentication.signature.secret.file | Major | . | Benjamin Teke | Tamas Domok | +| [HADOOP-17858](https://issues.apache.org/jira/browse/HADOOP-17858) | Avoid possible class loading deadlock with VerifierNone initialization | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17886](https://issues.apache.org/jira/browse/HADOOP-17886) | Upgrade ant to 1.10.11 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [YARN-10901](https://issues.apache.org/jira/browse/YARN-10901) | Permission checking error on an existing directory in LogAggregationFileController#verifyAndCreateRemoteLogDir | Major | nodemanager | Tamas Domok | Tamas Domok | +| [HDFS-16187](https://issues.apache.org/jira/browse/HDFS-16187) | SnapshotDiff behaviour with Xattrs and Acls is not consistent across NN restarts with checkpointing | Major | snapshots | Srinivasu Majeti | Shashikant Banerjee | +| [HDFS-16198](https://issues.apache.org/jira/browse/HDFS-16198) | Short circuit read leaks Slot objects when InvalidToken exception is thrown | Major | . | Eungsop Yoo | Eungsop Yoo | +| [HADOOP-17917](https://issues.apache.org/jira/browse/HADOOP-17917) | Backport HADOOP-15993 to branch-3.2 which address CVE-2014-4611 | Major | . | Brahma Reddy Battula | Brahma Reddy Battula | +| [HDFS-16233](https://issues.apache.org/jira/browse/HDFS-16233) | Do not use exception handler to implement copy-on-write for EnumCounters | Major | namenode | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HDFS-16235](https://issues.apache.org/jira/browse/HDFS-16235) | Deadlock in LeaseRenewer for static remove method | Major | hdfs | angerszhu | angerszhu | +| [HADOOP-17940](https://issues.apache.org/jira/browse/HADOOP-17940) | Upgrade Kafka to 2.8.1 | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-16272](https://issues.apache.org/jira/browse/HDFS-16272) | Int overflow in computing safe length during EC block recovery | Critical | 3.1.1 | daimin | daimin | +| [HADOOP-17971](https://issues.apache.org/jira/browse/HADOOP-17971) | Exclude IBM Java security classes from being shaded/relocated | Major | build | Nicholas Marion | Nicholas Marion | +| [HADOOP-17972](https://issues.apache.org/jira/browse/HADOOP-17972) | Backport HADOOP-17683 for branch-3.2 | Major | security | Ananya Singh | Ananya Singh | +| [HADOOP-17993](https://issues.apache.org/jira/browse/HADOOP-17993) | Disable JIRA plugin for YETUS on Hadoop | Critical | build | Gautham Banasandra | Gautham Banasandra | +| [HDFS-16182](https://issues.apache.org/jira/browse/HDFS-16182) | numOfReplicas is given the wrong value in BlockPlacementPolicyDefault$chooseTarget can cause DataStreamer to fail with Heterogeneous Storage | Major | namanode | Max Xie | Max Xie | +| [HDFS-16350](https://issues.apache.org/jira/browse/HDFS-16350) | Datanode start time should be set after RPC server starts successfully | Minor | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-13500](https://issues.apache.org/jira/browse/HADOOP-13500) | Synchronizing iteration of Configuration properties object | Major | conf | Jason Darrell Lowe | Dhananjay Badaya | +| [HDFS-16317](https://issues.apache.org/jira/browse/HDFS-16317) | Backport HDFS-14729 for branch-3.2 | Major | security | Ananya Singh | Ananya Singh | +| [HDFS-14099](https://issues.apache.org/jira/browse/HDFS-14099) | Unknown frame descriptor when decompressing multiple frames in ZStandardDecompressor | Major | . | xuzq | xuzq | +| [HDFS-16410](https://issues.apache.org/jira/browse/HDFS-16410) | Insecure Xml parsing in OfflineEditsXmlLoader | Minor | . | Ashutosh Gupta | Ashutosh Gupta | +| [HDFS-16420](https://issues.apache.org/jira/browse/HDFS-16420) | Avoid deleting unique data blocks when deleting redundancy striped blocks | Critical | ec, erasure-coding | qinyuren | Jackson Wang | +| [HDFS-16428](https://issues.apache.org/jira/browse/HDFS-16428) | Source path with storagePolicy cause wrong typeConsumed while rename | Major | hdfs, namenode | lei w | lei w | +| [HDFS-16437](https://issues.apache.org/jira/browse/HDFS-16437) | ReverseXML processor doesn't accept XML files without the SnapshotDiffSection. | Critical | hdfs | yanbin.zhang | yanbin.zhang | +| [HDFS-16422](https://issues.apache.org/jira/browse/HDFS-16422) | Fix thread safety of EC decoding during concurrent preads | Critical | dfsclient, ec, erasure-coding | daimin | daimin | + + +### TESTS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [YARN-9338](https://issues.apache.org/jira/browse/YARN-9338) | Timeline related testcases are failing | Major | . | Prabhu Joseph | Abhishek Modi | +| [HDFS-15092](https://issues.apache.org/jira/browse/HDFS-15092) | TestRedudantBlocks#testProcessOverReplicatedAndRedudantBlock sometimes fails | Minor | test | Hui Fei | Hui Fei | + + +### SUB-TASKS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-15775](https://issues.apache.org/jira/browse/HADOOP-15775) | [JDK9] Add missing javax.activation-api dependency | Critical | test | Akira Ajisaka | Akira Ajisaka | +| [YARN-9875](https://issues.apache.org/jira/browse/YARN-9875) | FSSchedulerConfigurationStore fails to update with hdfs path | Major | capacityscheduler | Prabhu Joseph | Prabhu Joseph | +| [HADOOP-16764](https://issues.apache.org/jira/browse/HADOOP-16764) | Rewrite Python example codes using Python3 | Minor | documentation | Kengo Seki | Kengo Seki | +| [HADOOP-16905](https://issues.apache.org/jira/browse/HADOOP-16905) | Update jackson-databind to 2.10.3 to relieve us from the endless CVE patches | Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [YARN-10337](https://issues.apache.org/jira/browse/YARN-10337) | TestRMHATimelineCollectors fails on hadoop trunk | Major | test, yarn | Ahmed Hussein | Bilwa S T | +| [HDFS-15464](https://issues.apache.org/jira/browse/HDFS-15464) | ViewFsOverloadScheme should work when -fs option pointing to remote cluster without mount links | Major | viewfsOverloadScheme | Uma Maheswara Rao G | Uma Maheswara Rao G | +| [HDFS-15478](https://issues.apache.org/jira/browse/HDFS-15478) | When Empty mount points, we are assigning fallback link to self. But it should not use full URI for target fs. | Major | . | Uma Maheswara Rao G | Uma Maheswara Rao G | +| [HDFS-15459](https://issues.apache.org/jira/browse/HDFS-15459) | TestBlockTokenWithDFSStriped fails intermittently | Major | hdfs | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15461](https://issues.apache.org/jira/browse/HDFS-15461) | TestDFSClientRetries#testGetFileChecksum fails intermittently | Major | dfsclient, test | Ahmed Hussein | Ahmed Hussein | +| [HDFS-9776](https://issues.apache.org/jira/browse/HDFS-9776) | TestHAAppend#testMultipleAppendsDuringCatchupTailing is flaky | Major | . | Vinayakumar B | Ahmed Hussein | +| [HDFS-15457](https://issues.apache.org/jira/browse/HDFS-15457) | TestFsDatasetImpl fails intermittently | Major | hdfs | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17330](https://issues.apache.org/jira/browse/HADOOP-17330) | Backport HADOOP-16005-"NativeAzureFileSystem does not support setXAttr" to branch-3.2 | Major | fs/azure | Sally Zuo | Sally Zuo | +| [HDFS-15643](https://issues.apache.org/jira/browse/HDFS-15643) | EC: Fix checksum computation in case of native encoders | Blocker | . | Ahmed Hussein | Ayush Saxena | +| [HADOOP-17325](https://issues.apache.org/jira/browse/HADOOP-17325) | WASB: Test failures | Major | fs/azure, test | Sneha Vijayarajan | Steve Loughran | +| [HDFS-15716](https://issues.apache.org/jira/browse/HDFS-15716) | TestUpgradeDomainBlockPlacementPolicy flaky | Major | namenode, test | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15762](https://issues.apache.org/jira/browse/HDFS-15762) | TestMultipleNNPortQOP#testMultipleNNPortOverwriteDownStream fails intermittently | Minor | . | Toshihiko Uchida | Toshihiko Uchida | +| [HDFS-15672](https://issues.apache.org/jira/browse/HDFS-15672) | TestBalancerWithMultipleNameNodes#testBalancingBlockpoolsWithBlockPoolPolicy fails on trunk | Major | . | Ahmed Hussein | Masatake Iwasaki | +| [HDFS-15818](https://issues.apache.org/jira/browse/HDFS-15818) | Fix TestFsDatasetImpl.testReadLockCanBeDisabledByConfig | Minor | test | Leon Gao | Leon Gao | +| [HADOOP-16748](https://issues.apache.org/jira/browse/HADOOP-16748) | Migrate to Python 3 and upgrade Yetus to 0.13.0 | Major | . | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15890](https://issues.apache.org/jira/browse/HDFS-15890) | Improve the Logs for File Concat Operation | Minor | namenode | Bhavik Patel | Bhavik Patel | +| [HDFS-13975](https://issues.apache.org/jira/browse/HDFS-13975) | TestBalancer#testMaxIterationTime fails sporadically | Major | . | Jason Darrell Lowe | Toshihiko Uchida | +| [YARN-10688](https://issues.apache.org/jira/browse/YARN-10688) | ClusterMetrics should support GPU capacity related metrics. | Major | metrics, resourcemanager | Qi Zhu | Qi Zhu | +| [HDFS-15902](https://issues.apache.org/jira/browse/HDFS-15902) | Improve the log for HTTPFS server operation | Minor | httpfs | Bhavik Patel | Bhavik Patel | +| [HDFS-15940](https://issues.apache.org/jira/browse/HDFS-15940) | Some tests in TestBlockRecovery are consistently failing | Major | . | Viraj Jasani | Viraj Jasani | +| [YARN-10702](https://issues.apache.org/jira/browse/YARN-10702) | Add cluster metric for amount of CPU used by RM Event Processor | Minor | yarn | Jim Brennan | Jim Brennan | +| [HADOOP-17630](https://issues.apache.org/jira/browse/HADOOP-17630) | [JDK 15] TestPrintableString fails due to Unicode 13.0 support | Major | . | Akira Ajisaka | Akira Ajisaka | +| [YARN-10723](https://issues.apache.org/jira/browse/YARN-10723) | Change CS nodes page in UI to support custom resource. | Major | . | Qi Zhu | Qi Zhu | +| [HADOOP-17112](https://issues.apache.org/jira/browse/HADOOP-17112) | whitespace not allowed in paths when saving files to s3a via committer | Blocker | fs/s3 | Krzysztof Adamski | Krzysztof Adamski | +| [HADOOP-17661](https://issues.apache.org/jira/browse/HADOOP-17661) | mvn versions:set fails to parse pom.xml | Blocker | build | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [YARN-10642](https://issues.apache.org/jira/browse/YARN-10642) | Race condition: AsyncDispatcher can get stuck by the changes introduced in YARN-8995 | Critical | resourcemanager | zhengchenyu | zhengchenyu | +| [HDFS-15659](https://issues.apache.org/jira/browse/HDFS-15659) | Set dfs.namenode.redundancy.considerLoad to false in MiniDFSCluster | Major | test | Akira Ajisaka | Ahmed Hussein | +| [HADOOP-17840](https://issues.apache.org/jira/browse/HADOOP-17840) | Backport HADOOP-17837 to branch-3.2 | Minor | . | Bryan Beaudreault | Bryan Beaudreault | +| [HADOOP-17126](https://issues.apache.org/jira/browse/HADOOP-17126) | implement non-guava Precondition checkNotNull | Major | . | Ahmed Hussein | Ahmed Hussein | + + +### OTHER: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HDFS-15870](https://issues.apache.org/jira/browse/HDFS-15870) | Remove unused configuration dfs.namenode.stripe.min | Minor | . | tomscut | tomscut | +| [HDFS-15808](https://issues.apache.org/jira/browse/HDFS-15808) | Add metrics for FSNamesystem read/write lock hold long time | Major | hdfs | tomscut | tomscut | +| [HDFS-15873](https://issues.apache.org/jira/browse/HDFS-15873) | Add namenode address in logs for block report | Minor | datanode, hdfs | tomscut | tomscut | +| [HDFS-15906](https://issues.apache.org/jira/browse/HDFS-15906) | Close FSImage and FSNamesystem after formatting is complete | Minor | . | tomscut | tomscut | +| [HDFS-15892](https://issues.apache.org/jira/browse/HDFS-15892) | Add metric for editPendingQ in FSEditLogAsync | Minor | . | tomscut | tomscut | +| [HDFS-16078](https://issues.apache.org/jira/browse/HDFS-16078) | Remove unused parameters for DatanodeManager.handleLifeline() | Minor | . | tomscut | tomscut | +| [YARN-10278](https://issues.apache.org/jira/browse/YARN-10278) | CapacityScheduler test framework ProportionalCapacityPreemptionPolicyMockFramework need some review | Major | . | Gergely Pollák | Szilard Nemeth | +| [HDFS-15731](https://issues.apache.org/jira/browse/HDFS-15731) | Reduce threadCount for unit tests to reduce the memory usage | Major | build, test | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17571](https://issues.apache.org/jira/browse/HADOOP-17571) | Upgrade com.fasterxml.woodstox:woodstox-core for security reasons | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-15895](https://issues.apache.org/jira/browse/HDFS-15895) | DFSAdmin#printOpenFiles has redundant String#format usage | Minor | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17614](https://issues.apache.org/jira/browse/HADOOP-17614) | Bump netty to the latest 4.1.61 | Blocker | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HADOOP-17627](https://issues.apache.org/jira/browse/HADOOP-17627) | Backport to branch-3.2 HADOOP-17371, HADOOP-17621, HADOOP-17625 to update Jetty to 9.4.39 | Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HDFS-15989](https://issues.apache.org/jira/browse/HDFS-15989) | Split TestBalancer into two classes | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17808](https://issues.apache.org/jira/browse/HADOOP-17808) | ipc.Client not setting interrupt flag after catching InterruptedException | Minor | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17834](https://issues.apache.org/jira/browse/HADOOP-17834) | Bump aliyun-sdk-oss to 3.13.0 | Major | . | Siyao Meng | Siyao Meng | +| [HADOOP-17955](https://issues.apache.org/jira/browse/HADOOP-17955) | Bump netty to the latest 4.1.68 | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [HADOOP-18061](https://issues.apache.org/jira/browse/HADOOP-18061) | Update the year to 2022 | Major | . | Ayush Saxena | Ayush Saxena | +| [HADOOP-18125](https://issues.apache.org/jira/browse/HADOOP-18125) | Utility to identify git commit / Jira fixVersion discrepancies for RC preparation | Major | . | Viraj Jasani | Viraj Jasani | + + diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/RELEASENOTES.3.2.3.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/RELEASENOTES.3.2.3.md new file mode 100644 index 0000000000000..5c53bb4cb876b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.2.3/RELEASENOTES.3.2.3.md @@ -0,0 +1,71 @@ + + +# Apache Hadoop 3.2.3 Release Notes + +These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements. + + +--- + +* [YARN-10036](https://issues.apache.org/jira/browse/YARN-10036) | *Major* | **Install yarnpkg and upgrade nodejs in Dockerfile** + +In the Dockerfile, nodejs is upgraded to 8.17.0 and yarn 1.12.1 is installed. + + +--- + +* [HADOOP-16054](https://issues.apache.org/jira/browse/HADOOP-16054) | *Major* | **Update Dockerfile to use Bionic** + +The build image has been upgraded to Bionic. + + +--- + +* [HDFS-15719](https://issues.apache.org/jira/browse/HDFS-15719) | *Critical* | **[Hadoop 3] Both NameNodes can crash simultaneously due to the short JN socket timeout** + +The default value of the configuration hadoop.http.idle\_timeout.ms (how long does Jetty disconnect an idle connection) is changed from 10000 to 60000. +This property is inlined during compile time, so an application that references this property must be recompiled in order for it to take effect. + + +--- + +* [HADOOP-16748](https://issues.apache.org/jira/browse/HADOOP-16748) | *Major* | **Migrate to Python 3 and upgrade Yetus to 0.13.0** + + +- Upgraded Yetus to 0.13.0. +- Removed determine-flaky-tests-hadoop.py. +- Temporarily disabled shelldocs check in the Jenkins jobs due to YETUS-1099. + + +--- + +* [HADOOP-16870](https://issues.apache.org/jira/browse/HADOOP-16870) | *Major* | **Use spotbugs-maven-plugin instead of findbugs-maven-plugin** + +Removed findbugs from the hadoop build images and added spotbugs instead. +Upgraded SpotBugs to 4.2.2 and spotbugs-maven-plugin to 4.2.0. + + +--- + +* [HDFS-15942](https://issues.apache.org/jira/browse/HDFS-15942) | *Major* | **Increase Quota initialization threads** + +The default quota initialization thread count during the NameNode startup process (dfs.namenode.quota.init-threads) is increased from 4 to 12. + + + From 0fbd96a2449ec49f840d93e1c7d290c5218ef4ea Mon Sep 17 00:00:00 2001 From: Masatake Iwasaki Date: Mon, 28 Mar 2022 08:07:50 +0000 Subject: [PATCH 071/145] Make upstream aware of 3.2.3 release. --- .../jdiff/Apache_Hadoop_HDFS_3.2.3.xml | 674 ++++++++++++++++++ 1 file changed, 674 insertions(+) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.2.3.xml diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.2.3.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.2.3.xml new file mode 100644 index 0000000000000..5454f53be9122 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.2.3.xml @@ -0,0 +1,674 @@ + + + + + + + + + + + A distributed implementation of {@link +org.apache.hadoop.fs.FileSystem}. This is loosely modelled after +Google's GFS.

+ +

The most important difference is that unlike GFS, Hadoop DFS files +have strictly one writer at any one time. Bytes are always appended +to the end of the writer's stream. There is no notion of "record appends" +or "mutations" that are then checked or reordered. Writers simply emit +a byte stream. That byte stream is guaranteed to be stored in the +order written.

]]> +
+
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method must return as quickly as possible, since it's called + in a critical section of the NameNode's operation. + + @param succeeded Whether authorization succeeded. + @param userName Name of the user executing the request. + @param addr Remote address of the request. + @param cmd The requested command. + @param src Path of affected source file. + @param dst Path of affected destination file (if any). + @param stat File information for operations that change the file's + metadata (permissions, owner, times, etc).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
From eb164213863a3e08592c0996eb3f3d1299a7e909 Mon Sep 17 00:00:00 2001 From: Owen O'Malley Date: Tue, 22 Mar 2022 09:51:51 -0700 Subject: [PATCH 072/145] HDFS-16517 Distance metric is wrong for non-DN machines in 2.10. Fixed in HADOOP-16161, but this test case adds value to ensure the two getWeight methods stay in sync. Fixes #4091 Signed-off-by: Owen O'Malley --- .../hadoop/net/TestClusterTopology.java | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java index 328cf11c20fa6..57b620fde6c0e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestClusterTopology.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Arrays; +import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.math3.stat.inference.ChiSquareTest; import org.apache.hadoop.conf.Configuration; import org.junit.Assert; @@ -248,4 +249,41 @@ private NodeElement getNewNode(String name, String rackLocation) { node.setNetworkLocation(rackLocation); return node; } + + private NodeElement getNewNode(NetworkTopology cluster, + String name, String rackLocation) { + NodeElement node = getNewNode(name, rackLocation); + cluster.add(node); + return node; + } + + @Test + @SuppressWarnings("unchecked") + public void testWeights() { + // create the topology + NetworkTopology cluster = NetworkTopology.getInstance(new Configuration()); + NodeElement node1 = getNewNode(cluster, "node1", "/r1"); + NodeElement node2 = getNewNode(cluster, "node2", "/r1"); + NodeElement node3 = getNewNode(cluster, "node3", "/r2"); + for (Pair test: new Pair[]{Pair.of(0, node1), + Pair.of(2, node2), Pair.of(4, node3)}) { + int expect = test.getLeft(); + assertEquals(test.toString(), expect, cluster.getWeight(node1, test.getRight())); + assertEquals(test.toString(), expect, + cluster.getWeightUsingNetworkLocation(node1, test.getRight())); + } + // Reset so that we can have 2 levels + cluster = NetworkTopology.getInstance(new Configuration()); + NodeElement node5 = getNewNode(cluster, "node5", "/pod1/r1"); + NodeElement node6 = getNewNode(cluster, "node6", "/pod1/r1"); + NodeElement node7 = getNewNode(cluster, "node7", "/pod1/r2"); + NodeElement node8 = getNewNode(cluster, "node8", "/pod2/r3"); + for (Pair test: new Pair[]{Pair.of(0, node5), + Pair.of(2, node6), Pair.of(4, node7), Pair.of(6, node8)}) { + int expect = test.getLeft(); + assertEquals(test.toString(), expect, cluster.getWeight(node5, test.getRight())); + assertEquals(test.toString(), expect, + cluster.getWeightUsingNetworkLocation(node5, test.getRight())); + } + } } From a9b43966c07fef4507015a7e63f2769f947a64ab Mon Sep 17 00:00:00 2001 From: Lei Yang Date: Wed, 23 Mar 2022 12:47:32 -0700 Subject: [PATCH 073/145] HDFS-16518: Add shutdownhook to invalidate the KeyProviders in the cache Fixes #4100 Signed-off-by: Owen O'Malley --- .../apache/hadoop/hdfs/KeyProviderCache.java | 25 +++++++++++++++++++ .../hadoop/hdfs/TestKeyProviderCache.java | 9 +++++++ 2 files changed, 34 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java index c70081c06fa49..d8dd485101bce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java @@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.key.KeyProvider; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.util.KMSUtil; import org.apache.hadoop.classification.VisibleForTesting; @@ -34,6 +35,7 @@ import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener; import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification; +import org.apache.hadoop.util.ShutdownHookManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,6 +67,9 @@ public void onRemoval( } }) .build(); + + ShutdownHookManager.get().addShutdownHook(new KeyProviderCacheFinalizer(), + SHUTDOWN_HOOK_PRIORITY); } public KeyProvider get(final Configuration conf, @@ -85,6 +90,26 @@ public KeyProvider call() throws Exception { } } + public static final int SHUTDOWN_HOOK_PRIORITY = FileSystem.SHUTDOWN_HOOK_PRIORITY - 1; + + private class KeyProviderCacheFinalizer implements Runnable { + @Override + public synchronized void run() { + invalidateCache(); + } + } + + /** + * Invalidate cache. KeyProviders in the cache will be closed by cache hook. + */ + @VisibleForTesting + synchronized void invalidateCache() { + LOG.debug("Invalidating all cached KeyProviders."); + if (cache != null) { + cache.invalidateAll(); + } + } + private URI createKeyProviderURI(Configuration conf) { final String providerUriStr = conf.getTrimmed( CommonConfigurationKeysPublic.HADOOP_SECURITY_KEY_PROVIDER_PATH); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java index 9fc6b3894940c..58011d7f15fce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestKeyProviderCache.java @@ -32,6 +32,8 @@ public class TestKeyProviderCache { public static class DummyKeyProvider extends KeyProvider { + public static int CLOSE_CALL_COUNT = 0; + public DummyKeyProvider(Configuration conf) { super(conf); } @@ -76,6 +78,10 @@ public KeyVersion rollNewVersion(String name, byte[] material) public void flush() throws IOException { } + @Override + public void close() { + CLOSE_CALL_COUNT += 1; + } } public static class Factory extends KeyProviderFactory { @@ -124,6 +130,9 @@ public void testCache() throws Exception { Assert.assertFalse("Same KeyProviders returned !!", keyProvider1 == keyProvider4); + kpCache.invalidateCache(); + Assert.assertEquals("Expected number of closing calls doesn't match", + 3, DummyKeyProvider.CLOSE_CALL_COUNT); } private URI getKeyProviderUriFromConf(Configuration conf) { From e386d6a6617450b827745ee6a0a856166d37d450 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Tue, 29 Mar 2022 09:52:39 +0200 Subject: [PATCH 074/145] YARN-10549. Decouple RM runner logic from SLSRunner. Contributed by Szilard Nemeth. --- .../org/apache/hadoop/yarn/sls/AMRunner.java | 1 + .../org/apache/hadoop/yarn/sls/RMRunner.java | 137 ++++++++++++++++++ .../org/apache/hadoop/yarn/sls/SLSRunner.java | 123 ++++------------ 3 files changed, 164 insertions(+), 97 deletions(-) create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RMRunner.java diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java index da95c687ee71c..301b4260f35cd 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java @@ -151,6 +151,7 @@ private void startAMFromSynthGenerator() throws YarnException, IOException { // if we use the nodeFile this could have been not initialized yet. if (stjp == null) { stjp = new SynthTraceJobProducer(conf, new Path(inputTraces[0])); + slsRunner.setStjp(stjp); } SynthJob job; diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RMRunner.java new file mode 100644 index 0000000000000..dbded4b306e19 --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RMRunner.java @@ -0,0 +1,137 @@ +/* + * 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.hadoop.yarn.sls; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.metrics2.source.JvmMetrics; +import org.apache.hadoop.net.DNSToSwitchMapping; +import org.apache.hadoop.net.TableMapping; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; +import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; +import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher; +import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler; +import org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler; +import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics; +import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper; +import java.util.HashMap; +import java.util.Map; + +public class RMRunner { + private ResourceManager rm; + private String metricsOutputDir; + private Configuration conf; + private SLSRunner slsRunner; + private String tableMapping; + private Map queueAppNumMap; + + public RMRunner(Configuration conf, SLSRunner slsRunner) { + this.conf = conf; + this.slsRunner = slsRunner; + this.queueAppNumMap = new HashMap<>(); + } + + public void startRM() throws ClassNotFoundException, YarnException { + Configuration rmConf = new YarnConfiguration(conf); + String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER); + + if (Class.forName(schedulerClass) == CapacityScheduler.class) { + rmConf.set(YarnConfiguration.RM_SCHEDULER, + SLSCapacityScheduler.class.getName()); + rmConf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true); + rmConf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES, + ProportionalCapacityPreemptionPolicy.class.getName()); + } else if (Class.forName(schedulerClass) == FairScheduler.class) { + rmConf.set(YarnConfiguration.RM_SCHEDULER, + SLSFairScheduler.class.getName()); + } else if (Class.forName(schedulerClass) == FifoScheduler.class) { + // TODO add support for FifoScheduler + throw new YarnException("Fifo Scheduler is not supported yet."); + } + rmConf.setClass( + CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + TableMapping.class, DNSToSwitchMapping.class); + rmConf.set( + CommonConfigurationKeysPublic.NET_TOPOLOGY_TABLE_MAPPING_FILE_KEY, + tableMapping); + rmConf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricsOutputDir); + + rm = new ResourceManager() { + @Override + protected ApplicationMasterLauncher createAMLauncher() { + return new MockAMLauncher(slsRunner, this.rmContext); + } + }; + + // Across runs of parametrized tests, the JvmMetrics objects is retained, + // but is not registered correctly + JvmMetrics jvmMetrics = JvmMetrics.initSingleton("ResourceManager", null); + jvmMetrics.registerIfNeeded(); + + // Init and start the actual ResourceManager + rm.init(rmConf); + rm.start(); + } + + public void increaseQueueAppNum(String queue) throws YarnException { + SchedulerWrapper wrapper = (SchedulerWrapper)rm.getResourceScheduler(); + String queueName = wrapper.getRealQueueName(queue); + Integer appNum = queueAppNumMap.get(queueName); + if (appNum == null) { + appNum = 1; + } else { + appNum = appNum + 1; + } + + queueAppNumMap.put(queueName, appNum); + SchedulerMetrics metrics = wrapper.getSchedulerMetrics(); + if (metrics != null) { + metrics.trackQueue(queueName); + } + } + + public void setMetricsOutputDir(String metricsOutputDir) { + this.metricsOutputDir = metricsOutputDir; + } + + public String getTableMapping() { + return tableMapping; + } + + public void setTableMapping(String tableMapping) { + this.tableMapping = tableMapping; + } + + public void stop() { + rm.stop(); + } + + public ResourceManager getRm() { + return rm; + } + + public Map getQueueAppNumMap() { + return queueAppNumMap; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index 2110e3c196ca3..e9ae7f51dbafa 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -53,11 +53,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.metrics2.source.JvmMetrics; -import org.apache.hadoop.net.DNSToSwitchMapping; -import org.apache.hadoop.net.TableMapping; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -66,24 +62,14 @@ import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher; -import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.sls.appmaster.AMSimulator; import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator; -import org.apache.hadoop.yarn.sls.resourcemanager.MockAMLauncher; -import org.apache.hadoop.yarn.sls.scheduler.SLSCapacityScheduler; -import org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler; -import org.apache.hadoop.yarn.sls.scheduler.SchedulerMetrics; import org.apache.hadoop.yarn.sls.scheduler.SchedulerWrapper; import org.apache.hadoop.yarn.sls.scheduler.TaskRunner; +import org.apache.hadoop.yarn.sls.scheduler.Tracker; import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer; import org.apache.hadoop.yarn.sls.utils.SLSUtils; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @@ -109,11 +95,8 @@ @Private @Unstable public class SLSRunner extends Configured implements Tool { - // RM, Runner - private ResourceManager rm; private static TaskRunner runner = new TaskRunner(); private String[] inputTraces; - private Map queueAppNumMap; private int poolSize; // NM simulator @@ -122,12 +105,10 @@ public class SLSRunner extends Configured implements Tool { private String nodeFile; // metrics - private String metricsOutputDir; private boolean printSimulation; // other simulation information private int numNMs, numRacks; - private String tableMapping; private final static Map simulateInfoMap = new HashMap<>(); @@ -136,6 +117,7 @@ public class SLSRunner extends Configured implements Tool { private static boolean exitAtTheFinish = false; private AMRunner amRunner; + private RMRunner rmRunner; /** * The type of trace in input. @@ -179,8 +161,8 @@ private void init(Configuration tempConf) throws ClassNotFoundException { setConf(tempConf); nmMap = new ConcurrentHashMap<>(); - queueAppNumMap = new HashMap<>(); amRunner = new AMRunner(runner, this); + rmRunner = new RMRunner(tempConf, this); // runner poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE, @@ -225,12 +207,12 @@ public static Map getSimulateInfoMap() { * @param inType * @param inTraces * @param nodes - * @param outDir + * @param metricsOutputDir * @param trackApps * @param printsimulation */ public void setSimulationParams(TraceType inType, String[] inTraces, - String nodes, String outDir, Set trackApps, + String nodes, String metricsOutputDir, Set trackApps, boolean printsimulation) { this.inputType = inType; @@ -240,8 +222,8 @@ public void setSimulationParams(TraceType inType, String[] inTraces, this.amRunner.setTrackedApps(trackApps); this.nodeFile = nodes; this.printSimulation = printsimulation; - metricsOutputDir = outDir; - tableMapping = outDir + "/tableMapping.csv"; + this.rmRunner.setMetricsOutputDir(metricsOutputDir); + this.rmRunner.setTableMapping(metricsOutputDir + "/tableMapping.csv"); } public void start() throws IOException, ClassNotFoundException, YarnException, @@ -250,17 +232,19 @@ public void start() throws IOException, ClassNotFoundException, YarnException, enableDNSCaching(getConf()); // start resource manager - startRM(); - amRunner.setResourceManager(rm); + rmRunner.startRM(); + amRunner.setResourceManager(rmRunner.getRm()); // start node managers startNM(); // start application masters amRunner.startAM(); + // set queue & tracked apps information - ((SchedulerWrapper) rm.getResourceScheduler()).getTracker() - .setQueueSet(this.queueAppNumMap.keySet()); - ((SchedulerWrapper) rm.getResourceScheduler()).getTracker() - .setTrackedAppSet(amRunner.getTrackedApps()); + SchedulerWrapper resourceScheduler = + (SchedulerWrapper) rmRunner.getRm().getResourceScheduler(); + Tracker tracker = resourceScheduler.getTracker(); + tracker.setQueueSet(rmRunner.getQueueAppNumMap().keySet()); + tracker.setTrackedAppSet(amRunner.getTrackedApps()); // print out simulation info printSimulationInfo(); // blocked until all nodes RUNNING @@ -286,49 +270,6 @@ static void enableDNSCaching(Configuration conf) { } } - private void startRM() throws ClassNotFoundException, YarnException { - Configuration rmConf = new YarnConfiguration(getConf()); - String schedulerClass = rmConf.get(YarnConfiguration.RM_SCHEDULER); - - if (Class.forName(schedulerClass) == CapacityScheduler.class) { - rmConf.set(YarnConfiguration.RM_SCHEDULER, - SLSCapacityScheduler.class.getName()); - rmConf.setBoolean(YarnConfiguration.RM_SCHEDULER_ENABLE_MONITORS, true); - rmConf.set(YarnConfiguration.RM_SCHEDULER_MONITOR_POLICIES, - ProportionalCapacityPreemptionPolicy.class.getName()); - } else if (Class.forName(schedulerClass) == FairScheduler.class) { - rmConf.set(YarnConfiguration.RM_SCHEDULER, - SLSFairScheduler.class.getName()); - } else if (Class.forName(schedulerClass) == FifoScheduler.class) { - // TODO add support for FifoScheduler - throw new YarnException("Fifo Scheduler is not supported yet."); - } - rmConf.setClass( - CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, - TableMapping.class, DNSToSwitchMapping.class); - rmConf.set( - CommonConfigurationKeysPublic.NET_TOPOLOGY_TABLE_MAPPING_FILE_KEY, - tableMapping); - rmConf.set(SLSConfiguration.METRICS_OUTPUT_DIR, metricsOutputDir); - - final SLSRunner se = this; - rm = new ResourceManager() { - @Override - protected ApplicationMasterLauncher createAMLauncher() { - return new MockAMLauncher(se, this.rmContext); - } - }; - - // Across runs of parametrized tests, the JvmMetrics objects is retained, - // but is not registered correctly - JvmMetrics jvmMetrics = JvmMetrics.initSingleton("ResourceManager", null); - jvmMetrics.registerIfNeeded(); - - // Init and start the actual ResourceManager - rm.init(rmConf); - rm.start(); - } - private void startNM() throws YarnException, IOException, InterruptedException { // nm configuration @@ -368,7 +309,7 @@ private void startNM() throws YarnException, IOException, throw new YarnException("No node! Please configure nodes."); } - SLSUtils.generateNodeTableMapping(nodeSet, tableMapping); + SLSUtils.generateNodeTableMapping(nodeSet, rmRunner.getTableMapping()); // create NM simulators Random random = new Random(); @@ -391,7 +332,7 @@ private void startNM() throws YarnException, IOException, Set nodeLabels = nodeDetails.getLabels(); nm.init(hostName, nmResource, random.nextInt(heartbeatInterval), - heartbeatInterval, rm, resourceUtilizationRatio, nodeLabels); + heartbeatInterval, rmRunner.getRm(), resourceUtilizationRatio, nodeLabels); nmMap.put(nm.getNode().getNodeID(), nm); runner.schedule(nm); rackSet.add(nm.getNode().getRackName()); @@ -411,7 +352,7 @@ private void waitForNodesRunning() throws InterruptedException { long startTimeMS = System.currentTimeMillis(); while (true) { int numRunningNodes = 0; - for (RMNode node : rm.getRMContext().getRMNodes().values()) { + for (RMNode node : rmRunner.getRm().getRMContext().getRMNodes().values()) { if (node.getState() == NodeState.RUNNING) { numRunningNodes++; } @@ -435,21 +376,8 @@ Resource getDefaultContainerResource() { return Resources.createResource(containerMemory, containerVCores); } - void increaseQueueAppNum(String queue) throws YarnException { - SchedulerWrapper wrapper = (SchedulerWrapper)rm.getResourceScheduler(); - String queueName = wrapper.getRealQueueName(queue); - Integer appNum = queueAppNumMap.get(queueName); - if (appNum == null) { - appNum = 1; - } else { - appNum = appNum + 1; - } - - queueAppNumMap.put(queueName, appNum); - SchedulerMetrics metrics = wrapper.getSchedulerMetrics(); - if (metrics != null) { - metrics.trackQueue(queueName); - } + public void increaseQueueAppNum(String queue) throws YarnException { + rmRunner.increaseQueueAppNum(queue); } private void printSimulationInfo() { @@ -457,6 +385,7 @@ private void printSimulationInfo() { final int numTasks = amRunner.getNumTasks(); final long maxRuntime = amRunner.getMaxRuntime(); Map amMap = amRunner.getAmMap(); + Map queueAppNumMap = rmRunner.getQueueAppNumMap(); if (printSimulation) { // node @@ -523,7 +452,7 @@ public static void exitSLSRunner() { } public void stop() throws InterruptedException { - rm.stop(); + rmRunner.stop(); runner.stop(); } @@ -696,14 +625,14 @@ public int hashCode() { } } - public ResourceManager getRm() { - return rm; - } - public SynthTraceJobProducer getStjp() { return stjp; } + public void setStjp(SynthTraceJobProducer stjp) { + this.stjp = stjp; + } + public AMSimulator getAMSimulatorByAppId(ApplicationId appId) { return amRunner.getAMSimulator(appId); } From 4e32318acee9f9ce6f7eafeb980a235d081f7be4 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Wed, 30 Mar 2022 00:44:55 +0900 Subject: [PATCH 075/145] HDFS-16523. Fix dependency error in hadoop-hdfs on M1 Mac (#4112) The version of hawtjni-runtime has been fixed to 1.11 to avoid build failures Contributed by Akira Ajisaka --- hadoop-project/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 93330d479e0b0..0a84f89a723fd 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1481,6 +1481,11 @@ leveldbjni-all 1.8
+ + org.fusesource.hawtjni + hawtjni-runtime + 1.11 + com.microsoft.azure From 6eea28c3f3813594279b81c5be9cc3087bf3d99f Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Wed, 30 Mar 2022 14:56:04 +0800 Subject: [PATCH 076/145] HDFS-16498. Fix NPE for checkBlockReportLease #4057. Contributed by tomscut. Reviewed-by: Ayush Saxena Signed-off-by: He Xiaoqiao --- .../server/blockmanagement/BlockManager.java | 3 ++ .../blockmanagement/DatanodeManager.java | 5 +++ .../server/namenode/NameNodeRpcServer.java | 2 +- .../blockmanagement/TestBlockReportLease.java | 45 +++++++++++++++++++ 4 files changed, 54 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 494c2f01c99ef..7b666f34327b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -2751,6 +2751,9 @@ public boolean checkBlockReportLease(BlockReportContext context, return true; } DatanodeDescriptor node = datanodeManager.getDatanode(nodeID); + if (node == null) { + throw new UnregisteredNodeException(nodeID, null); + } final long startTime = Time.monotonicNow(); return blockReportLeaseManager.checkLease(node, startTime, context.getLeaseId()); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index 092ef6502a6f2..a9850aa7f5a9b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -2198,5 +2198,10 @@ public DatanodeStorageReport[] getDatanodeStorageReport( } return reports; } + + @VisibleForTesting + public Map getDatanodeMap() { + return datanodeMap; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java index 4f400519ff416..73957bcafeb30 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java @@ -1641,7 +1641,7 @@ public DatanodeCommand blockReport(final DatanodeRegistration nodeReg, } } } catch (UnregisteredNodeException une) { - LOG.debug("Datanode {} is attempting to report but not register yet.", + LOG.warn("Datanode {} is attempting to report but not register yet.", nodeReg); return RegisterCommand.REGISTER; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportLease.java index a5acc14edd935..d1ae0b600fcfa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportLease.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportLease.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.server.protocol.FinalizeCommand; import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; +import org.apache.hadoop.hdfs.server.protocol.RegisterCommand; import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports; import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports; import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport; @@ -136,6 +137,50 @@ public void testCheckBlockReportLease() throws Exception { } } + @Test + public void testCheckBlockReportLeaseWhenDnUnregister() throws Exception { + HdfsConfiguration conf = new HdfsConfiguration(); + Random rand = new Random(); + + try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build()) { + FSNamesystem fsn = cluster.getNamesystem(); + BlockManager blockManager = fsn.getBlockManager(); + String poolId = cluster.getNamesystem().getBlockPoolId(); + NamenodeProtocols rpcServer = cluster.getNameNodeRpc(); + + // Remove the unique DataNode to simulate the unregistered situation. + // This is similar to starting NameNode, and DataNodes are not registered yet. + DataNode dn = cluster.getDataNodes().get(0); + blockManager.getDatanodeManager().getDatanodeMap().remove(dn.getDatanodeUuid()); + + // Trigger BlockReport. + DatanodeRegistration dnRegistration = dn.getDNRegistrationForBP(poolId); + StorageReport[] storages = dn.getFSDataset().getStorageReports(poolId); + ExecutorService pool = Executors.newFixedThreadPool(1); + BlockReportContext brContext = new BlockReportContext(1, 0, + rand.nextLong(), 1); + Future sendBRFuture = pool.submit(() -> { + // Build every storage with 100 blocks for sending report. + DatanodeStorage[] datanodeStorages + = new DatanodeStorage[storages.length]; + for (int i = 0; i < storages.length; i++) { + datanodeStorages[i] = storages[i].getStorage(); + } + StorageBlockReport[] reports = createReports(datanodeStorages, 100); + + // Send blockReport. + return rpcServer.blockReport(dnRegistration, poolId, reports, + brContext); + }); + + // When unregistered DataNode triggering the block report, will throw an + // UnregisteredNodeException. After NameNode processing, RegisterCommand + // is returned to the DataNode. + DatanodeCommand datanodeCommand = sendBRFuture.get(); + assertTrue(datanodeCommand instanceof RegisterCommand); + } + } + private StorageBlockReport[] createReports(DatanodeStorage[] dnStorages, int numBlocks) { int longsPerBlock = 3; From 08e6d0ce608647d57da647c71ceb216243ff16d9 Mon Sep 17 00:00:00 2001 From: zhongjingxiong <84573424+zhongjingxiong@users.noreply.github.com> Date: Wed, 30 Mar 2022 19:42:50 +0800 Subject: [PATCH 077/145] HADOOP-18145. Fileutil's unzip method causes unzipped files to lose their original permissions (#4036) Contributed by jingxiong zhong --- .../java/org/apache/hadoop/fs/FileUtil.java | 70 +++++++++++++-- .../org/apache/hadoop/fs/TestFileUtil.java | 85 ++++++++++++++----- 2 files changed, 126 insertions(+), 29 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index b788c7ec6b664..0d5ced79fc4cf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -36,15 +36,18 @@ import java.nio.charset.CharsetEncoder; import java.nio.charset.StandardCharsets; import java.nio.file.AccessDeniedException; +import java.nio.file.attribute.PosixFilePermission; import java.nio.file.FileSystems; import java.nio.file.Files; import java.nio.file.LinkOption; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Enumeration; +import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -53,13 +56,13 @@ import java.util.jar.JarOutputStream; import java.util.jar.Manifest; import java.util.zip.GZIPInputStream; -import java.util.zip.ZipEntry; -import java.util.zip.ZipFile; -import java.util.zip.ZipInputStream; import org.apache.commons.collections.map.CaseInsensitiveMap; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.archivers.zip.ZipArchiveEntry; +import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream; +import org.apache.commons.compress.archivers.zip.ZipFile; import org.apache.commons.io.FileUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -644,12 +647,12 @@ public static long getDU(File dir) { */ public static void unZip(InputStream inputStream, File toDir) throws IOException { - try (ZipInputStream zip = new ZipInputStream(inputStream)) { + try (ZipArchiveInputStream zip = new ZipArchiveInputStream(inputStream)) { int numOfFailedLastModifiedSet = 0; String targetDirPath = toDir.getCanonicalPath() + File.separator; - for(ZipEntry entry = zip.getNextEntry(); + for(ZipArchiveEntry entry = zip.getNextZipEntry(); entry != null; - entry = zip.getNextEntry()) { + entry = zip.getNextZipEntry()) { if (!entry.isDirectory()) { File file = new File(toDir, entry.getName()); if (!file.getCanonicalPath().startsWith(targetDirPath)) { @@ -668,6 +671,9 @@ public static void unZip(InputStream inputStream, File toDir) if (!file.setLastModified(entry.getTime())) { numOfFailedLastModifiedSet++; } + if (entry.getPlatform() == ZipArchiveEntry.PLATFORM_UNIX) { + Files.setPosixFilePermissions(file.toPath(), permissionsFromMode(entry.getUnixMode())); + } } } if (numOfFailedLastModifiedSet > 0) { @@ -677,6 +683,49 @@ public static void unZip(InputStream inputStream, File toDir) } } + /** + * The permission operation of this method only involves users, user groups, and others. + * If SUID is set, only executable permissions are reserved. + * @param mode Permissions are represented by numerical values + * @return The original permissions for files are stored in collections + */ + private static Set permissionsFromMode(int mode) { + EnumSet permissions = + EnumSet.noneOf(PosixFilePermission.class); + addPermissions(permissions, mode, PosixFilePermission.OTHERS_READ, + PosixFilePermission.OTHERS_WRITE, PosixFilePermission.OTHERS_EXECUTE); + addPermissions(permissions, mode >> 3, PosixFilePermission.GROUP_READ, + PosixFilePermission.GROUP_WRITE, PosixFilePermission.GROUP_EXECUTE); + addPermissions(permissions, mode >> 6, PosixFilePermission.OWNER_READ, + PosixFilePermission.OWNER_WRITE, PosixFilePermission.OWNER_EXECUTE); + return permissions; + } + + /** + * Assign the original permissions to the file + * @param permissions The original permissions for files are stored in collections + * @param mode Use a value of type int to indicate permissions + * @param r Read permission + * @param w Write permission + * @param x Execute permission + */ + private static void addPermissions( + Set permissions, + int mode, + PosixFilePermission r, + PosixFilePermission w, + PosixFilePermission x) { + if ((mode & 1L) == 1L) { + permissions.add(x); + } + if ((mode & 2L) == 2L) { + permissions.add(w); + } + if ((mode & 4L) == 4L) { + permissions.add(r); + } + } + /** * Given a File input it will unzip it in the unzip directory. * passed as the second parameter @@ -685,14 +734,14 @@ public static void unZip(InputStream inputStream, File toDir) * @throws IOException An I/O exception has occurred */ public static void unZip(File inFile, File unzipDir) throws IOException { - Enumeration entries; + Enumeration entries; ZipFile zipFile = new ZipFile(inFile); try { - entries = zipFile.entries(); + entries = zipFile.getEntries(); String targetDirPath = unzipDir.getCanonicalPath() + File.separator; while (entries.hasMoreElements()) { - ZipEntry entry = entries.nextElement(); + ZipArchiveEntry entry = entries.nextElement(); if (!entry.isDirectory()) { InputStream in = zipFile.getInputStream(entry); try { @@ -717,6 +766,9 @@ public static void unZip(File inFile, File unzipDir) throws IOException { } finally { out.close(); } + if (entry.getPlatform() == ZipArchiveEntry.PLATFORM_UNIX) { + Files.setPosixFilePermissions(file.toPath(), permissionsFromMode(entry.getUnixMode())); + } } finally { in.close(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java index 29eafb9e4dac0..c884e223365c9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java @@ -53,11 +53,11 @@ import java.util.jar.Attributes; import java.util.jar.JarFile; import java.util.jar.Manifest; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; import org.apache.commons.compress.archivers.tar.TarArchiveEntry; import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; +import org.apache.commons.compress.archivers.zip.ZipArchiveEntry; +import org.apache.commons.compress.archivers.zip.ZipArchiveOutputStream; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.test.GenericTestUtils; @@ -773,26 +773,71 @@ public void testCreateLocalTempFile() throws IOException { public void testUnZip() throws Exception { // make sa simple zip final File simpleZip = new File(del, FILE); - OutputStream os = new FileOutputStream(simpleZip); - ZipOutputStream tos = new ZipOutputStream(os); - try { - ZipEntry ze = new ZipEntry("foo"); - byte[] data = "some-content".getBytes("UTF-8"); - ze.setSize(data.length); - tos.putNextEntry(ze); - tos.write(data); - tos.closeEntry(); + try (OutputStream os = new FileOutputStream(simpleZip); + ZipArchiveOutputStream tos = new ZipArchiveOutputStream(os)) { + List ZipArchiveList = new ArrayList<>(7); + int count = 0; + // create 7 files to verify permissions + for (int i = 0; i < 7; i++) { + ZipArchiveList.add(new ZipArchiveEntry("foo_" + i)); + ZipArchiveEntry archiveEntry = ZipArchiveList.get(i); + archiveEntry.setUnixMode(count += 0100); + byte[] data = "some-content".getBytes("UTF-8"); + archiveEntry.setSize(data.length); + tos.putArchiveEntry(archiveEntry); + tos.write(data); + } + tos.closeArchiveEntry(); tos.flush(); tos.finish(); - } finally { - tos.close(); } - + // successfully unzip it into an existing dir: FileUtil.unZip(simpleZip, tmp); + File foo0 = new File(tmp, "foo_0"); + File foo1 = new File(tmp, "foo_1"); + File foo2 = new File(tmp, "foo_2"); + File foo3 = new File(tmp, "foo_3"); + File foo4 = new File(tmp, "foo_4"); + File foo5 = new File(tmp, "foo_5"); + File foo6 = new File(tmp, "foo_6"); // check result: - Verify.exists(new File(tmp, "foo")); - assertEquals(12, new File(tmp, "foo").length()); + assertTrue(foo0.exists()); + assertTrue(foo1.exists()); + assertTrue(foo2.exists()); + assertTrue(foo3.exists()); + assertTrue(foo4.exists()); + assertTrue(foo5.exists()); + assertTrue(foo6.exists()); + assertEquals(12, foo0.length()); + // tests whether file foo_0 has executable permissions + assertTrue("file lacks execute permissions", foo0.canExecute()); + assertFalse("file has write permissions", foo0.canWrite()); + assertFalse("file has read permissions", foo0.canRead()); + // tests whether file foo_1 has writable permissions + assertFalse("file has execute permissions", foo1.canExecute()); + assertTrue("file lacks write permissions", foo1.canWrite()); + assertFalse("file has read permissions", foo1.canRead()); + // tests whether file foo_2 has executable and writable permissions + assertTrue("file lacks execute permissions", foo2.canExecute()); + assertTrue("file lacks write permissions", foo2.canWrite()); + assertFalse("file has read permissions", foo2.canRead()); + // tests whether file foo_3 has readable permissions + assertFalse("file has execute permissions", foo3.canExecute()); + assertFalse("file has write permissions", foo3.canWrite()); + assertTrue("file lacks read permissions", foo3.canRead()); + // tests whether file foo_4 has readable and executable permissions + assertTrue("file lacks execute permissions", foo4.canExecute()); + assertFalse("file has write permissions", foo4.canWrite()); + assertTrue("file lacks read permissions", foo4.canRead()); + // tests whether file foo_5 has readable and writable permissions + assertFalse("file has execute permissions", foo5.canExecute()); + assertTrue("file lacks write permissions", foo5.canWrite()); + assertTrue("file lacks read permissions", foo5.canRead()); + // tests whether file foo_6 has readable, writable and executable permissions + assertTrue("file lacks execute permissions", foo6.canExecute()); + assertTrue("file lacks write permissions", foo6.canWrite()); + assertTrue("file lacks read permissions", foo6.canRead()); final File regularFile = Verify.createNewFile(new File(tmp, "QuickBrownFoxJumpsOverTheLazyDog")); @@ -804,14 +849,14 @@ public void testUnZip2() throws IOException { // make a simple zip final File simpleZip = new File(del, FILE); OutputStream os = new FileOutputStream(simpleZip); - try (ZipOutputStream tos = new ZipOutputStream(os)) { + try (ZipArchiveOutputStream tos = new ZipArchiveOutputStream(os)) { // Add an entry that contains invalid filename - ZipEntry ze = new ZipEntry("../foo"); + ZipArchiveEntry ze = new ZipArchiveEntry("../foo"); byte[] data = "some-content".getBytes(StandardCharsets.UTF_8); ze.setSize(data.length); - tos.putNextEntry(ze); + tos.putArchiveEntry(ze); tos.write(data); - tos.closeEntry(); + tos.closeArchiveEntry(); tos.flush(); tos.finish(); } From dc4a680da8bcacf152cc8638d86dd171a7901245 Mon Sep 17 00:00:00 2001 From: Kengo Seki Date: Wed, 30 Mar 2022 22:47:45 +0900 Subject: [PATCH 078/145] MAPREDUCE-7373. Building MapReduce NativeTask fails on Fedora 34+ (#4120) --- .../hadoop-mapreduce-client-nativetask/src/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt index ae3b9c6029e57..4c32838afb0b4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/CMakeLists.txt @@ -27,6 +27,7 @@ set(GTEST_SRC_DIR ${CMAKE_SOURCE_DIR}/../../../../hadoop-common-project/hadoop-c # Add extra compiler and linker flags. # -Wno-sign-compare hadoop_add_compiler_flags("-DNDEBUG -DSIMPLE_MEMCPY -fno-strict-aliasing -fsigned-char") +set(CMAKE_CXX_STANDARD 11) # Source location. set(SRC main/native) From ac50657c378fe0d6554011a54bb162b1b8c49958 Mon Sep 17 00:00:00 2001 From: litao Date: Wed, 30 Mar 2022 23:51:17 +0800 Subject: [PATCH 079/145] HDFS-16413. Reconfig dfs usage parameters for datanode (#3863) --- .../apache/hadoop/fs/CachingGetSpaceUsed.java | 13 ++- .../hadoop/hdfs/server/datanode/DataNode.java | 50 ++++++++++- .../datanode/fsdataset/FsDatasetSpi.java | 6 ++ .../fsdataset/impl/BlockPoolSlice.java | 41 ++++++++- .../fsdataset/impl/FsDatasetImpl.java | 5 ++ .../datanode/fsdataset/impl/FsVolumeImpl.java | 4 + .../server/datanode/SimulatedFSDataset.java | 6 ++ .../datanode/TestDataNodeReconfiguration.java | 83 +++++++++++++++++++ .../extdataset/ExternalDatasetImpl.java | 6 ++ .../hadoop/hdfs/tools/TestDFSAdmin.java | 2 +- 10 files changed, 211 insertions(+), 5 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java index a437995885cd8..362d125b09df5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java @@ -19,6 +19,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,10 +155,20 @@ boolean running() { /** * How long in between runs of the background refresh. */ - long getRefreshInterval() { + @VisibleForTesting + public long getRefreshInterval() { return refreshInterval; } + /** + * Randomize the refresh interval timing by this amount, the actual interval will be chosen + * uniformly between {@code interval-jitter} and {@code interval+jitter}. + */ + @VisibleForTesting + public long getJitter() { + return jitter; + } + /** * Reset the current used data amount. This should be called * when the cached value is re-computed. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 15e8a9e359799..c3b1aa1c67203 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -20,6 +20,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT; @@ -149,6 +153,8 @@ import org.apache.hadoop.hdfs.server.common.DataNodeLockManager.LockLevel; import org.apache.hadoop.hdfs.server.datanode.checker.DatasetVolumeChecker; import org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.BlockPoolSlice; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.hdfs.util.DataTransferThrottler; import org.apache.hadoop.util.*; import org.apache.hadoop.hdfs.client.BlockReportOptions; @@ -341,7 +347,9 @@ public class DataNode extends ReconfigurableBase DFS_DATANODE_FILEIO_PROFILING_SAMPLING_PERCENTAGE_KEY, DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, - DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY)); + DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY, + FS_DU_INTERVAL_KEY, + FS_GETSPACEUSED_JITTER_KEY)); public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog"); @@ -673,6 +681,9 @@ public String reconfigurePropertyImpl(String property, String newVal) case DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY: case DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY: return reconfSlowDiskParameters(property, newVal); + case FS_DU_INTERVAL_KEY: + case FS_GETSPACEUSED_JITTER_KEY: + return reconfDfsUsageParameters(property, newVal); default: break; } @@ -854,6 +865,43 @@ private String reconfSlowDiskParameters(String property, String newVal) } } + private String reconfDfsUsageParameters(String property, String newVal) + throws ReconfigurationException { + String result = null; + try { + LOG.info("Reconfiguring {} to {}", property, newVal); + if (property.equals(FS_DU_INTERVAL_KEY)) { + Preconditions.checkNotNull(data, "FsDatasetSpi has not been initialized."); + long interval = (newVal == null ? FS_DU_INTERVAL_DEFAULT : + Long.parseLong(newVal)); + result = Long.toString(interval); + List volumeList = data.getVolumeList(); + for (FsVolumeImpl fsVolume : volumeList) { + Map blockPoolSlices = fsVolume.getBlockPoolSlices(); + for (BlockPoolSlice value : blockPoolSlices.values()) { + value.updateDfsUsageConfig(interval, null); + } + } + } else if (property.equals(FS_GETSPACEUSED_JITTER_KEY)) { + Preconditions.checkNotNull(data, "FsDatasetSpi has not been initialized."); + long jitter = (newVal == null ? FS_GETSPACEUSED_JITTER_DEFAULT : + Long.parseLong(newVal)); + result = Long.toString(jitter); + List volumeList = data.getVolumeList(); + for (FsVolumeImpl fsVolume : volumeList) { + Map blockPoolSlices = fsVolume.getBlockPoolSlices(); + for (BlockPoolSlice value : blockPoolSlices.values()) { + value.updateDfsUsageConfig(null, jitter); + } + } + } + LOG.info("RECONFIGURE* changed {} to {}", property, newVal); + return result; + } catch (IllegalArgumentException | IOException e) { + throw new ReconfigurationException(property, newVal, getConf().get(property), e); + } + } + /** * Get a list of the keys of the re-configurable properties in configuration. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java index e39ef817b6f29..8d1d10bccd2fc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java @@ -37,6 +37,7 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MountVolumeMap; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.protocol.Block; @@ -679,4 +680,9 @@ ReplicaInfo moveBlockAcrossVolumes(final ExtendedBlock block, * @throws IOException */ MountVolumeMap getMountVolumeMap() throws IOException; + + /** + * Get the volume list. + */ + List getVolumeList(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java index eff079a353da6..8357b860b25e2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java @@ -46,6 +46,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.hdfs.server.datanode.FSCachingGetSpaceUsed; +import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -77,6 +78,9 @@ import org.apache.hadoop.classification.VisibleForTesting; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_KEY; + /** * A block pool slice represents a portion of a block pool stored on a volume. * Taken together, all BlockPoolSlices sharing a block pool ID across a @@ -84,7 +88,7 @@ * * This class is synchronized by {@link FsVolumeImpl}. */ -class BlockPoolSlice { +public class BlockPoolSlice { static final Logger LOG = LoggerFactory.getLogger(BlockPoolSlice.class); private final String bpid; @@ -115,6 +119,8 @@ class BlockPoolSlice { private final Timer timer; private final int maxDataLength; private final FileIoProvider fileIoProvider; + private final Configuration config; + private final File bpDir; private static ForkJoinPool addReplicaThreadPool = null; private static final int VOLUMES_REPLICA_ADD_THREADPOOL_SIZE = Runtime @@ -128,7 +134,7 @@ public int compare(File f1, File f2) { }; // TODO:FEDERATION scalability issue - a thread per DU is needed - private final GetSpaceUsed dfsUsage; + private volatile GetSpaceUsed dfsUsage; /** * Create a blook pool slice @@ -141,6 +147,8 @@ public int compare(File f1, File f2) { */ BlockPoolSlice(String bpid, FsVolumeImpl volume, File bpDir, Configuration conf, Timer timer) throws IOException { + this.config = conf; + this.bpDir = bpDir; this.bpid = bpid; this.volume = volume; this.fileIoProvider = volume.getFileIoProvider(); @@ -232,6 +240,35 @@ public void run() { SHUTDOWN_HOOK_PRIORITY); } + public void updateDfsUsageConfig(Long interval, Long jitter) throws IOException { + // Close the old dfsUsage if it is CachingGetSpaceUsed. + if (dfsUsage instanceof CachingGetSpaceUsed) { + ((CachingGetSpaceUsed) dfsUsage).close(); + } + if (interval != null) { + Preconditions.checkArgument(interval > 0, + FS_DU_INTERVAL_KEY + " should be larger than 0"); + config.setLong(FS_DU_INTERVAL_KEY, interval); + } + if (jitter != null) { + Preconditions.checkArgument(jitter >= 0, + FS_GETSPACEUSED_JITTER_KEY + " should be larger than or equal to 0"); + config.setLong(FS_GETSPACEUSED_JITTER_KEY, jitter); + } + // Start new dfsUsage. + this.dfsUsage = new FSCachingGetSpaceUsed.Builder().setBpid(bpid) + .setVolume(volume) + .setPath(bpDir) + .setConf(config) + .setInitialUsed(loadDfsUsed()) + .build(); + } + + @VisibleForTesting + public GetSpaceUsed getDfsUsage() { + return dfsUsage; + } + private synchronized static void initializeAddReplicaPool(Configuration conf, FsDatasetImpl dataset) { if (addReplicaThreadPool == null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index 002d99abc5ba7..aaf37aa09c8c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -3688,5 +3688,10 @@ void stopAllDataxceiverThreads(FsVolumeImpl volume) { } } } + + @Override + public List getVolumeList() { + return volumes.getVolumes(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java index 8f15d8a70932e..806afbdb2d115 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsVolumeImpl.java @@ -548,6 +548,10 @@ long getRecentReserved() { return recentReserved; } + public Map getBlockPoolSlices() { + return bpSlices; + } + long getReserved(){ return reserved != null ? reserved.getReserved() : 0; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java index 3313c7c7a0360..29eb051cb0210 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MountVolumeMap; import org.apache.hadoop.thirdparty.com.google.common.math.LongMath; import org.apache.commons.lang3.ArrayUtils; @@ -1605,5 +1606,10 @@ public Set deepCopyReplica(String bpid) public MountVolumeMap getMountVolumeMap() { return null; } + + @Override + public List getVolumeList() { + return null; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java index 1a9d6024acd0f..172a44557c1ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java @@ -19,6 +19,10 @@ package org.apache.hadoop.hdfs.server.datanode; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY; @@ -49,15 +53,21 @@ import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; +import java.util.List; +import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.ReconfigurationException; +import org.apache.hadoop.fs.CachingGetSpaceUsed; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.GetSpaceUsed; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.BlockPoolSlice; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.test.LambdaTestUtils; import org.junit.After; import org.junit.Assert; @@ -673,4 +683,77 @@ public void testSlowDiskParameters() throws ReconfigurationException, IOExceptio dn.getDiskMetrics().getSlowDiskDetector().getLowThresholdMs()); } } + + @Test + public void testDfsUsageParameters() throws ReconfigurationException { + String[] dfsUsageParameters = { + FS_DU_INTERVAL_KEY, + FS_GETSPACEUSED_JITTER_KEY}; + + for (int i = 0; i < NUM_DATA_NODE; i++) { + DataNode dn = cluster.getDataNodes().get(i); + + // Try invalid values. + for (String parameter : dfsUsageParameters) { + try { + dn.reconfigureProperty(parameter, "text"); + fail("ReconfigurationException expected"); + } catch (ReconfigurationException expected) { + assertTrue("expecting NumberFormatException", + expected.getCause() instanceof NumberFormatException); + } + + try { + dn.reconfigureProperty(parameter, String.valueOf(-1)); + fail("ReconfigurationException expected"); + } catch (ReconfigurationException expected) { + assertTrue("expecting IllegalArgumentException", + expected.getCause() instanceof IllegalArgumentException); + } + } + + // Change and verify properties. + for (String parameter : dfsUsageParameters) { + dn.reconfigureProperty(parameter, "99"); + } + List volumeList = dn.data.getVolumeList(); + for (FsVolumeImpl fsVolume : volumeList) { + Map blockPoolSlices = fsVolume.getBlockPoolSlices(); + for (Map.Entry entry : blockPoolSlices.entrySet()) { + GetSpaceUsed dfsUsage = entry.getValue().getDfsUsage(); + if (dfsUsage instanceof CachingGetSpaceUsed) { + assertEquals(99, + ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getRefreshInterval()); + assertEquals(99, + ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getJitter()); + } + } + } + + // Revert to default and verify. + for (String parameter : dfsUsageParameters) { + dn.reconfigureProperty(parameter, null); + } + for (FsVolumeImpl fsVolume : volumeList) { + Map blockPoolSlices = fsVolume.getBlockPoolSlices(); + for (Map.Entry entry : blockPoolSlices.entrySet()) { + GetSpaceUsed dfsUsage = entry.getValue().getDfsUsage(); + if (dfsUsage instanceof CachingGetSpaceUsed) { + assertEquals(String.format("expect %s is not configured", + FS_DU_INTERVAL_KEY), FS_DU_INTERVAL_DEFAULT, + ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getRefreshInterval()); + assertEquals(String.format("expect %s is not configured", + FS_GETSPACEUSED_JITTER_KEY), FS_GETSPACEUSED_JITTER_DEFAULT, + ((CachingGetSpaceUsed) entry.getValue().getDfsUsage()).getJitter()); + } + assertEquals(String.format("expect %s is not configured", + FS_DU_INTERVAL_KEY), null, + dn.getConf().get(FS_DU_INTERVAL_KEY)); + assertEquals(String.format("expect %s is not configured", + FS_GETSPACEUSED_JITTER_KEY), null, + dn.getConf().get(FS_GETSPACEUSED_JITTER_KEY)); + } + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java index 1c6597eb4541a..77e2e2077d1f5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalDatasetImpl.java @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.server.common.AutoCloseDataSetLock; import org.apache.hadoop.hdfs.server.common.DataNodeLockManager; +import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MountVolumeMap; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockListAsLongs; @@ -467,4 +468,9 @@ public Set deepCopyReplica(String bpid) public MountVolumeMap getMountVolumeMap() { return null; } + + @Override + public List getVolumeList() { + return null; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java index 0f8c4cdc8a4ec..648fb854e52b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java @@ -339,7 +339,7 @@ public void testDataNodeGetReconfigurableProperties() throws IOException { final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("datanode", address, outs, errs); - assertEquals(16, outs.size()); + assertEquals(18, outs.size()); assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, outs.get(1)); } From 6e00a799e772b4d698ca56d12e9611c0c4aa6563 Mon Sep 17 00:00:00 2001 From: Junfan Zhang Date: Wed, 30 Mar 2022 21:57:25 +0800 Subject: [PATCH 080/145] YARN-11106. Fix the test failure due to missing conf of yarn.resourcemanager.node-labels.am.default-node-label-expression. Contributed by Junfan Zhang --- .../src/main/resources/yarn-default.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index 1416665a4fb5d..45b05b7a337be 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -3452,6 +3452,14 @@ 30000 + + + Overwrites default-node-label-expression only for the ApplicationMaster + container. It is disabled by default. + + yarn.resourcemanager.node-labels.am.default-node-label-expression + + From ab8c36062047e5c4c76ab0399162af9765988690 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Sat, 26 Dec 2020 23:59:30 +0100 Subject: [PATCH 081/145] YARN-10550. Decouple NM runner logic from SLSRunner. Contributed by Szilard Nemeth --- .../org/apache/hadoop/yarn/sls/AMRunner.java | 10 +- .../org/apache/hadoop/yarn/sls/NMRunner.java | 238 ++++++++++++++++++ .../org/apache/hadoop/yarn/sls/SLSRunner.java | 213 ++++------------ .../hadoop/yarn/sls/nodemanager/NodeInfo.java | 1 - .../yarn/sls/scheduler/RMNodeWrapper.java | 1 - .../yarn/sls/scheduler/SchedulerMetrics.java | 12 +- 6 files changed, 296 insertions(+), 179 deletions(-) create mode 100644 hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java index 301b4260f35cd..d80337688d5e2 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java @@ -63,7 +63,6 @@ public class AMRunner { private Map amClassMap; private TraceType inputType; private String[] inputTraces; - private SynthTraceJobProducer stjp; private TaskRunner runner; private SLSRunner slsRunner; private int numAMs, numTasks; @@ -148,16 +147,15 @@ private void startAMFromSLSTrace(String inputTrace) throws IOException { private void startAMFromSynthGenerator() throws YarnException, IOException { Configuration localConf = new Configuration(); localConf.set("fs.defaultFS", "file:///"); - // if we use the nodeFile this could have been not initialized yet. - if (stjp == null) { - stjp = new SynthTraceJobProducer(conf, new Path(inputTraces[0])); - slsRunner.setStjp(stjp); + //if we use the nodeFile this could have been not initialized yet. + if (slsRunner.getStjp() == null) { + slsRunner.setStjp(new SynthTraceJobProducer(conf, new Path(inputTraces[0]))); } SynthJob job; // we use stjp, a reference to the job producer instantiated during node // creation - while ((job = (SynthJob) stjp.getNextJob()) != null) { + while ((job = (SynthJob) slsRunner.getStjp().getNextJob()) != null) { ReservationId reservationId = null; if (job.hasDeadline()) { reservationId = ReservationId diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java new file mode 100644 index 0000000000000..224e1e373ff0b --- /dev/null +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java @@ -0,0 +1,238 @@ +/* + * 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.hadoop.yarn.sls; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.api.records.NodeState; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.ResourceInformation; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; +import org.apache.hadoop.yarn.sls.SLSRunner.TraceType; +import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; +import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator; +import org.apache.hadoop.yarn.sls.scheduler.TaskRunner; +import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer; +import org.apache.hadoop.yarn.sls.utils.SLSUtils; +import org.apache.hadoop.yarn.util.resource.ResourceUtils; +import org.apache.hadoop.yarn.util.resource.Resources; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +public class NMRunner { + private static final Logger LOG = LoggerFactory.getLogger(NMRunner.class); + + // other simulation information + private int numNMs, numRacks; + + // NM simulator + private Map nmMap; + private Resource nodeManagerResource; + private String nodeFile; + private TaskRunner taskRunner; + private Configuration conf; + private ResourceManager rm; + private String tableMapping; + private int threadPoolSize; + private TraceType inputType; + private String[] inputTraces; + private SynthTraceJobProducer stjp; + + public NMRunner(TaskRunner taskRunner, Configuration conf, ResourceManager rm, String tableMapping, int threadPoolSize) { + this.taskRunner = taskRunner; + this.conf = conf; + this.rm = rm; + this.tableMapping = tableMapping; + this.threadPoolSize = threadPoolSize; + this.nmMap = new ConcurrentHashMap<>(); + this.nodeManagerResource = getNodeManagerResourceFromConf(); + } + + public void startNM() throws YarnException, IOException, + InterruptedException { + // nm configuration + int heartbeatInterval = conf.getInt( + SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS, + SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT); + float resourceUtilizationRatio = conf.getFloat( + SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO, + SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO_DEFAULT); + // nm information (fetch from topology file, or from sls/rumen json file) + Set nodeSet = null; + if (nodeFile.isEmpty()) { + for (String inputTrace : inputTraces) { + switch (inputType) { + case SLS: + nodeSet = SLSUtils.parseNodesFromSLSTrace(inputTrace); + break; + case RUMEN: + nodeSet = SLSUtils.parseNodesFromRumenTrace(inputTrace); + break; + case SYNTH: + stjp = new SynthTraceJobProducer(conf, new Path(inputTraces[0])); + nodeSet = SLSUtils.generateNodes(stjp.getNumNodes(), + stjp.getNumNodes()/stjp.getNodesPerRack()); + break; + default: + throw new YarnException("Input configuration not recognized, " + + "trace type should be SLS, RUMEN, or SYNTH"); + } + } + } else { + nodeSet = SLSUtils.parseNodesFromNodeFile(nodeFile, + nodeManagerResource); + } + + if (nodeSet == null || nodeSet.isEmpty()) { + throw new YarnException("No node! Please configure nodes."); + } + + SLSUtils.generateNodeTableMapping(nodeSet, tableMapping); + + // create NM simulators + Random random = new Random(); + Set rackSet = ConcurrentHashMap.newKeySet(); + int threadPoolSize = Math.max(this.threadPoolSize, + SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT); + ExecutorService executorService = Executors. + newFixedThreadPool(threadPoolSize); + for (SLSRunner.NodeDetails nodeDetails : nodeSet) { + executorService.submit(new Runnable() { + @Override public void run() { + try { + // we randomize the heartbeat start time from zero to 1 interval + NMSimulator nm = new NMSimulator(); + Resource nmResource = nodeManagerResource; + String hostName = nodeDetails.getHostname(); + if (nodeDetails.getNodeResource() != null) { + nmResource = nodeDetails.getNodeResource(); + } + Set nodeLabels = nodeDetails.getLabels(); + nm.init(hostName, nmResource, + random.nextInt(heartbeatInterval), + heartbeatInterval, rm, resourceUtilizationRatio, nodeLabels); + nmMap.put(nm.getNode().getNodeID(), nm); + taskRunner.schedule(nm); + rackSet.add(nm.getNode().getRackName()); + } catch (IOException | YarnException e) { + LOG.error("Got an error while adding node", e); + } + } + }); + } + executorService.shutdown(); + executorService.awaitTermination(10, TimeUnit.MINUTES); + numRacks = rackSet.size(); + numNMs = nmMap.size(); + } + + void waitForNodesRunning() throws InterruptedException { + long startTimeMS = System.currentTimeMillis(); + while (true) { + int numRunningNodes = 0; + for (RMNode node : rm.getRMContext().getRMNodes().values()) { + if (node.getState() == NodeState.RUNNING) { + numRunningNodes++; + } + } + if (numRunningNodes == numNMs) { + break; + } + LOG.info("SLSRunner is waiting for all nodes RUNNING." + + " {} of {} NMs initialized.", numRunningNodes, numNMs); + Thread.sleep(1000); + } + LOG.info("SLSRunner takes {} ms to launch all nodes.", + System.currentTimeMillis() - startTimeMS); + } + + private Resource getNodeManagerResourceFromConf() { + Resource resource = Resources.createResource(0); + ResourceInformation[] infors = ResourceUtils.getResourceTypesArray(); + for (ResourceInformation info : infors) { + long value; + if (info.getName().equals(ResourceInformation.MEMORY_URI)) { + value = conf.getInt(SLSConfiguration.NM_MEMORY_MB, + SLSConfiguration.NM_MEMORY_MB_DEFAULT); + } else if (info.getName().equals(ResourceInformation.VCORES_URI)) { + value = conf.getInt(SLSConfiguration.NM_VCORES, + SLSConfiguration.NM_VCORES_DEFAULT); + } else { + value = conf.getLong(SLSConfiguration.NM_PREFIX + + info.getName(), SLSConfiguration.NM_RESOURCE_DEFAULT); + } + + resource.setResourceValue(info.getName(), value); + } + + return resource; + } + + public void setNodeFile(String nodeFile) { + this.nodeFile = nodeFile; + } + + + public void setInputType(TraceType inputType) { + this.inputType = inputType; + } + + public void setInputTraces(String[] inputTraces) { + this.inputTraces = inputTraces; + } + + public int getNumNMs() { + return numNMs; + } + + public int getNumRacks() { + return numRacks; + } + + public Resource getNodeManagerResource() { + return nodeManagerResource; + } + + public Map getNmMap() { + return nmMap; + } + + public SynthTraceJobProducer getStjp() { + return stjp; + } + + public void setTableMapping(String tableMapping) { + this.tableMapping = tableMapping; + } + + public void setRm(ResourceManager rm) { + this.rm = rm; + } +} diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index e9ae7f51dbafa..318476427a69b 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -59,11 +59,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeLabel; -import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.exceptions.YarnException; -import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.sls.appmaster.AMSimulator; import org.apache.hadoop.yarn.sls.conf.SLSConfiguration; import org.apache.hadoop.yarn.sls.nodemanager.NMSimulator; @@ -71,8 +69,6 @@ import org.apache.hadoop.yarn.sls.scheduler.TaskRunner; import org.apache.hadoop.yarn.sls.scheduler.Tracker; import org.apache.hadoop.yarn.sls.synthetic.SynthTraceJobProducer; -import org.apache.hadoop.yarn.sls.utils.SLSUtils; -import org.apache.hadoop.yarn.util.resource.ResourceUtils; import org.apache.hadoop.yarn.util.resource.Resources; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,32 +81,19 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; -import java.util.Random; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; @Private @Unstable public class SLSRunner extends Configured implements Tool { private static TaskRunner runner = new TaskRunner(); private String[] inputTraces; - private int poolSize; - - // NM simulator - private Map nmMap; - private Resource nodeManagerResource; - private String nodeFile; // metrics private boolean printSimulation; - // other simulation information - private int numNMs, numRacks; - - private final static Map simulateInfoMap = new HashMap<>(); + private final static Map simulateInfoMap = + new HashMap<>(); // logger public final static Logger LOG = LoggerFactory.getLogger(SLSRunner.class); @@ -118,6 +101,9 @@ public class SLSRunner extends Configured implements Tool { private static boolean exitAtTheFinish = false; private AMRunner amRunner; private RMRunner rmRunner; + private NMRunner nmRunner; + + private SynthTraceJobProducer stjp; /** * The type of trace in input. @@ -130,19 +116,16 @@ public enum TraceType { public static final String NETWORK_NEGATIVE_CACHE_TTL = "networkaddress.cache.negative.ttl"; - private TraceType inputType; - private SynthTraceJobProducer stjp; - public static int getRemainingApps() { return AMRunner.REMAINING_APPS; } - public SLSRunner() throws ClassNotFoundException { + public SLSRunner() throws ClassNotFoundException, YarnException { Configuration tempConf = new Configuration(false); init(tempConf); } - public SLSRunner(Configuration tempConf) throws ClassNotFoundException { + public SLSRunner(Configuration tempConf) throws ClassNotFoundException, YarnException { init(tempConf); } @@ -156,43 +139,31 @@ public void setConf(Configuration conf) { super.setConf(conf); } - private void init(Configuration tempConf) throws ClassNotFoundException { + private void init(Configuration tempConf) throws ClassNotFoundException, YarnException { // runner configuration setConf(tempConf); - - nmMap = new ConcurrentHashMap<>(); - amRunner = new AMRunner(runner, this); - rmRunner = new RMRunner(tempConf, this); - - // runner - poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE, + + int poolSize = tempConf.getInt(SLSConfiguration.RUNNER_POOL_SIZE, SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT); SLSRunner.runner.setQueueSize(poolSize); + rmRunner = new RMRunner(getConf(), this); + nmRunner = new NMRunner(runner, getConf(), rmRunner.getRm(), rmRunner.getTableMapping(), poolSize); + amRunner = new AMRunner(runner, this); amRunner.init(tempConf); - nodeManagerResource = getNodeManagerResource(); } - private Resource getNodeManagerResource() { - Resource resource = Resources.createResource(0); - ResourceInformation[] infors = ResourceUtils.getResourceTypesArray(); - for (ResourceInformation info : infors) { - long value; - if (info.getName().equals(ResourceInformation.MEMORY_URI)) { - value = getConf().getInt(SLSConfiguration.NM_MEMORY_MB, - SLSConfiguration.NM_MEMORY_MB_DEFAULT); - } else if (info.getName().equals(ResourceInformation.VCORES_URI)) { - value = getConf().getInt(SLSConfiguration.NM_VCORES, - SLSConfiguration.NM_VCORES_DEFAULT); - } else { - value = getConf().getLong(SLSConfiguration.NM_PREFIX + - info.getName(), SLSConfiguration.NM_RESOURCE_DEFAULT); + private SynthTraceJobProducer getSynthJobTraceProducer() throws YarnException { + // if we use the nodeFile this could have been not initialized yet. + if (nmRunner.getStjp() != null) { + return nmRunner.getStjp(); + } else { + try { + return new SynthTraceJobProducer(getConf(), new Path(inputTraces[0])); + } catch (IOException e) { + throw new YarnException("Failed to initialize SynthTraceJobProducer", e); } - - resource.setResourceValue(info.getName(), value); } - - return resource; } /** @@ -213,29 +184,37 @@ public static Map getSimulateInfoMap() { */ public void setSimulationParams(TraceType inType, String[] inTraces, String nodes, String metricsOutputDir, Set trackApps, - boolean printsimulation) { - - this.inputType = inType; + boolean printsimulation) throws YarnException { this.inputTraces = inTraces.clone(); - this.amRunner.setInputType(this.inputType); + this.amRunner.setInputType(inType); this.amRunner.setInputTraces(this.inputTraces); this.amRunner.setTrackedApps(trackApps); - this.nodeFile = nodes; + this.nmRunner.setNodeFile(nodes); + this.nmRunner.setInputType(inType); + this.nmRunner.setInputTraces(this.inputTraces); this.printSimulation = printsimulation; this.rmRunner.setMetricsOutputDir(metricsOutputDir); - this.rmRunner.setTableMapping(metricsOutputDir + "/tableMapping.csv"); + String tableMapping = metricsOutputDir + "/tableMapping.csv"; + this.rmRunner.setTableMapping(tableMapping); + this.nmRunner.setTableMapping(tableMapping); + + //We need this.inputTraces to set before creating SynthTraceJobProducer + if (inType == TraceType.SYNTH) { + this.stjp = getSynthJobTraceProducer(); + } } public void start() throws IOException, ClassNotFoundException, YarnException, InterruptedException { - enableDNSCaching(getConf()); // start resource manager rmRunner.startRM(); + nmRunner.setRm(rmRunner.getRm()); amRunner.setResourceManager(rmRunner.getRm()); + // start node managers - startNM(); + nmRunner.startNM(); // start application masters amRunner.startAM(); @@ -248,7 +227,7 @@ public void start() throws IOException, ClassNotFoundException, YarnException, // print out simulation info printSimulationInfo(); // blocked until all nodes RUNNING - waitForNodesRunning(); + nmRunner.waitForNodesRunning(); // starting the runner once everything is ready to go, runner.start(); } @@ -270,104 +249,6 @@ static void enableDNSCaching(Configuration conf) { } } - private void startNM() throws YarnException, IOException, - InterruptedException { - // nm configuration - int heartbeatInterval = getConf().getInt( - SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS, - SLSConfiguration.NM_HEARTBEAT_INTERVAL_MS_DEFAULT); - float resourceUtilizationRatio = getConf().getFloat( - SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO, - SLSConfiguration.NM_RESOURCE_UTILIZATION_RATIO_DEFAULT); - // nm information (fetch from topology file, or from sls/rumen json file) - Set nodeSet = null; - if (nodeFile.isEmpty()) { - for (String inputTrace : inputTraces) { - switch (inputType) { - case SLS: - nodeSet = SLSUtils.parseNodesFromSLSTrace(inputTrace); - break; - case RUMEN: - nodeSet = SLSUtils.parseNodesFromRumenTrace(inputTrace); - break; - case SYNTH: - stjp = new SynthTraceJobProducer(getConf(), new Path(inputTraces[0])); - nodeSet = SLSUtils.generateNodes(stjp.getNumNodes(), - stjp.getNumNodes()/stjp.getNodesPerRack()); - break; - default: - throw new YarnException("Input configuration not recognized, " - + "trace type should be SLS, RUMEN, or SYNTH"); - } - } - } else { - nodeSet = SLSUtils.parseNodesFromNodeFile(nodeFile, - nodeManagerResource); - } - - if (nodeSet == null || nodeSet.isEmpty()) { - throw new YarnException("No node! Please configure nodes."); - } - - SLSUtils.generateNodeTableMapping(nodeSet, rmRunner.getTableMapping()); - - // create NM simulators - Random random = new Random(); - Set rackSet = ConcurrentHashMap.newKeySet(); - int threadPoolSize = Math.max(poolSize, - SLSConfiguration.RUNNER_POOL_SIZE_DEFAULT); - ExecutorService executorService = Executors. - newFixedThreadPool(threadPoolSize); - for (NodeDetails nodeDetails : nodeSet) { - executorService.submit(new Runnable() { - @Override public void run() { - try { - // we randomize the heartbeat start time from zero to 1 interval - NMSimulator nm = new NMSimulator(); - Resource nmResource = nodeManagerResource; - String hostName = nodeDetails.getHostname(); - if (nodeDetails.getNodeResource() != null) { - nmResource = nodeDetails.getNodeResource(); - } - Set nodeLabels = nodeDetails.getLabels(); - nm.init(hostName, nmResource, - random.nextInt(heartbeatInterval), - heartbeatInterval, rmRunner.getRm(), resourceUtilizationRatio, nodeLabels); - nmMap.put(nm.getNode().getNodeID(), nm); - runner.schedule(nm); - rackSet.add(nm.getNode().getRackName()); - } catch (IOException | YarnException e) { - LOG.error("Got an error while adding node", e); - } - } - }); - } - executorService.shutdown(); - executorService.awaitTermination(10, TimeUnit.MINUTES); - numRacks = rackSet.size(); - numNMs = nmMap.size(); - } - - private void waitForNodesRunning() throws InterruptedException { - long startTimeMS = System.currentTimeMillis(); - while (true) { - int numRunningNodes = 0; - for (RMNode node : rmRunner.getRm().getRMContext().getRMNodes().values()) { - if (node.getState() == NodeState.RUNNING) { - numRunningNodes++; - } - } - if (numRunningNodes == numNMs) { - break; - } - LOG.info("SLSRunner is waiting for all nodes RUNNING." - + " {} of {} NMs initialized.", numRunningNodes, numNMs); - Thread.sleep(1000); - } - LOG.info("SLSRunner takes {} ms to launch all nodes.", - System.currentTimeMillis() - startTimeMS); - } - Resource getDefaultContainerResource() { int containerMemory = getConf().getInt(SLSConfiguration.CONTAINER_MEMORY_MB, SLSConfiguration.CONTAINER_MEMORY_MB_DEFAULT); @@ -392,7 +273,7 @@ private void printSimulationInfo() { LOG.info("------------------------------------"); LOG.info("# nodes = {}, # racks = {}, capacity " + "of each node {}.", - numNMs, numRacks, nodeManagerResource); + nmRunner.getNumNMs(), nmRunner.getNumRacks(), nmRunner.getNodeManagerResource()); LOG.info("------------------------------------"); // job LOG.info("# applications = {}, # total " + @@ -416,12 +297,12 @@ private void printSimulationInfo() { LOG.info("------------------------------------"); } // package these information in the simulateInfoMap used by other places - simulateInfoMap.put("Number of racks", numRacks); - simulateInfoMap.put("Number of nodes", numNMs); + simulateInfoMap.put("Number of racks", nmRunner.getNumRacks()); + simulateInfoMap.put("Number of nodes", nmRunner.getNumNMs()); simulateInfoMap.put("Node memory (MB)", - nodeManagerResource.getResourceValue(ResourceInformation.MEMORY_URI)); + nmRunner.getNodeManagerResource().getResourceValue(ResourceInformation.MEMORY_URI)); simulateInfoMap.put("Node VCores", - nodeManagerResource.getResourceValue(ResourceInformation.VCORES_URI)); + nmRunner.getNodeManagerResource().getResourceValue(ResourceInformation.VCORES_URI)); simulateInfoMap.put("Number of applications", numAMs); simulateInfoMap.put("Number of tasks", numTasks); simulateInfoMap.put("Average tasks per applicaion", @@ -434,7 +315,7 @@ private void printSimulationInfo() { } public Map getNmMap() { - return nmMap; + return nmRunner.getNmMap(); } public static void decreaseRemainingApps() { @@ -458,7 +339,6 @@ public void stop() throws InterruptedException { public int run(final String[] argv) throws IOException, InterruptedException, ParseException, ClassNotFoundException, YarnException { - Options options = new Options(); // Left for compatibility @@ -524,7 +404,6 @@ public int run(final String[] argv) throws IOException, InterruptedException, case "RUMEN": tempTraceType = TraceType.RUMEN; break; - case "SYNTH": tempTraceType = TraceType.SYNTH; break; @@ -537,7 +416,7 @@ public int run(final String[] argv) throws IOException, InterruptedException, setSimulationParams(tempTraceType, inputFiles, tempNodeFile, output, trackedJobSet, cmd.hasOption("printsimulation")); - + start(); return 0; diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java index 32567db666ef3..a22230f86616b 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java @@ -186,7 +186,6 @@ public Set getNodeLabels() { @Override public List pullNewlyIncreasedContainers() { - // TODO Auto-generated method stub return null; } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java index 26d35ac897235..dbbc88fb52d00 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/RMNodeWrapper.java @@ -176,7 +176,6 @@ public Set getNodeLabels() { @Override public List pullNewlyIncreasedContainers() { - // TODO Auto-generated method stub return Collections.emptyList(); } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java index 26a9da4cd8bc8..26fbcd78f3969 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java @@ -316,7 +316,7 @@ private void registerClusterResourceMetrics() { new Gauge() { @Override public Long getValue() { - if (scheduler.getRootQueueMetrics() == null) { + if (isMetricsAvailable()) { return 0L; } else { return scheduler.getRootQueueMetrics().getAllocatedMB(); @@ -328,7 +328,7 @@ public Long getValue() { new Gauge() { @Override public Integer getValue() { - if (scheduler.getRootQueueMetrics() == null) { + if (isMetricsAvailable()) { return 0; } else { return scheduler.getRootQueueMetrics().getAllocatedVirtualCores(); @@ -340,7 +340,7 @@ public Integer getValue() { new Gauge() { @Override public Long getValue() { - if (scheduler.getRootQueueMetrics() == null) { + if (isMetricsAvailable()) { return 0L; } else { return scheduler.getRootQueueMetrics().getAvailableMB(); @@ -352,7 +352,7 @@ public Long getValue() { new Gauge() { @Override public Integer getValue() { - if (scheduler.getRootQueueMetrics() == null) { + if (isMetricsAvailable()) { return 0; } else { return scheduler.getRootQueueMetrics().getAvailableVirtualCores(); @@ -362,6 +362,10 @@ public Integer getValue() { ); } + private boolean isMetricsAvailable() { + return scheduler.getRootQueueMetrics() == null; + } + private void registerContainerAppNumMetrics() { metrics.register("variable.running.application", new Gauge() { From 2bf78e2416875c8f82f485b457420fafd42aa977 Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Thu, 31 Mar 2022 14:00:38 +0800 Subject: [PATCH 082/145] HDFS-16511. Improve lock type for ReplicaMap under fine-grain lock mode. (#4085). Contributed by limingxiang. Signed-off-by: He Xiaoqiao --- .../datanode/fsdataset/impl/ReplicaMap.java | 16 +++--- .../fsdataset/impl/TestFsDatasetImpl.java | 51 +++++++++++++++++++ 2 files changed, 59 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java index 25d302b86c20a..6ecc48a95fd2d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/ReplicaMap.java @@ -120,12 +120,12 @@ ReplicaInfo get(String bpid, long blockId) { ReplicaInfo add(String bpid, ReplicaInfo replicaInfo) { checkBlockPool(bpid); checkBlock(replicaInfo); - try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m == null) { // Add an entry for block pool if it does not exist already - m = new LightWeightResizableGSet(); - map.put(bpid, m); + map.putIfAbsent(bpid, new LightWeightResizableGSet()); + m = map.get(bpid); } return m.put(replicaInfo); } @@ -138,12 +138,12 @@ ReplicaInfo add(String bpid, ReplicaInfo replicaInfo) { ReplicaInfo addAndGet(String bpid, ReplicaInfo replicaInfo) { checkBlockPool(bpid); checkBlock(replicaInfo); - try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m == null) { // Add an entry for block pool if it does not exist already - m = new LightWeightResizableGSet(); - map.put(bpid, m); + map.putIfAbsent(bpid, new LightWeightResizableGSet()); + m = map.get(bpid); } ReplicaInfo oldReplicaInfo = m.get(replicaInfo); if (oldReplicaInfo != null) { @@ -202,7 +202,7 @@ void mergeAll(ReplicaMap other) { ReplicaInfo remove(String bpid, Block block) { checkBlockPool(bpid); checkBlock(block); - try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m != null) { ReplicaInfo replicaInfo = m.get(block); @@ -224,7 +224,7 @@ ReplicaInfo remove(String bpid, Block block) { */ ReplicaInfo remove(String bpid, long blockId) { checkBlockPool(bpid); - try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { LightWeightResizableGSet m = map.get(bpid); if (m != null) { return m.remove(new Block(blockId)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java index 23a72f9acfa4e..f250eea2920d3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java @@ -21,6 +21,10 @@ import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.function.Supplier; import org.apache.hadoop.fs.DF; @@ -602,6 +606,53 @@ public void run() {} + "volumeMap.", 0, totalNumReplicas); } + @Test(timeout = 30000) + public void testConcurrentWriteAndDeleteBlock() throws Exception { + // Feed FsDataset with block metadata. + final int numBlocks = 1000; + final int threadCount = 10; + // Generate data blocks. + ExecutorService pool = Executors.newFixedThreadPool(threadCount); + List> futureList = new ArrayList<>(); + Random random = new Random(); + // Random write block and delete half of them. + for (int i = 0; i < threadCount; i++) { + Thread thread = new Thread() { + @Override + public void run() { + try { + String bpid = BLOCK_POOL_IDS[random.nextInt(BLOCK_POOL_IDS.length)]; + for (int blockId = 0; blockId < numBlocks; blockId++) { + ExtendedBlock eb = new ExtendedBlock(bpid, blockId); + ReplicaHandler replica = null; + try { + replica = dataset.createRbw(StorageType.DEFAULT, null, eb, + false); + if (blockId % 2 > 0) { + dataset.invalidate(bpid, new Block[]{eb.getLocalBlock()}); + } + } finally { + if (replica != null) { + replica.close(); + } + } + } + // Just keep final consistency no need to care exception. + } catch (Exception ignore) {} + } + }; + thread.setName("AddBlock" + i); + futureList.add(pool.submit(thread)); + } + // Wait for data generation + for (Future f : futureList) { + f.get(); + } + for (String bpid : dataset.volumeMap.getBlockPoolList()) { + assertEquals(numBlocks / 2, dataset.volumeMap.size(bpid)); + } + } + @Test(timeout = 5000) public void testRemoveNewlyAddedVolume() throws IOException { final int numExistingVolumes = getNumVolumes(); From 9a4dddd64080bc1a1bf3a6e2d6116ce29e024e49 Mon Sep 17 00:00:00 2001 From: litao Date: Thu, 31 Mar 2022 14:01:48 +0800 Subject: [PATCH 083/145] HDFS-16507. [SBN read] Avoid purging edit log which is in progress (#4082) --- .../apache/hadoop/hdfs/server/namenode/FSEditLog.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java index 53c663307d738..850b2fc570884 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java @@ -1515,11 +1515,12 @@ public synchronized void purgeLogsOlderThan(final long minTxIdToKeep) { if (!isOpenForWrite()) { return; } - - assert curSegmentTxId == HdfsServerConstants.INVALID_TXID || // on format this is no-op - minTxIdToKeep <= curSegmentTxId : - "cannot purge logs older than txid " + minTxIdToKeep + - " when current segment starts at " + curSegmentTxId; + + Preconditions.checkArgument( + curSegmentTxId == HdfsServerConstants.INVALID_TXID || // on format this is no-op + minTxIdToKeep <= curSegmentTxId, + "cannot purge logs older than txid " + minTxIdToKeep + + " when current segment starts at " + curSegmentTxId); if (minTxIdToKeep == 0) { return; } From e044a46f97dcc7998dc0737f15cf3956dca170c4 Mon Sep 17 00:00:00 2001 From: Junfan Zhang Date: Thu, 24 Mar 2022 22:33:59 +0800 Subject: [PATCH 084/145] YARN-11088. Introduce the config to control the AM allocated to non-exclusive nodes. Contributed by Junfan Zhang --- .../hadoop/yarn/conf/YarnConfiguration.java | 3 + .../src/main/resources/yarn-default.xml | 9 ++ .../allocator/RegularContainerAllocator.java | 7 +- ...estAMAllocatedToNonExclusivePartition.java | 123 ++++++++++++++++++ 4 files changed, 139 insertions(+), 3 deletions(-) create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAMAllocatedToNonExclusivePartition.java diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java index df01a12f0a395..d42562cf6140a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java @@ -4412,6 +4412,9 @@ public static boolean areNodeLabelsEnabled( public static final String AM_DEFAULT_NODE_LABEL = RM_NODE_LABELS_PREFIX + "am.default-node-label-expression"; + public static final String AM_ALLOW_NON_EXCLUSIVE_ALLOCATION = + RM_NODE_LABELS_PREFIX + "am.allow-non-exclusive-allocation"; + public static final String RM_NODE_LABELS_PROVIDER_CONFIG = RM_NODE_LABELS_PREFIX + "provider"; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml index 45b05b7a337be..407ef74d3d062 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml @@ -3460,6 +3460,15 @@ yarn.resourcemanager.node-labels.am.default-node-label-expression + + + Flag to indicate whether the AM can be allocated to non-exclusive nodes or not. + Default is false. + + yarn.resourcemanager.node-labels.am.allow-non-exclusive-allocation + false + + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java index 669435ecba0ff..c46b0341f74d0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java @@ -60,6 +60,8 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.AM_ALLOW_NON_EXCLUSIVE_ALLOCATION; + /** * Allocate normal (new) containers, considers locality/label, etc. Using * delayed scheduling mechanism to get better locality allocation. @@ -121,10 +123,9 @@ private ContainerAllocation preCheckForNodeCandidateSet(FiCaSchedulerNode node, return ContainerAllocation.PRIORITY_SKIPPED; } - // AM container allocation doesn't support non-exclusive allocation to - // avoid painful of preempt an AM container if (schedulingMode == SchedulingMode.IGNORE_PARTITION_EXCLUSIVITY) { - if (application.isWaitingForAMContainer()) { + if (application.isWaitingForAMContainer() && !rmContext.getYarnConfiguration() + .getBoolean(AM_ALLOW_NON_EXCLUSIVE_ALLOCATION, false)) { LOG.debug("Skip allocating AM container to app_attempt={}," + " don't allow to allocate AM container in non-exclusive mode", application.getApplicationAttemptId()); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAMAllocatedToNonExclusivePartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAMAllocatedToNonExclusivePartition.java new file mode 100644 index 0000000000000..469d12c149866 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestAMAllocatedToNonExclusivePartition.java @@ -0,0 +1,123 @@ +/** + * 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.hadoop.yarn.server.resourcemanager.scheduler.capacity; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.Sets; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.MockAM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmissionData; +import org.apache.hadoop.yarn.server.resourcemanager.MockRMAppSubmitter; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.NullRMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; + +public class TestAMAllocatedToNonExclusivePartition { + private Configuration conf; + private RMNodeLabelsManager mgr; + + @Before + public void setUp() throws Exception { + conf = new YarnConfiguration(); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true); + conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, true); + conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName()); + mgr = new NullRMNodeLabelsManager(); + mgr.init(conf); + } + + @SuppressWarnings("unchecked") + private Set toSet(E... elements) { + Set set = Sets.newHashSet(elements); + return set; + } + + /** + * Test that AM can be allocated to non-exclusive partition when the config + * of {@code YarnConfiguration.AM_ALLOW_NON_EXCLUSIVE_ALLOCATION} is true. + */ + @Test + public void testAMAllowToNonExclusivePartition() throws Exception { + conf.setBoolean(YarnConfiguration.AM_ALLOW_NON_EXCLUSIVE_ALLOCATION, true); + + mgr.addToCluserNodeLabels( + Arrays.asList(NodeLabel.newInstance("x", false)) + ); + mgr.addLabelsToNode( + ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")) + ); + + conf = TestUtils.getConfigurationWithDefaultQueueLabels(conf); + + MockRM rm1 = new MockRM(conf) { + @Override + public RMNodeLabelsManager createNodeLabelManager() { + return mgr; + } + }; + rm1.getRMContext().setNodeLabelManager(mgr); + rm1.start(); + + MockNM nm1 = rm1.registerNode("h1:1234", 8000); // label = x + + MockRMAppSubmissionData data2 = + MockRMAppSubmissionData + .Builder + .createWithMemory(200, rm1) + .withAppName("app") + .withUser("user") + .withAcls(null) + .withQueue("c1") + .withUnmanagedAM(false) + .build(); + + RMApp app1 = MockRMAppSubmitter.submit(rm1, data2); + // Wait the AM allocated to non-partition node of h1 + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm1, nm1); + + ContainerId containerId; + + // Request a container and it also should be allocated to non-partition node of h1 + am1.allocate("*", 1024, 1, new ArrayList()); + containerId = ContainerId.newContainerId(am1.getApplicationAttemptId(), 2); + Assert.assertTrue(rm1.waitForState(nm1, containerId, RMContainerState.ALLOCATED)); + + rm1.close(); + } +} From 94031b729d01d337268dc2f5ae286de71c768b02 Mon Sep 17 00:00:00 2001 From: Szilard Nemeth Date: Tue, 29 Mar 2022 21:05:51 +0200 Subject: [PATCH 085/145] YARN-11103. SLS cleanup after previously merged SLS refactor jiras. Contributed by Szilard Nemeth --- .../org/apache/hadoop/yarn/sls/AMRunner.java | 12 +- .../yarn/sls/ReservationClientUtil.java | 4 +- .../hadoop/yarn/sls/RumenToSLSConverter.java | 12 +- .../org/apache/hadoop/yarn/sls/SLSRunner.java | 134 +++++++++--------- .../yarn/sls/nodemanager/NMSimulator.java | 4 +- .../hadoop/yarn/sls/nodemanager/NodeInfo.java | 5 +- .../sls/resourcemanager/MockAMLauncher.java | 1 - .../sls/scheduler/SLSSchedulerCommons.java | 6 +- .../yarn/sls/scheduler/SchedulerMetrics.java | 14 +- .../hadoop/yarn/sls/synthetic/SynthJob.java | 4 +- .../sls/synthetic/SynthTraceJobProducer.java | 3 +- .../hadoop/yarn/sls/utils/SLSUtils.java | 6 +- .../hadoop/yarn/sls/BaseSLSRunnerTest.java | 2 +- .../yarn/sls/TestSynthJobGeneration.java | 5 +- .../hadoop/yarn/sls/utils/TestSLSUtils.java | 4 +- 15 files changed, 107 insertions(+), 109 deletions(-) diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java index d80337688d5e2..8ce8d29a01a2c 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java @@ -53,7 +53,7 @@ public class AMRunner { private static final Logger LOG = LoggerFactory.getLogger(AMRunner.class); - static int REMAINING_APPS = 0; + static int remainingApps = 0; private final Configuration conf; private int AM_ID; @@ -63,8 +63,8 @@ public class AMRunner { private Map amClassMap; private TraceType inputType; private String[] inputTraces; - private TaskRunner runner; - private SLSRunner slsRunner; + private final TaskRunner runner; + private final SLSRunner slsRunner; private int numAMs, numTasks; private long maxRuntime; private ResourceManager rm; @@ -81,8 +81,8 @@ public void init(Configuration conf) throws ClassNotFoundException { amClassMap = new HashMap<>(); appIdAMSim = new ConcurrentHashMap<>(); // map - for (Map.Entry e : conf) { - String key = e.getKey().toString(); + for (Map.Entry e : conf) { + String key = e.getKey(); if (key.startsWith(SLSConfiguration.AM_TYPE_PREFIX)) { String amType = key.substring(SLSConfiguration.AM_TYPE_PREFIX.length()); amClassMap.put(amType, Class.forName(conf.get(key))); @@ -112,7 +112,7 @@ public void startAM() throws YarnException, IOException { } numAMs = amMap.size(); - REMAINING_APPS = numAMs; + remainingApps = numAMs; } /** diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java index 7c10a57b1df32..2d04bb240c1a6 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/ReservationClientUtil.java @@ -70,9 +70,7 @@ public static ReservationSubmissionRequest createMRReservation( deadline, reservationRequests, name); // outermost request - ReservationSubmissionRequest request = ReservationSubmissionRequest + return ReservationSubmissionRequest .newInstance(resDef, queueName, reservationId); - - return request; } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java index 76bcb157188e5..2cdfe236c410d 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/RumenToSLSConverter.java @@ -25,6 +25,7 @@ import java.io.OutputStreamWriter; import java.io.Reader; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Iterator; import java.util.LinkedHashMap; @@ -54,7 +55,7 @@ public class RumenToSLSConverter { private static Map> rackNodeMap = new TreeMap>(); - public static void main(String args[]) throws Exception { + public static void main(String[] args) throws Exception { Options options = new Options(); options.addOption("input", true, "input rumen json file"); options.addOption("outputJobs", true, "output jobs file"); @@ -121,9 +122,10 @@ public static void main(String args[]) throws Exception { private static void generateSLSLoadFile(String inputFile, String outputFile) throws IOException { try (Reader input = - new InputStreamReader(new FileInputStream(inputFile), "UTF-8")) { + new InputStreamReader(new FileInputStream(inputFile), + StandardCharsets.UTF_8)) { try (Writer output = - new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) { + new OutputStreamWriter(new FileOutputStream(outputFile), StandardCharsets.UTF_8)) { ObjectMapper mapper = new ObjectMapper(); ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter(); Iterator i = mapper.readValues( @@ -140,7 +142,7 @@ private static void generateSLSLoadFile(String inputFile, String outputFile) private static void generateSLSNodeFile(String outputFile) throws IOException { try (Writer output = - new OutputStreamWriter(new FileOutputStream(outputFile), "UTF-8")) { + new OutputStreamWriter(new FileOutputStream(outputFile), StandardCharsets.UTF_8)) { ObjectMapper mapper = new ObjectMapper(); ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter(); for (Map.Entry> entry : rackNodeMap.entrySet()) { @@ -218,7 +220,7 @@ private static List createSLSTasks(String taskType, task.put("container.priority", priority); task.put("container.type", taskType); array.add(task); - String rackHost[] = SLSUtils.getRackHostName(hostname); + String[] rackHost = SLSUtils.getRackHostName(hostname); if (rackNodeMap.containsKey(rackHost[0])) { rackNodeMap.get(rackHost[0]).add(rackHost[1]); } else { diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index 318476427a69b..0909a45cc4773 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -18,16 +18,11 @@ package org.apache.hadoop.yarn.sls; import java.io.File; -import java.io.FileInputStream; import java.io.IOException; -import java.io.InputStreamReader; -import java.io.Reader; -import java.nio.charset.StandardCharsets; import java.security.Security; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.Map; import java.util.Objects; import java.util.Random; @@ -38,10 +33,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; -import com.fasterxml.jackson.core.JsonFactory; -import com.fasterxml.jackson.databind.JavaType; -import com.fasterxml.jackson.databind.ObjectMapper; - import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.GnuParser; @@ -73,20 +64,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.security.Security; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - @Private @Unstable public class SLSRunner extends Configured implements Tool { - private static TaskRunner runner = new TaskRunner(); + private static final TaskRunner runner = new TaskRunner(); private String[] inputTraces; // metrics @@ -103,6 +84,7 @@ public class SLSRunner extends Configured implements Tool { private RMRunner rmRunner; private NMRunner nmRunner; + private TraceType inputType; private SynthTraceJobProducer stjp; /** @@ -117,7 +99,7 @@ public enum TraceType { "networkaddress.cache.negative.ttl"; public static int getRemainingApps() { - return AMRunner.REMAINING_APPS; + return AMRunner.remainingApps; } public SLSRunner() throws ClassNotFoundException, YarnException { @@ -175,31 +157,32 @@ public static Map getSimulateInfoMap() { /** * This is invoked before start. - * @param inType - * @param inTraces - * @param nodes - * @param metricsOutputDir - * @param trackApps - * @param printsimulation + * @param inputType The trace type + * @param inTraces Input traces + * @param nodes The node file + * @param metricsOutputDir Output dir for metrics + * @param trackApps Track these applications + * @param printSimulation Whether to print the simulation */ - public void setSimulationParams(TraceType inType, String[] inTraces, + public void setSimulationParams(TraceType inputType, String[] inTraces, String nodes, String metricsOutputDir, Set trackApps, - boolean printsimulation) throws YarnException { + boolean printSimulation) throws YarnException { + this.inputType = inputType; this.inputTraces = inTraces.clone(); - this.amRunner.setInputType(inType); + this.amRunner.setInputType(inputType); this.amRunner.setInputTraces(this.inputTraces); this.amRunner.setTrackedApps(trackApps); this.nmRunner.setNodeFile(nodes); - this.nmRunner.setInputType(inType); + this.nmRunner.setInputType(inputType); this.nmRunner.setInputTraces(this.inputTraces); - this.printSimulation = printsimulation; + this.printSimulation = printSimulation; this.rmRunner.setMetricsOutputDir(metricsOutputDir); String tableMapping = metricsOutputDir + "/tableMapping.csv"; this.rmRunner.setTableMapping(tableMapping); this.nmRunner.setTableMapping(tableMapping); //We need this.inputTraces to set before creating SynthTraceJobProducer - if (inType == TraceType.SYNTH) { + if (inputType == TraceType.SYNTH) { this.stjp = getSynthJobTraceProducer(); } } @@ -319,8 +302,8 @@ public Map getNmMap() { } public static void decreaseRemainingApps() { - AMRunner.REMAINING_APPS--; - if (AMRunner.REMAINING_APPS == 0) { + AMRunner.remainingApps--; + if (AMRunner.remainingApps == 0) { exitSLSRunner(); } } @@ -359,24 +342,15 @@ public int run(final String[] argv) throws IOException, InterruptedException, CommandLineParser parser = new GnuParser(); CommandLine cmd = parser.parse(options, argv); - String traceType = null; - String traceLocation = null; - // compatibility with old commandline - if (cmd.hasOption("inputrumen")) { - traceType = "RUMEN"; - traceLocation = cmd.getOptionValue("inputrumen"); - } - if (cmd.hasOption("inputsls")) { - traceType = "SLS"; - traceLocation = cmd.getOptionValue("inputsls"); - } - - if (cmd.hasOption("tracetype")) { - traceType = cmd.getOptionValue("tracetype"); - traceLocation = cmd.getOptionValue("tracelocation"); - } - + boolean hasInputRumenOption = cmd.hasOption("inputrumen"); + boolean hasInputSlsOption = cmd.hasOption("inputsls"); + boolean hasTraceTypeOption = cmd.hasOption("tracetype"); + TraceType traceType = determineTraceType(cmd, hasInputRumenOption, + hasInputSlsOption, hasTraceTypeOption); + String traceLocation = determineTraceLocation(cmd, hasInputRumenOption, + hasInputSlsOption, hasTraceTypeOption); + String output = cmd.getOptionValue("output"); File outputFile = new File(output); @@ -396,30 +370,56 @@ public int run(final String[] argv) throws IOException, InterruptedException, String tempNodeFile = cmd.hasOption("nodes") ? cmd.getOptionValue("nodes") : ""; - TraceType tempTraceType; + String[] inputFiles = traceLocation.split(","); + + setSimulationParams(traceType, inputFiles, tempNodeFile, output, + trackedJobSet, cmd.hasOption("printsimulation")); + + start(); + + return 0; + } + + private TraceType determineTraceType(CommandLine cmd, boolean hasInputRumenOption, + boolean hasInputSlsOption, boolean hasTraceTypeOption) throws YarnException { + String traceType = null; + if (hasInputRumenOption) { + traceType = "RUMEN"; + } + if (hasInputSlsOption) { + traceType = "SLS"; + } + if (hasTraceTypeOption) { + traceType = cmd.getOptionValue("tracetype"); + } + if (traceType == null) { + throw new YarnException("Misconfigured input"); + } switch (traceType) { case "SLS": - tempTraceType = TraceType.SLS; - break; + return TraceType.SLS; case "RUMEN": - tempTraceType = TraceType.RUMEN; - break; + return TraceType.RUMEN; case "SYNTH": - tempTraceType = TraceType.SYNTH; - break; + return TraceType.SYNTH; default: printUsage(); throw new YarnException("Misconfigured input"); } + } - String[] inputFiles = traceLocation.split(","); - - setSimulationParams(tempTraceType, inputFiles, tempNodeFile, output, - trackedJobSet, cmd.hasOption("printsimulation")); - - start(); - - return 0; + private String determineTraceLocation(CommandLine cmd, boolean hasInputRumenOption, + boolean hasInputSlsOption, boolean hasTraceTypeOption) throws YarnException { + if (hasInputRumenOption) { + return cmd.getOptionValue("inputrumen"); + } + if (hasInputSlsOption) { + return cmd.getOptionValue("inputsls"); + } + if (hasTraceTypeOption) { + return cmd.getOptionValue("tracelocation"); + } + throw new YarnException("Misconfigured input! "); } public static void main(String[] argv) throws Exception { diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java index 0d304419351c2..4898ea736f9c7 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NMSimulator.java @@ -85,7 +85,7 @@ public void init(String nodeIdStr, Resource nodeResource, int dispatchTime, super.init(dispatchTime, dispatchTime + 1000000L * heartBeatInterval, heartBeatInterval); // create resource - String rackHostName[] = SLSUtils.getRackHostName(nodeIdStr); + String[] rackHostName = SLSUtils.getRackHostName(nodeIdStr); this.node = NodeInfo.newNodeInfo(rackHostName[0], rackHostName[1], Resources.clone(nodeResource)); this.rm = pRm; @@ -128,7 +128,7 @@ public void firstStep() { @Override public void middleStep() throws Exception { // we check the lifetime for each running containers - ContainerSimulator cs = null; + ContainerSimulator cs; synchronized(completedContainerList) { while ((cs = containerQueue.poll()) != null) { runningContainers.remove(cs.getId()); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java index a22230f86616b..5d1069a5ca298 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/nodemanager/NodeInfo.java @@ -258,9 +258,8 @@ public static RMNode newNodeInfo(String rackName, String hostName, final Resource resource, int port) { final NodeId nodeId = newNodeID(hostName, port); final String nodeAddr = hostName + ":" + port; - final String httpAddress = hostName; - - return new FakeRMNodeImpl(nodeId, nodeAddr, httpAddress, + + return new FakeRMNodeImpl(nodeId, nodeAddr, hostName, resource, rackName, "Me good", port, hostName, null); } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java index e46dea521c551..aec9f0f105b46 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/resourcemanager/MockAMLauncher.java @@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.sls.SLSRunner; import org.apache.hadoop.yarn.sls.appmaster.AMSimulator; -import java.util.Map; public class MockAMLauncher extends ApplicationMasterLauncher implements EventHandler { diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java index d83fe5c6d0e69..ee70dfbe3e96c 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java @@ -61,7 +61,7 @@ public class SLSSchedulerCommons { private final Map appQueueMap = new ConcurrentHashMap<>(); private final Tracker tracker; - public SLSSchedulerCommons(AbstractYarnScheduler scheduler) { + public SLSSchedulerCommons(AbstractYarnScheduler scheduler) { this.scheduler = scheduler; this.tracker = new Tracker(); } @@ -174,7 +174,7 @@ private void updateQueueWithAllocateRequest(Allocation allocation, } } // containers released/preemption from scheduler - Set preemptionContainers = new HashSet(); + Set preemptionContainers = new HashSet<>(); if (allocation.getContainerPreemptions() != null) { preemptionContainers.addAll(allocation.getContainerPreemptions()); } @@ -277,7 +277,7 @@ public void handle(SchedulerEvent schedulerEvent) { AppAttemptAddedSchedulerEvent appAddEvent = (AppAttemptAddedSchedulerEvent) schedulerEvent; SchedulerApplication app = - (SchedulerApplication) scheduler.getSchedulerApplications() + scheduler.getSchedulerApplications() .get(appAddEvent.getApplicationAttemptId().getApplicationId()); appQueueMap.put(appAddEvent.getApplicationAttemptId(), app.getQueue() .getQueueName()); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java index 26fbcd78f3969..a1e530a6f77f7 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerMetrics.java @@ -20,6 +20,7 @@ import java.io.BufferedWriter; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -184,7 +185,7 @@ void init(ResourceScheduler resourceScheduler, Configuration config) // application running information jobRuntimeLogBW = new BufferedWriter(new OutputStreamWriter(new FileOutputStream( - metricsOutputDir + "/jobruntime.csv"), "UTF-8")); + metricsOutputDir + "/jobruntime.csv"), StandardCharsets.UTF_8)); jobRuntimeLogBW.write("JobID,real_start_time,real_end_time," + "simulate_start_time,simulate_end_time" + EOL); jobRuntimeLogBW.flush(); @@ -560,7 +561,7 @@ class MetricsLogRunnable implements Runnable { try { metricsLogBW = new BufferedWriter(new OutputStreamWriter(new FileOutputStream( - metricsOutputDir + "/realtimetrack.json"), "UTF-8")); + metricsOutputDir + "/realtimetrack.json"), StandardCharsets.UTF_8)); metricsLogBW.write("["); } catch (IOException e) { LOG.info(e.getMessage()); @@ -717,11 +718,10 @@ public void addAMRuntime(ApplicationId appId, long traceStartTimeMS, long traceEndTimeMS, long simulateStartTimeMS, long simulateEndTimeMS) { try { // write job runtime information - StringBuilder sb = new StringBuilder(); - sb.append(appId).append(",").append(traceStartTimeMS).append(",") - .append(traceEndTimeMS).append(",").append(simulateStartTimeMS) - .append(",").append(simulateEndTimeMS); - jobRuntimeLogBW.write(sb.toString() + EOL); + String runtimeInfo = appId + "," + traceStartTimeMS + "," + + traceEndTimeMS + "," + simulateStartTimeMS + + "," + simulateEndTimeMS; + jobRuntimeLogBW.write(runtimeInfo + EOL); jobRuntimeLogBW.flush(); } catch (IOException e) { LOG.info(e.getMessage()); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java index 86a1dcf8b5351..eea0a8b64266f 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthJob.java @@ -184,9 +184,9 @@ protected SynthJob(JDKRandomGenerator rand, Configuration conf, int num = task.count.getInt(); String taskType = task.type; long memory = task.max_memory.getLong(); - memory = memory < MIN_MEMORY ? MIN_MEMORY: memory; + memory = Math.max(memory, MIN_MEMORY); long vcores = task.max_vcores.getLong(); - vcores = vcores < MIN_VCORES ? MIN_VCORES : vcores; + vcores = Math.max(vcores, MIN_VCORES); int priority = task.priority; ExecutionType executionType = task.executionType == null ? ExecutionType.GUARANTEED diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java index 3527d6b7668d8..18b1c034bdf3a 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/synthetic/SynthTraceJobProducer.java @@ -310,7 +310,6 @@ public void init(JDKRandomGenerator random){ // Initialize job weights job_weights = new ArrayList<>(); - job_weights = new ArrayList<>(); for(JobDefinition j : job_classes){ job_weights.add(j.class_weight); } @@ -638,7 +637,7 @@ public String getString(){ public String toString(){ switch(mode){ case CONST: - return "value: " + Double.toString(val); + return "value: " + val; case DIST: return "value: " + this.val + " std: " + this.std + " dist: " + this.dist.name(); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java index e529d1841a671..af0b4f6caf3ab 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/utils/SLSUtils.java @@ -85,7 +85,7 @@ public static Set parseNodesFromRumenTrace( JobTraceReader reader = new JobTraceReader( new Path(fin.getAbsolutePath()), conf); try { - LoggedJob job = null; + LoggedJob job; while ((job = reader.getNext()) != null) { for(LoggedTask mapTask : job.getMapTasks()) { // select the last attempt @@ -123,7 +123,7 @@ public static Set parseNodesFromSLSTrace( JsonFactory jsonF = new JsonFactory(); ObjectMapper mapper = new ObjectMapper(); Reader input = - new InputStreamReader(new FileInputStream(jobTrace), "UTF-8"); + new InputStreamReader(new FileInputStream(jobTrace), StandardCharsets.UTF_8); try { Iterator i = mapper.readValues(jsonF.createParser(input), Map.class); while (i.hasNext()) { @@ -170,7 +170,7 @@ public static Set parseNodesFromNodeFile( JsonFactory jsonF = new JsonFactory(); ObjectMapper mapper = new ObjectMapper(); Reader input = - new InputStreamReader(new FileInputStream(nodeFile), "UTF-8"); + new InputStreamReader(new FileInputStream(nodeFile), StandardCharsets.UTF_8); try { Iterator i = mapper.readValues(jsonF.createParser(input), Map.class); while (i.hasNext()) { diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java index 1bb4710d56d16..513dbe5a436b6 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java +++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/BaseSLSRunnerTest.java @@ -103,7 +103,7 @@ public void uncaughtException(Thread t, Throwable e) { } if (nodeFile != null) { - args = ArrayUtils.addAll(args, new String[] {"-nodes", nodeFile }); + args = ArrayUtils.addAll(args, "-nodes", nodeFile); } // enable continuous invariant checks diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java index 14e74751577cf..dd12a10f94612 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java +++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/TestSynthJobGeneration.java @@ -37,6 +37,7 @@ import static com.fasterxml.jackson.databind.DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; /** @@ -88,7 +89,7 @@ public void testWorkloadGenerateTime() } Assert.assertTrue(bucket0 > 0); - Assert.assertTrue(bucket1 == 0); + assertEquals(0, bucket1); Assert.assertTrue(bucket2 > 0); Assert.assertTrue(bucket3 > 0); Assert.assertTrue(bucket2 > bucket0); @@ -255,7 +256,7 @@ private void validateJob(SynthJob js) { assertTrue(js.getTasks().size() > 0); for (SynthJob.SynthTask t : js.getTasks()) { - assertTrue(t.getType() != null); + assertNotNull(t.getType()); assertTrue(t.getTime() > 0); assertTrue(t.getMemory() > 0); assertTrue(t.getVcores() > 0); diff --git a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java index 423d6b2c7fe84..5376a55cc639b 100644 --- a/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java +++ b/hadoop-tools/hadoop-sls/src/test/java/org/apache/hadoop/yarn/sls/utils/TestSLSUtils.java @@ -36,7 +36,7 @@ public class TestSLSUtils { @Test public void testGetRackHostname() { String str = "/rack1/node1"; - String rackHostname[] = SLSUtils.getRackHostName(str); + String[] rackHostname = SLSUtils.getRackHostName(str); Assert.assertEquals("rack1", rackHostname[0]); Assert.assertEquals("node1", rackHostname[1]); @@ -81,7 +81,7 @@ public void testParseNodesFromNodeFile() throws Exception { } else if(nodeLabel.getName().equals("label2")) { Assert.assertFalse(nodeLabel.isExclusive()); } else { - Assert.assertTrue("Unexepected label", false); + Assert.fail("Unexpected label"); } } } else if (nodeDetail.getHostname().equals("/rack1/node4")) { From 15a5ea2c955a7d1b89aea0cb127727a57db76c76 Mon Sep 17 00:00:00 2001 From: Xing Lin Date: Thu, 31 Mar 2022 14:59:09 -0700 Subject: [PATCH 086/145] HADOOP-18169. getDelegationTokens in ViewFs should also fetch the token from fallback FS (#4094) HADOOP-18169. getDelegationTokens in ViewFs should also fetch the token from the fallback FS --- .../org/apache/hadoop/fs/viewfs/ViewFs.java | 11 ++++++++++ .../fs/viewfs/TestViewFsLinkFallback.java | 22 +++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java index 6d619b1d6779d..6e35ddf19053b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java @@ -746,6 +746,17 @@ public List> getDelegationTokens(String renewer) throws IOException { result.addAll(tokens); } } + + // Add tokens from fallback FS + if (this.fsState.getRootFallbackLink() != null) { + AbstractFileSystem rootFallbackFs = + this.fsState.getRootFallbackLink().getTargetFileSystem(); + List> tokens = rootFallbackFs.getDelegationTokens(renewer); + if (tokens != null) { + result.addAll(tokens); + } + } + return result; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java index 09e02be640e5e..ba4a80ca11549 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsLinkFallback.java @@ -29,6 +29,7 @@ import java.net.URISyntaxException; import java.util.EnumSet; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.AbstractFileSystem; import org.apache.hadoop.fs.FileAlreadyExistsException; @@ -45,6 +46,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.test.LambdaTestUtils; import org.junit.AfterClass; import org.junit.Assert; @@ -182,6 +184,26 @@ public void testMkdirOfNewDirWithOutMatchingToMountOrFallbackDirTree() assertTrue(fsTarget.exists(test)); } + /** + * Test getDelegationToken when fallback is configured. + */ + @Test + public void testGetDelegationToken() throws IOException { + Configuration conf = new Configuration(); + conf.setBoolean(Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS, false); + ConfigUtil.addLink(conf, "/user", + new Path(targetTestRoot.toString(), "user").toUri()); + ConfigUtil.addLink(conf, "/data", + new Path(targetTestRoot.toString(), "data").toUri()); + ConfigUtil.addLinkFallback(conf, targetTestRoot.toUri()); + + FileContext fcView = + FileContext.getFileContext(FsConstants.VIEWFS_URI, conf); + List> tokens = fcView.getDelegationTokens(new Path("/"), "tester"); + // Two tokens from the two mount points and one token from fallback + assertEquals(3, tokens.size()); + } + /** * Tests that when the parent dirs does not exist in fallback but the parent * dir is same as mount internal directory, then we create parent structure From 4b1a6bfb10cacf1b5e070edf09b74ff8a3c680a7 Mon Sep 17 00:00:00 2001 From: 9uapaw Date: Fri, 1 Apr 2022 18:24:37 +0200 Subject: [PATCH 087/145] YARN-11102. Fix spotbugs error in hadoop-sls module. Contributed by Szilard Nemeth, Andras Gyori. --- .../java/org/apache/hadoop/yarn/sls/AMRunner.java | 7 +++++-- .../java/org/apache/hadoop/yarn/sls/NMRunner.java | 2 +- .../java/org/apache/hadoop/yarn/sls/SLSRunner.java | 11 ++++++----- .../yarn/sls/scheduler/SLSCapacityScheduler.java | 12 ++++++++++++ .../hadoop/yarn/sls/scheduler/SLSFairScheduler.java | 13 +++++++++++++ .../yarn/sls/scheduler/SLSSchedulerCommons.java | 5 +++-- .../hadoop/yarn/sls/scheduler/SchedulerWrapper.java | 5 +++++ 7 files changed, 45 insertions(+), 10 deletions(-) diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java index 8ce8d29a01a2c..2dc09de665368 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/AMRunner.java @@ -44,8 +44,11 @@ import java.io.InputStreamReader; import java.io.Reader; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; @@ -53,7 +56,7 @@ public class AMRunner { private static final Logger LOG = LoggerFactory.getLogger(AMRunner.class); - static int remainingApps = 0; + int remainingApps = 0; private final Configuration conf; private int AM_ID; @@ -263,7 +266,7 @@ public void setInputType(TraceType inputType) { } public void setInputTraces(String[] inputTraces) { - this.inputTraces = inputTraces; + this.inputTraces = inputTraces.clone(); } public void setResourceManager(ResourceManager rm) { diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java index 224e1e373ff0b..14524a4e04ac5 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/NMRunner.java @@ -205,7 +205,7 @@ public void setInputType(TraceType inputType) { } public void setInputTraces(String[] inputTraces) { - this.inputTraces = inputTraces; + this.inputTraces = inputTraces.clone(); } public int getNumNMs() { diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java index 0909a45cc4773..c14d5685b7db0 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/SLSRunner.java @@ -98,8 +98,8 @@ public enum TraceType { public static final String NETWORK_NEGATIVE_CACHE_TTL = "networkaddress.cache.negative.ttl"; - public static int getRemainingApps() { - return AMRunner.remainingApps; + public int getRemainingApps() { + return amRunner.remainingApps; } public SLSRunner() throws ClassNotFoundException, YarnException { @@ -204,6 +204,7 @@ public void start() throws IOException, ClassNotFoundException, YarnException, // set queue & tracked apps information SchedulerWrapper resourceScheduler = (SchedulerWrapper) rmRunner.getRm().getResourceScheduler(); + resourceScheduler.setSLSRunner(this); Tracker tracker = resourceScheduler.getTracker(); tracker.setQueueSet(rmRunner.getQueueAppNumMap().keySet()); tracker.setTrackedAppSet(amRunner.getTrackedApps()); @@ -301,9 +302,9 @@ public Map getNmMap() { return nmRunner.getNmMap(); } - public static void decreaseRemainingApps() { - AMRunner.remainingApps--; - if (AMRunner.remainingApps == 0) { + public void decreaseRemainingApps() { + amRunner.remainingApps--; + if (amRunner.remainingApps == 0) { exitSLSRunner(); } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java index 39170b18727af..18e12cca05f11 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSCapacityScheduler.java @@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.sls.SLSRunner; @Private @Unstable @@ -43,6 +44,7 @@ public class SLSCapacityScheduler extends CapacityScheduler implements private final SLSSchedulerCommons schedulerCommons; private Configuration conf; + private SLSRunner runner; public SLSCapacityScheduler() { schedulerCommons = new SLSSchedulerCommons(this); @@ -138,5 +140,15 @@ public Configuration getConf() { public Tracker getTracker() { return schedulerCommons.getTracker(); } + + @Override + public void setSLSRunner(SLSRunner runner) { + this.runner = runner; + } + + @Override + public SLSRunner getSLSRunner() { + return this.runner; + } } diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java index 2835b413cabe0..1b4d5ced69bde 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSFairScheduler.java @@ -30,6 +30,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler; +import org.apache.hadoop.yarn.sls.SLSRunner; + import java.util.List; @Private @@ -37,6 +39,7 @@ public class SLSFairScheduler extends FairScheduler implements SchedulerWrapper, Configurable { private final SLSSchedulerCommons schedulerCommons; + private SLSRunner runner; public SLSFairScheduler() { schedulerCommons = new SLSSchedulerCommons(this); @@ -99,4 +102,14 @@ public SchedulerMetrics getSchedulerMetrics() { public Tracker getTracker() { return schedulerCommons.getTracker(); } + + @Override + public void setSLSRunner(SLSRunner runner) { + this.runner = runner; + } + + @Override + public SLSRunner getSLSRunner() { + return this.runner; + } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java index ee70dfbe3e96c..a99fe53eff91a 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SLSSchedulerCommons.java @@ -205,6 +205,7 @@ private void updateQueueWithAllocateRequest(Allocation allocation, } public void handle(SchedulerEvent schedulerEvent) { + SchedulerWrapper wrapper = (SchedulerWrapper) scheduler; if (!metricsON) { ((SchedulerWrapper)scheduler).propagatedHandle(schedulerEvent); return; @@ -259,11 +260,11 @@ public void handle(SchedulerEvent schedulerEvent) { if (schedulerEvent.getType() == SchedulerEventType.APP_ATTEMPT_REMOVED && schedulerEvent instanceof AppAttemptRemovedSchedulerEvent) { - SLSRunner.decreaseRemainingApps(); + wrapper.getSLSRunner().decreaseRemainingApps(); AppAttemptRemovedSchedulerEvent appRemoveEvent = (AppAttemptRemovedSchedulerEvent) schedulerEvent; appQueueMap.remove(appRemoveEvent.getApplicationAttemptID()); - if (SLSRunner.getRemainingApps() == 0) { + if (wrapper.getSLSRunner().getRemainingApps() == 0) { try { schedulerMetrics.tearDown(); SLSRunner.exitSLSRunner(); diff --git a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java index 5ee088d792a2f..cc9844a8f2af9 100644 --- a/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java +++ b/hadoop-tools/hadoop-sls/src/main/java/org/apache/hadoop/yarn/sls/scheduler/SchedulerWrapper.java @@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ContainerUpdates; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.sls.SLSRunner; import java.util.List; @@ -48,4 +49,8 @@ Allocation allocatePropagated(ApplicationAttemptId attemptId, List blacklistAdditions, List blacklistRemovals, ContainerUpdates updateRequests); + + void setSLSRunner(SLSRunner runner); + + SLSRunner getSLSRunner(); } From 34b3275bf4ba0483b3dfe0e7389b1a6eb59457e7 Mon Sep 17 00:00:00 2001 From: litao Date: Sun, 3 Apr 2022 04:06:03 +0800 Subject: [PATCH 088/145] HDFS-16477. [SPS]: Add metric PendingSPSPaths for getting the number of paths to be processed by SPS (#4009). Contributed by tomscut. Signed-off-by: Ayush Saxena --- .../src/site/markdown/Metrics.md | 1 + .../federation/metrics/FederationMBean.java | 7 +++ .../metrics/NamenodeBeanMetrics.java | 10 +++++ .../server/federation/metrics/RBFMetrics.java | 6 +++ .../resolver/MembershipNamenodeResolver.java | 1 + .../resolver/NamenodeStatusReport.java | 18 ++++++-- .../router/NamenodeHeartbeatService.java | 3 +- .../store/records/MembershipStats.java | 4 ++ .../impl/pb/MembershipStatsPBImpl.java | 10 +++++ .../src/main/proto/FederationProtocol.proto | 1 + .../federation/metrics/TestRBFMetrics.java | 5 +++ .../store/FederationStateStoreTestUtils.java | 1 + .../server/blockmanagement/BlockManager.java | 8 ++++ .../hdfs/server/namenode/FSNamesystem.java | 6 +++ .../namenode/metrics/FSNamesystemMBean.java | 7 +++ .../sps/StoragePolicySatisfyManager.java | 31 +++++++------ .../TestExternalStoragePolicySatisfier.java | 43 +++++++++++++++---- 17 files changed, 137 insertions(+), 25 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md index 190f2a835804c..b0fc525776bbe 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md @@ -299,6 +299,7 @@ Each metrics record contains tags such as HAState and Hostname as additional inf | `FSN(Read/Write)Lock`*OperationName*`NanosAvgTime` | Average time of holding the lock by operations in nanoseconds | | `FSN(Read/Write)LockOverallNanosNumOps` | Total number of acquiring lock by all operations | | `FSN(Read/Write)LockOverallNanosAvgTime` | Average time of holding the lock by all operations in nanoseconds | +| `PendingSPSPaths` | The number of paths to be processed by storage policy satisfier | JournalNode ----------- diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java index b9ea8709e90f9..e8b00d0b5dcfb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java @@ -343,4 +343,11 @@ public interface FederationMBean { * with the highest risk of loss. */ long getHighestPriorityLowRedundancyECBlocks(); + + /** + * Returns the number of paths to be processed by storage policy satisfier. + * + * @return The number of paths to be processed by sps. + */ + int getPendingSPSPaths(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java index c48728a923c0d..0c62922146311 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java @@ -874,6 +874,16 @@ public long getCurrentTokensCount() { return 0; } + @Override + public int getPendingSPSPaths() { + try { + return getRBFMetrics().getPendingSPSPaths(); + } catch (IOException e) { + LOG.debug("Failed to get number of paths to be processed by sps", e); + } + return 0; + } + private Router getRouter() throws IOException { if (this.router == null) { throw new IOException("Router is not initialized"); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java index 752fbfc628d23..d5eabd1a3da82 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java @@ -746,6 +746,12 @@ public long getHighestPriorityLowRedundancyECBlocks() { MembershipStats::getHighestPriorityLowRedundancyECBlocks); } + @Override + public int getPendingSPSPaths() { + return getNameserviceAggregatedInt( + MembershipStats::getPendingSPSPaths); + } + @Override @Metric({"RouterFederationRenameCount", "Number of federation rename"}) public int getRouterFederationRenameCount() { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java index 13593e694a80e..9f0f78067aedd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MembershipNamenodeResolver.java @@ -306,6 +306,7 @@ public boolean registerNamenode(NamenodeStatusReport report) report.getHighestPriorityLowRedundancyReplicatedBlocks()); stats.setHighestPriorityLowRedundancyECBlocks( report.getHighestPriorityLowRedundancyECBlocks()); + stats.setPendingSPSPaths(report.getPendingSPSPaths()); record.setStats(stats); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java index feb5a86dba83b..d7da11e6420ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/NamenodeStatusReport.java @@ -75,6 +75,7 @@ public class NamenodeStatusReport { private long numberOfMissingBlocksWithReplicationFactorOne = -1; private long highestPriorityLowRedundancyReplicatedBlocks = -1; private long highestPriorityLowRedundancyECBlocks = -1; + private int pendingSPSPaths = -1; /** If the fields are valid. */ private boolean registrationValid = false; @@ -367,12 +368,13 @@ public int getNumEnteringMaintenanceDataNodes() { * @param numBlocksPendingReplication Number of blocks pending replication. * @param numBlocksUnderReplicated Number of blocks under replication. * @param numBlocksPendingDeletion Number of blocks pending deletion. - * @param providedSpace Space in provided storage. + * @param providedStorageSpace Space in provided storage. + * @param numPendingSPSPaths The number of paths to be processed by storage policy satisfier. */ public void setNamesystemInfo(long available, long total, long numFiles, long numBlocks, long numBlocksMissing, long numBlocksPendingReplication, long numBlocksUnderReplicated, - long numBlocksPendingDeletion, long providedSpace) { + long numBlocksPendingDeletion, long providedStorageSpace, int numPendingSPSPaths) { this.totalSpace = total; this.availableSpace = available; this.numOfBlocks = numBlocks; @@ -382,7 +384,8 @@ public void setNamesystemInfo(long available, long total, this.numOfBlocksPendingDeletion = numBlocksPendingDeletion; this.numOfFiles = numFiles; this.statsValid = true; - this.providedSpace = providedSpace; + this.providedSpace = providedStorageSpace; + this.pendingSPSPaths = numPendingSPSPaths; } /** @@ -460,6 +463,15 @@ public long getHighestPriorityLowRedundancyECBlocks() { return this.highestPriorityLowRedundancyECBlocks; } + /** + * Returns the number of paths to be processed by storage policy satisfier. + * + * @return The number of paths to be processed by sps. + */ + public int getPendingSPSPaths() { + return this.pendingSPSPaths; + } + /** * Get the number of blocks. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java index e1c8a6846fcac..ad9d5e2c2a72c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/NamenodeHeartbeatService.java @@ -478,7 +478,8 @@ private void getFsNamesystemMetrics(String address, jsonObject.getLong("PendingReplicationBlocks"), jsonObject.getLong("UnderReplicatedBlocks"), jsonObject.getLong("PendingDeletionBlocks"), - jsonObject.optLong("ProvidedCapacityTotal")); + jsonObject.optLong("ProvidedCapacityTotal"), + jsonObject.getInt("PendingSPSPaths")); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java index 21c8c2f79fce4..3e05a12cd9b9a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MembershipStats.java @@ -133,6 +133,10 @@ public abstract void setHighestPriorityLowRedundancyECBlocks( public abstract long getHighestPriorityLowRedundancyECBlocks(); + public abstract void setPendingSPSPaths(int pendingSPSPaths); + + public abstract int getPendingSPSPaths(); + @Override public SortedMap getPrimaryKeys() { // This record is not stored directly, no key needed diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java index 2caa59dfca7e5..9dff84befa4ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/impl/pb/MembershipStatsPBImpl.java @@ -297,4 +297,14 @@ public long getHighestPriorityLowRedundancyECBlocks() { return this.translator.getProtoOrBuilder() .getHighestPriorityLowRedundancyECBlocks(); } + + @Override + public void setPendingSPSPaths(int pendingSPSPaths) { + this.translator.getBuilder().setPendingSPSPaths(pendingSPSPaths); + } + + @Override + public int getPendingSPSPaths() { + return this.translator.getProtoOrBuilder().getPendingSPSPaths(); + } } \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto index 4a83ebc6ca267..336130e419a3a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/proto/FederationProtocol.proto @@ -54,6 +54,7 @@ message NamenodeMembershipStatsRecordProto { optional uint64 numberOfMissingBlocksWithReplicationFactorOne = 31; optional uint64 highestPriorityLowRedundancyReplicatedBlocks = 32; optional uint64 HighestPriorityLowRedundancyECBlocks = 33; + optional uint32 pendingSPSPaths = 34; } message NamenodeMembershipRecordProto { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestRBFMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestRBFMetrics.java index 25473f8df9233..c86397b511de6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestRBFMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/metrics/TestRBFMetrics.java @@ -219,6 +219,8 @@ public void testNameserviceStatsDataSource() json.getLong("numOfEnteringMaintenanceDataNodes")); assertEquals(stats.getProvidedSpace(), json.getLong("providedSpace")); + assertEquals(stats.getPendingSPSPaths(), + json.getInt("pendingSPSPaths")); nameservicesFound++; } assertEquals(getNameservices().size(), nameservicesFound); @@ -296,6 +298,7 @@ private void validateClusterStatsFederationBean(FederationMBean bean) { long highestPriorityLowRedundancyReplicatedBlocks = 0; long highestPriorityLowRedundancyECBlocks = 0; long numFiles = 0; + int pendingSPSPaths = 0; for (MembershipState mock : getActiveMemberships()) { MembershipStats stats = mock.getStats(); numBlocks += stats.getNumOfBlocks(); @@ -316,6 +319,7 @@ private void validateClusterStatsFederationBean(FederationMBean bean) { stats.getHighestPriorityLowRedundancyReplicatedBlocks(); highestPriorityLowRedundancyECBlocks += stats.getHighestPriorityLowRedundancyECBlocks(); + pendingSPSPaths += stats.getPendingSPSPaths(); } assertEquals(numBlocks, bean.getNumBlocks()); @@ -342,6 +346,7 @@ private void validateClusterStatsFederationBean(FederationMBean bean) { bean.getHighestPriorityLowRedundancyReplicatedBlocks()); assertEquals(highestPriorityLowRedundancyECBlocks, bean.getHighestPriorityLowRedundancyECBlocks()); + assertEquals(pendingSPSPaths, bean.getPendingSPSPaths()); } private void validateClusterStatsRouterBean(RouterMBean bean) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java index 0fad76de050bd..50840460a3943 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/FederationStateStoreTestUtils.java @@ -269,6 +269,7 @@ public static MembershipState createMockRegistrationForNamenode( stats.setNumOfDecomActiveDatanodes(15); stats.setNumOfDecomDeadDatanodes(5); stats.setNumOfBlocks(10); + stats.setPendingSPSPaths(10); entry.setStats(stats); return entry; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 7b666f34327b1..bfa8457dd4e31 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -298,6 +298,14 @@ public long getTotalECBlockGroups() { return blocksMap.getECBlockGroups(); } + /** Used by metrics. */ + public int getPendingSPSPaths() { + if (spsManager != null) { + return spsManager.getPendingSPSPaths(); + } + return 0; + } + /** * redundancyRecheckInterval is how often namenode checks for new * reconstruction work. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 389bd6455c92f..f080a101f679d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -4875,6 +4875,12 @@ public long getCurrentTokensCount() { dtSecretManager.getCurrentTokensSize() : -1; } + @Override + @Metric({"PendingSPSPaths", "The number of paths to be processed by storage policy satisfier"}) + public int getPendingSPSPaths() { + return blockManager.getPendingSPSPaths(); + } + /** * Returns the length of the wait Queue for the FSNameSystemLock. * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java index 7e5f108167ccc..59bb01f0063cf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/metrics/FSNamesystemMBean.java @@ -254,4 +254,11 @@ public interface FSNamesystemMBean { * @return number of DTs */ long getCurrentTokensCount(); + + /** + * Returns the number of paths to be processed by storage policy satisfier. + * + * @return The number of paths to be processed by sps. + */ + int getPendingSPSPaths(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java index 40e3faa5550e5..2c7f36a690bd1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java @@ -60,7 +60,7 @@ public class StoragePolicySatisfyManager { private final StoragePolicySatisfier spsService; private final boolean storagePolicyEnabled; private volatile StoragePolicySatisfierMode mode; - private final Queue pathsToBeTraveresed; + private final Queue pathsToBeTraversed; private final int outstandingPathsLimit; private final Namesystem namesystem; @@ -77,7 +77,7 @@ public StoragePolicySatisfyManager(Configuration conf, DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY, DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_DEFAULT); mode = StoragePolicySatisfierMode.fromString(modeVal); - pathsToBeTraveresed = new LinkedList(); + pathsToBeTraversed = new LinkedList(); this.namesystem = namesystem; // instantiate SPS service by just keeps config reference and not starting // any supporting threads. @@ -218,8 +218,8 @@ public boolean isSatisfierRunning() { * storages. */ public Long getNextPathId() { - synchronized (pathsToBeTraveresed) { - return pathsToBeTraveresed.poll(); + synchronized (pathsToBeTraversed) { + return pathsToBeTraversed.poll(); } } @@ -228,7 +228,7 @@ public Long getNextPathId() { * @throws IOException */ public void verifyOutstandingPathQLimit() throws IOException { - long size = pathsToBeTraveresed.size(); + long size = pathsToBeTraversed.size(); // Checking that the SPS call Q exceeds the allowed limit. if (outstandingPathsLimit - size <= 0) { LOG.debug("Satisifer Q - outstanding limit:{}, current size:{}", @@ -244,15 +244,15 @@ public void verifyOutstandingPathQLimit() throws IOException { * @throws IOException */ private void clearPathIds(){ - synchronized (pathsToBeTraveresed) { - Iterator iterator = pathsToBeTraveresed.iterator(); + synchronized (pathsToBeTraversed) { + Iterator iterator = pathsToBeTraversed.iterator(); while (iterator.hasNext()) { Long trackId = iterator.next(); try { namesystem.removeXattr(trackId, HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY); } catch (IOException e) { - LOG.debug("Failed to remove sps xatttr!", e); + LOG.debug("Failed to remove sps xattr!", e); } iterator.remove(); } @@ -263,8 +263,8 @@ private void clearPathIds(){ * Clean up all sps path ids. */ public void removeAllPathIds() { - synchronized (pathsToBeTraveresed) { - pathsToBeTraveresed.clear(); + synchronized (pathsToBeTraversed) { + pathsToBeTraversed.clear(); } } @@ -273,8 +273,8 @@ public void removeAllPathIds() { * @param id */ public void addPathId(long id) { - synchronized (pathsToBeTraveresed) { - pathsToBeTraveresed.add(id); + synchronized (pathsToBeTraversed) { + pathsToBeTraversed.add(id); } } @@ -292,4 +292,11 @@ public boolean isEnabled() { public StoragePolicySatisfierMode getMode() { return mode; } + + /** + * @return the number of paths to be processed by storage policy satisfier. + */ + public int getPendingSPSPaths() { + return pathsToBeTraversed.size(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java index 361d61d54e258..4c75cb3f294a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java @@ -36,6 +36,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; @@ -202,7 +203,15 @@ private void createCluster() throws IOException { private void createCluster(boolean createMoverPath) throws IOException { getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE); setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES, - STORAGES_PER_DATANODE, CAPACITY, createMoverPath)); + STORAGES_PER_DATANODE, CAPACITY, createMoverPath, true)); + getFS(); + writeContent(FILE); + } + + private void createClusterDoNotStartSPS() throws IOException { + getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE); + setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES, + STORAGES_PER_DATANODE, CAPACITY, true, false)); getFS(); writeContent(FILE); } @@ -211,12 +220,12 @@ private MiniDFSCluster startCluster(final Configuration conf, StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn, long nodeCapacity) throws IOException { return startCluster(conf, storageTypes, numberOfDatanodes, storagesPerDn, - nodeCapacity, false); + nodeCapacity, false, true); } private MiniDFSCluster startCluster(final Configuration conf, StorageType[][] storageTypes, int numberOfDatanodes, int storagesPerDn, - long nodeCapacity, boolean createMoverPath) throws IOException { + long nodeCapacity, boolean createMoverPath, boolean startSPS) throws IOException { long[][] capacities = new long[numberOfDatanodes][storagesPerDn]; for (int i = 0; i < numberOfDatanodes; i++) { for (int j = 0; j < storagesPerDn; j++) { @@ -228,14 +237,16 @@ private MiniDFSCluster startCluster(final Configuration conf, .storageTypes(storageTypes).storageCapacities(capacities).build(); cluster.waitActive(); - nnc = DFSTestUtil.getNameNodeConnector(getConf(), - HdfsServerConstants.MOVER_ID_PATH, 1, createMoverPath); + if (startSPS) { + nnc = DFSTestUtil.getNameNodeConnector(getConf(), + HdfsServerConstants.MOVER_ID_PATH, 1, createMoverPath); - externalSps = new StoragePolicySatisfier(getConf()); - externalCtxt = new ExternalSPSContext(externalSps, nnc); + externalSps = new StoragePolicySatisfier(getConf()); + externalCtxt = new ExternalSPSContext(externalSps, nnc); - externalSps.init(externalCtxt); - externalSps.start(StoragePolicySatisfierMode.EXTERNAL); + externalSps.init(externalCtxt); + externalSps.start(StoragePolicySatisfierMode.EXTERNAL); + } return cluster; } @@ -1515,6 +1526,20 @@ public void testMoveBlocksWithUnderReplicatedBlocks() throws Exception { } } + @Test(timeout = 300000) + public void testExternalSPSMetrics() + throws Exception { + + try { + createClusterDoNotStartSPS(); + dfs.satisfyStoragePolicy(new Path(FILE)); + // Assert metrics. + assertEquals(1, hdfsCluster.getNamesystem().getPendingSPSPaths()); + } finally { + shutdownCluster(); + } + } + private static void createDirectoryTree(DistributedFileSystem dfs) throws Exception { // tree structure From 4ef1d3eef9340b59f132421fba443dc456671ee4 Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Tue, 5 Apr 2022 22:59:11 +0530 Subject: [PATCH 089/145] HDFS-16472. Make HDFS setrep tool cross platform (#4130) * The source files for hdfs_setrep uses getopt for parsing the command line arguments. * getopt is available only on Linux and thus, isn't cross platform. * We need to replace getopt with boost::program_options to make this tool cross platform. --- .../libhdfspp/tests/tools/CMakeLists.txt | 3 + .../libhdfspp/tests/tools/hdfs-setrep-mock.cc | 56 +++++ .../libhdfspp/tests/tools/hdfs-setrep-mock.h | 68 ++++++ .../libhdfspp/tests/tools/hdfs-tool-tests.cc | 24 +- .../native/libhdfspp/tools/CMakeLists.txt | 3 +- .../tools/hdfs-setrep/CMakeLists.txt | 27 +++ .../tools/hdfs-setrep/hdfs-setrep.cc | 220 ++++++++++++++++++ .../libhdfspp/tools/hdfs-setrep/hdfs-setrep.h | 96 ++++++++ .../libhdfspp/tools/hdfs-setrep/main.cc | 52 +++++ .../native/libhdfspp/tools/hdfs_setrep.cc | 172 -------------- .../tools/internal/set-replication-state.h | 72 ++++++ 11 files changed, 617 insertions(+), 176 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/CMakeLists.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/main.cc delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_setrep.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/internal/set-replication-state.h diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt index 75c5ad1ff9816..bec9343903923 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt @@ -38,6 +38,7 @@ add_executable(hdfs_tool_tests hdfs-get-mock.cc hdfs-find-mock.cc hdfs-ls-mock.cc + hdfs-setrep-mock.cc main.cc) target_include_directories(hdfs_tool_tests PRIVATE ../tools @@ -60,6 +61,7 @@ target_include_directories(hdfs_tool_tests PRIVATE ../../tools/hdfs-get ../../tools/hdfs-find ../../tools/hdfs-ls + ../../tools/hdfs-setrep ../../tools/hdfs-cat) target_link_libraries(hdfs_tool_tests PRIVATE gmock_main @@ -81,5 +83,6 @@ target_link_libraries(hdfs_tool_tests PRIVATE hdfs_get_lib hdfs_find_lib hdfs_ls_lib + hdfs_setrep_lib hdfs_cat_lib) add_test(hdfs_tool_tests hdfs_tool_tests) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.cc new file mode 100644 index 0000000000000..d33f49b6aec67 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.cc @@ -0,0 +1,56 @@ +/** + * 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. + */ + +#include +#include +#include +#include + +#include +#include + +#include "hdfs-setrep-mock.h" +#include "hdfs-tool-tests.h" + +namespace hdfs::tools::test { +SetrepMock::~SetrepMock() = default; + +void SetrepMock::SetExpectations( + std::function()> test_case, + const std::vector &args) const { + // Get the pointer to the function that defines the test case + const auto test_case_func = + test_case.target (*)()>(); + ASSERT_NE(test_case_func, nullptr); + + // Set the expected method calls and their corresponding arguments for each + // test case + if (*test_case_func == &CallHelp) { + EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true)); + return; + } + + if (*test_case_func == &PassPermissionsAndAPath) { + const auto number = args[0]; + const auto path = args[1]; + EXPECT_CALL(*this, HandlePath(path, number)) + .Times(1) + .WillOnce(testing::Return(true)); + } +} +} // namespace hdfs::tools::test diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.h new file mode 100644 index 0000000000000..db1e0960ae0a7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-setrep-mock.h @@ -0,0 +1,68 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_SETREP_MOCK +#define LIBHDFSPP_TOOLS_HDFS_SETREP_MOCK + +#include +#include +#include +#include + +#include + +#include "hdfs-setrep.h" + +namespace hdfs::tools::test { +/** + * {@class SetrepMock} is an {@class Setrep} whereby it mocks the + * HandleHelp and HandlePath methods for testing their functionality. + */ +class SetrepMock : public hdfs::tools::Setrep { +public: + /** + * {@inheritdoc} + */ + SetrepMock(const int argc, char **argv) : Setrep(argc, argv) {} + + // Abiding to the Rule of 5 + SetrepMock(const SetrepMock &) = delete; + SetrepMock(SetrepMock &&) = delete; + SetrepMock &operator=(const SetrepMock &) = delete; + SetrepMock &operator=(SetrepMock &&) = delete; + ~SetrepMock() override; + + /** + * Defines the methods and the corresponding arguments that are expected + * to be called on this instance of {@link HdfsTool} for the given test case. + * + * @param test_case An {@link std::function} object that points to the + * function defining the test case + * @param args The arguments that are passed to this test case + */ + void SetExpectations(std::function()> test_case, + const std::vector &args = {}) const; + + MOCK_METHOD(bool, HandleHelp, (), (const, override)); + + MOCK_METHOD(bool, HandlePath, (const std::string &, const std::string &), + (const, override)); +}; +} // namespace hdfs::tools::test + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc index 97169293f495b..1bdf82f9afd3b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc @@ -38,6 +38,7 @@ #include "hdfs-move-to-local-mock.h" #include "hdfs-rename-snapshot-mock.h" #include "hdfs-rm-mock.h" +#include "hdfs-setrep-mock.h" #include "hdfs-tool-test-fixtures.h" #include "hdfs-tool-tests.h" @@ -156,6 +157,11 @@ INSTANTIATE_TEST_SUITE_P( PassMOptPermissionsAndAPath, PassNOptAndAPath)); +INSTANTIATE_TEST_SUITE_P( + HdfsSetrep, HdfsToolBasicTest, + testing::Values(CallHelp, + PassPermissionsAndAPath)); + // Negative tests INSTANTIATE_TEST_SUITE_P( HdfsAllowSnapshot, HdfsToolNegativeTestThrows, @@ -245,6 +251,20 @@ INSTANTIATE_TEST_SUITE_P( PassMOpt, PassNOpt)); +INSTANTIATE_TEST_SUITE_P( + HdfsChgrp, HdfsToolNegativeTestThrows, + testing::Values(PassNOptAndAPath)); + +INSTANTIATE_TEST_SUITE_P( + HdfsSetrep, HdfsToolNegativeTestThrows, + testing::Values( + Pass3Paths, + PassRecursiveOwnerAndAPath, + PassRecursive, + PassMPOptsPermissionsAndAPath, + PassMOpt, + PassNOpt)); + INSTANTIATE_TEST_SUITE_P( HdfsRm, HdfsToolNegativeTestNoThrow, testing::Values(PassRecursive)); @@ -302,5 +322,5 @@ INSTANTIATE_TEST_SUITE_P( testing::Values(PassAPath)); INSTANTIATE_TEST_SUITE_P( - HdfsChgrp, HdfsToolNegativeTestThrows, - testing::Values(PassNOptAndAPath)); + HdfsSetrep, HdfsToolNegativeTestNoThrow, + testing::Values(PassAPath)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt index 7cbbe49b5580f..0d9a684c8caa5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt @@ -64,8 +64,7 @@ add_subdirectory(hdfs-copy-to-local) add_subdirectory(hdfs-move-to-local) -add_executable(hdfs_setrep hdfs_setrep.cc) -target_link_libraries(hdfs_setrep tools_common hdfspp_static) +add_subdirectory(hdfs-setrep) add_subdirectory(hdfs-allow-snapshot) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/CMakeLists.txt new file mode 100644 index 0000000000000..a0d8bafa63080 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/CMakeLists.txt @@ -0,0 +1,27 @@ +# +# 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. +# + +add_library(hdfs_setrep_lib STATIC $ hdfs-setrep.cc) +target_include_directories(hdfs_setrep_lib PRIVATE ../../tools ${Boost_INCLUDE_DIRS}) +target_link_libraries(hdfs_setrep_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static) + +add_executable(hdfs_setrep main.cc) +target_include_directories(hdfs_setrep PRIVATE ../../tools) +target_link_libraries(hdfs_setrep PRIVATE hdfs_setrep_lib) + +install(TARGETS hdfs_setrep RUNTIME DESTINATION bin) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.cc new file mode 100644 index 0000000000000..542659b29f141 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.cc @@ -0,0 +1,220 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include + +#include "hdfs-setrep.h" +#include "internal/set-replication-state.h" +#include "tools_common.h" + +namespace hdfs::tools { +Setrep::Setrep(const int argc, char **argv) : HdfsTool(argc, argv) {} + +bool Setrep::Initialize() { + auto add_options = opt_desc_.add_options(); + add_options("help,h", + "Changes the replication factor of a file at PATH. If PATH is a " + "directory then the command recursively changes the replication " + "factor of all files under the directory tree rooted at PATH."); + add_options( + "replication-factor", po::value(), + "The replication factor to set for the given path and its children."); + add_options("path", po::value(), + "The path for which the replication factor needs to be set."); + + // We allow only one positional argument to be passed to this tool. An + // exception is thrown if multiple arguments are passed. + pos_opt_desc_.add("replication-factor", 1); + pos_opt_desc_.add("path", 1); + + po::store(po::command_line_parser(argc_, argv_) + .options(opt_desc_) + .positional(pos_opt_desc_) + .run(), + opt_val_); + po::notify(opt_val_); + return true; +} + +bool Setrep::ValidateConstraints() const { + // Only "help" is allowed as single argument. + if (argc_ == 2) { + return opt_val_.count("help"); + } + + // Rest of the cases must contain more than 2 arguments on the command line. + return argc_ > 2; +} + +std::string Setrep::GetDescription() const { + std::stringstream desc; + desc << "Usage: hdfs_setrep [OPTION] NUM_REPLICAS PATH" << std::endl + << std::endl + << "Changes the replication factor of a file at PATH. If PATH is a " + "directory then the command" + << std::endl + << "recursively changes the replication factor of all files under the " + "directory tree rooted at PATH." + << std::endl + << std::endl + << " -h display this help and exit" << std::endl + << std::endl + << "Examples:" << std::endl + << "hdfs_setrep 5 hdfs://localhost.localdomain:8020/dir/file" + << std::endl + << "hdfs_setrep 3 /dir1/dir2" << std::endl; + return desc.str(); +} + +bool Setrep::Do() { + if (!Initialize()) { + std::cerr << "Unable to initialize HDFS setrep tool" << std::endl; + return false; + } + + if (!ValidateConstraints()) { + std::cout << GetDescription(); + return false; + } + + if (opt_val_.count("help") > 0) { + return HandleHelp(); + } + + if (opt_val_.count("path") > 0 && opt_val_.count("replication-factor") > 0) { + const auto replication_factor = + opt_val_["replication-factor"].as(); + const auto path = opt_val_["path"].as(); + return HandlePath(path, replication_factor); + } + + return false; +} + +bool Setrep::HandleHelp() const { + std::cout << GetDescription(); + return true; +} + +bool Setrep::HandlePath(const std::string &path, + const std::string &replication_factor) const { + // Building a URI object from the given path. + auto uri = hdfs::parse_path_or_exit(path); + + const auto fs = hdfs::doConnect(uri, true); + if (!fs) { + std::cerr << "Could not connect to the file system." << std::endl; + return false; + } + + /* + * Wrap async FileSystem::SetReplication with promise to make it a blocking + * call. + */ + auto promise = std::make_shared>(); + std::future future(promise->get_future()); + auto handler = [promise](const hdfs::Status &s) { promise->set_value(s); }; + + const auto replication = static_cast( + std::strtol(replication_factor.c_str(), nullptr, 8)); + /* + * Allocating shared state, which includes: + * replication to be set, handler to be called, request counter, and a boolean + * to keep track if find is done + */ + auto state = + std::make_shared(replication, handler, 0, false); + + /* + * Keep requesting more from Find until we process the entire listing. Call + * handler when Find is done and request counter is 0. Find guarantees that + * the handler will only be called once at a time so we do not need locking in + * handler_find. + */ + auto handler_find = [fs, state](const hdfs::Status &status_find, + const std::vector &stat_infos, + const bool has_more_results) -> bool { + /* + * For each result returned by Find we call async SetReplication with the + * handler below. SetReplication DOES NOT guarantee that the handler will + * only be called once at a time, so we DO need locking in + * handler_set_replication. + */ + auto handler_set_replication = + [state](const hdfs::Status &status_set_replication) { + std::lock_guard guard(state->lock); + + // Decrement the counter once since we are done with this async call. + if (!status_set_replication.ok() && state->status.ok()) { + // We make sure we set state->status only on the first error. + state->status = status_set_replication; + } + state->request_counter--; + if (state->request_counter == 0 && state->find_is_done) { + state->handler(state->status); // Exit. + } + }; + if (!stat_infos.empty() && state->status.ok()) { + for (hdfs::StatInfo const &stat_info : stat_infos) { + // Launch an asynchronous call to SetReplication for every returned + // file. + if (stat_info.file_type == hdfs::StatInfo::IS_FILE) { + state->request_counter++; + fs->SetReplication(stat_info.full_path, state->replication, + handler_set_replication); + } + } + } + + /* + * Lock this section because handlerSetReplication might be accessing the + * same shared variables simultaneously. + */ + std::lock_guard guard(state->lock); + if (!status_find.ok() && state->status.ok()) { + // We make sure we set state->status only on the first error. + state->status = status_find; + } + if (!has_more_results) { + state->find_is_done = true; + if (state->request_counter == 0) { + state->handler(state->status); // Exit. + } + return false; + } + return true; + }; + + // Asynchronous call to Find. + fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), + handler_find); + + // Block until promise is set. + const auto status = future.get(); + if (!status.ok()) { + std::cerr << "Error: " << status.ToString() << std::endl; + return false; + } + return true; +} +} // namespace hdfs::tools diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.h new file mode 100644 index 0000000000000..20ee7405b6de9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/hdfs-setrep.h @@ -0,0 +1,96 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_SETREP +#define LIBHDFSPP_TOOLS_HDFS_SETREP + +#include + +#include + +#include "hdfs-tool.h" + +namespace hdfs::tools { +/** + * {@class Setrep} is an {@class HdfsTool} that changes the replication factor + * of a file at a given path. If the path is a directory, then it recursively + * changes the replication factor of all files under the directory tree rooted + * at the given path. + */ +class Setrep : public HdfsTool { +public: + /** + * {@inheritdoc} + */ + Setrep(int argc, char **argv); + + // Abiding to the Rule of 5 + Setrep(const Setrep &) = default; + Setrep(Setrep &&) = default; + Setrep &operator=(const Setrep &) = delete; + Setrep &operator=(Setrep &&) = delete; + ~Setrep() override = default; + + /** + * {@inheritdoc} + */ + [[nodiscard]] std::string GetDescription() const override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Do() override; + +protected: + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Initialize() override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool ValidateConstraints() const override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool HandleHelp() const override; + + /** + * Handle the path argument that's passed to this tool. + * + * @param path The path to the directory for which we need setrep info. + * @param replication_factor The replication factor to set to given path and + * its children. + * + * @return A boolean indicating the result of this operation. + */ + [[nodiscard]] virtual bool + HandlePath(const std::string &path, + const std::string &replication_factor) const; + +private: + /** + * A boost data-structure containing the description of positional arguments + * passed to the command-line. + */ + po::positional_options_description pos_opt_desc_; +}; +} // namespace hdfs::tools +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/main.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/main.cc new file mode 100644 index 0000000000000..a3d8399c575ab --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-setrep/main.cc @@ -0,0 +1,52 @@ +/** + * 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. + */ + +#include +#include +#include + +#include + +#include "hdfs-setrep.h" + +int main(int argc, char *argv[]) { + const auto result = std::atexit([]() -> void { + // Clean up static data on exit and prevent valgrind memory leaks + google::protobuf::ShutdownProtobufLibrary(); + }); + if (result != 0) { + std::cerr << "Error: Unable to schedule clean-up tasks for HDFS setrep " + "tool, exiting" + << std::endl; + std::exit(EXIT_FAILURE); + } + + hdfs::tools::Setrep setrep(argc, argv); + auto success = false; + + try { + success = setrep.Do(); + } catch (const std::exception &e) { + std::cerr << "Error: " << e.what() << std::endl; + } + + if (!success) { + std::exit(EXIT_FAILURE); + } + return 0; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_setrep.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_setrep.cc deleted file mode 100644 index 019e24d63fea0..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_setrep.cc +++ /dev/null @@ -1,172 +0,0 @@ -/* - 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. -*/ - -#include -#include -#include -#include "tools_common.h" - -void usage(){ - std::cout << "Usage: hdfs_setrep [OPTION] NUM_REPLICAS PATH" - << std::endl - << std::endl << "Changes the replication factor of a file at PATH. If PATH is a directory then the command" - << std::endl << "recursively changes the replication factor of all files under the directory tree rooted at PATH." - << std::endl - << std::endl << " -h display this help and exit" - << std::endl - << std::endl << "Examples:" - << std::endl << "hdfs_setrep 5 hdfs://localhost.localdomain:8020/dir/file" - << std::endl << "hdfs_setrep 3 /dir1/dir2" - << std::endl; -} - -struct SetReplicationState { - const uint16_t replication; - const std::function handler; - //The request counter is incremented once every time SetReplication async call is made - uint64_t request_counter; - //This boolean will be set when find returns the last result - bool find_is_done; - //Final status to be returned - hdfs::Status status; - //Shared variables will need protection with a lock - std::mutex lock; - SetReplicationState(const uint16_t replication_, const std::function & handler_, - uint64_t request_counter_, bool find_is_done_) - : replication(replication_), - handler(handler_), - request_counter(request_counter_), - find_is_done(find_is_done_), - status(), - lock() { - } -}; - -int main(int argc, char *argv[]) { - //We should have 3 or 4 parameters - if (argc < 3) { - usage(); - exit(EXIT_FAILURE); - } - - int input; - - //Using GetOpt to read in the values - opterr = 0; - while ((input = getopt(argc, argv, "h")) != -1) { - switch (input) - { - case 'h': - usage(); - exit(EXIT_SUCCESS); - case '?': - if (isprint(optopt)) - std::cerr << "Unknown option `-" << (char) optopt << "'." << std::endl; - else - std::cerr << "Unknown option character `" << (char) optopt << "'." << std::endl; - usage(); - exit(EXIT_FAILURE); - default: - exit(EXIT_FAILURE); - } - } - std::string repl = argv[optind]; - std::string uri_path = argv[optind + 1]; - - //Building a URI object from the given uri_path - hdfs::URI uri = hdfs::parse_path_or_exit(uri_path); - - std::shared_ptr fs = hdfs::doConnect(uri, true); - if (!fs) { - std::cerr << "Could not connect the file system. " << std::endl; - exit(EXIT_FAILURE); - } - - /* wrap async FileSystem::SetReplication with promise to make it a blocking call */ - std::shared_ptr> promise = std::make_shared>(); - std::future future(promise->get_future()); - auto handler = [promise](const hdfs::Status &s) { - promise->set_value(s); - }; - - uint16_t replication = std::stoi(repl.c_str(), NULL, 8); - //Allocating shared state, which includes: - //replication to be set, handler to be called, request counter, and a boolean to keep track if find is done - std::shared_ptr state = std::make_shared(replication, handler, 0, false); - - // Keep requesting more from Find until we process the entire listing. Call handler when Find is done and reques counter is 0. - // Find guarantees that the handler will only be called once at a time so we do not need locking in handlerFind. - auto handlerFind = [fs, state](const hdfs::Status &status_find, const std::vector & stat_infos, bool has_more_results) -> bool { - - //For each result returned by Find we call async SetReplication with the handler below. - //SetReplication DOES NOT guarantee that the handler will only be called once at a time, so we DO need locking in handlerSetReplication. - auto handlerSetReplication = [state](const hdfs::Status &status_set_replication) { - std::lock_guard guard(state->lock); - - //Decrement the counter once since we are done with this async call - if (!status_set_replication.ok() && state->status.ok()){ - //We make sure we set state->status only on the first error. - state->status = status_set_replication; - } - state->request_counter--; - if(state->request_counter == 0 && state->find_is_done){ - state->handler(state->status); //exit - } - }; - if(!stat_infos.empty() && state->status.ok()) { - for (hdfs::StatInfo const& s : stat_infos) { - //Launch an asynchronous call to SetReplication for every returned file - if(s.file_type == hdfs::StatInfo::IS_FILE){ - state->request_counter++; - fs->SetReplication(s.full_path, state->replication, handlerSetReplication); - } - } - } - - //Lock this section because handlerSetReplication might be accessing the same - //shared variables simultaneously - std::lock_guard guard(state->lock); - if (!status_find.ok() && state->status.ok()){ - //We make sure we set state->status only on the first error. - state->status = status_find; - } - if(!has_more_results){ - state->find_is_done = true; - if(state->request_counter == 0){ - state->handler(state->status); //exit - } - return false; - } - return true; - }; - - //Asynchronous call to Find - fs->Find(uri.get_path(), "*", hdfs::FileSystem::GetDefaultFindMaxDepth(), handlerFind); - - /* block until promise is set */ - hdfs::Status status = future.get(); - if (!status.ok()) { - std::cerr << "Error: " << status.ToString() << std::endl; - exit(EXIT_FAILURE); - } - - // Clean up static data and prevent valgrind memory leaks - google::protobuf::ShutdownProtobufLibrary(); - return 0; -} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/internal/set-replication-state.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/internal/set-replication-state.h new file mode 100644 index 0000000000000..5d432eddbf711 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/internal/set-replication-state.h @@ -0,0 +1,72 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_SET_REPLICATION_STATE +#define LIBHDFSPP_TOOLS_HDFS_SET_REPLICATION_STATE + +#include +#include + +#include "hdfspp/hdfspp.h" + +namespace hdfs::tools { +/** + * {@class SetReplicationState} helps in handling the intermediate results while + * running {@link Setrep}. + */ +struct SetReplicationState { + SetReplicationState(const uint16_t replication, + std::function handler, + const uint64_t request_counter, const bool find_is_done) + : replication{replication}, handler{std::move(handler)}, + request_counter{request_counter}, find_is_done{find_is_done} {} + + /** + * The replication factor. + */ + const uint16_t replication; + + /** + * Handle the given {@link hdfs::Status}. + */ + const std::function handler; + + /** + * The request counter is incremented once every time SetReplication async + * call is made. + */ + uint64_t request_counter; + + /** + * This boolean will be set when find returns the last result. + */ + bool find_is_done; + + /** + * Final status to be returned. + */ + hdfs::Status status; + + /** + * Shared variables will need protection with a lock. + */ + std::mutex lock; +}; +} // namespace hdfs::tools + +#endif From 966b773a7c9e15e6be3ceaf703b210128fc2e307 Mon Sep 17 00:00:00 2001 From: litao Date: Wed, 6 Apr 2022 13:34:24 +0800 Subject: [PATCH 090/145] HDFS-16527. Add global timeout rule for TestRouterDistCpProcedure (#4129) Reviewed-by: Inigo Goiri Reviewed-by: Ayush Saxena Signed-off-by: Akira Ajisaka --- .../tools/fedbalance/TestDistCpProcedure.java | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestDistCpProcedure.java b/hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestDistCpProcedure.java index 0bed359903694..39986d865aea4 100644 --- a/hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestDistCpProcedure.java +++ b/hadoop-tools/hadoop-federation-balance/src/test/java/org/apache/hadoop/tools/fedbalance/TestDistCpProcedure.java @@ -33,7 +33,9 @@ import org.apache.hadoop.tools.fedbalance.procedure.BalanceProcedureScheduler; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.Timeout; import java.io.IOException; import java.io.OutputStream; @@ -44,6 +46,7 @@ import java.io.ByteArrayInputStream; import java.net.URI; import java.util.Random; +import java.util.concurrent.TimeUnit; import static junit.framework.TestCase.assertTrue; import static org.apache.hadoop.tools.fedbalance.FedBalanceConfigs.SCHEDULER_JOURNAL_URI; @@ -74,6 +77,11 @@ public class TestDistCpProcedure { new FileEntry(SRCDAT + "/b/c", false)}; private static String nnUri; + @Rule + // There are multiple unit tests with different timeouts that fail multiple times because of + // DataStreamer#waitAndQueuePacket, so we set a larger global timeout. + public Timeout globalTimeout = new Timeout(180000, TimeUnit.MILLISECONDS); + @BeforeClass public static void beforeClass() throws IOException { DistCpProcedure.enableForTest(); @@ -98,7 +106,7 @@ public static void afterClass() { } } - @Test(timeout = 90000) + @Test public void testSuccessfulDistCpProcedure() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = @@ -140,7 +148,7 @@ public void testSuccessfulDistCpProcedure() throws Exception { cleanup(fs, new Path(testRoot)); } - @Test(timeout = 30000) + @Test public void testInitDistCp() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = @@ -198,7 +206,7 @@ public void testDiffThreshold() throws Exception { cleanup(fs, new Path(testRoot)); } - @Test(timeout = 30000) + @Test public void testDiffDistCp() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = @@ -235,7 +243,7 @@ public void testDiffDistCp() throws Exception { cleanup(fs, new Path(testRoot)); } - @Test(timeout = 30000) + @Test public void testStageFinalDistCp() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = @@ -260,7 +268,7 @@ public void testStageFinalDistCp() throws Exception { cleanup(fs, new Path(testRoot)); } - @Test(timeout = 30000) + @Test public void testStageFinish() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = @@ -293,7 +301,7 @@ public void testStageFinish() throws Exception { cleanup(fs, new Path(testRoot)); } - @Test(timeout = 30000) + @Test public void testRecoveryByStage() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = @@ -334,7 +342,7 @@ public void testRecoveryByStage() throws Exception { cleanup(fs, new Path(testRoot)); } - @Test(timeout = 30000) + @Test public void testShutdown() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = @@ -359,7 +367,7 @@ public void testShutdown() throws Exception { cleanup(fs, new Path(testRoot)); } - @Test(timeout = 30000) + @Test public void testDisableWrite() throws Exception { String testRoot = nnUri + "/user/foo/testdir." + getMethodName(); DistributedFileSystem fs = From bbfe3500cf76ce661e0b10bb5d16cb419f9b9b10 Mon Sep 17 00:00:00 2001 From: Stephen O'Donnell Date: Wed, 6 Apr 2022 08:49:52 +0100 Subject: [PATCH 091/145] HDFS-16530. setReplication debug log creates a new string even if debug is disabled (#4142) --- .../org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java index db1baab66b3dc..04913d1a7cee0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java @@ -415,7 +415,7 @@ static BlockInfo[] unprotectedSetReplication( bm.setReplication(oldBR, targetReplication, b); } - if (oldBR != -1) { + if (oldBR != -1 && FSDirectory.LOG.isDebugEnabled()) { if (oldBR > targetReplication) { FSDirectory.LOG.debug("Decreasing replication from {} to {} for {}", oldBR, targetReplication, iip.getPath()); From 61bbdfd3a73a18cf05aeb2c40429c34c4b9b0732 Mon Sep 17 00:00:00 2001 From: wangzhaohui <32935220+wzhallright@users.noreply.github.com> Date: Wed, 6 Apr 2022 16:29:43 +0800 Subject: [PATCH 092/145] HDFS-16529. Remove unnecessary setObserverRead in TestConsistentReadsObserver (#4131) Signed-off-by: Akira Ajisaka --- .../hdfs/server/namenode/ha/TestConsistentReadsObserver.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java index 18f987db97481..decf85c06a46f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java @@ -108,8 +108,6 @@ public static void shutDownCluster() throws IOException { @Test public void testRequeueCall() throws Exception { - setObserverRead(true); - // Update the configuration just for the observer, by enabling // IPC backoff and using the test scheduler class, which starts to backoff // after certain number of calls. From 7c20602b17725cf266075477d70219a2a03721aa Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 6 Apr 2022 14:47:02 +0530 Subject: [PATCH 093/145] HDFS-16522. Set Http and Ipc ports for Datanodes in MiniDFSCluster (#4108) Signed-off-by: Akira Ajisaka --- .../apache/hadoop/hdfs/MiniDFSCluster.java | 83 +++++++++++++---- .../hdfs/MiniDFSClusterWithNodeGroup.java | 6 +- .../hadoop/hdfs/TestMiniDFSCluster.java | 92 +++++++++++++++++++ .../TestBalancerLongRunningTasks.java | 2 +- .../hadoop/hdfs/server/mover/TestMover.java | 6 +- ...StoragePolicySatisfierWithStripedFile.java | 4 +- .../TestExternalStoragePolicySatisfier.java | 2 +- .../tools/dynamometer/SimulatedDataNodes.java | 2 +- 8 files changed, 167 insertions(+), 30 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index 3e13e600b5189..d8d633f2c861e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -203,6 +203,8 @@ public static class Builder { private int nameNodeHttpPort = 0; private final Configuration conf; private int numDataNodes = 1; + private int[] dnHttpPorts = null; + private int[] dnIpcPorts = null; private StorageType[][] storageTypes = null; private StorageType[] storageTypes1D = null; private int storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE; @@ -277,6 +279,16 @@ public Builder numDataNodes(int val) { return this; } + public Builder setDnHttpPorts(int... ports) { + this.dnHttpPorts = ports; + return this; + } + + public Builder setDnIpcPorts(int... ports) { + this.dnIpcPorts = ports; + return this; + } + /** * Default: DEFAULT_STORAGES_PER_DATANODE */ @@ -599,7 +611,9 @@ protected MiniDFSCluster(Builder builder) throws IOException { builder.checkDataNodeHostConfig, builder.dnConfOverlays, builder.skipFsyncForTesting, - builder.useConfiguredTopologyMappingClass); + builder.useConfiguredTopologyMappingClass, + builder.dnHttpPorts, + builder.dnIpcPorts); } public static class DataNodeProperties { @@ -873,7 +887,7 @@ public MiniDFSCluster(int nameNodePort, operation, null, racks, hosts, null, simulatedCapacities, null, true, false, MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0), - true, false, false, null, true, false); + true, false, false, null, true, false, null, null); } private void initMiniDFSCluster( @@ -891,7 +905,9 @@ private void initMiniDFSCluster( boolean checkDataNodeHostConfig, Configuration[] dnConfOverlays, boolean skipFsyncForTesting, - boolean useConfiguredTopologyMappingClass) + boolean useConfiguredTopologyMappingClass, + int[] dnHttpPorts, + int[] dnIpcPorts) throws IOException { boolean success = false; try { @@ -974,9 +990,9 @@ private void initMiniDFSCluster( // Start the DataNodes startDataNodes(conf, numDataNodes, storageTypes, manageDataDfsDirs, - dnStartOpt != null ? dnStartOpt : startOpt, - racks, hosts, storageCapacities, simulatedCapacities, setupHostsFile, - checkDataNodeAddrConfig, checkDataNodeHostConfig, dnConfOverlays); + dnStartOpt != null ? dnStartOpt : startOpt, racks, hosts, storageCapacities, + simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, checkDataNodeHostConfig, + dnConfOverlays, dnHttpPorts, dnIpcPorts); waitClusterUp(); //make sure ProxyUsers uses the latest conf ProxyUsers.refreshSuperUserGroupsConfiguration(conf); @@ -1598,8 +1614,8 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, String[] racks, String[] hosts, long[] simulatedCapacities, boolean setupHostsFile) throws IOException { - startDataNodes(conf, numDataNodes, null, manageDfsDirs, operation, racks, hosts, - null, simulatedCapacities, setupHostsFile, false, false, null); + startDataNodes(conf, numDataNodes, null, manageDfsDirs, operation, racks, hosts, null, + simulatedCapacities, setupHostsFile, false, false, null, null, null); } public synchronized void startDataNodes(Configuration conf, int numDataNodes, @@ -1608,14 +1624,14 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, long[] simulatedCapacities, boolean setupHostsFile, boolean checkDataNodeAddrConfig) throws IOException { - startDataNodes(conf, numDataNodes, null, manageDfsDirs, operation, racks, hosts, - null, simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, false, null); + startDataNodes(conf, numDataNodes, null, manageDfsDirs, operation, racks, hosts, null, + simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, false, null, null, null); } /** * Modify the config and start up additional DataNodes. The info port for * DataNodes is guaranteed to use a free port. - * + * * Data nodes can run with the name node in the mini cluster or * a real name node. For example, running with a real name node is useful * when running simulated data nodes with a real name node. @@ -1625,6 +1641,7 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, * @param conf the base configuration to use in starting the DataNodes. This * will be modified as necessary. * @param numDataNodes Number of DataNodes to start; may be zero + * @param storageTypes Storage Types for DataNodes. * @param manageDfsDirs if true, the data directories for DataNodes will be * created and {@link DFSConfigKeys#DFS_DATANODE_DATA_DIR_KEY} will be * set in the conf @@ -1632,13 +1649,16 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, * or StartupOption.FORMAT, then StartupOption.REGULAR will be used. * @param racks array of strings indicating the rack that each DataNode is on * @param hosts array of strings indicating the hostnames for each DataNode + * @param storageCapacities array of Storage Capacities to be used while testing. * @param simulatedCapacities array of capacities of the simulated data nodes * @param setupHostsFile add new nodes to dfs hosts files * @param checkDataNodeAddrConfig if true, only set DataNode port addresses if not already set in config * @param checkDataNodeHostConfig if true, only set DataNode hostname key if not already set in config * @param dnConfOverlays An array of {@link Configuration} objects that will overlay the * global MiniDFSCluster Configuration for the corresponding DataNode. - * @throws IllegalStateException if NameNode has been shutdown + * @param dnHttpPorts An array of Http ports if present, to be used for DataNodes. + * @param dnIpcPorts An array of Ipc ports if present, to be used for DataNodes. + * @throws IOException If the DFS daemons experience some issues. */ public synchronized void startDataNodes(Configuration conf, int numDataNodes, StorageType[][] storageTypes, boolean manageDfsDirs, StartupOption operation, @@ -1648,7 +1668,9 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, boolean setupHostsFile, boolean checkDataNodeAddrConfig, boolean checkDataNodeHostConfig, - Configuration[] dnConfOverlays) throws IOException { + Configuration[] dnConfOverlays, + int[] dnHttpPorts, + int[] dnIpcPorts) throws IOException { assert storageCapacities == null || simulatedCapacities == null; assert storageTypes == null || storageTypes.length == numDataNodes; assert storageCapacities == null || storageCapacities.length == numDataNodes; @@ -1656,6 +1678,19 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, if (operation == StartupOption.RECOVER) { return; } + + if (dnHttpPorts != null && dnHttpPorts.length != numDataNodes) { + throw new IllegalArgumentException( + "Num of http ports (" + dnHttpPorts.length + ") should match num of DataNodes (" + + numDataNodes + ")"); + } + + if (dnIpcPorts != null && dnIpcPorts.length != numDataNodes) { + throw new IllegalArgumentException( + "Num of ipc ports (" + dnIpcPorts.length + ") should match num of DataNodes (" + + numDataNodes + ")"); + } + if (checkDataNodeHostConfig) { conf.setIfUnset(DFS_DATANODE_HOST_NAME_KEY, "127.0.0.1"); } else { @@ -1711,7 +1746,15 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, dnConf.addResource(dnConfOverlays[i]); } // Set up datanode address - setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig); + int httpPort = 0; + int ipcPort = 0; + if(dnHttpPorts != null) { + httpPort = dnHttpPorts[i - curDatanodesNum]; + } + if(dnIpcPorts != null) { + ipcPort = dnIpcPorts[i - curDatanodesNum]; + } + setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig, httpPort, ipcPort); if (manageDfsDirs) { String dirs = makeDataNodeDirs(i, storageTypes == null ? null : storageTypes[i - curDatanodesNum]); @@ -3363,9 +3406,9 @@ public void setBlockRecoveryTimeout(long timeout) { timeout); } } - + protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile, - boolean checkDataNodeAddrConfig) throws IOException { + boolean checkDataNodeAddrConfig, int httpPort, int ipcPort) throws IOException { if (setupHostsFile) { String hostsFile = conf.get(DFS_HOSTS, "").trim(); if (hostsFile.length() == 0) { @@ -3388,11 +3431,11 @@ protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile, } } if (checkDataNodeAddrConfig) { - conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0"); - conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0"); + conf.setIfUnset(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:" + httpPort); + conf.setIfUnset(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:" + ipcPort); } else { - conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:0"); - conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:0"); + conf.set(DFS_DATANODE_HTTP_ADDRESS_KEY, "127.0.0.1:" + httpPort); + conf.set(DFS_DATANODE_IPC_ADDRESS_KEY, "127.0.0.1:" + ipcPort); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java index 8b4e9e5ef7379..f7d1cb10e8f60 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java @@ -118,7 +118,7 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, for (int i = curDatanodesNum; i < curDatanodesNum+numDataNodes; i++) { Configuration dnConf = new HdfsConfiguration(conf); // Set up datanode address - setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig); + setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig, 0, 0); if (manageDfsDirs) { String dirs = makeDataNodeDirs(i, storageTypes == null ? null : storageTypes[i]); dnConf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dirs); @@ -235,7 +235,9 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes, boolean setupHostsFile, boolean checkDataNodeAddrConfig, boolean checkDataNodeHostConfig, - Configuration[] dnConfOverlays) throws IOException { + Configuration[] dnConfOverlays, + int[] dnHttpPorts, + int[] dnIpcPorts) throws IOException { startDataNodes(conf, numDataNodes, storageTypes, manageDfsDirs, operation, racks, NODE_GROUPS, hosts, storageCapacities, simulatedCapacities, setupHostsFile, checkDataNodeAddrConfig, checkDataNodeHostConfig); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java index 737795b88d4d1..0b6d07bc2f6f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assume.assumeTrue; @@ -26,6 +27,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Random; +import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeoutException; @@ -38,9 +40,13 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.test.PathUtils; import org.junit.Before; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.util.Preconditions; @@ -52,6 +58,8 @@ */ public class TestMiniDFSCluster { + private static final Logger LOG = LoggerFactory.getLogger(TestMiniDFSCluster.class); + private static final String CLUSTER_1 = "cluster1"; private static final String CLUSTER_2 = "cluster2"; private static final String CLUSTER_3 = "cluster3"; @@ -319,4 +327,88 @@ public void testSetUpFederatedCluster() throws Exception { cluster.restartNameNode(1); } } + + // There is a possibility that this test might fail if any other concurrently running + // test could bind same port as one of the ports returned by NetUtils.getFreeSocketPorts(6) + // before datanodes are started. + @Test + public void testStartStopWithPorts() throws Exception { + Configuration conf = new Configuration(); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of http ports (1) should match num of DataNodes (3)", + "MiniJournalCluster port validation failed", + () -> { + new MiniDFSCluster.Builder(conf).numDataNodes(3).setDnHttpPorts(8481).build(); + }); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of ipc ports (2) should match num of DataNodes (1)", + "MiniJournalCluster port validation failed", + () -> { + new MiniDFSCluster.Builder(conf).setDnIpcPorts(8481, 8482).build(); + }); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of ipc ports (1) should match num of DataNodes (3)", + "MiniJournalCluster port validation failed", + () -> { + new MiniDFSCluster.Builder(conf).numDataNodes(3).setDnHttpPorts(800, 9000, 10000) + .setDnIpcPorts(8481).build(); + }); + + LambdaTestUtils.intercept( + IllegalArgumentException.class, + "Num of http ports (4) should match num of DataNodes (3)", + "MiniJournalCluster port validation failed", + () -> { + new MiniDFSCluster.Builder(conf).setDnHttpPorts(800, 9000, 1000, 2000) + .setDnIpcPorts(8481, 8482, 8483).numDataNodes(3).build(); + }); + + final Set httpAndIpcPorts = NetUtils.getFreeSocketPorts(6); + LOG.info("Free socket ports: {}", httpAndIpcPorts); + + assertThat(httpAndIpcPorts).doesNotContain(0); + + final int[] httpPorts = new int[3]; + final int[] ipcPorts = new int[3]; + int httpPortIdx = 0; + int ipcPortIdx = 0; + for (Integer httpAndIpcPort : httpAndIpcPorts) { + if (httpPortIdx < 3) { + httpPorts[httpPortIdx++] = httpAndIpcPort; + } else { + ipcPorts[ipcPortIdx++] = httpAndIpcPort; + } + } + + LOG.info("Http ports selected: {}", httpPorts); + LOG.info("Ipc ports selected: {}", ipcPorts); + + try (MiniDFSCluster miniDfsCluster = new MiniDFSCluster.Builder(conf) + .setDnHttpPorts(httpPorts) + .setDnIpcPorts(ipcPorts) + .numDataNodes(3).build()) { + miniDfsCluster.waitActive(); + + assertEquals(httpPorts[0], + miniDfsCluster.getDataNode(ipcPorts[0]).getInfoPort()); + assertEquals(httpPorts[1], + miniDfsCluster.getDataNode(ipcPorts[1]).getInfoPort()); + assertEquals(httpPorts[2], + miniDfsCluster.getDataNode(ipcPorts[2]).getInfoPort()); + + assertEquals(ipcPorts[0], + miniDfsCluster.getDataNode(ipcPorts[0]).getIpcPort()); + assertEquals(ipcPorts[1], + miniDfsCluster.getDataNode(ipcPorts[1]).getIpcPort()); + assertEquals(ipcPorts[2], + miniDfsCluster.getDataNode(ipcPorts[2]).getIpcPort()); + } + } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerLongRunningTasks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerLongRunningTasks.java index 0d6300cf05e91..ba36a42e0c3db 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerLongRunningTasks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerLongRunningTasks.java @@ -262,7 +262,7 @@ public void testBalancerWithRamDisk() throws Exception { long[][] storageCapacities = new long[][]{{ramDiskStorageLimit, diskStorageLimit}}; cluster.startDataNodes(conf, replicationFactor, storageTypes, true, null, - null, null, storageCapacities, null, false, false, false, null); + null, null, storageCapacities, null, false, false, false, null, null, null); cluster.triggerHeartbeats(); Collection namenodes = DFSUtil.getInternalNsRpcUris(conf); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index d78b598bd3f5c..90353c352ea41 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -949,7 +949,7 @@ public void testMoverWithStripedFile() throws Exception { {StorageType.ARCHIVE, StorageType.ARCHIVE}, {StorageType.ARCHIVE, StorageType.ARCHIVE}, {StorageType.ARCHIVE, StorageType.ARCHIVE}}, - true, null, null, null,capacities, null, false, false, false, null); + true, null, null, null, capacities, null, false, false, false, null, null, null); cluster.triggerHeartbeats(); // move file to ARCHIVE @@ -982,7 +982,7 @@ public void testMoverWithStripedFile() throws Exception { { StorageType.SSD, StorageType.DISK }, { StorageType.SSD, StorageType.DISK }, { StorageType.SSD, StorageType.DISK } }, - true, null, null, null, capacities, null, false, false, false, null); + true, null, null, null, capacities, null, false, false, false, null, null, null); cluster.triggerHeartbeats(); // move file blocks to ONE_SSD policy @@ -1372,7 +1372,7 @@ private void startAdditionalDNs(final Configuration conf, final MiniDFSCluster cluster) throws IOException { cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null, - null, null, null, false, false, false, null); + null, null, null, false, false, false, null, null, null); cluster.triggerHeartbeats(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java index 664f459ebae01..8e0364078357d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/sps/TestStoragePolicySatisfierWithStripedFile.java @@ -186,7 +186,7 @@ public void testMoverWithFullStripe() throws Exception { {StorageType.ARCHIVE, StorageType.ARCHIVE}, {StorageType.ARCHIVE, StorageType.ARCHIVE}, {StorageType.ARCHIVE, StorageType.ARCHIVE}}, - true, null, null, null, capacities, null, false, false, false, null); + true, null, null, null, capacities, null, false, false, false, null, null, null); cluster.triggerHeartbeats(); // move file to ARCHIVE @@ -294,7 +294,7 @@ public void testWhenOnlyFewTargetNodesAreAvailableToSatisfyStoragePolicy() new StorageType[][]{ {StorageType.ARCHIVE, StorageType.ARCHIVE}, {StorageType.ARCHIVE, StorageType.ARCHIVE}}, - true, null, null, null, capacities, null, false, false, false, null); + true, null, null, null, capacities, null, false, false, false, null, null, null); cluster.triggerHeartbeats(); // Move file to ARCHIVE. Only 5 datanodes are available with ARCHIVE diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java index 4c75cb3f294a5..7ef83c392e215 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java @@ -1712,7 +1712,7 @@ private void startAdditionalDNs(final Configuration conf, } cluster.startDataNodes(conf, newNodesRequired, newTypes, true, null, null, - null, capacities, null, false, false, false, null); + null, capacities, null, false, false, false, null, null, null); cluster.triggerHeartbeats(); } diff --git a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/SimulatedDataNodes.java b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/SimulatedDataNodes.java index 520077e0823cb..0189d1fd1a1f1 100644 --- a/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/SimulatedDataNodes.java +++ b/hadoop-tools/hadoop-dynamometer/hadoop-dynamometer-infra/src/main/java/org/apache/hadoop/tools/dynamometer/SimulatedDataNodes.java @@ -131,7 +131,7 @@ public int run(String[] args) throws Exception { + " block listing files; launching DataNodes accordingly."); mc.startDataNodes(getConf(), blockListFiles.size(), null, false, StartupOption.REGULAR, null, null, null, null, false, true, true, - null); + null, null, null); long startTime = Time.monotonicNow(); System.out.println("Waiting for DataNodes to connect to NameNode and " + "init storage directories."); From 4b786c797a76d00ef33237fde76f2bd6edda7f3a Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Thu, 7 Apr 2022 09:19:36 +0200 Subject: [PATCH 094/145] HADOOP-18178. Upgrade jackson to 2.13.2 and jackson-databind to 2.13.2.2 (#4111) Signed-off-by: Akira Ajisaka --- LICENSE-binary | 12 ++++++------ hadoop-project/pom.xml | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index de8df181fc3d1..ef0db1bb6a4d7 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -218,12 +218,12 @@ com.aliyun.oss:aliyun-sdk-oss:3.13.2 com.amazonaws:aws-java-sdk-bundle:1.11.901 com.cedarsoftware:java-util:1.9.0 com.cedarsoftware:json-io:2.5.1 -com.fasterxml.jackson.core:jackson-annotations:2.13.0 -com.fasterxml.jackson.core:jackson-core:2.13.0 -com.fasterxml.jackson.core:jackson-databind:2.13.0 -com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.13.0 -com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.13.0 -com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.13.0 +com.fasterxml.jackson.core:jackson-annotations:2.13.2 +com.fasterxml.jackson.core:jackson-core:2.13.2 +com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.13.2 +com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.13.2 +com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.13.2 com.fasterxml.uuid:java-uuid-generator:3.1.4 com.fasterxml.woodstox:woodstox-core:5.3.0 com.github.davidmoten:rxjava-extras:0.8.0.17 diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 0a84f89a723fd..bdca6ebb9e7aa 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -70,8 +70,8 @@ 1.9.13 - 2.13.0 - 2.13.0 + 2.13.2 + 2.13.2.2 2.1.1 From f70935522b0fd771dedb9604b92397135fd0c8ad Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Thu, 7 Apr 2022 13:59:45 +0530 Subject: [PATCH 095/145] HADOOP-18188. Support touch command for directory (#4135) Signed-off-by: Akira Ajisaka --- .../apache/hadoop/fs/shell/TouchCommands.java | 3 - .../apache/hadoop/fs/TestFsShellTouch.java | 72 ++++++ .../apache/hadoop/hdfs/TestDFSShellTouch.java | 239 ++++++++++++++++++ 3 files changed, 311 insertions(+), 3 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java index 902eada98dbd1..457cd86ab703c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/TouchCommands.java @@ -147,9 +147,6 @@ protected void processOptions(LinkedList args) { @Override protected void processPath(PathData item) throws IOException { - if (item.stat.isDirectory()) { - throw new PathIsDirectoryException(item.toString()); - } touch(item); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java index 49bbd5af04f25..c2bd5b2133d47 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShellTouch.java @@ -200,6 +200,78 @@ public void testTouch() throws Exception { FileStatus fileStatus = lfs.getFileStatus(newFile); assertThat(fileStatus.getAccessTime()).isEqualTo(dateObj.getTime()); assertThat(fileStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + lfs.delete(newFile, true); + assertThat(lfs.exists(newFile)).isFalse(); + + } + + @Test + public void testTouchDir() throws Exception { + String strTime; + final String newFileName = "dir3/newFile3"; + Date dateObj; + final Path newFile = new Path(newFileName); + FileStatus fstatus; + Path dirPath = new Path("dir3"); + lfs.delete(dirPath, true); + lfs.mkdirs(dirPath); + lfs.delete(newFile, true); + assertThat(lfs.exists(newFile)).isFalse(); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-t", strTime, newFileName)).as( + "Expected successful touch on a new file with a specified timestamp").isEqualTo(0); + FileStatus newStatus = lfs.getFileStatus(newFile); + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + Thread.sleep(500); + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-m", "-a", "-t", strTime, "dir3")).as( + "Expected successful touch with a specified modification time").isEqualTo(0); + + newStatus = lfs.getFileStatus(dirPath); + // Verify if both modification and access times are recorded correctly + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + fstatus = lfs.getFileStatus(dirPath); + Thread.sleep(500); + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-m", "-t", strTime, "dir3")).as( + "Expected successful touch with a specified modification time").isEqualTo(0); + + newStatus = lfs.getFileStatus(dirPath); + // Verify if modification time is recorded correctly (and access time + // remains unchanged). + assertThat(newStatus.getAccessTime()).isEqualTo(fstatus.getAccessTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + fstatus = lfs.getFileStatus(dirPath); + Thread.sleep(500); + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-a", "-t", strTime, "dir3")).as( + "Expected successful touch with a specified modification time").isEqualTo(0); + + newStatus = lfs.getFileStatus(dirPath); + // Verify if access time is recorded correctly (and modification time + // remains unchanged). + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(fstatus.getModificationTime()); + + lfs.delete(newFile, true); + lfs.delete(dirPath, true); + assertThat(lfs.exists(newFile)).isFalse(); + assertThat(lfs.exists(dirPath)).isFalse(); } private String formatTimestamp(long timeInMillis) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java new file mode 100644 index 0000000000000..d0334cbf42747 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShellTouch.java @@ -0,0 +1,239 @@ +/** + * 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.hadoop.hdfs; + +import java.io.IOException; +import java.text.ParseException; +import java.util.Date; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FsShell; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.shell.TouchCommands; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.StringUtils; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests to perform Touch operations on DFS. + */ +public class TestDFSShellTouch { + + private static final Logger LOG = LoggerFactory.getLogger(TestDFSShellTouch.class); + + private static MiniDFSCluster miniCluster; + private static DistributedFileSystem dfs; + private static FsShell shell; + + @BeforeClass + public static void setup() throws IOException { + final Configuration conf = new Configuration(); + conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, + GenericTestUtils.getTestDir("TestDFSShellTouch").getAbsolutePath()); + + miniCluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + miniCluster.waitActive(); + dfs = miniCluster.getFileSystem(); + shell = new FsShell(dfs.getConf()); + } + + @AfterClass + public static void tearDown() { + if (miniCluster != null) { + miniCluster.shutdown(true, true); + } + } + + @Test + public void testTouch() throws Exception { + final String newFileName = "newFile1"; + final Path newFile = new Path(newFileName); + dfs.delete(newFile, true); + assertThat(dfs.exists(newFile)).isFalse(); + dfs.create(newFile); + assertThat(dfs.exists(newFile)).isTrue(); + + String strTime = formatTimestamp(System.currentTimeMillis()); + Date dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-t", strTime, newFileName)).as( + "Expected successful touch on a new file" + " with a specified timestamp").isEqualTo(0); + FileStatus newStatus = dfs.getFileStatus(newFile); + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + FileStatus fileStatus = dfs.getFileStatus(newFile); + Thread.sleep(500); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-a", "-t", strTime, newFileName)).as( + "Expected successful touch with a specified access time").isEqualTo(0); + newStatus = dfs.getFileStatus(newFile); + // Verify if access time is recorded correctly (and modification time + // remains unchanged). + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(fileStatus.getModificationTime()); + + fileStatus = dfs.getFileStatus(newFile); + Thread.sleep(500); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-m", "-t", strTime, newFileName)).as( + "Expected successful touch with a specified modification time").isEqualTo(0); + // Verify if modification time is recorded correctly (and access time + // remains unchanged). + newStatus = dfs.getFileStatus(newFile); + assertThat(newStatus.getAccessTime()).isEqualTo(fileStatus.getAccessTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-t", strTime, newFileName)).as( + "Expected successful touch with a specified timestamp").isEqualTo(0); + + // Verify if both modification and access times are recorded correctly + newStatus = dfs.getFileStatus(newFile); + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-a", "-m", "-t", strTime, newFileName)).as( + "Expected successful touch with a specified timestamp").isEqualTo(0); + + // Verify if both modification and access times are recorded correctly + newStatus = dfs.getFileStatus(newFile); + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + assertThat(shellRun("-touch", "-t", newFileName)).as( + "Expected failed touch with a missing timestamp").isNotEqualTo(0); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + assertThat(shellRun("-touch", "-c", "-t", strTime, newFileName)).as( + "Expected successful touch on a non-existent file with -c option").isEqualTo(0); + fileStatus = dfs.getFileStatus(newFile); + assertThat(fileStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(fileStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + dfs.delete(newFile, true); + assertThat(dfs.exists(newFile)).isFalse(); + + } + + @Test + public void testTouchDirs() throws IOException, ParseException, InterruptedException { + final String newFileName = "dir2/newFile2"; + final Path newFile = new Path(newFileName); + FileStatus newStatus; + FileStatus fileStatus; + String strTime; + Date dateObj; + Path dirPath = new Path("dir2"); + dfs.mkdirs(dirPath); + dfs.delete(newFile, true); + assertThat(dfs.exists(newFile)).isFalse(); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-t", strTime, newFileName)).as( + "Expected successful touch on a new file with a specified timestamp").isEqualTo(0); + newStatus = dfs.getFileStatus(newFile); + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + Thread.sleep(500); + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-m", "-a", "-t", strTime, "dir2")).as( + "Expected successful touch with a specified modification time").isEqualTo(0); + + newStatus = dfs.getFileStatus(dirPath); + // Verify if both modification and access times are recorded correctly + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + fileStatus = dfs.getFileStatus(dirPath); + Thread.sleep(500); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-m", "-t", strTime, "dir2")).as( + "Expected successful touch with a specified modification time").isEqualTo(0); + + newStatus = dfs.getFileStatus(dirPath); + // Verify if modification time is recorded correctly (and access time + // remains unchanged). + assertThat(newStatus.getAccessTime()).isEqualTo(fileStatus.getAccessTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(dateObj.getTime()); + + fileStatus = dfs.getFileStatus(dirPath); + Thread.sleep(500); + + strTime = formatTimestamp(System.currentTimeMillis()); + dateObj = parseTimestamp(strTime); + + assertThat(shellRun("-touch", "-a", "-t", strTime, "dir2")).as( + "Expected successful touch with a specified modification time").isEqualTo(0); + + newStatus = dfs.getFileStatus(dirPath); + // Verify if access time is recorded correctly (and modification time + // remains unchanged). + assertThat(newStatus.getAccessTime()).isEqualTo(dateObj.getTime()); + assertThat(newStatus.getModificationTime()).isEqualTo(fileStatus.getModificationTime()); + + dfs.delete(newFile, true); + dfs.delete(dirPath, true); + assertThat(dfs.exists(newFile)).isFalse(); + assertThat(dfs.exists(dirPath)).isFalse(); + } + + private int shellRun(String... args) { + int exitCode = shell.run(args); + LOG.info("exit " + exitCode + " - " + StringUtils.join(" ", args)); + return exitCode; + } + + private String formatTimestamp(long timeInMillis) { + return (new TouchCommands.Touch()).getDateFormat().format(new Date(timeInMillis)); + } + + private Date parseTimestamp(String tstamp) throws ParseException { + return (new TouchCommands.Touch()).getDateFormat().parse(tstamp); + } + +} From 807a428b556cf16d7da7d7062ba3f264e1d9191b Mon Sep 17 00:00:00 2001 From: singer-bin Date: Fri, 8 Apr 2022 09:27:35 +0800 Subject: [PATCH 096/145] HDFS-16457.Make fs.getspaceused.classname reconfigurable (#4069) --- .../hadoop/hdfs/server/datanode/DataNode.java | 34 ++++++++++++++++--- .../fsdataset/impl/BlockPoolSlice.java | 8 ++++- .../datanode/TestDataNodeReconfiguration.java | 31 +++++++++++++++++ .../hadoop/hdfs/tools/TestDFSAdmin.java | 2 +- 4 files changed, 69 insertions(+), 6 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index c3b1aa1c67203..57aa2c31cab67 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_CLASSNAME; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_DEFAULT; @@ -87,6 +88,9 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.DF; +import org.apache.hadoop.fs.DU; +import org.apache.hadoop.fs.GetSpaceUsed; +import org.apache.hadoop.fs.WindowsGetSpaceUsed; import org.apache.hadoop.hdfs.protocol.proto.ReconfigurationProtocolProtos.ReconfigurationProtocolService; import java.io.BufferedOutputStream; @@ -349,7 +353,8 @@ public class DataNode extends ReconfigurableBase DFS_DATANODE_MIN_OUTLIER_DETECTION_DISKS_KEY, DFS_DATANODE_SLOWDISK_LOW_THRESHOLD_MS_KEY, FS_DU_INTERVAL_KEY, - FS_GETSPACEUSED_JITTER_KEY)); + FS_GETSPACEUSED_JITTER_KEY, + FS_GETSPACEUSED_CLASSNAME)); public static final Log METRICS_LOG = LogFactory.getLog("DataNodeMetricsLog"); @@ -683,6 +688,7 @@ public String reconfigurePropertyImpl(String property, String newVal) return reconfSlowDiskParameters(property, newVal); case FS_DU_INTERVAL_KEY: case FS_GETSPACEUSED_JITTER_KEY: + case FS_GETSPACEUSED_CLASSNAME: return reconfDfsUsageParameters(property, newVal); default: break; @@ -879,7 +885,7 @@ private String reconfDfsUsageParameters(String property, String newVal) for (FsVolumeImpl fsVolume : volumeList) { Map blockPoolSlices = fsVolume.getBlockPoolSlices(); for (BlockPoolSlice value : blockPoolSlices.values()) { - value.updateDfsUsageConfig(interval, null); + value.updateDfsUsageConfig(interval, null, null); } } } else if (property.equals(FS_GETSPACEUSED_JITTER_KEY)) { @@ -891,13 +897,33 @@ private String reconfDfsUsageParameters(String property, String newVal) for (FsVolumeImpl fsVolume : volumeList) { Map blockPoolSlices = fsVolume.getBlockPoolSlices(); for (BlockPoolSlice value : blockPoolSlices.values()) { - value.updateDfsUsageConfig(null, jitter); + value.updateDfsUsageConfig(null, jitter, null); + } + } + } else if (property.equals(FS_GETSPACEUSED_CLASSNAME)) { + Preconditions.checkNotNull(data, "FsDatasetSpi has not been initialized."); + Class klass; + if (newVal == null) { + if (Shell.WINDOWS) { + klass = DU.class; + } else { + klass = WindowsGetSpaceUsed.class; + } + } else { + klass = Class.forName(newVal).asSubclass(GetSpaceUsed.class); + } + result = klass.getName(); + List volumeList = data.getVolumeList(); + for (FsVolumeImpl fsVolume : volumeList) { + Map blockPoolSlices = fsVolume.getBlockPoolSlices(); + for (BlockPoolSlice value : blockPoolSlices.values()) { + value.updateDfsUsageConfig(null, null, klass); } } } LOG.info("RECONFIGURE* changed {} to {}", property, newVal); return result; - } catch (IllegalArgumentException | IOException e) { + } catch (IllegalArgumentException | IOException | ClassNotFoundException e) { throw new ReconfigurationException(property, newVal, getConf().get(property), e); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java index 8357b860b25e2..23f3602a456c8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java @@ -80,6 +80,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_CLASSNAME; /** * A block pool slice represents a portion of a block pool stored on a volume. @@ -240,7 +241,8 @@ public void run() { SHUTDOWN_HOOK_PRIORITY); } - public void updateDfsUsageConfig(Long interval, Long jitter) throws IOException { + public void updateDfsUsageConfig(Long interval, Long jitter, Class klass) + throws IOException { // Close the old dfsUsage if it is CachingGetSpaceUsed. if (dfsUsage instanceof CachingGetSpaceUsed) { ((CachingGetSpaceUsed) dfsUsage).close(); @@ -255,6 +257,10 @@ public void updateDfsUsageConfig(Long interval, Long jitter) throws IOException FS_GETSPACEUSED_JITTER_KEY + " should be larger than or equal to 0"); config.setLong(FS_GETSPACEUSED_JITTER_KEY, jitter); } + + if (klass != null) { + config.setClass(FS_GETSPACEUSED_CLASSNAME, klass, CachingGetSpaceUsed.class); + } // Start new dfsUsage. this.dfsUsage = new FSCachingGetSpaceUsed.Builder().setBpid(bpid) .setVolume(volume) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java index 172a44557c1ce..14e3f63691bd5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeReconfiguration.java @@ -23,6 +23,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_DU_INTERVAL_KEY; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_JITTER_KEY; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_GETSPACEUSED_CLASSNAME; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY; @@ -86,6 +87,7 @@ public class TestDataNodeReconfiguration { private final int NUM_NAME_NODE = 1; private final int NUM_DATA_NODE = 10; private MiniDFSCluster cluster; + private static long counter = 0; @Before public void Setup() throws IOException { @@ -756,4 +758,33 @@ public void testDfsUsageParameters() throws ReconfigurationException { } } } + + public static class DummyCachingGetSpaceUsed extends CachingGetSpaceUsed { + public DummyCachingGetSpaceUsed(Builder builder) throws IOException { + super(builder.setInterval(1000).setJitter(0L)); + } + + @Override + protected void refresh() { + counter++; + } + } + + @Test + public void testDfsUsageKlass() throws ReconfigurationException, InterruptedException { + + long lastCounter = counter; + Thread.sleep(5000); + assertEquals(lastCounter, counter); + + for (int i = 0; i < NUM_DATA_NODE; i++) { + DataNode dn = cluster.getDataNodes().get(i); + dn.reconfigurePropertyImpl(FS_GETSPACEUSED_CLASSNAME, + DummyCachingGetSpaceUsed.class.getName()); + } + + lastCounter = counter; + Thread.sleep(5000); + assertTrue(counter > lastCounter); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java index 648fb854e52b1..d2b5f1b218940 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java @@ -339,7 +339,7 @@ public void testDataNodeGetReconfigurableProperties() throws IOException { final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("datanode", address, outs, errs); - assertEquals(18, outs.size()); + assertEquals(19, outs.size()); assertEquals(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, outs.get(1)); } From 5412fbf6d4b59836df6ad9118f230812c8a50f1c Mon Sep 17 00:00:00 2001 From: litao Date: Fri, 8 Apr 2022 12:26:39 +0800 Subject: [PATCH 097/145] HDFS-16460. [SPS]: Handle failure retries for moving tasks (#4001) --- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 4 ++ .../server/common/sps/BlockDispatcher.java | 10 +--- .../sps/ExternalSPSBlockMoveTaskHandler.java | 38 +++++++++------ .../server/sps/ExternalSPSFaultInjector.java | 46 +++++++++++++++++++ .../src/main/resources/hdfs-default.xml | 8 ++++ .../TestExternalStoragePolicySatisfier.java | 23 ++++++++++ 6 files changed, 105 insertions(+), 24 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFaultInjector.java 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 6216f6e7a1ded..cf1755cd9f9b8 100755 --- 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 @@ -827,6 +827,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.storage.policy.satisfier.retry.max.attempts"; public static final int DFS_STORAGE_POLICY_SATISFIER_MAX_RETRY_ATTEMPTS_DEFAULT = 3; + public static final String DFS_STORAGE_POLICY_SATISFIER_MOVE_TASK_MAX_RETRY_ATTEMPTS_KEY = + "dfs.storage.policy.satisfier.move.task.retry.max.attempts"; + public static final int DFS_STORAGE_POLICY_SATISFIER_MOVE_TASK_MAX_RETRY_ATTEMPTS_DEFAULT = + 3; public static final String DFS_STORAGE_DEFAULT_POLICY = "dfs.storage.default.policy"; public static final HdfsConstants.StoragePolicy diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java index f87fcaef054c0..f7756c74851a6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/sps/BlockDispatcher.java @@ -101,7 +101,7 @@ public BlockDispatcher(int sockTimeout, int ioFileBuffSize, */ public BlockMovementStatus moveBlock(BlockMovingInfo blkMovingInfo, SaslDataTransferClient saslClient, ExtendedBlock eb, Socket sock, - DataEncryptionKeyFactory km, Token accessToken) { + DataEncryptionKeyFactory km, Token accessToken) throws IOException { LOG.info("Start moving block:{} from src:{} to destin:{} to satisfy " + "storageType, sourceStoragetype:{} and destinStoragetype:{}", blkMovingInfo.getBlock(), blkMovingInfo.getSource(), @@ -149,14 +149,6 @@ public BlockMovementStatus moveBlock(BlockMovingInfo blkMovingInfo, LOG.debug("Pinned block can't be moved, so skipping block:{}", blkMovingInfo.getBlock(), e); return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_SUCCESS; - } catch (IOException e) { - // TODO: handle failure retries - LOG.warn( - "Failed to move block:{} from src:{} to destin:{} to satisfy " - + "storageType:{}", - blkMovingInfo.getBlock(), blkMovingInfo.getSource(), - blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), e); - return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE; } finally { IOUtils.closeStream(out); IOUtils.closeStream(in); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java index 64dec8bbc5c3c..ec3837424cc20 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java @@ -80,11 +80,15 @@ public class ExternalSPSBlockMoveTaskHandler implements BlockMoveTaskHandler { private Daemon movementTrackerThread; private final SPSService service; private final BlockDispatcher blkDispatcher; + private final int maxRetry; public ExternalSPSBlockMoveTaskHandler(Configuration conf, NameNodeConnector nnc, SPSService spsService) { int moverThreads = conf.getInt(DFSConfigKeys.DFS_MOVER_MOVERTHREADS_KEY, DFSConfigKeys.DFS_MOVER_MOVERTHREADS_DEFAULT); + maxRetry = conf.getInt( + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MOVE_TASK_MAX_RETRY_ATTEMPTS_KEY, + DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MOVE_TASK_MAX_RETRY_ATTEMPTS_DEFAULT); moveExecutor = initializeBlockMoverThreadPool(moverThreads); mCompletionServ = new ExecutorCompletionService<>(moveExecutor); this.nnc = nnc; @@ -151,7 +155,7 @@ public void submitMoveTask(BlockMovingInfo blkMovingInfo) throws IOException { // during block movement assignment logic. In the internal movement, // remaining space is bookkeeping at the DatanodeDescriptor, please refer // IntraSPSNameNodeBlockMoveTaskHandler#submitMoveTask implementation and - // updating via the funcation call - + // updating via the function call - // dn.incrementBlocksScheduled(blkMovingInfo.getTargetStorageType()); LOG.debug("Received BlockMovingTask {}", blkMovingInfo); BlockMovingTask blockMovingTask = new BlockMovingTask(blkMovingInfo); @@ -195,21 +199,25 @@ private BlockMovementStatus moveBlock() { final KeyManager km = nnc.getKeyManager(); Token accessToken; - try { - accessToken = km.getAccessToken(eb, - new StorageType[]{blkMovingInfo.getTargetStorageType()}, - new String[0]); - } catch (IOException e) { - // TODO: handle failure retries - LOG.warn( - "Failed to move block:{} from src:{} to destin:{} to satisfy " - + "storageType:{}", - blkMovingInfo.getBlock(), blkMovingInfo.getSource(), - blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), e); - return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE; + int retry = 0; + while (retry <= maxRetry) { + try { + ExternalSPSFaultInjector.getInstance().mockAnException(retry); + accessToken = km.getAccessToken(eb, + new StorageType[]{blkMovingInfo.getTargetStorageType()}, + new String[0]); + return blkDispatcher.moveBlock(blkMovingInfo, saslClient, eb, + new Socket(), km, accessToken); + } catch (IOException e) { + LOG.warn( + "Failed to move block:{} from src:{} to dest:{} to satisfy " + + "storageType:{}, retry: {}", + blkMovingInfo.getBlock(), blkMovingInfo.getSource(), + blkMovingInfo.getTarget(), blkMovingInfo.getTargetStorageType(), retry, e); + retry++; + } } - return blkDispatcher.moveBlock(blkMovingInfo, saslClient, eb, - new Socket(), km, accessToken); + return BlockMovementStatus.DN_BLK_STORAGE_MOVEMENT_FAILURE; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFaultInjector.java new file mode 100644 index 0000000000000..5ddf1ee3c0f6b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSFaultInjector.java @@ -0,0 +1,46 @@ +/** + * 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.hadoop.hdfs.server.sps; + +import org.apache.hadoop.classification.VisibleForTesting; + +import java.io.IOException; + +/** + * Used to inject certain faults for testing. + */ +public class ExternalSPSFaultInjector { + @VisibleForTesting + private static ExternalSPSFaultInjector instance = + new ExternalSPSFaultInjector(); + + @VisibleForTesting + public static ExternalSPSFaultInjector getInstance() { + return instance; + } + + @VisibleForTesting + public static void setInstance(ExternalSPSFaultInjector instance) { + ExternalSPSFaultInjector.instance = instance; + } + + @VisibleForTesting + public void mockAnException(int retry) throws IOException { + } +} 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 d45f8eb5b7ec5..a75a76d7af4ed 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -5507,6 +5507,14 @@ + + dfs.storage.policy.satisfier.move.task.retry.max.attempts + 3 + + Max retries for moving task to satisfy the block storage policy. + + + dfs.storage.policy.satisfier.datanode.cache.refresh.interval.ms 300000 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java index 7ef83c392e215..9b4dc437ce414 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java @@ -33,6 +33,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_KEYTAB_FILE_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SPS_MAX_OUTSTANDING_PATHS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MOVE_TASK_MAX_RETRY_ATTEMPTS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY; import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_SATISFY_STORAGE_POLICY; @@ -130,6 +131,14 @@ public class TestExternalStoragePolicySatisfier { private static final int DEFAULT_BLOCK_SIZE = 1024; private static final Logger LOG = LoggerFactory.getLogger(TestExternalStoragePolicySatisfier.class); + private final ExternalSPSFaultInjector injector = new ExternalSPSFaultInjector() { + @Override + public void mockAnException(int retry) throws IOException { + if (retry < DFS_STORAGE_POLICY_SATISFIER_MOVE_TASK_MAX_RETRY_ATTEMPTS_DEFAULT) { + throw new IOException("IO exception"); + } + } + }; @Before public void setUp() { @@ -480,6 +489,20 @@ public void testInfiniteStartWhenAnotherSPSRunning() } } + @Test(timeout = 300000) + public void testWhenStoragePolicySetToCOLDWithException() + throws Exception { + + try { + createCluster(); + // Mock an IOException 3 times, and moving tasks should succeed finally. + ExternalSPSFaultInjector.setInstance(injector); + doTestWhenStoragePolicySetToCOLD(); + } finally { + shutdownCluster(); + } + } + private void doTestWhenStoragePolicySetToCOLD() throws Exception { // Change policy to COLD dfs.setStoragePolicy(new Path(FILE), COLD); From bfde9102be77654861c8025eab08fc8568ce41dd Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Fri, 8 Apr 2022 11:24:09 +0200 Subject: [PATCH 098/145] HADOOP-18195. Make jackson 1 a runtime scope dependency (#4149) Contributed by PJ Fanning --- hadoop-project/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index bdca6ebb9e7aa..871d7fc8082c8 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1186,21 +1186,25 @@ org.codehaus.jackson jackson-mapper-asl ${jackson.version} + runtime org.codehaus.jackson jackson-core-asl ${jackson.version} + runtime org.codehaus.jackson jackson-jaxrs ${jackson.version} + runtime org.codehaus.jackson jackson-xc ${jackson.version} + runtime com.fasterxml.jackson.core From 37650ced81964efe7a9ea45bf505af96b8035d53 Mon Sep 17 00:00:00 2001 From: Takanobu Asanuma Date: Fri, 8 Apr 2022 18:38:40 +0900 Subject: [PATCH 099/145] HDFS-16497. EC: Add param comment for liveBusyBlockIndices with HDFS-14768. Contributed by caozhiqiang. --- .../hadoop/hdfs/server/blockmanagement/BlockManager.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index bfa8457dd4e31..1d4937739dd64 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -2443,6 +2443,10 @@ private DatanodeDescriptor getDatanodeDescriptorFromStorage( * replicas of the given block. * @param liveBlockIndices List to be populated with indices of healthy * blocks in a striped block group + * @param liveBusyBlockIndices List to be populated with indices of healthy + * blocks in a striped block group in busy DN, + * which the recovery work have reached their + * replication limits * @param priority integer representing replication priority of the given * block * @return the array of DatanodeDescriptor of the chosen nodes from which to From b69ede7154d44538a4a66824c34f7ba143deef25 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Fri, 8 Apr 2022 18:01:54 +0530 Subject: [PATCH 100/145] HADOOP-18191. Log retry count while handling exceptions in RetryInvocationHandler (#4133) --- .../apache/hadoop/io/retry/RetryInvocationHandler.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java index fcc5975987ebc..3960b189665f6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java @@ -387,12 +387,12 @@ private RetryInfo handleException(final Method method, final int callId, throw retryInfo.getFailException(); } - log(method, retryInfo.isFailover(), counters.failovers, retryInfo.delay, e); + log(method, retryInfo.isFailover(), counters.failovers, counters.retries, retryInfo.delay, e); return retryInfo; } - private void log(final Method method, final boolean isFailover, - final int failovers, final long delay, final Exception ex) { + private void log(final Method method, final boolean isFailover, final int failovers, + final int retries, final long delay, final Exception ex) { boolean info = true; // If this is the first failover to this proxy, skip logging at INFO level if (!failedAtLeastOnce.contains(proxyDescriptor.getProxyInfo().toString())) @@ -408,13 +408,15 @@ private void log(final Method method, final boolean isFailover, } final StringBuilder b = new StringBuilder() - .append(ex + ", while invoking ") + .append(ex) + .append(", while invoking ") .append(proxyDescriptor.getProxyInfo().getString(method.getName())); if (failovers > 0) { b.append(" after ").append(failovers).append(" failover attempts"); } b.append(isFailover? ". Trying to failover ": ". Retrying "); b.append(delay > 0? "after sleeping for " + delay + "ms.": "immediately."); + b.append(" Current retry count: ").append(retries).append("."); if (info) { LOG.info(b.toString()); From d5e97fe4d6baf43a5576cbd1700c22b788dba01e Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Fri, 8 Apr 2022 23:06:39 +0530 Subject: [PATCH 101/145] HDFS-16473. Make HDFS stat tool cross platform (#4145) * The source files for hdfs_stat uses getopt for parsing the command line arguments. * getopt is available only on Linux and thus, isn't cross platform. * We need to replace getopt with boost::program_options to make this tool cross platform. --- .../libhdfspp/tests/tools/CMakeLists.txt | 3 + .../libhdfspp/tests/tools/hdfs-stat-mock.cc | 55 +++++++++ .../libhdfspp/tests/tools/hdfs-stat-mock.h | 67 +++++++++++ .../libhdfspp/tests/tools/hdfs-tool-tests.cc | 17 +++ .../native/libhdfspp/tools/CMakeLists.txt | 3 +- .../libhdfspp/tools/hdfs-stat/CMakeLists.txt | 27 +++++ .../libhdfspp/tools/hdfs-stat/hdfs-stat.cc | 111 ++++++++++++++++++ .../libhdfspp/tools/hdfs-stat/hdfs-stat.h | 90 ++++++++++++++ .../native/libhdfspp/tools/hdfs-stat/main.cc | 52 ++++++++ .../main/native/libhdfspp/tools/hdfs_stat.cc | 87 -------------- 10 files changed, 423 insertions(+), 89 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/CMakeLists.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/main.cc delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_stat.cc diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt index bec9343903923..a5364d334fb74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt @@ -39,6 +39,7 @@ add_executable(hdfs_tool_tests hdfs-find-mock.cc hdfs-ls-mock.cc hdfs-setrep-mock.cc + hdfs-stat-mock.cc main.cc) target_include_directories(hdfs_tool_tests PRIVATE ../tools @@ -62,6 +63,7 @@ target_include_directories(hdfs_tool_tests PRIVATE ../../tools/hdfs-find ../../tools/hdfs-ls ../../tools/hdfs-setrep + ../../tools/hdfs-stat ../../tools/hdfs-cat) target_link_libraries(hdfs_tool_tests PRIVATE gmock_main @@ -84,5 +86,6 @@ target_link_libraries(hdfs_tool_tests PRIVATE hdfs_find_lib hdfs_ls_lib hdfs_setrep_lib + hdfs_stat_lib hdfs_cat_lib) add_test(hdfs_tool_tests hdfs_tool_tests) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.cc new file mode 100644 index 0000000000000..efa773ce70fdc --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.cc @@ -0,0 +1,55 @@ +/** + * 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. + */ + +#include +#include +#include +#include + +#include +#include + +#include "hdfs-stat-mock.h" +#include "hdfs-tool-tests.h" + +namespace hdfs::tools::test { +StatMock::~StatMock() = default; + +void StatMock::SetExpectations( + std::function()> test_case, + const std::vector &args) const { + // Get the pointer to the function that defines the test case + const auto test_case_func = + test_case.target (*)()>(); + ASSERT_NE(test_case_func, nullptr); + + // Set the expected method calls and their corresponding arguments for each + // test case + if (*test_case_func == &CallHelp) { + EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true)); + return; + } + + if (*test_case_func == &PassAPath) { + const auto path = args[0]; + EXPECT_CALL(*this, HandlePath(path)) + .Times(1) + .WillOnce(testing::Return(true)); + } +} +} // namespace hdfs::tools::test diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.h new file mode 100644 index 0000000000000..01781f5f1751c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-stat-mock.h @@ -0,0 +1,67 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_STAT_MOCK +#define LIBHDFSPP_TOOLS_HDFS_STAT_MOCK + +#include +#include +#include +#include + +#include + +#include "hdfs-stat.h" + +namespace hdfs::tools::test { +/** + * {@class StatMock} is an {@class Stat} whereby it mocks the + * HandleHelp and HandlePath methods for testing their functionality. + */ +class StatMock : public hdfs::tools::Stat { +public: + /** + * {@inheritdoc} + */ + StatMock(const int argc, char **argv) : Stat(argc, argv) {} + + // Abiding to the Rule of 5 + StatMock(const StatMock &) = delete; + StatMock(StatMock &&) = delete; + StatMock &operator=(const StatMock &) = delete; + StatMock &operator=(StatMock &&) = delete; + ~StatMock() override; + + /** + * Defines the methods and the corresponding arguments that are expected + * to be called on this instance of {@link HdfsTool} for the given test case. + * + * @param test_case An {@link std::function} object that points to the + * function defining the test case + * @param args The arguments that are passed to this test case + */ + void SetExpectations(std::function()> test_case, + const std::vector &args = {}) const; + + MOCK_METHOD(bool, HandleHelp, (), (const, override)); + + MOCK_METHOD(bool, HandlePath, (const std::string &), (const, override)); +}; +} // namespace hdfs::tools::test + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc index 1bdf82f9afd3b..6f6104ea6e202 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc @@ -39,6 +39,7 @@ #include "hdfs-rename-snapshot-mock.h" #include "hdfs-rm-mock.h" #include "hdfs-setrep-mock.h" +#include "hdfs-stat-mock.h" #include "hdfs-tool-test-fixtures.h" #include "hdfs-tool-tests.h" @@ -162,6 +163,11 @@ INSTANTIATE_TEST_SUITE_P( testing::Values(CallHelp, PassPermissionsAndAPath)); +INSTANTIATE_TEST_SUITE_P( + HdfsStat, HdfsToolBasicTest, + testing::Values(CallHelp, + PassAPath)); + // Negative tests INSTANTIATE_TEST_SUITE_P( HdfsAllowSnapshot, HdfsToolNegativeTestThrows, @@ -265,6 +271,17 @@ INSTANTIATE_TEST_SUITE_P( PassMOpt, PassNOpt)); +INSTANTIATE_TEST_SUITE_P( + HdfsStat, HdfsToolNegativeTestThrows, + testing::Values(Pass2Paths, + Pass3Paths, + PassRecursiveOwnerAndAPath, + PassRecursive, + PassRecursivePath, + PassMPOptsPermissionsAndAPath, + PassMOpt, + PassNOpt)); + INSTANTIATE_TEST_SUITE_P( HdfsRm, HdfsToolNegativeTestNoThrow, testing::Values(PassRecursive)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt index 0d9a684c8caa5..bda2c1886ea43 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt @@ -49,8 +49,7 @@ add_subdirectory(hdfs-rm) add_subdirectory(hdfs-ls) -add_executable(hdfs_stat hdfs_stat.cc) -target_link_libraries(hdfs_stat tools_common hdfspp_static) +add_subdirectory(hdfs-stat) add_subdirectory(hdfs-count) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/CMakeLists.txt new file mode 100644 index 0000000000000..3a390cc2ed20a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/CMakeLists.txt @@ -0,0 +1,27 @@ +# +# 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. +# + +add_library(hdfs_stat_lib STATIC $ hdfs-stat.cc) +target_include_directories(hdfs_stat_lib PRIVATE ../../tools ${Boost_INCLUDE_DIRS}) +target_link_libraries(hdfs_stat_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static) + +add_executable(hdfs_stat main.cc) +target_include_directories(hdfs_stat PRIVATE ../../tools) +target_link_libraries(hdfs_stat PRIVATE hdfs_stat_lib) + +install(TARGETS hdfs_stat RUNTIME DESTINATION bin) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.cc new file mode 100644 index 0000000000000..e20fc4576d31c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.cc @@ -0,0 +1,111 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include + +#include "hdfs-stat.h" +#include "tools_common.h" + +namespace hdfs::tools { +Stat::Stat(const int argc, char **argv) : HdfsTool(argc, argv) {} + +bool Stat::Initialize() { + auto add_options = opt_desc_.add_options(); + add_options("help,h", "Displays the stat information for the given path. The " + "path can be a file or a directory."); + add_options("path", po::value(), + "The path in the filesystem for which to display the " + "stat information."); + + // We allow only one positional argument to be passed to this tool. An + // exception is thrown if multiple arguments are passed. + pos_opt_desc_.add("path", 1); + + po::store(po::command_line_parser(argc_, argv_) + .options(opt_desc_) + .positional(pos_opt_desc_) + .run(), + opt_val_); + po::notify(opt_val_); + return true; +} + +std::string Stat::GetDescription() const { + std::stringstream desc; + desc << "Usage: hdfs_stat PATH" << std::endl + << std::endl + << "Displays the stat information for the given path." << std::endl + << "The path can be a file or a directory." << std::endl + << "Examples:" << std::endl + << "hdfs_stat hdfs://localhost.localdomain:8020/dir/file" << std::endl; + return desc.str(); +} + +bool Stat::Do() { + if (!Initialize()) { + std::cerr << "Unable to initialize HDFS stat tool" << std::endl; + return false; + } + + if (!ValidateConstraints()) { + std::cout << GetDescription(); + return false; + } + + if (opt_val_.count("help") > 0) { + return HandleHelp(); + } + + if (opt_val_.count("path") > 0) { + const auto path = opt_val_["path"].as(); + return HandlePath(path); + } + + return false; +} + +bool Stat::HandleHelp() const { + std::cout << GetDescription(); + return true; +} + +bool Stat::HandlePath(const std::string &path) const { + // Building a URI object from the given uri_path + auto uri = hdfs::parse_path_or_exit(path); + + const auto fs = hdfs::doConnect(uri, false); + if (!fs) { + std::cerr << "Could not connect the file system. " << std::endl; + return false; + } + + hdfs::StatInfo stat_info; + const auto status = fs->GetFileInfo(uri.get_path(), stat_info); + if (!status.ok()) { + std::cerr << "Error: " << status.ToString() << std::endl; + return false; + } + std::cout << stat_info.str() << std::endl; + return true; +} +} // namespace hdfs::tools diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.h new file mode 100644 index 0000000000000..a23affc1c461c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/hdfs-stat.h @@ -0,0 +1,90 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_STAT +#define LIBHDFSPP_TOOLS_HDFS_STAT + +#include + +#include + +#include "hdfs-tool.h" + +namespace hdfs::tools { +/** + * {@class Stat} is an {@class HdfsTool} that displays the stat information for + * the given path. The path can be a file or a directory. + */ +class Stat : public HdfsTool { +public: + /** + * {@inheritdoc} + */ + Stat(int argc, char **argv); + + // Abiding to the Rule of 5 + Stat(const Stat &) = default; + Stat(Stat &&) = default; + Stat &operator=(const Stat &) = delete; + Stat &operator=(Stat &&) = delete; + ~Stat() override = default; + + /** + * {@inheritdoc} + */ + [[nodiscard]] std::string GetDescription() const override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Do() override; + +protected: + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Initialize() override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool ValidateConstraints() const override { return argc_ > 1; } + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool HandleHelp() const override; + + /** + * Handle the path argument that's passed to this tool. + * + * @param path The path to the directory for which we need the stat info. + * + * @return A boolean indicating the result of this operation. + */ + [[nodiscard]] virtual bool HandlePath(const std::string &path) const; + +private: + /** + * A boost data-structure containing the description of positional arguments + * passed to the command-line. + */ + po::positional_options_description pos_opt_desc_; +}; +} // namespace hdfs::tools +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/main.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/main.cc new file mode 100644 index 0000000000000..4437419259ca0 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-stat/main.cc @@ -0,0 +1,52 @@ +/** + * 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. + */ + +#include +#include +#include + +#include + +#include "hdfs-stat.h" + +int main(int argc, char *argv[]) { + const auto result = std::atexit([]() -> void { + // Clean up static data on exit and prevent valgrind memory leaks + google::protobuf::ShutdownProtobufLibrary(); + }); + if (result != 0) { + std::cerr << "Error: Unable to schedule the clean-up tasks for HDFS stat" + "tool, exiting" + << std::endl; + std::exit(EXIT_FAILURE); + } + + hdfs::tools::Stat stat(argc, argv); + auto success = false; + + try { + success = stat.Do(); + } catch (const std::exception &e) { + std::cerr << "Error: " << e.what() << std::endl; + } + + if (!success) { + std::exit(EXIT_FAILURE); + } + return 0; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_stat.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_stat.cc deleted file mode 100644 index 59d513b21afd3..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_stat.cc +++ /dev/null @@ -1,87 +0,0 @@ -/* - 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. -*/ - -#include -#include -#include "tools_common.h" - -void usage(){ - std::cout << "Usage: hdfs_rm [OPTION] FILE" - << std::endl - << std::endl << "Display FILE status." - << std::endl - << std::endl << " -h display this help and exit" - << std::endl - << std::endl << "Examples:" - << std::endl << "hdfs_rm hdfs://localhost.localdomain:8020/dir/file" - << std::endl << "hdfs_rm -R /dir1/dir2" - << std::endl; -} - -int main(int argc, char *argv[]) { - //We should have at least 2 arguments - if (argc < 2) { - usage(); - exit(EXIT_FAILURE); - } - - int input; - - //Using GetOpt to read in the values - opterr = 0; - while ((input = getopt(argc, argv, "h")) != -1) { - switch (input) - { - case 'h': - usage(); - exit(EXIT_SUCCESS); - case '?': - if (isprint(optopt)) - std::cerr << "Unknown option `-" << (char) optopt << "'." << std::endl; - else - std::cerr << "Unknown option character `" << (char) optopt << "'." << std::endl; - usage(); - exit(EXIT_FAILURE); - default: - exit(EXIT_FAILURE); - } - } - std::string uri_path = argv[optind]; - - //Building a URI object from the given uri_path - hdfs::URI uri = hdfs::parse_path_or_exit(uri_path); - - std::shared_ptr fs = hdfs::doConnect(uri, false); - if (!fs) { - std::cerr << "Could not connect the file system. " << std::endl; - exit(EXIT_FAILURE); - } - - hdfs::StatInfo stat_info; - hdfs::Status status = fs->GetFileInfo(uri.get_path(), stat_info); - if (!status.ok()) { - std::cerr << "Error: " << status.ToString() << std::endl; - exit(EXIT_FAILURE); - } - std::cout << stat_info.str() << std::endl; - - // Clean up static data and prevent valgrind memory leaks - google::protobuf::ShutdownProtobufLibrary(); - return 0; -} From 5de78ceb0e7f522289ad7d008a42db529f9a6700 Mon Sep 17 00:00:00 2001 From: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Date: Mon, 11 Apr 2022 15:54:00 +0800 Subject: [PATCH 102/145] HDFS-16516. Fix Fsshell wrong params (#4090). Contributed by GuoPhilipse. --- .../hadoop-common/src/site/markdown/FileSystemShell.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md index 331cf872bf2bc..9a690a8c5ccdd 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/FileSystemShell.md @@ -328,16 +328,16 @@ Returns 0 on success and -1 on error. get --- -Usage: `hadoop fs -get [-ignorecrc] [-crc] [-p] [-f] [-t ] [-q ] ... ` +Usage: `hadoop fs -get [-ignoreCrc] [-crc] [-p] [-f] [-t ] [-q ] ... ` -Copy files to the local file system. Files that fail the CRC check may be copied with the -ignorecrc option. Files and CRCs may be copied using the -crc option. +Copy files to the local file system. Files that fail the CRC check may be copied with the -ignoreCrc option. Files and CRCs may be copied using the -crc option. Options: * `-p` : Preserves access and modification times, ownership and the permissions. (assuming the permissions can be propagated across filesystems) * `-f` : Overwrites the destination if it already exists. -* `-ignorecrc` : Skip CRC checks on the file(s) downloaded. +* `-ignoreCrc` : Skip CRC checks on the file(s) downloaded. * `-crc`: write CRC checksums for the files downloaded. * `-t ` : Number of threads to be used, default is 1. Useful when downloading directories containing more than 1 file. From e89fd9645bbd2568d7785d70f9df3a53f4a57015 Mon Sep 17 00:00:00 2001 From: Gautham B A Date: Tue, 12 Apr 2022 20:31:24 +0530 Subject: [PATCH 103/145] HDFS-16474. Make HDFS tail tool cross platform (#4157) * The source files for hdfs_tail uses getopt for parsing the command line arguments. * getopt is available only on Linux and thus, isn't cross platform. * We need to replace getopt with boost::program_options to make these tools cross platform. --- .../libhdfspp/tests/tools/CMakeLists.txt | 3 + .../libhdfspp/tests/tools/hdfs-tail-mock.cc | 64 ++++++++ .../libhdfspp/tests/tools/hdfs-tail-mock.h | 68 ++++++++ .../libhdfspp/tests/tools/hdfs-tool-tests.cc | 25 +++ .../libhdfspp/tests/tools/hdfs-tool-tests.h | 25 +++ .../native/libhdfspp/tools/CMakeLists.txt | 24 +-- .../libhdfspp/tools/hdfs-tail/CMakeLists.txt | 27 ++++ .../libhdfspp/tools/hdfs-tail/hdfs-tail.cc | 150 ++++++++++++++++++ .../libhdfspp/tools/hdfs-tail/hdfs-tail.h | 102 ++++++++++++ .../native/libhdfspp/tools/hdfs-tail/main.cc | 52 ++++++ .../main/native/libhdfspp/tools/hdfs_tail.cc | 124 --------------- 11 files changed, 517 insertions(+), 147 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/CMakeLists.txt create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/main.cc delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_tail.cc diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt index a5364d334fb74..7bbe63153b77c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/CMakeLists.txt @@ -40,6 +40,7 @@ add_executable(hdfs_tool_tests hdfs-ls-mock.cc hdfs-setrep-mock.cc hdfs-stat-mock.cc + hdfs-tail-mock.cc main.cc) target_include_directories(hdfs_tool_tests PRIVATE ../tools @@ -64,6 +65,7 @@ target_include_directories(hdfs_tool_tests PRIVATE ../../tools/hdfs-ls ../../tools/hdfs-setrep ../../tools/hdfs-stat + ../../tools/hdfs-tail ../../tools/hdfs-cat) target_link_libraries(hdfs_tool_tests PRIVATE gmock_main @@ -87,5 +89,6 @@ target_link_libraries(hdfs_tool_tests PRIVATE hdfs_ls_lib hdfs_setrep_lib hdfs_stat_lib + hdfs_tail_lib hdfs_cat_lib) add_test(hdfs_tool_tests hdfs_tool_tests) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.cc new file mode 100644 index 0000000000000..59c9f3a23b371 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.cc @@ -0,0 +1,64 @@ +/** + * 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. + */ + +#include +#include +#include +#include + +#include +#include + +#include "hdfs-tail-mock.h" +#include "hdfs-tool-tests.h" + +namespace hdfs::tools::test { +TailMock::~TailMock() = default; + +void TailMock::SetExpectations( + std::function()> test_case, + const std::vector &args) const { + // Get the pointer to the function that defines the test case + const auto test_case_func = + test_case.target (*)()>(); + ASSERT_NE(test_case_func, nullptr); + + // Set the expected method calls and their corresponding arguments for each + // test case + if (*test_case_func == &CallHelp) { + EXPECT_CALL(*this, HandleHelp()).Times(1).WillOnce(testing::Return(true)); + return; + } + + if (*test_case_func == &PassAPath) { + const auto path = args[0]; + EXPECT_CALL(*this, HandlePath(path, false)) + .Times(1) + .WillOnce(testing::Return(true)); + } + + if (*test_case_func == &PassFOptAndAPath) { + const auto f_opt = args[0]; + const auto path = args[1]; + ASSERT_EQ(f_opt, "-f"); + EXPECT_CALL(*this, HandlePath(path, true)) + .Times(1) + .WillOnce(testing::Return(true)); + } +} +} // namespace hdfs::tools::test diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.h new file mode 100644 index 0000000000000..e9fb9b7521bfa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tail-mock.h @@ -0,0 +1,68 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_TAIL_MOCK +#define LIBHDFSPP_TOOLS_HDFS_TAIL_MOCK + +#include +#include +#include +#include + +#include + +#include "hdfs-tail.h" + +namespace hdfs::tools::test { +/** + * {@class TailMock} is an {@class Tail} whereby it mocks the + * HandleHelp and HandlePath methods for testing their functionality. + */ +class TailMock : public hdfs::tools::Tail { +public: + /** + * {@inheritdoc} + */ + TailMock(const int argc, char **argv) : Tail(argc, argv) {} + + // Abiding to the Rule of 5 + TailMock(const TailMock &) = delete; + TailMock(TailMock &&) = delete; + TailMock &operator=(const TailMock &) = delete; + TailMock &operator=(TailMock &&) = delete; + ~TailMock() override; + + /** + * Defines the methods and the corresponding arguments that are expected + * to be called on this instance of {@link HdfsTool} for the given test case. + * + * @param test_case An {@link std::function} object that points to the + * function defining the test case + * @param args The arguments that are passed to this test case + */ + void SetExpectations(std::function()> test_case, + const std::vector &args = {}) const; + + MOCK_METHOD(bool, HandleHelp, (), (const, override)); + + MOCK_METHOD(bool, HandlePath, (const std::string &, const bool), + (const, override)); +}; +} // namespace hdfs::tools::test + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc index 6f6104ea6e202..2a16c3c0d97f0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.cc @@ -40,6 +40,7 @@ #include "hdfs-rm-mock.h" #include "hdfs-setrep-mock.h" #include "hdfs-stat-mock.h" +#include "hdfs-tail-mock.h" #include "hdfs-tool-test-fixtures.h" #include "hdfs-tool-tests.h" @@ -168,6 +169,12 @@ INSTANTIATE_TEST_SUITE_P( testing::Values(CallHelp, PassAPath)); +INSTANTIATE_TEST_SUITE_P( + HdfsTail, HdfsToolBasicTest, + testing::Values(PassAPath, + CallHelp, + PassFOptAndAPath)); + // Negative tests INSTANTIATE_TEST_SUITE_P( HdfsAllowSnapshot, HdfsToolNegativeTestThrows, @@ -282,6 +289,20 @@ INSTANTIATE_TEST_SUITE_P( PassMOpt, PassNOpt)); +INSTANTIATE_TEST_SUITE_P( + HdfsTail, HdfsToolNegativeTestThrows, + testing::Values(Pass2Paths, + Pass3Paths, + PassNOptAndAPath, + PassRecursiveOwnerAndAPath, + PassMOpt, + PassRecursive, + PassRecursivePath, + PassNOpt, + PassOwnerAndAPath, + PassMPOptsPermissionsAndAPath, + PassPermissionsAndAPath)); + INSTANTIATE_TEST_SUITE_P( HdfsRm, HdfsToolNegativeTestNoThrow, testing::Values(PassRecursive)); @@ -341,3 +362,7 @@ INSTANTIATE_TEST_SUITE_P( INSTANTIATE_TEST_SUITE_P( HdfsSetrep, HdfsToolNegativeTestNoThrow, testing::Values(PassAPath)); + +INSTANTIATE_TEST_SUITE_P( + HdfsTail, HdfsToolNegativeTestNoThrow, + testing::Values(PassFOpt)); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h index f27a2b022da68..4fef261d0e72f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h @@ -69,6 +69,19 @@ template std::unique_ptr PassRecursivePath() { return hdfs_tool; } +template std::unique_ptr PassFOptAndAPath() { + constexpr auto argc = 3; + static std::string exe("hdfs_tool_name"); + static std::string arg1("-f"); + static std::string arg2("a/b/c"); + + static char *argv[] = {exe.data(), arg1.data(), arg2.data()}; + + auto hdfs_tool = std::make_unique(argc, argv); + hdfs_tool->SetExpectations(PassFOptAndAPath, {arg1, arg2}); + return hdfs_tool; +} + template std::unique_ptr CallHelp() { constexpr auto argc = 2; static std::string exe("hdfs_tool_name"); @@ -226,6 +239,18 @@ template std::unique_ptr PassMOpt() { return hdfs_tool; } +template std::unique_ptr PassFOpt() { + constexpr auto argc = 2; + static std::string exe("hdfs_tool_name"); + static std::string arg1("-f"); + + static char *argv[] = {exe.data(), arg1.data()}; + + auto hdfs_tool = std::make_unique(argc, argv); + hdfs_tool->SetExpectations(PassFOpt, {arg1}); + return hdfs_tool; +} + template std::unique_ptr PassPOptAndPath() { constexpr auto argc = 3; static std::string exe("hdfs_tool_name"); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt index bda2c1886ea43..52caf01915653 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/CMakeLists.txt @@ -34,46 +34,24 @@ add_library(hdfs_tool_obj OBJECT hdfs-tool.cc) target_include_directories(hdfs_tool_obj PRIVATE ../tools) add_subdirectory(hdfs-cat) - add_subdirectory(hdfs-chgrp) - add_subdirectory(hdfs-chown) - add_subdirectory(hdfs-chmod) - add_subdirectory(hdfs-find) - add_subdirectory(hdfs-mkdir) - add_subdirectory(hdfs-rm) - add_subdirectory(hdfs-ls) - add_subdirectory(hdfs-stat) - add_subdirectory(hdfs-count) - add_subdirectory(hdfs-df) - add_subdirectory(hdfs-du) - add_subdirectory(hdfs-get) - add_subdirectory(hdfs-copy-to-local) - add_subdirectory(hdfs-move-to-local) - add_subdirectory(hdfs-setrep) - add_subdirectory(hdfs-allow-snapshot) - add_subdirectory(hdfs-disallow-snapshot) - add_subdirectory(hdfs-create-snapshot) - add_subdirectory(hdfs-rename-snapshot) - add_subdirectory(hdfs-delete-snapshot) - -add_executable(hdfs_tail hdfs_tail.cc) -target_link_libraries(hdfs_tail tools_common hdfspp_static) +add_subdirectory(hdfs-tail) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/CMakeLists.txt new file mode 100644 index 0000000000000..5169e00f1996f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/CMakeLists.txt @@ -0,0 +1,27 @@ +# +# 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. +# + +add_library(hdfs_tail_lib STATIC $ hdfs-tail.cc) +target_include_directories(hdfs_tail_lib PRIVATE ../../tools ${Boost_INCLUDE_DIRS}) +target_link_libraries(hdfs_tail_lib PRIVATE Boost::boost Boost::program_options tools_common hdfspp_static) + +add_executable(hdfs_tail main.cc) +target_include_directories(hdfs_tail PRIVATE ../../tools) +target_link_libraries(hdfs_tail PRIVATE hdfs_tail_lib) + +install(TARGETS hdfs_tail RUNTIME DESTINATION bin) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.cc new file mode 100644 index 0000000000000..7eaa4602c894f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.cc @@ -0,0 +1,150 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include +#include +#include + +#include "hdfs-tail.h" +#include "tools_common.h" + +namespace hdfs::tools { +Tail::Tail(const int argc, char **argv) : HdfsTool(argc, argv) {} + +bool Tail::Initialize() { + auto add_options = opt_desc_.add_options(); + add_options("help,h", "Displays last kilobyte of the file to stdout."); + add_options("follow,f", + "Append data to the output as the file grows, as in Unix."); + add_options("path", po::value(), + "The path indicating the filesystem that needs to be tailed."); + + // We allow only one positional argument to be passed to this tool. An + // exception is thrown if multiple arguments are passed. + pos_opt_desc_.add("path", 1); + + po::store(po::command_line_parser(argc_, argv_) + .options(opt_desc_) + .positional(pos_opt_desc_) + .run(), + opt_val_); + po::notify(opt_val_); + return true; +} + +std::string Tail::GetDescription() const { + std::stringstream desc; + desc << "Usage: hdfs_tail [OPTION] FILE" << std::endl + << std::endl + << "Displays last kilobyte of the file to stdout." << std::endl + << std::endl + << " -f append data to the output as the file grows, as in Unix" + << std::endl + << " -h display this help and exit" << std::endl + << std::endl + << "Examples:" << std::endl + << "hdfs_tail hdfs://localhost.localdomain:8020/dir/file" << std::endl + << "hdfs_tail /dir/file" << std::endl; + return desc.str(); +} + +bool Tail::Do() { + if (!Initialize()) { + std::cerr << "Unable to initialize HDFS tail tool" << std::endl; + return false; + } + + if (!ValidateConstraints()) { + std::cout << GetDescription(); + return false; + } + + if (opt_val_.count("help") > 0) { + return HandleHelp(); + } + + if (opt_val_.count("path") > 0) { + const auto path = opt_val_["path"].as(); + const auto follow = opt_val_.count("follow") > 0; + return HandlePath(path, follow); + } + + return false; +} + +bool Tail::HandleHelp() const { + std::cout << GetDescription(); + return true; +} + +bool Tail::HandlePath(const std::string &path, const bool follow) const { + // Building a URI object from the given path. + auto uri = hdfs::parse_path_or_exit(path); + + const auto fs = hdfs::doConnect(uri, true); + if (!fs) { + std::cerr << "Could not connect to the file system." << std::endl; + return false; + } + + // We need to get the size of the file using stat. + hdfs::StatInfo stat_info; + auto status = fs->GetFileInfo(uri.get_path(), stat_info); + if (!status.ok()) { + std::cerr << "Error: " << status.ToString() << std::endl; + return false; + } + + // Determine where to start reading. + off_t offset{0}; + if (stat_info.length > tail_size_in_bytes) { + offset = static_cast(stat_info.length - tail_size_in_bytes); + } + + do { + const auto current_length = static_cast(stat_info.length); + readFile(fs, uri.get_path(), offset, stdout, false); + + // Exit if -f flag was not set. + if (!follow) { + break; + } + + do { + // Sleep for the refresh rate. + std::this_thread::sleep_for(std::chrono::seconds(refresh_rate_in_sec)); + + // Use stat to check the new file size. + status = fs->GetFileInfo(uri.get_path(), stat_info); + if (!status.ok()) { + std::cerr << "Error: " << status.ToString() << std::endl; + return false; + } + + // If file became longer, loop back and print the difference. + } while (static_cast(stat_info.length) <= current_length); + } while (true); + + return true; +} +} // namespace hdfs::tools diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.h new file mode 100644 index 0000000000000..f103d0d5b9470 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/hdfs-tail.h @@ -0,0 +1,102 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_TOOLS_HDFS_TAIL +#define LIBHDFSPP_TOOLS_HDFS_TAIL + +#include + +#include + +#include "hdfs-tool.h" + +namespace hdfs::tools { +/** + * {@class Tail} is an {@class HdfsTool} displays last kilobyte of the file to + * stdout. + */ +class Tail : public HdfsTool { +public: + /** + * {@inheritdoc} + */ + Tail(int argc, char **argv); + + // Abiding to the Rule of 5 + Tail(const Tail &) = default; + Tail(Tail &&) = default; + Tail &operator=(const Tail &) = delete; + Tail &operator=(Tail &&) = delete; + ~Tail() override = default; + + /** + * {@inheritdoc} + */ + [[nodiscard]] std::string GetDescription() const override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Do() override; + +protected: + /** + * {@inheritdoc} + */ + [[nodiscard]] bool Initialize() override; + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool ValidateConstraints() const override { return argc_ > 1; } + + /** + * {@inheritdoc} + */ + [[nodiscard]] bool HandleHelp() const override; + + /** + * Handle the path argument that's passed to this tool. + * + * @param path The path to the file which needs to be tailed. + * @param follow Append data to the output as the file grows, as in Unix. + * + * @return A boolean indicating the result of this operation. + */ + [[nodiscard]] virtual bool HandlePath(const std::string &path, + bool follow) const; + + /** + * The tail size in bytes. + */ + static constexpr uint64_t tail_size_in_bytes{1024}; + + /** + * The refresh rate for {@link hdfs::tools::Tail} in seconds. + */ + static constexpr int refresh_rate_in_sec{1}; + +private: + /** + * A boost data-structure containing the description of positional arguments + * passed to the command-line. + */ + po::positional_options_description pos_opt_desc_; +}; +} // namespace hdfs::tools +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/main.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/main.cc new file mode 100644 index 0000000000000..fb85915e1d3f2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-tail/main.cc @@ -0,0 +1,52 @@ +/** + * 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. + */ + +#include +#include +#include + +#include + +#include "hdfs-tail.h" + +int main(int argc, char *argv[]) { + const auto result = std::atexit([]() -> void { + // Clean up static data on exit and prevent valgrind memory leaks + google::protobuf::ShutdownProtobufLibrary(); + }); + if (result != 0) { + std::cerr << "Error: Unable to schedule clean-up tasks for HDFS tail tool, " + "exiting" + << std::endl; + std::exit(EXIT_FAILURE); + } + + hdfs::tools::Tail tail(argc, argv); + auto success = false; + + try { + success = tail.Do(); + } catch (const std::exception &e) { + std::cerr << "Error: " << e.what() << std::endl; + } + + if (!success) { + std::exit(EXIT_FAILURE); + } + return 0; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_tail.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_tail.cc deleted file mode 100644 index 8125bdcfecfeb..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs_tail.cc +++ /dev/null @@ -1,124 +0,0 @@ -/* - 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. -*/ - -#include -#include -#include "tools_common.h" - -void usage(){ - std::cout << "Usage: hdfs_tail [OPTION] FILE" - << std::endl - << std::endl << "Displays last kilobyte of the file to stdout." - << std::endl - << std::endl << " -f output appended data as the file grows, as in Unix" - << std::endl << " -h display this help and exit" - << std::endl - << std::endl << "Examples:" - << std::endl << "hdfs_tail hdfs://localhost.localdomain:8020/dir/file" - << std::endl << "hdfs_tail /dir/file" - << std::endl; -} - -#define TAIL_SIZE 1024 -#define REFRESH_RATE 1 //seconds - -int main(int argc, char *argv[]) { - if (argc < 2) { - usage(); - exit(EXIT_FAILURE); - } - - bool follow = false; - int input; - - //Using GetOpt to read in the values - opterr = 0; - while ((input = getopt(argc, argv, "hf")) != -1) { - switch (input) - { - case 'h': - usage(); - exit(EXIT_SUCCESS); - case 'f': - follow = true; - break; - case '?': - if (isprint(optopt)) - std::cerr << "Unknown option `-" << (char) optopt << "'." << std::endl; - else - std::cerr << "Unknown option character `" << (char) optopt << "'." << std::endl; - usage(); - exit(EXIT_FAILURE); - default: - exit(EXIT_FAILURE); - } - } - - std::string uri_path = argv[optind]; - - //Building a URI object from the given uri_path - hdfs::URI uri = hdfs::parse_path_or_exit(uri_path); - - std::shared_ptr fs = hdfs::doConnect(uri, false); - if (!fs) { - std::cerr << "Could not connect the file system. " << std::endl; - exit(EXIT_FAILURE); - } - - //We need to get the size of the file using stat - hdfs::StatInfo stat_info; - hdfs::Status status = fs->GetFileInfo(uri.get_path(), stat_info); - if (!status.ok()) { - std::cerr << "Error: " << status.ToString() << std::endl; - exit(EXIT_FAILURE); - } - - //Determine where to start reading - off_t offset = 0; - if(stat_info.length > TAIL_SIZE){ - offset = stat_info.length - TAIL_SIZE; - } - - do { - off_t current_length = (off_t) stat_info.length; - readFile(fs, uri.get_path(), offset, stdout, false); - - //Exit if -f flag was not set - if(!follow){ - break; - } - - do{ - //Sleep for the REFRESH_RATE - sleep(REFRESH_RATE); - //Use stat to check the new filesize. - status = fs->GetFileInfo(uri.get_path(), stat_info); - if (!status.ok()) { - std::cerr << "Error: " << status.ToString() << std::endl; - exit(EXIT_FAILURE); - } - //If file became longer, loop back and print the difference - } - while((off_t) stat_info.length <= current_length); - } while (true); - - // Clean up static data and prevent valgrind memory leaks - google::protobuf::ShutdownProtobufLibrary(); - return 0; -} From 45394433a112334e48087bd60674538af739922a Mon Sep 17 00:00:00 2001 From: qinyuren <1476659627@qq.com> Date: Wed, 13 Apr 2022 10:27:37 +0800 Subject: [PATCH 104/145] HDFS-16484. [SPS]: Fix an infinite loop bug in SPSPathIdProcessor thread (#4032) --- .../sps/BlockStorageMovementNeeded.java | 16 +++++- .../TestExternalStoragePolicySatisfier.java | 53 +++++++++++++++++++ 2 files changed, 68 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java index 9a1faed952595..b6fc15d2104a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.namenode.sps; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.HashMap; import java.util.LinkedList; @@ -227,15 +228,18 @@ public synchronized void clearQueuesWithNotification() { * ID's to process for satisfy the policy. */ private class SPSPathIdProcessor implements Runnable { + private static final int MAX_RETRY_COUNT = 3; @Override public void run() { LOG.info("Starting SPSPathIdProcessor!."); Long startINode = null; + int retryCount = 0; while (ctxt.isRunning()) { try { if (!ctxt.isInSafeMode()) { if (startINode == null) { + retryCount = 0; startINode = ctxt.getNextSPSPath(); } // else same id will be retried if (startINode == null) { @@ -248,7 +252,12 @@ public void run() { pendingWorkForDirectory.get(startINode); if (dirPendingWorkInfo != null && dirPendingWorkInfo.isDirWorkDone()) { - ctxt.removeSPSHint(startINode); + try { + ctxt.removeSPSHint(startINode); + } catch (FileNotFoundException e) { + // ignore if the file doesn't already exist + startINode = null; + } pendingWorkForDirectory.remove(startINode); } } @@ -268,6 +277,11 @@ public void run() { LOG.info("Interrupted while waiting in SPSPathIdProcessor", t); break; } + retryCount++; + if (retryCount >= MAX_RETRY_COUNT) { + LOG.warn("Skipping this inode {} due to too many retries.", startINode); + startINode = null; + } } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java index 9b4dc437ce414..2559836a0a171 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java @@ -201,6 +201,20 @@ private void shutdownCluster() { } } + private void stopExternalSps() { + if (externalSps != null) { + externalSps.stopGracefully(); + } + } + + private void startExternalSps() { + externalSps = new StoragePolicySatisfier(getConf()); + externalCtxt = new ExternalSPSContext(externalSps, nnc); + + externalSps.init(externalCtxt); + externalSps.start(StoragePolicySatisfierMode.EXTERNAL); + } + private void createCluster() throws IOException { getConf().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE); setCluster(startCluster(getConf(), allDiskTypes, NUM_OF_DATANODES, @@ -1445,6 +1459,45 @@ public void testSPSWithDirectoryTreeWithoutFile() throws Exception { } } + /** + * Test SPS that satisfy the files and then delete the files before start SPS. + */ + @Test(timeout = 300000) + public void testSPSSatisfyAndThenDeleteFileBeforeStartSPS() throws Exception { + try { + createCluster(); + HdfsAdmin hdfsAdmin = + new HdfsAdmin(FileSystem.getDefaultUri(config), config); + + StorageType[][] newtypes = + new StorageType[][]{{StorageType.DISK, StorageType.ARCHIVE}, + {StorageType.DISK, StorageType.ARCHIVE}, + {StorageType.DISK, StorageType.ARCHIVE}}; + startAdditionalDNs(config, 3, NUM_OF_DATANODES, newtypes, + STORAGES_PER_DATANODE, CAPACITY, hdfsCluster); + + stopExternalSps(); + + dfs.setStoragePolicy(new Path(FILE), COLD); + hdfsAdmin.satisfyStoragePolicy(new Path(FILE)); + dfs.delete(new Path(FILE), true); + + startExternalSps(); + + String file1 = "/testMoveToSatisfyStoragePolicy_1"; + writeContent(file1); + dfs.setStoragePolicy(new Path(file1), COLD); + hdfsAdmin.satisfyStoragePolicy(new Path(file1)); + + hdfsCluster.triggerHeartbeats(); + DFSTestUtil.waitExpectedStorageType(file1, StorageType.ARCHIVE, 3, 30000, + dfs); + } finally { + shutdownCluster(); + } + } + + /** * Test SPS for directory which has multilevel directories. */ From a7b4e8f03e55e2b5862c1a7328a15bd08f4e0279 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 13 Apr 2022 14:15:27 +0100 Subject: [PATCH 105/145] HDFS-14478. Add libhdfs APIs for openFile (#4166) Contributed by Sahil Takiar --- .../native/libhdfs-tests/test_libhdfs_ops.c | 62 +++ .../src/main/native/libhdfs/hdfs.c | 500 +++++++++++++++++- .../main/native/libhdfs/include/hdfs/hdfs.h | 135 +++++ .../src/main/native/libhdfs/jclasses.c | 4 + .../src/main/native/libhdfs/jclasses.h | 7 + .../main/native/libhdfspp/tests/hdfs_shim.c | 59 +++ .../libhdfspp/tests/libhdfs_wrapper_defines.h | 17 + .../libhdfspp/tests/libhdfs_wrapper_undefs.h | 17 + .../tests/libhdfspp_wrapper_defines.h | 17 + 9 files changed, 807 insertions(+), 11 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_ops.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_ops.c index a3058bbe6ec06..359cc2f4e3269 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_ops.c +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/test_libhdfs_ops.c @@ -454,6 +454,68 @@ int main(int argc, char **argv) { hdfsCloseFile(lfs, localFile); } + + { + // HDFS Open File Builder tests + + exists = hdfsExists(fs, readPath); + + if (exists) { + fprintf(stderr, "Failed to validate existence of %s\n", readPath); + shutdown_and_exit(cl, -1); + } + + hdfsOpenFileBuilder *builder; + builder = hdfsOpenFileBuilderAlloc(fs, readPath); + hdfsOpenFileBuilderOpt(builder, "hello", "world"); + + hdfsOpenFileFuture *future; + future = hdfsOpenFileBuilderBuild(builder); + + readFile = hdfsOpenFileFutureGet(future); + if (!hdfsOpenFileFutureCancel(future, 0)) { + fprintf(stderr, "Cancel on a completed Future should return false"); + shutdown_and_exit(cl, -1); + } + hdfsOpenFileFutureFree(future); + + memset(buffer, 0, sizeof(buffer)); + num_read_bytes = hdfsRead(fs, readFile, (void *) buffer, + sizeof(buffer)); + if (strncmp(fileContents, buffer, strlen(fileContents)) != 0) { + fprintf(stderr, + "Failed to read. Expected %s but got %s (%d bytes)\n", + fileContents, buffer, num_read_bytes); + shutdown_and_exit(cl, -1); + } + hdfsCloseFile(fs, readFile); + + builder = hdfsOpenFileBuilderAlloc(fs, readPath); + hdfsOpenFileBuilderOpt(builder, "hello", "world"); + + future = hdfsOpenFileBuilderBuild(builder); + + readFile = hdfsOpenFileFutureGetWithTimeout(future, 1, jDays); + if (!hdfsOpenFileFutureCancel(future, 0)) { + fprintf(stderr, "Cancel on a completed Future should return " + "false"); + shutdown_and_exit(cl, -1); + } + hdfsOpenFileFutureFree(future); + + memset(buffer, 0, sizeof(buffer)); + num_read_bytes = hdfsRead(fs, readFile, (void*)buffer, + sizeof(buffer)); + if (strncmp(fileContents, buffer, strlen(fileContents)) != 0) { + fprintf(stderr, "Failed to read. Expected %s but got " + "%s (%d bytes)\n", fileContents, buffer, + num_read_bytes); + shutdown_and_exit(cl, -1); + } + memset(buffer, 0, strlen(fileContents + 1)); + hdfsCloseFile(fs, readFile); + } + totalResult = 0; result = 0; { diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c index 60f2826c74173..ed150925cdb81 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/hdfs.c @@ -38,6 +38,10 @@ #define KERBEROS_TICKET_CACHE_PATH "hadoop.security.kerberos.ticket.cache.path" +// StreamCapability flags taken from o.a.h.fs.StreamCapabilities +#define IS_READ_BYTE_BUFFER_CAPABILITY "in:readbytebuffer" +#define IS_PREAD_BYTE_BUFFER_CAPABILITY "in:preadbytebuffer" + // Bit fields for hdfsFile_internal flags #define HDFS_FILE_SUPPORTS_DIRECT_READ (1<<0) #define HDFS_FILE_SUPPORTS_DIRECT_PREAD (1<<1) @@ -1075,6 +1079,27 @@ static int hdfsHasStreamCapability(jobject jFile, return 0; } +/** + * Sets the flags of the given hdfsFile based on the capabilities of the + * underlying stream. + * + * @param file file->flags will be updated based on the capabilities of jFile + * @param jFile the underlying stream to check for capabilities + */ +static void setFileFlagCapabilities(hdfsFile file, jobject jFile) { + // Check the StreamCapabilities of jFile to see if we can do direct + // reads + if (hdfsHasStreamCapability(jFile, IS_READ_BYTE_BUFFER_CAPABILITY)) { + file->flags |= HDFS_FILE_SUPPORTS_DIRECT_READ; + } + + // Check the StreamCapabilities of jFile to see if we can do direct + // preads + if (hdfsHasStreamCapability(jFile, IS_PREAD_BYTE_BUFFER_CAPABILITY)) { + file->flags |= HDFS_FILE_SUPPORTS_DIRECT_PREAD; + } +} + static hdfsFile hdfsOpenFileImpl(hdfsFS fs, const char *path, int flags, int32_t bufferSize, int16_t replication, int64_t blockSize) { @@ -1245,17 +1270,7 @@ static hdfsFile hdfsOpenFileImpl(hdfsFS fs, const char *path, int flags, file->flags = 0; if ((flags & O_WRONLY) == 0) { - // Check the StreamCapabilities of jFile to see if we can do direct - // reads - if (hdfsHasStreamCapability(jFile, "in:readbytebuffer")) { - file->flags |= HDFS_FILE_SUPPORTS_DIRECT_READ; - } - - // Check the StreamCapabilities of jFile to see if we can do direct - // preads - if (hdfsHasStreamCapability(jFile, "in:preadbytebuffer")) { - file->flags |= HDFS_FILE_SUPPORTS_DIRECT_PREAD; - } + setFileFlagCapabilities(file, jFile); } ret = 0; @@ -1288,6 +1303,469 @@ hdfsFile hdfsStreamBuilderBuild(struct hdfsStreamBuilder *bld) return file; } +/** + * A wrapper around o.a.h.fs.FutureDataInputStreamBuilder and the file name + * associated with the builder. + */ +struct hdfsOpenFileBuilder { + jobject jBuilder; + const char *path; +}; + +/** + * A wrapper around a java.util.concurrent.Future (created by calling + * FutureDataInputStreamBuilder#build) and the file name associated with the + * builder. + */ +struct hdfsOpenFileFuture { + jobject jFuture; + const char *path; +}; + +hdfsOpenFileBuilder *hdfsOpenFileBuilderAlloc(hdfsFS fs, + const char *path) { + int ret = 0; + jthrowable jthr; + jvalue jVal; + jobject jFS = (jobject) fs; + + jobject jPath = NULL; + jobject jBuilder = NULL; + + JNIEnv *env = getJNIEnv(); + if (!env) { + errno = EINTERNAL; + return NULL; + } + + hdfsOpenFileBuilder *builder; + builder = calloc(1, sizeof(hdfsOpenFileBuilder)); + if (!builder) { + fprintf(stderr, "hdfsOpenFileBuilderAlloc(%s): OOM when creating " + "hdfsOpenFileBuilder\n", path); + errno = ENOMEM; + goto done; + } + builder->path = path; + + jthr = constructNewObjectOfPath(env, path, &jPath); + if (jthr) { + errno = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileBuilderAlloc(%s): constructNewObjectOfPath", + path); + goto done; + } + + jthr = invokeMethod(env, &jVal, INSTANCE, jFS, JC_FILE_SYSTEM, + "openFile", JMETHOD1(JPARAM(HADOOP_PATH), JPARAM(HADOOP_FDISB)), + jPath); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileBuilderAlloc(%s): %s#openFile(Path) failed", + HADOOP_FS, path); + goto done; + } + jBuilder = jVal.l; + + builder->jBuilder = (*env)->NewGlobalRef(env, jBuilder); + if (!builder->jBuilder) { + printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "hdfsOpenFileBuilderAlloc(%s): NewGlobalRef(%s) failed", path, + HADOOP_FDISB); + ret = EINVAL; + goto done; + } + +done: + destroyLocalReference(env, jPath); + destroyLocalReference(env, jBuilder); + if (ret) { + if (builder) { + if (builder->jBuilder) { + (*env)->DeleteGlobalRef(env, builder->jBuilder); + } + free(builder); + } + errno = ret; + return NULL; + } + return builder; +} + +/** + * Used internally by hdfsOpenFileBuilderWithOption to switch between + * FSBuilder#must and #opt. + */ +typedef enum { must, opt } openFileBuilderOptionType; + +/** + * Shared implementation of hdfsOpenFileBuilderMust and hdfsOpenFileBuilderOpt + * that switches between each method depending on the value of + * openFileBuilderOptionType. + */ +static hdfsOpenFileBuilder *hdfsOpenFileBuilderWithOption( + hdfsOpenFileBuilder *builder, const char *key, + const char *value, openFileBuilderOptionType optionType) { + int ret = 0; + jthrowable jthr; + jvalue jVal; + jobject localJBuilder = NULL; + jobject globalJBuilder; + jstring jKeyString = NULL; + jstring jValueString = NULL; + + // If the builder was not previously created by a prior call to + // hdfsOpenFileBuilderAlloc then exit + if (builder == NULL || builder->jBuilder == NULL) { + errno = EINVAL; + return NULL; + } + + JNIEnv *env = getJNIEnv(); + if (!env) { + errno = EINTERNAL; + return NULL; + } + jthr = newJavaStr(env, key, &jKeyString); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileBuilderWithOption(%s): newJavaStr(%s)", + builder->path, key); + goto done; + } + jthr = newJavaStr(env, value, &jValueString); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileBuilderWithOption(%s): newJavaStr(%s)", + builder->path, value); + goto done; + } + + const char *optionTypeMethodName; + switch (optionType) { + case must: + optionTypeMethodName = "must"; + break; + case opt: + optionTypeMethodName = "opt"; + break; + default: + ret = EINTERNAL; + goto done; + } + + jthr = invokeMethod(env, &jVal, INSTANCE, builder->jBuilder, + JC_FUTURE_DATA_IS_BUILDER, optionTypeMethodName, + JMETHOD2(JPARAM(JAVA_STRING), JPARAM(JAVA_STRING), + JPARAM(HADOOP_FS_BLDR)), jKeyString, + jValueString); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileBuilderWithOption(%s): %s#%s(%s, %s) failed", + builder->path, HADOOP_FS_BLDR, optionTypeMethodName, key, + value); + goto done; + } + + localJBuilder = jVal.l; + globalJBuilder = (*env)->NewGlobalRef(env, localJBuilder); + if (!globalJBuilder) { + printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "hdfsOpenFileBuilderWithOption(%s): NewGlobalRef(%s) failed", + builder->path, HADOOP_FDISB); + ret = EINVAL; + goto done; + } + (*env)->DeleteGlobalRef(env, builder->jBuilder); + builder->jBuilder = globalJBuilder; + +done: + destroyLocalReference(env, jKeyString); + destroyLocalReference(env, jValueString); + destroyLocalReference(env, localJBuilder); + if (ret) { + errno = ret; + return NULL; + } + return builder; +} + +hdfsOpenFileBuilder *hdfsOpenFileBuilderMust(hdfsOpenFileBuilder *builder, + const char *key, const char *value) { + openFileBuilderOptionType optionType; + optionType = must; + return hdfsOpenFileBuilderWithOption(builder, key, value, optionType); +} + +hdfsOpenFileBuilder *hdfsOpenFileBuilderOpt(hdfsOpenFileBuilder *builder, + const char *key, const char *value) { + openFileBuilderOptionType optionType; + optionType = opt; + return hdfsOpenFileBuilderWithOption(builder, key, value, optionType); +} + +hdfsOpenFileFuture *hdfsOpenFileBuilderBuild(hdfsOpenFileBuilder *builder) { + int ret = 0; + jthrowable jthr; + jvalue jVal; + + jobject jFuture = NULL; + + // If the builder was not previously created by a prior call to + // hdfsOpenFileBuilderAlloc then exit + if (builder == NULL || builder->jBuilder == NULL) { + ret = EINVAL; + return NULL; + } + + JNIEnv *env = getJNIEnv(); + if (!env) { + errno = EINTERNAL; + return NULL; + } + + hdfsOpenFileFuture *future; + future = calloc(1, sizeof(hdfsOpenFileFuture)); + if (!future) { + fprintf(stderr, "hdfsOpenFileBuilderBuild: OOM when creating " + "hdfsOpenFileFuture\n"); + errno = ENOMEM; + goto done; + } + future->path = builder->path; + + jthr = invokeMethod(env, &jVal, INSTANCE, builder->jBuilder, + JC_FUTURE_DATA_IS_BUILDER, "build", + JMETHOD1("", JPARAM(JAVA_CFUTURE))); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileBuilderBuild(%s): %s#build() failed", + builder->path, HADOOP_FDISB); + goto done; + } + jFuture = jVal.l; + + future->jFuture = (*env)->NewGlobalRef(env, jFuture); + if (!future->jFuture) { + printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "hdfsOpenFileBuilderBuild(%s): NewGlobalRef(%s) failed", + builder->path, JAVA_CFUTURE); + ret = EINVAL; + goto done; + } + +done: + destroyLocalReference(env, jFuture); + if (ret) { + if (future) { + if (future->jFuture) { + (*env)->DeleteGlobalRef(env, future->jFuture); + } + free(future); + } + hdfsOpenFileBuilderFree(builder); + errno = ret; + return NULL; + } + hdfsOpenFileBuilderFree(builder); + return future; +} + +void hdfsOpenFileBuilderFree(hdfsOpenFileBuilder *builder) { + JNIEnv *env; + env = getJNIEnv(); + if (!env) { + return; + } + if (builder->jBuilder) { + (*env)->DeleteGlobalRef(env, builder->jBuilder); + builder->jBuilder = NULL; + } + free(builder); +} + +/** + * Shared implementation of hdfsOpenFileFutureGet and + * hdfsOpenFileFutureGetWithTimeout. If a timeout is specified, calls + * Future#get() otherwise it calls Future#get(long, TimeUnit). + */ +static hdfsFile fileFutureGetWithTimeout(hdfsOpenFileFuture *future, + int64_t timeout, jobject jTimeUnit) { + int ret = 0; + jthrowable jthr; + jvalue jVal; + + hdfsFile file = NULL; + jobject jFile = NULL; + + JNIEnv *env = getJNIEnv(); + if (!env) { + ret = EINTERNAL; + return NULL; + } + + if (!jTimeUnit) { + jthr = invokeMethod(env, &jVal, INSTANCE, future->jFuture, + JC_CFUTURE, "get", JMETHOD1("", JPARAM(JAVA_OBJECT))); + } else { + jthr = invokeMethod(env, &jVal, INSTANCE, future->jFuture, + JC_CFUTURE, "get", JMETHOD2("J", + JPARAM(JAVA_TIMEUNIT), JPARAM(JAVA_OBJECT)), timeout, + jTimeUnit); + } + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileFutureGet(%s): %s#get failed", future->path, + JAVA_CFUTURE); + goto done; + } + + file = calloc(1, sizeof(struct hdfsFile_internal)); + if (!file) { + fprintf(stderr, "hdfsOpenFileFutureGet(%s): OOM when creating " + "hdfsFile\n", future->path); + ret = ENOMEM; + goto done; + } + jFile = jVal.l; + file->file = (*env)->NewGlobalRef(env, jFile); + if (!file->file) { + ret = printPendingExceptionAndFree(env, PRINT_EXC_ALL, + "hdfsOpenFileFutureGet(%s): NewGlobalRef(jFile) failed", + future->path); + goto done; + } + + file->type = HDFS_STREAM_INPUT; + file->flags = 0; + + setFileFlagCapabilities(file, jFile); + +done: + destroyLocalReference(env, jTimeUnit); + destroyLocalReference(env, jFile); + if (ret) { + if (file) { + if (file->file) { + (*env)->DeleteGlobalRef(env, file->file); + } + free(file); + } + errno = ret; + return NULL; + } + return file; +} + +hdfsFile hdfsOpenFileFutureGet(hdfsOpenFileFuture *future) { + return fileFutureGetWithTimeout(future, -1, NULL); +} + +hdfsFile hdfsOpenFileFutureGetWithTimeout(hdfsOpenFileFuture *future, + int64_t timeout, javaConcurrentTimeUnit timeUnit) { + int ret = 0; + jthrowable jthr; + jobject jTimeUnit = NULL; + + JNIEnv *env = getJNIEnv(); + if (!env) { + ret = EINTERNAL; + return NULL; + } + + const char *timeUnitEnumName; + switch (timeUnit) { + case jNanoseconds: + timeUnitEnumName = "NANOSECONDS"; + break; + case jMicroseconds: + timeUnitEnumName = "MICROSECONDS"; + break; + case jMilliseconds: + timeUnitEnumName = "MILLISECONDS"; + break; + case jSeconds: + timeUnitEnumName = "SECONDS"; + break; + case jMinutes: + timeUnitEnumName = "MINUTES"; + break; + case jHours: + timeUnitEnumName = "HOURS"; + break; + case jDays: + timeUnitEnumName = "DAYS"; + break; + default: + ret = EINTERNAL; + goto done; + } + + jthr = fetchEnumInstance(env, JAVA_TIMEUNIT, timeUnitEnumName, &jTimeUnit); + + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileFutureGet(%s): %s#get failed", future->path, + JAVA_CFUTURE); + goto done; + } + return fileFutureGetWithTimeout(future, timeout, jTimeUnit); + +done: + if (ret) { + errno = ret; + } + return NULL; +} + +int hdfsOpenFileFutureCancel(hdfsOpenFileFuture *future, + int mayInterruptIfRunning) { + int ret = 0; + jthrowable jthr; + jvalue jVal; + + jboolean jMayInterruptIfRunning; + + JNIEnv *env = getJNIEnv(); + if (!env) { + ret = EINTERNAL; + return -1; + } + + jMayInterruptIfRunning = mayInterruptIfRunning ? JNI_TRUE : JNI_FALSE; + jthr = invokeMethod(env, &jVal, INSTANCE, future->jFuture, JC_CFUTURE, + "cancel", JMETHOD1("Z", "Z"), jMayInterruptIfRunning); + if (jthr) { + ret = printExceptionAndFree(env, jthr, PRINT_EXC_ALL, + "hdfsOpenFileFutureCancel(%s): %s#cancel failed", future->path, + JAVA_CFUTURE); + goto done; + } + +done: + if (ret) { + errno = ret; + return -1; + } + if (!jVal.z) { + return -1; + } + return 0; +} + +void hdfsOpenFileFutureFree(hdfsOpenFileFuture *future) { + JNIEnv *env; + env = getJNIEnv(); + if (!env) { + return; + } + if (future->jFuture) { + (*env)->DeleteGlobalRef(env, future->jFuture); + future->jFuture = NULL; + } + free(future); +} + int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength) { jobject jFS = (jobject)fs; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h index e58a6232d205a..eba50ff6eb277 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/include/hdfs/hdfs.h @@ -82,6 +82,29 @@ extern "C" { } tObjectKind; struct hdfsStreamBuilder; + /** + * The C reflection of the enum values from java.util.concurrent.TimeUnit . + */ + typedef enum javaConcurrentTimeUnit { + jNanoseconds, + jMicroseconds, + jMilliseconds, + jSeconds, + jMinutes, + jHours, + jDays, + } javaConcurrentTimeUnit; + + /** + * The C reflection of java.util.concurrent.Future specifically used for + * opening HDFS files asynchronously. + */ + typedef struct hdfsOpenFileFuture hdfsOpenFileFuture; + + /** + * The C reflection of o.a.h.fs.FutureDataInputStreamBuilder . + */ + typedef struct hdfsOpenFileBuilder hdfsOpenFileBuilder; /** * The C reflection of org.apache.org.hadoop.FileSystem . @@ -429,6 +452,118 @@ extern "C" { hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags, int bufferSize, short replication, tSize blocksize); + /** + * hdfsOpenFileBuilderAlloc - Allocate a HDFS open file builder. + * + * @param fs The configured filesystem handle. + * @param path The full path to the file. + * @return Returns the hdfsOpenFileBuilder, or NULL on error. + */ + LIBHDFS_EXTERNAL + hdfsOpenFileBuilder *hdfsOpenFileBuilderAlloc(hdfsFS fs, + const char *path); + + /** + * hdfsOpenFileBuilderMust - Specifies a mandatory parameter for the open + * file builder. While the underlying FsBuilder supports various various + * types for the value (boolean, int, float, double), currently only + * strings are supported. + * + * @param builder The open file builder to set the config for. + * @param key The config key + * @param value The config value + * @return Returns the hdfsOpenFileBuilder, or NULL on error. + */ + LIBHDFS_EXTERNAL + hdfsOpenFileBuilder *hdfsOpenFileBuilderMust(hdfsOpenFileBuilder *builder, + const char *key, const char *value); + + /** + * hdfsOpenFileBuilderOpt - Specifies an optional parameter for the open + * file builder. While the underlying FsBuilder supports various various + * types for the value (boolean, int, float, double), currently only + * strings are supported. + * + * @param builder The open file builder to set the config for. + * @param key The config key + * @param value The config value + * @return Returns the hdfsOpenFileBuilder, or NULL on error. + */ + LIBHDFS_EXTERNAL + hdfsOpenFileBuilder *hdfsOpenFileBuilderOpt(hdfsOpenFileBuilder *builder, + const char *key, const char *value); + + /** + * hdfsOpenFileBuilderBuild - Builds the open file builder and returns a + * hdfsOpenFileFuture which tracks the asynchronous call to open the + * specified file. + * + * @param builder The open file builder to build. + * @return Returns the hdfsOpenFileFuture, or NULL on error. + */ + LIBHDFS_EXTERNAL + hdfsOpenFileFuture *hdfsOpenFileBuilderBuild(hdfsOpenFileBuilder *builder); + + /** + * hdfsOpenFileBuilderFree - Free a HDFS open file builder. + * + * It is normally not necessary to call this function since + * hdfsOpenFileBuilderBuild frees the builder. + * + * @param builder The hdfsOpenFileBuilder to free. + */ + LIBHDFS_EXTERNAL + void hdfsOpenFileBuilderFree(hdfsOpenFileBuilder *builder); + + /** + * hdfsOpenFileFutureGet - Call Future#get() on the underlying Java Future + * object. A call to #get() will block until the asynchronous operation has + * completed. In this case, until the open file call has completed. This + * method blocks indefinitely until blocking call completes. + * + * @param future The hdfsOpenFileFuture to call #get on + * @return Returns the opened hdfsFile, or NULL on error. + */ + LIBHDFS_EXTERNAL + hdfsFile hdfsOpenFileFutureGet(hdfsOpenFileFuture *future); + + /** + * hdfsOpenFileFutureGetWithTimeout - Call Future#get(long, TimeUnit) on + * the underlying Java Future object. A call to #get(long, TimeUnit) will + * block until the asynchronous operation has completed (in this case, + * until the open file call has completed) or the specified timeout has + * been reached. + * + * @param future The hdfsOpenFileFuture to call #get on + * @return Returns the opened hdfsFile, or NULL on error or if the timeout + * has been reached. + */ + LIBHDFS_EXTERNAL + hdfsFile hdfsOpenFileFutureGetWithTimeout(hdfsOpenFileFuture *future, + int64_t timeout, javaConcurrentTimeUnit timeUnit); + + /** + * hdfsOpenFileFutureCancel - Call Future#cancel(boolean) on the + * underlying Java Future object. The value of mayInterruptedIfRunning + * controls whether the Java thread running the Future should be + * interrupted or not. + * + * @param future The hdfsOpenFileFuture to call #cancel on + * @param mayInterruptIfRunning if true, interrupts the running thread + * @return Returns 0 if the thread was successfully cancelled, else -1 + */ + LIBHDFS_EXTERNAL + int hdfsOpenFileFutureCancel(hdfsOpenFileFuture *future, + int mayInterruptIfRunning); + + /** + * hdfsOpenFileFutureFree - Free a HDFS open file future. + * + * @param hdfsOpenFileFuture The hdfsOpenFileFuture to free. + */ + LIBHDFS_EXTERNAL + void hdfsOpenFileFutureFree(hdfsOpenFileFuture *future); + /** * hdfsStreamBuilderAlloc - Allocate an HDFS stream builder. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.c index cf880e91b7596..9f589ac257aa1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.c +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.c @@ -98,6 +98,8 @@ jthrowable initCachedClasses(JNIEnv* env) { "org/apache/hadoop/hdfs/ReadStatistics"; cachedJavaClasses[JC_HDFS_DATA_INPUT_STREAM].className = "org/apache/hadoop/hdfs/client/HdfsDataInputStream"; + cachedJavaClasses[JC_FUTURE_DATA_IS_BUILDER].className = + "org/apache/hadoop/fs/FutureDataInputStreamBuilder"; cachedJavaClasses[JC_DOMAIN_SOCKET].className = "org/apache/hadoop/net/unix/DomainSocket"; cachedJavaClasses[JC_URI].className = @@ -108,6 +110,8 @@ jthrowable initCachedClasses(JNIEnv* env) { "java/util/EnumSet"; cachedJavaClasses[JC_EXCEPTION_UTILS].className = "org/apache/commons/lang3/exception/ExceptionUtils"; + cachedJavaClasses[JC_CFUTURE].className = + "java/util/concurrent/CompletableFuture"; // Create and set the jclass objects based on the class names set above jthrowable jthr; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.h index 92cdd542e2371..0b174e1fecc56 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jclasses.h @@ -54,11 +54,13 @@ typedef enum { JC_FS_PERMISSION, JC_READ_STATISTICS, JC_HDFS_DATA_INPUT_STREAM, + JC_FUTURE_DATA_IS_BUILDER, JC_DOMAIN_SOCKET, JC_URI, JC_BYTE_BUFFER, JC_ENUM_SET, JC_EXCEPTION_UTILS, + JC_CFUTURE, // A special marker enum that counts the number of cached jclasses NUM_CACHED_CLASSES } CachedJavaClass; @@ -95,6 +97,8 @@ const char *getClassName(CachedJavaClass cachedJavaClass); #define HADOOP_FSPERM "org/apache/hadoop/fs/permission/FsPermission" #define HADOOP_RSTAT "org/apache/hadoop/hdfs/ReadStatistics" #define HADOOP_HDISTRM "org/apache/hadoop/hdfs/client/HdfsDataInputStream" +#define HADOOP_FDISB "org/apache/hadoop/fs/FutureDataInputStreamBuilder" +#define HADOOP_FS_BLDR "org/apache/hadoop/fs/FSBuilder" #define HADOOP_RO "org/apache/hadoop/fs/ReadOption" #define HADOOP_DS "org/apache/hadoop/net/unix/DomainSocket" @@ -104,6 +108,9 @@ const char *getClassName(CachedJavaClass cachedJavaClass); #define JAVA_BYTEBUFFER "java/nio/ByteBuffer" #define JAVA_STRING "java/lang/String" #define JAVA_ENUMSET "java/util/EnumSet" +#define JAVA_CFUTURE "java/util/concurrent/CompletableFuture" +#define JAVA_TIMEUNIT "java/util/concurrent/TimeUnit" +#define JAVA_OBJECT "java/lang/Object" /* Some frequently used third-party class names */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c index bda27b9a43202..2d265b8f03c0c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_shim.c @@ -250,6 +250,65 @@ hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags, return ret; } +hdfsOpenFileBuilder *hdfsOpenFileBuilderAlloc(hdfsFS fs, + const char *path) { + return libhdfs_hdfsOpenFileBuilderAlloc(fs->libhdfsRep, path); +} + +hdfsOpenFileBuilder *hdfsOpenFileBuilderMust( + hdfsOpenFileBuilder *builder, const char *key, + const char *value) { + return libhdfs_hdfsOpenFileBuilderMust(builder, key, value); +} + +hdfsOpenFileBuilder *hdfsOpenFileBuilderOpt( + hdfsOpenFileBuilder *builder, const char *key, + const char *value) { + return libhdfs_hdfsOpenFileBuilderOpt(builder, key, value); +} + +hdfsOpenFileFuture *hdfsOpenFileBuilderBuild( + hdfsOpenFileBuilder *builder) { + return libhdfs_hdfsOpenFileBuilderBuild(builder); +} + +void hdfsOpenFileBuilderFree(hdfsOpenFileBuilder *builder) { + libhdfs_hdfsOpenFileBuilderFree(builder); +} + +hdfsFile hdfsOpenFileFutureGet(hdfsOpenFileFuture *future) { + hdfsFile ret = calloc(1, sizeof(struct hdfsFile_internal)); + ret->libhdfsppRep = 0; + ret->libhdfsRep = libhdfs_hdfsOpenFileFutureGet(future); + if (!ret->libhdfsRep) { + free(ret); + ret = NULL; + } + return ret; +} + +hdfsFile hdfsOpenFileFutureGetWithTimeout(hdfsOpenFileFuture *future, + int64_t timeout, javaConcurrentTimeUnit timeUnit) { + hdfsFile ret = calloc(1, sizeof(struct hdfsFile_internal)); + ret->libhdfsppRep = 0; + ret->libhdfsRep = libhdfs_hdfsOpenFileFutureGetWithTimeout(future, timeout, + timeUnit); + if (!ret->libhdfsRep) { + free(ret); + ret = NULL; + } + return ret; +} + +int hdfsOpenFileFutureCancel(hdfsOpenFileFuture *future, + int mayInterruptIfRunning) { + return libhdfs_hdfsOpenFileFutureCancel(future, mayInterruptIfRunning); +} + +void hdfsOpenFileFutureFree(hdfsOpenFileFuture *future) { + libhdfs_hdfsOpenFileFutureFree(future); +} + int hdfsTruncateFile(hdfsFS fs, const char* path, tOffset newlength) { return libhdfs_hdfsTruncateFile(fs->libhdfsRep, path, newlength); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_defines.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_defines.h index 0d014341b4c57..165744142558a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_defines.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_defines.h @@ -39,6 +39,23 @@ #define hdfsConfStrFree libhdfs_hdfsConfStrFree #define hdfsDisconnect libhdfs_hdfsDisconnect #define hdfsOpenFile libhdfs_hdfsOpenFile +#define hdfsOpenFileBuilderAlloc libhdfs_hdfsOpenFileBuilderAlloc +#define hdfsOpenFileBuilderMust libhdfs_hdfsOpenFileBuilderMust +#define hdfsOpenFileBuilderOpt libhdfs_hdfsOpenFileBuilderOpt +#define hdfsOpenFileBuilderBuild libhdfs_hdfsOpenFileBuilderBuild +#define hdfsOpenFileBuilderFree libhdfs_hdfsOpenFileBuilderFree +#define hdfsOpenFileFutureGet libhdfs_hdfsOpenFileFutureGet +#define javaConcurrentTimeUnit libhdfs_javaConcurrentTimeUnit +#define jNanoseconds libhdfs_jNanoseconds +#define jMicroseconds libhdfs_jMicroseconds +#define jMilliseconds libhdfs_jMilliseconds +#define jSeconds libhdfsj_jSeconds +#define jMinutes libhdfs_jMinutes +#define jHours libhdfs_jHours +#define jDays libhdfs_jDays +#define hdfsOpenFileFutureGetWithTimeout libhdfs_hdfsOpenFileFutureGetWithTimeout +#define hdfsOpenFileFutureCancel libhdfs_hdfsOpenFileFutureCancel +#define hdfsOpenFileFutureFree libhdfs_hdfsOpenFileFutureFree #define hdfsTruncateFile libhdfs_hdfsTruncateFile #define hdfsUnbufferFile libhdfs_hdfsUnbufferFile #define hdfsCloseFile libhdfs_hdfsCloseFile diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_undefs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_undefs.h index d46768c02ad39..d84b8ba287525 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_undefs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfs_wrapper_undefs.h @@ -39,6 +39,23 @@ #undef hdfsConfStrFree #undef hdfsDisconnect #undef hdfsOpenFile +#undef hdfsOpenFileBuilderAlloc +#undef hdfsOpenFileBuilderMust +#undef hdfsOpenFileBuilderOpt +#undef hdfsOpenFileBuilderBuild +#undef hdfsOpenFileBuilderFree +#undef hdfsOpenFileFutureGet +#undef javaConcurrentTimeUnit +#undef jNanoseconds +#undef jMicroseconds +#undef jMilliseconds +#undef jSeconds +#undef jMinutes +#undef jHours +#undef jDays +#undef hdfsOpenFileFutureGetWithTimeout +#undef hdfsOpenFileFutureCancel +#undef hdfsOpenFileFutureFree #undef hdfsTruncateFile #undef hdfsUnbufferFile #undef hdfsCloseFile diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfspp_wrapper_defines.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfspp_wrapper_defines.h index 4b08d0556c3aa..0a6d987409fec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfspp_wrapper_defines.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/libhdfspp_wrapper_defines.h @@ -39,6 +39,23 @@ #define hdfsConfStrFree libhdfspp_hdfsConfStrFree #define hdfsDisconnect libhdfspp_hdfsDisconnect #define hdfsOpenFile libhdfspp_hdfsOpenFile +#define hdfsOpenFileBuilderAlloc libhdfspp_hdfsOpenFileBuilderAlloc +#define hdfsOpenFileBuilderMust libhdfspp_hdfsOpenFileBuilderMust +#define hdfsOpenFileBuilderOpt libhdfspp_hdfsOpenFileBuilderOpt +#define hdfsOpenFileBuilderBuild libhdfspp_hdfsOpenFileBuilderBuild +#define hdfsOpenFileBuilderFree libhdfspp_hdfsOpenFileBuilderFree +#define hdfsOpenFileFutureGet libhdfspp_hdfsOpenFileFutureGet +#define javaConcurrentTimeUnit libhdfspp_javaConcurrentTimeUnit +#define jNanoseconds libhdfspp_jNanoseconds +#define jMicroseconds libhdfspp_jMicroseconds +#define jMilliseconds libhdfspp_jMilliseconds +#define jSeconds libhdfspp_jSeconds +#define jMinutes libhdfspp_jMinutes +#define jHours libhdfspp_jHours +#define jDays libhdfspp_jDays +#define hdfsOpenFileFutureGetWithTimeout libhdfspp_hdfsOpenFileFutureGetWithTimeout +#define hdfsOpenFileFutureCancel libhdfspp_hdfsOpenFileFutureCancel +#define hdfsOpenFileFutureFree libhdfspp_hdfsOpenFileFutureFree #define hdfsTruncateFile libhdfspp_hdfsTruncateFile #define hdfsUnbufferFile libhdfspp_hdfsUnbufferFile #define hdfsCloseFile libhdfspp_hdfsCloseFile From 3b46aae977e078cf7eb5e6bbbc55aca7cecee4c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=AB=A0=E9=94=A1=E5=B9=B3?= <40832063+zhangxiping1@users.noreply.github.com> Date: Fri, 8 Apr 2022 15:19:17 +0800 Subject: [PATCH 106/145] YARN-11107. When NodeLabel is enabled for a YARN cluster, AM blacklist program does not work properly. Contributed by zhangxiping1 --- .../resourcemanager/DefaultAMSProcessor.java | 16 +- .../TestApplicationMasterServiceCapacity.java | 163 ++++++++++++++++++ 2 files changed, 178 insertions(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java index 12a3b21682a60..3797a6ed3a189 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java @@ -351,7 +351,21 @@ public void allocate(ApplicationAttemptId appAttemptId, ((AbstractYarnScheduler)getScheduler()) .getApplicationAttempt(appAttemptId).pullUpdateContainerErrors()); - response.setNumClusterNodes(getScheduler().getNumClusterNodes()); + String label=""; + try { + label = rmContext.getScheduler() + .getQueueInfo(app.getQueue(), false, false) + .getDefaultNodeLabelExpression(); + } catch (Exception e){ + //Queue may not exist since it could be auto-created in case of + // dynamic queues + } + + if (label == null || label.equals("")) { + response.setNumClusterNodes(getScheduler().getNumClusterNodes()); + } else { + response.setNumClusterNodes(rmContext.getNodeLabelManager().getActiveNMCountPerLabel(label)); + } // add collector address for this application if (timelineServiceV2Enabled) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java index 182016a6cfab6..ccbc375359aee 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java @@ -16,36 +16,46 @@ package org.apache.hadoop.yarn.server.resourcemanager; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse; import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerUpdateType; +import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.UpdateContainerRequest; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer .RMContainerEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; +import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; + +import com.google.common.collect.ImmutableMap; import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSet; import static org.junit.Assert.fail; /** @@ -208,4 +218,157 @@ public void testPriorityInAllocatedResponse() throws Exception { Assert.assertEquals(appPriority2, response2.getApplicationPriority()); rm.stop(); } + + @Test(timeout = 300000) + public void testGetNMNumInAllocatedResponseWithOutNodeLabel() throws Exception { + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); + MockRM rm = new MockRM(conf); + rm.start(); + + // Register node1 node2 node3 node4 + MockNM nm1 = rm.registerNode("host1:1234", 6 * GB); + MockNM nm2 = rm.registerNode("host2:1234", 6 * GB); + MockNM nm3 = rm.registerNode("host3:1234", 6 * GB); + + // Submit an application + MockRMAppSubmissionData data = MockRMAppSubmissionData.Builder + .createWithMemory(2048, rm) + .build(); + RMApp app1 = MockRMAppSubmitter.submit(rm, data); + + nm1.nodeHeartbeat(true); + nm2.nodeHeartbeat(true); + nm3.nodeHeartbeat(true); + + RMAppAttempt attempt1 = app1.getCurrentAppAttempt(); + MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId()); + am1.registerAppAttempt(); + + AllocateRequestPBImpl allocateRequest = new AllocateRequestPBImpl(); + List release = new ArrayList(); + List ask = new ArrayList(); + allocateRequest.setReleaseList(release); + allocateRequest.setAskList(ask); + + AllocateResponse response1 = am1.allocate(allocateRequest); + Assert.assertEquals(3, response1.getNumClusterNodes()); + + rm.stop(); + } + + private Configuration getConfigurationWithQueueLabels(Configuration config) { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(config); + + // Define top-level queues + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"}); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100); + conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100); + + final String A = CapacitySchedulerConfiguration.ROOT + ".a"; + conf.setCapacity(A, 50); + conf.setMaximumCapacity(A, 100); + conf.setAccessibleNodeLabels(A, toSet("x")); + conf.setDefaultNodeLabelExpression(A, "x"); + conf.setCapacityByLabel(A, "x", 100); + + final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + conf.setCapacity(B, 50); + conf.setMaximumCapacity(B, 100); + conf.setAccessibleNodeLabels(B, toSet("y")); + conf.setDefaultNodeLabelExpression(B, "y"); + conf.setCapacityByLabel(B, "y", 100); + + return conf; + } + + @Test(timeout = 300000) + public void testGetNMNumInAllocatedResponseWithNodeLabel() throws Exception { + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, ResourceScheduler.class); + conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true); + MockRM rm = new MockRM(getConfigurationWithQueueLabels(conf)) { + @Override + protected RMNodeLabelsManager createNodeLabelManager() { + RMNodeLabelsManager mgr = new RMNodeLabelsManager(); + mgr.init(getConfig()); + return mgr; + } + }; + + // add node label "x","y" and set node to label mapping + Set clusterNodeLabels = new HashSet(); + clusterNodeLabels.add("x"); + clusterNodeLabels.add("y"); + + RMNodeLabelsManager nodeLabelManager = rm.getRMContext().getNodeLabelManager(); + nodeLabelManager. + addToCluserNodeLabelsWithDefaultExclusivity(clusterNodeLabels); + + //has 3 nodes with node label "x",1 node with node label "y" + nodeLabelManager + .addLabelsToNode(ImmutableMap.of(NodeId.newInstance("host1", 1234), toSet("x"))); + nodeLabelManager + .addLabelsToNode(ImmutableMap.of(NodeId.newInstance("host2", 1234), toSet("x"))); + nodeLabelManager + .addLabelsToNode(ImmutableMap.of(NodeId.newInstance("host3", 1234), toSet("x"))); + nodeLabelManager + .addLabelsToNode(ImmutableMap.of(NodeId.newInstance("host4", 1234), toSet("y"))); + rm.start(); + + // Register node1 node2 node3 node4 + MockNM nm1 = rm.registerNode("host1:1234", 6 * GB); + MockNM nm2 = rm.registerNode("host2:1234", 6 * GB); + MockNM nm3 = rm.registerNode("host3:1234", 6 * GB); + MockNM nm4 = rm.registerNode("host4:1234", 6 * GB); + + // submit an application to queue root.a expression as "x" + MockRMAppSubmissionData data1 = MockRMAppSubmissionData.Builder + .createWithMemory(2048, rm) + .withAppName("someApp1") + .withUser("someUser") + .withQueue("root.a") + .build(); + RMApp app1 = MockRMAppSubmitter.submit(rm, data1); + MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm1); + + // submit an application to queue root.b expression as "y" + MockRMAppSubmissionData data2 = MockRMAppSubmissionData.Builder + .createWithMemory(2048, rm) + .withAppName("someApp2") + .withUser("someUser") + .withQueue("root.b") + .build(); + RMApp app2 = MockRMAppSubmitter.submit(rm, data2); + MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm4); + + AllocateRequestPBImpl allocateRequest = new AllocateRequestPBImpl(); + List release = new ArrayList(); + List ask = new ArrayList(); + allocateRequest.setReleaseList(release); + allocateRequest.setAskList(ask); + + AllocateResponse response1 = am1.allocate(allocateRequest); + AllocateResponse response2 = am2.allocate(allocateRequest); + + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + RMNode rmNode1 = rm.getRMContext().getRMNodes().get(nm1.getNodeId()); + RMNode rmNode2 = rm.getRMContext().getRMNodes().get(nm2.getNodeId()); + RMNode rmNode3 = rm.getRMContext().getRMNodes().get(nm3.getNodeId()); + RMNode rmNode4 = rm.getRMContext().getRMNodes().get(nm4.getNodeId()); + + // Do node heartbeats many times + for (int i = 0; i < 3; i++) { + cs.handle(new NodeUpdateSchedulerEvent(rmNode1)); + cs.handle(new NodeUpdateSchedulerEvent(rmNode2)); + cs.handle(new NodeUpdateSchedulerEvent(rmNode3)); + cs.handle(new NodeUpdateSchedulerEvent(rmNode4)); + } + + //has 3 nodes with node label "x" + Assert.assertEquals(3, response1.getNumClusterNodes()); + + //has 1 node with node label "y" + Assert.assertEquals(1, response2.getNumClusterNodes()); + + rm.stop(); + } } From 2efab92959ca0a68c52ed6a9c721704e57afbcc7 Mon Sep 17 00:00:00 2001 From: Takanobu Asanuma Date: Thu, 14 Apr 2022 11:23:38 +0900 Subject: [PATCH 107/145] HDFS-16479. EC: NameNode should not send a reconstruction work when the source datanodes are insufficient (#4138) --- .../server/blockmanagement/BlockManager.java | 10 ++ .../blockmanagement/TestBlockManager.java | 96 +++++++++++++++++++ 2 files changed, 106 insertions(+) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 1d4937739dd64..59b3fb32e38a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -2163,6 +2163,16 @@ BlockReconstructionWork scheduleReconstruction(BlockInfo block, return null; } + // skip if source datanodes for reconstructing ec block are not enough + if (block.isStriped()) { + BlockInfoStriped stripedBlock = (BlockInfoStriped) block; + if (stripedBlock.getRealDataBlockNum() > srcNodes.length) { + LOG.debug("Block {} cannot be reconstructed due to shortage of source datanodes ", block); + NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled(); + return null; + } + } + // liveReplicaNodes can include READ_ONLY_SHARED replicas which are // not included in the numReplicas.liveReplicas() count assert liveReplicaNodes.size() >= numReplicas.liveReplicas(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java index d5e0a99fe789b..a507fce34f31a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java @@ -852,6 +852,102 @@ public void testChooseSrcDNWithDupECInDecommissioningNode() throws Exception { 0, numReplicas.redundantInternalBlocks()); } + @Test + public void testSkipReconstructionWithManyBusyNodes() { + long blockId = -9223372036854775776L; // real ec block id + // RS-3-2 EC policy + ErasureCodingPolicy ecPolicy = + SystemErasureCodingPolicies.getPolicies().get(1); + + // create an EC block group: 3 data blocks + 2 parity blocks + Block aBlockGroup = new Block(blockId, ecPolicy.getCellSize() * ecPolicy.getNumDataUnits(), 0); + BlockInfoStriped aBlockInfoStriped = new BlockInfoStriped(aBlockGroup, ecPolicy); + + // create 4 storageInfo, which means 1 block is missing + DatanodeStorageInfo ds1 = DFSTestUtil.createDatanodeStorageInfo( + "storage1", "1.1.1.1", "rack1", "host1"); + DatanodeStorageInfo ds2 = DFSTestUtil.createDatanodeStorageInfo( + "storage2", "2.2.2.2", "rack2", "host2"); + DatanodeStorageInfo ds3 = DFSTestUtil.createDatanodeStorageInfo( + "storage3", "3.3.3.3", "rack3", "host3"); + DatanodeStorageInfo ds4 = DFSTestUtil.createDatanodeStorageInfo( + "storage4", "4.4.4.4", "rack4", "host4"); + + // link block with storage + aBlockInfoStriped.addStorage(ds1, aBlockGroup); + aBlockInfoStriped.addStorage(ds2, new Block(blockId + 1, 0, 0)); + aBlockInfoStriped.addStorage(ds3, new Block(blockId + 2, 0, 0)); + aBlockInfoStriped.addStorage(ds4, new Block(blockId + 3, 0, 0)); + + addEcBlockToBM(blockId, ecPolicy); + aBlockInfoStriped.setBlockCollectionId(mockINodeId); + + // reconstruction should be scheduled + BlockReconstructionWork work = bm.scheduleReconstruction(aBlockInfoStriped, 3); + assertNotNull(work); + + // simulate the 2 nodes reach maxReplicationStreams + for(int i = 0; i < bm.maxReplicationStreams; i++){ + ds3.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); + ds4.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); + } + + // reconstruction should be skipped since the number of non-busy nodes are not enough + work = bm.scheduleReconstruction(aBlockInfoStriped, 3); + assertNull(work); + } + + @Test + public void testSkipReconstructionWithManyBusyNodes2() { + long blockId = -9223372036854775776L; // real ec block id + // RS-3-2 EC policy + ErasureCodingPolicy ecPolicy = + SystemErasureCodingPolicies.getPolicies().get(1); + + // create an EC block group: 2 data blocks + 2 parity blocks + Block aBlockGroup = new Block(blockId, + ecPolicy.getCellSize() * (ecPolicy.getNumDataUnits() - 1), 0); + BlockInfoStriped aBlockInfoStriped = new BlockInfoStriped(aBlockGroup, ecPolicy); + + // create 3 storageInfo, which means 1 block is missing + DatanodeStorageInfo ds1 = DFSTestUtil.createDatanodeStorageInfo( + "storage1", "1.1.1.1", "rack1", "host1"); + DatanodeStorageInfo ds2 = DFSTestUtil.createDatanodeStorageInfo( + "storage2", "2.2.2.2", "rack2", "host2"); + DatanodeStorageInfo ds3 = DFSTestUtil.createDatanodeStorageInfo( + "storage3", "3.3.3.3", "rack3", "host3"); + + // link block with storage + aBlockInfoStriped.addStorage(ds1, aBlockGroup); + aBlockInfoStriped.addStorage(ds2, new Block(blockId + 1, 0, 0)); + aBlockInfoStriped.addStorage(ds3, new Block(blockId + 2, 0, 0)); + + addEcBlockToBM(blockId, ecPolicy); + aBlockInfoStriped.setBlockCollectionId(mockINodeId); + + // reconstruction should be scheduled + BlockReconstructionWork work = bm.scheduleReconstruction(aBlockInfoStriped, 3); + assertNotNull(work); + + // simulate the 1 node reaches maxReplicationStreams + for(int i = 0; i < bm.maxReplicationStreams; i++){ + ds2.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); + } + + // reconstruction should still be scheduled since there are 2 source nodes to create 2 blocks + work = bm.scheduleReconstruction(aBlockInfoStriped, 3); + assertNotNull(work); + + // simulate the 1 more node reaches maxReplicationStreams + for(int i = 0; i < bm.maxReplicationStreams; i++){ + ds3.getDatanodeDescriptor().incrementPendingReplicationWithoutTargets(); + } + + // reconstruction should be skipped since the number of non-busy nodes are not enough + work = bm.scheduleReconstruction(aBlockInfoStriped, 3); + assertNull(work); + } + @Test public void testFavorDecomUntilHardLimit() throws Exception { bm.maxReplicationStreams = 0; From c65c383b7ebef48c638607f15ba35d61554982cb Mon Sep 17 00:00:00 2001 From: daimin Date: Thu, 14 Apr 2022 11:07:06 +0800 Subject: [PATCH 108/145] HDFS-16509. Fix decommission UnsupportedOperationException (#4077). Contributed by daimin. --- .../DatanodeAdminDefaultMonitor.java | 6 ++++-- .../apache/hadoop/hdfs/TestDecommission.java | 17 +++++++++++++++++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java index c1d62e35c0375..4149d198c72f8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java @@ -390,8 +390,10 @@ private void processBlocksInternal( // Remove the block from the list if it's no longer in the block map, // e.g. the containing file has been deleted if (blockManager.blocksMap.getStoredBlock(block) == null) { - LOG.trace("Removing unknown block {}", block); - it.remove(); + if (pruneReliableBlocks) { + LOG.trace("Removing unknown block {}", block); + it.remove(); + } continue; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java index f3b81acf37437..670ca5fd9a6fe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java @@ -54,6 +54,7 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries; import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; +import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates; @@ -64,6 +65,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFileEntry; import org.apache.hadoop.hdfs.protocol.OpenFilesIterator; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; @@ -673,6 +675,21 @@ public void testDecommissionWithOpenfile() fdos.close(); } + @Test(timeout = 20000) + public void testDecommissionWithUnknownBlock() throws IOException { + startCluster(1, 3); + + FSNamesystem ns = getCluster().getNamesystem(0); + DatanodeManager datanodeManager = ns.getBlockManager().getDatanodeManager(); + + BlockInfo blk = new BlockInfoContiguous(new Block(1L), (short) 1); + DatanodeDescriptor dn = datanodeManager.getDatanodes().iterator().next(); + dn.getStorageInfos()[0].addBlock(blk, blk); + + datanodeManager.getDatanodeAdminManager().startDecommission(dn); + waitNodeState(dn, DatanodeInfo.AdminStates.DECOMMISSIONED); + } + private static String scanIntoString(final ByteArrayOutputStream baos) { final TextStringBuilder sb = new TextStringBuilder(); final Scanner scanner = new Scanner(baos.toString()); From cee8c62498f55794f911ce62edfd4be9e88a7361 Mon Sep 17 00:00:00 2001 From: caozhiqiang Date: Thu, 14 Apr 2022 17:42:39 +0800 Subject: [PATCH 109/145] HDFS-16456. EC: Decommission a rack with only on dn will fail when the rack number is equal with replication (#4126) --- .../apache/hadoop/net/NetworkTopology.java | 115 +++++++++++++++- .../BlockPlacementPolicyDefault.java | 7 +- ...BlockPlacementPolicyRackFaultTolerant.java | 59 +++++---- .../blockmanagement/DatanodeAdminManager.java | 6 +- .../hdfs/server/namenode/FSNamesystem.java | 5 +- ...BlockPlacementPolicyRackFaultTolerant.java | 124 ++++++++++++++++++ .../hadoop/net/TestNetworkTopology.java | 16 +++ 7 files changed, 294 insertions(+), 38 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java index 6e0d88f2a7b7b..137c940001c0c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java @@ -101,6 +101,13 @@ protected NetworkTopology init(InnerNode.Factory factory) { private int depthOfAllLeaves = -1; /** rack counter */ protected int numOfRacks = 0; + /** empty rack map, rackname->nodenumber. */ + private HashMap> rackMap = + new HashMap>(); + /** decommission nodes, contained stoped nodes. */ + private HashSet decommissionNodes = new HashSet<>(); + /** empty rack counter. */ + private int numOfEmptyRacks = 0; /** * Whether or not this cluster has ever consisted of more than 1 rack, @@ -150,6 +157,7 @@ public void add(Node node) { if (rack == null) { incrementRacks(); } + interAddNodeWithEmptyRack(node); if (depthOfAllLeaves == -1) { depthOfAllLeaves = node.getLevel(); } @@ -224,6 +232,7 @@ public void remove(Node node) { if (rack == null) { numOfRacks--; } + interRemoveNodeWithEmptyRack(node); } LOG.debug("NetworkTopology became:\n{}", this); } finally { @@ -1015,4 +1024,108 @@ protected static boolean isNodeInScope(Node node, String scope) { String nodeLocation = NodeBase.getPath(node) + NodeBase.PATH_SEPARATOR_STR; return nodeLocation.startsWith(scope); } -} \ No newline at end of file + + /** @return the number of nonempty racks */ + public int getNumOfNonEmptyRacks() { + return numOfRacks - numOfEmptyRacks; + } + + /** + * Update empty rack number when add a node like recommission. + * @param node node to be added; can be null + */ + public void recommissionNode(Node node) { + if (node == null) { + return; + } + if (node instanceof InnerNode) { + throw new IllegalArgumentException( + "Not allow to remove an inner node: " + NodeBase.getPath(node)); + } + netlock.writeLock().lock(); + try { + decommissionNodes.remove(node.getName()); + interAddNodeWithEmptyRack(node); + } finally { + netlock.writeLock().unlock(); + } + } + + /** + * Update empty rack number when remove a node like decommission. + * @param node node to be added; can be null + */ + public void decommissionNode(Node node) { + if (node == null) { + return; + } + if (node instanceof InnerNode) { + throw new IllegalArgumentException( + "Not allow to remove an inner node: " + NodeBase.getPath(node)); + } + netlock.writeLock().lock(); + try { + decommissionNodes.add(node.getName()); + interRemoveNodeWithEmptyRack(node); + } finally { + netlock.writeLock().unlock(); + } + } + + /** + * Internal function for update empty rack number + * for add or recommission a node. + * @param node node to be added; can be null + */ + private void interAddNodeWithEmptyRack(Node node) { + if (node == null) { + return; + } + String rackname = node.getNetworkLocation(); + Set nodes = rackMap.get(rackname); + if (nodes == null) { + nodes = new HashSet(); + } + if (!decommissionNodes.contains(node.getName())) { + nodes.add(node.getName()); + } + rackMap.put(rackname, nodes); + countEmptyRacks(); + } + + /** + * Internal function for update empty rack number + * for remove or decommission a node. + * @param node node to be removed; can be null + */ + private void interRemoveNodeWithEmptyRack(Node node) { + if (node == null) { + return; + } + String rackname = node.getNetworkLocation(); + Set nodes = rackMap.get(rackname); + if (nodes != null) { + InnerNode rack = (InnerNode) getNode(node.getNetworkLocation()); + if (rack == null) { + // this node and its rack are both removed. + rackMap.remove(rackname); + } else if (nodes.contains(node.getName())) { + // this node is decommissioned or removed. + nodes.remove(node.getName()); + rackMap.put(rackname, nodes); + } + countEmptyRacks(); + } + } + + private void countEmptyRacks() { + int count = 0; + for (Set nodes : rackMap.values()) { + if (nodes != null && nodes.isEmpty()) { + count++; + } + } + numOfEmptyRacks = count; + LOG.debug("Current numOfEmptyRacks is {}", numOfEmptyRacks); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java index dec98d85b52df..39a40f52ae5f7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java @@ -304,7 +304,7 @@ private DatanodeStorageInfo[] chooseTarget(int numOfReplicas, && stats.isAvoidingStaleDataNodesForWrite()); boolean avoidLocalRack = (addBlockFlags != null && addBlockFlags.contains(AddBlockFlag.NO_LOCAL_RACK) && writer != null - && clusterMap.getNumOfRacks() > 2); + && clusterMap.getNumOfNonEmptyRacks() > 2); boolean avoidLocalNode = (addBlockFlags != null && addBlockFlags.contains(AddBlockFlag.NO_LOCAL_WRITE) && writer != null @@ -385,7 +385,7 @@ protected int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) { totalNumOfReplicas = clusterSize; } // No calculation needed when there is only one rack or picking one node. - int numOfRacks = clusterMap.getNumOfRacks(); + int numOfRacks = clusterMap.getNumOfNonEmptyRacks(); // HDFS-14527 return default when numOfRacks = 0 to avoid // ArithmeticException when calc maxNodesPerRack at following logic. if (numOfRacks <= 1 || totalNumOfReplicas <= 1) { @@ -1173,7 +1173,7 @@ public BlockPlacementStatus verifyBlockPlacement(DatanodeInfo[] locs, .map(dn -> dn.getNetworkLocation()).distinct().count(); return new BlockPlacementStatusDefault(Math.toIntExact(rackCount), - minRacks, clusterMap.getNumOfRacks()); + minRacks, clusterMap.getNumOfNonEmptyRacks()); } /** @@ -1370,4 +1370,3 @@ public boolean getExcludeSlowNodesEnabled() { return excludeSlowNodesEnabled; } } - diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolerant.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolerant.java index dad877fdc76fe..a3b3f482e8c23 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolerant.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyRackFaultTolerant.java @@ -42,7 +42,7 @@ protected int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) { totalNumOfReplicas = clusterSize; } // No calculation needed when there is only one rack or picking one node. - int numOfRacks = clusterMap.getNumOfRacks(); + int numOfRacks = clusterMap.getNumOfNonEmptyRacks(); // HDFS-14527 return default when numOfRacks = 0 to avoid // ArithmeticException when calc maxNodesPerRack at following logic. if (numOfRacks <= 1 || totalNumOfReplicas <= 1) { @@ -90,38 +90,39 @@ protected Node chooseTargetInOrder(int numOfReplicas, EnumMap storageTypes) throws NotEnoughReplicasException { int totalReplicaExpected = results.size() + numOfReplicas; - int numOfRacks = clusterMap.getNumOfRacks(); - if (totalReplicaExpected < numOfRacks || - totalReplicaExpected % numOfRacks == 0) { - writer = chooseOnce(numOfReplicas, writer, excludedNodes, blocksize, - maxNodesPerRack, results, avoidStaleNodes, storageTypes); - return writer; - } + int numOfRacks = clusterMap.getNumOfNonEmptyRacks(); - assert totalReplicaExpected > (maxNodesPerRack -1) * numOfRacks; + try { + if (totalReplicaExpected < numOfRacks || + totalReplicaExpected % numOfRacks == 0) { + writer = chooseOnce(numOfReplicas, writer, excludedNodes, blocksize, + maxNodesPerRack, results, avoidStaleNodes, storageTypes); + return writer; + } - // Calculate numOfReplicas for filling each rack exactly (maxNodesPerRack-1) - // replicas. - HashMap rackCounts = new HashMap<>(); - for (DatanodeStorageInfo dsInfo : results) { - String rack = dsInfo.getDatanodeDescriptor().getNetworkLocation(); - Integer count = rackCounts.get(rack); - if (count != null) { - rackCounts.put(rack, count + 1); - } else { - rackCounts.put(rack, 1); + assert totalReplicaExpected > (maxNodesPerRack -1) * numOfRacks; + + // Calculate numOfReplicas for filling each rack exactly (maxNodesPerRack-1) + // replicas. + HashMap rackCounts = new HashMap<>(); + for (DatanodeStorageInfo dsInfo : results) { + String rack = dsInfo.getDatanodeDescriptor().getNetworkLocation(); + Integer count = rackCounts.get(rack); + if (count != null) { + rackCounts.put(rack, count + 1); + } else { + rackCounts.put(rack, 1); + } } - } - int excess = 0; // Sum of the above (maxNodesPerRack-1) part of nodes in results - for (int count : rackCounts.values()) { - if (count > maxNodesPerRack -1) { - excess += count - (maxNodesPerRack -1); + int excess = 0; // Sum of the above (maxNodesPerRack-1) part of nodes in results + for (int count : rackCounts.values()) { + if (count > maxNodesPerRack -1) { + excess += count - (maxNodesPerRack -1); + } } - } - numOfReplicas = Math.min(totalReplicaExpected - results.size(), - (maxNodesPerRack -1) * numOfRacks - (results.size() - excess)); + numOfReplicas = Math.min(totalReplicaExpected - results.size(), + (maxNodesPerRack -1) * numOfRacks - (results.size() - excess)); - try { // Try to spread the replicas as evenly as possible across racks. // This is done by first placing with (maxNodesPerRack-1), then spreading // the remainder by calling again with maxNodesPerRack. @@ -243,7 +244,7 @@ public BlockPlacementStatus verifyBlockPlacement(DatanodeInfo[] locs, racks.add(dn.getNetworkLocation()); } return new BlockPlacementStatusDefault(racks.size(), numberOfReplicas, - clusterMap.getNumOfRacks()); + clusterMap.getNumOfNonEmptyRacks()); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java index 42b6ddd8c78fb..1c95c26a190c3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java @@ -177,6 +177,8 @@ public void startDecommission(DatanodeDescriptor node) { if (!node.isDecommissionInProgress() && !node.isDecommissioned()) { // Update DN stats maintained by HeartbeatManager hbManager.startDecommission(node); + // Update cluster's emptyRack + blockManager.getDatanodeManager().getNetworkTopology().decommissionNode(node); // hbManager.startDecommission will set dead node to decommissioned. if (node.isDecommissionInProgress()) { for (DatanodeStorageInfo storage : node.getStorageInfos()) { @@ -201,6 +203,8 @@ public void stopDecommission(DatanodeDescriptor node) { if (node.isDecommissionInProgress() || node.isDecommissioned()) { // Update DN stats maintained by HeartbeatManager hbManager.stopDecommission(node); + // Update cluster's emptyRack + blockManager.getDatanodeManager().getNetworkTopology().recommissionNode(node); // extra redundancy blocks will be detected and processed when // the dead node comes back and send in its full block report. if (node.isAlive()) { @@ -413,4 +417,4 @@ void runMonitorForTest() throws ExecutionException, InterruptedException { executor.submit(monitor).get(); } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index f080a101f679d..0a00aa65c0bcb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -8405,7 +8405,7 @@ public ECTopologyVerifierResult getECTopologyResultForPolicies( getBlockManager().getDatanodeManager().getNumOfDataNodes(); int numOfRacks = getBlockManager().getDatanodeManager().getNetworkTopology() - .getNumOfRacks(); + .getNumOfNonEmptyRacks(); result = ECTopologyVerifier .getECTopologyVerifierResult(numOfRacks, numOfDataNodes, policies); } @@ -8950,7 +8950,7 @@ private ECTopologyVerifierResult getEcTopologyVerifierResultForEnabledPolicies() int numOfDataNodes = getBlockManager().getDatanodeManager().getNumOfDataNodes(); int numOfRacks = getBlockManager().getDatanodeManager().getNetworkTopology() - .getNumOfRacks(); + .getNumOfNonEmptyRacks(); ErasureCodingPolicy[] enabledEcPolicies = getErasureCodingPolicyManager().getCopyOfEnabledPolicies(); return ECTopologyVerifier @@ -9012,4 +9012,3 @@ public void checkErasureCodingSupported(String operationName) } } } - diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolerant.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolerant.java index 88b7d2bf7f23d..3beea47800d7a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolerant.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockPlacementPolicyRackFaultTolerant.java @@ -19,24 +19,35 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.net.DFSNetworkTopology; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.net.StaticMapping; +import org.apache.hadoop.test.GenericTestUtils; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.io.IOException; import java.util.*; +import java.util.function.Supplier; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -167,6 +178,108 @@ private void doTestChooseTargetSpecialCase() throws Exception { } } + /** + * Verify decommission a dn which is an only node in its rack. + */ + @Test + public void testPlacementWithOnlyOneNodeInRackDecommission() throws Exception { + Configuration conf = new HdfsConfiguration(); + final String[] racks = {"/RACK0", "/RACK0", "/RACK2", "/RACK3", "/RACK4", "/RACK5", "/RACK2"}; + final String[] hosts = {"/host0", "/host1", "/host2", "/host3", "/host4", "/host5", "/host6"}; + + // enables DFSNetworkTopology + conf.setClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, + BlockPlacementPolicyRackFaultTolerant.class, + BlockPlacementPolicy.class); + conf.setBoolean(DFSConfigKeys.DFS_USE_DFS_NETWORK_TOPOLOGY_KEY, true); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE); + conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, + DEFAULT_BLOCK_SIZE / 2); + + if (cluster != null) { + cluster.shutdown(); + } + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(7).racks(racks) + .hosts(hosts).build(); + cluster.waitActive(); + nameNodeRpc = cluster.getNameNodeRpc(); + namesystem = cluster.getNamesystem(); + DistributedFileSystem fs = cluster.getFileSystem(); + fs.enableErasureCodingPolicy("RS-3-2-1024k"); + fs.setErasureCodingPolicy(new Path("/"), "RS-3-2-1024k"); + + final BlockManager bm = cluster.getNamesystem().getBlockManager(); + final DatanodeManager dm = bm.getDatanodeManager(); + assertTrue(dm.getNetworkTopology() instanceof DFSNetworkTopology); + + String clientMachine = "/host4"; + String clientRack = "/RACK4"; + String src = "/test"; + + final DatanodeManager dnm = namesystem.getBlockManager().getDatanodeManager(); + DatanodeDescriptor dnd4 = dnm.getDatanode(cluster.getDataNodes().get(4).getDatanodeId()); + assertEquals(dnd4.getNetworkLocation(), clientRack); + dnm.getDatanodeAdminManager().startDecommission(dnd4); + short replication = 5; + short additionalReplication = 1; + + try { + // Create the file with client machine + HdfsFileStatus fileStatus = namesystem.startFile(src, perm, + clientMachine, clientMachine, EnumSet.of(CreateFlag.CREATE), true, + replication, DEFAULT_BLOCK_SIZE * 1024 * 10, null, null, null, false); + + //test chooseTarget for new file + LocatedBlock locatedBlock = nameNodeRpc.addBlock(src, clientMachine, + null, null, fileStatus.getFileId(), null, null); + HashMap racksCount = new HashMap(); + doTestLocatedBlockRacks(racksCount, replication, 4, locatedBlock); + + //test chooseTarget for existing file. + LocatedBlock additionalLocatedBlock = + nameNodeRpc.getAdditionalDatanode(src, fileStatus.getFileId(), + locatedBlock.getBlock(), locatedBlock.getLocations(), + locatedBlock.getStorageIDs(), DatanodeInfo.EMPTY_ARRAY, + additionalReplication, clientMachine); + + racksCount.clear(); + doTestLocatedBlockRacks(racksCount, additionalReplication + replication, + 4, additionalLocatedBlock); + assertEquals(racksCount.get("/RACK0"), (Integer)2); + assertEquals(racksCount.get("/RACK2"), (Integer)2); + } finally { + dnm.getDatanodeAdminManager().stopDecommission(dnd4); + } + + //test if decommission succeeded + DatanodeDescriptor dnd3 = dnm.getDatanode(cluster.getDataNodes().get(3).getDatanodeId()); + cluster.getNamesystem().writeLock(); + try { + dm.getDatanodeAdminManager().startDecommission(dnd3); + } finally { + cluster.getNamesystem().writeUnlock(); + } + + // make sure the decommission finishes and the block in on 4 racks + GenericTestUtils.waitFor(new Supplier() { + @Override + public Boolean get() { + return dnd3.isDecommissioned(); + } + }, 1000, 10 * 1000); + + LocatedBlocks locatedBlocks = + cluster.getFileSystem().getClient().getLocatedBlocks( + src, 0, DEFAULT_BLOCK_SIZE); + assertEquals(4, bm.getDatanodeManager(). + getNetworkTopology().getNumOfNonEmptyRacks()); + for (LocatedBlock block : locatedBlocks.getLocatedBlocks()) { + BlockPlacementStatus status = bm.getStriptedBlockPlacementPolicy() + .verifyBlockPlacement(block.getLocations(), 5); + Assert.assertTrue(status.isPlacementPolicySatisfied()); + } + } + private void shuffle(DatanodeInfo[] locs, String[] storageIDs) { int length = locs.length; Object[][] pairs = new Object[length][]; @@ -198,6 +311,17 @@ private void doTestLocatedBlock(int replication, LocatedBlock locatedBlock) { assertTrue(maxCount - minCount <= 1); } + private void doTestLocatedBlockRacks(HashMap racksCount, int replication, + int validracknum, LocatedBlock locatedBlock) { + assertEquals(replication, locatedBlock.getLocations().length); + + for (DatanodeInfo node : + locatedBlock.getLocations()) { + addToRacksCount(node.getNetworkLocation(), racksCount); + } + assertEquals(validracknum, racksCount.size()); + } + private void addToRacksCount(String rack, HashMap racksCount) { Integer count = racksCount.get(rack); if (count == null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java index 5758fe7986099..531f1eb0df021 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/net/TestNetworkTopology.java @@ -633,4 +633,20 @@ public void testCountNumOfAvailableNodes() { numNodes = cluster.countNumOfAvailableNodes(NodeBase.ROOT, excludedNodes); assertEquals(12, numNodes); } + + @Test + public void testAddAndRemoveNodeWithEmptyRack() { + DatanodeDescriptor n1 = DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d2/r3"); + DatanodeDescriptor n2 = DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3"); + DatanodeDescriptor n3 = DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3"); + + cluster.decommissionNode(n1); + assertEquals(6, cluster.getNumOfNonEmptyRacks()); + cluster.decommissionNode(n2); + cluster.decommissionNode(n3); + assertEquals(5, cluster.getNumOfNonEmptyRacks()); + + cluster.recommissionNode(n1); + assertEquals(6, cluster.getNumOfNonEmptyRacks()); + } } From a6ebc42671f35c82038ddc5a7d0002c19a553127 Mon Sep 17 00:00:00 2001 From: Daniel Carl Jones Date: Thu, 14 Apr 2022 16:21:34 +0100 Subject: [PATCH 110/145] HADOOP-18201. Remove endpoint config overrides for ITestS3ARequesterPays (#4169) Contributed by Daniel Carl Jones. --- .../java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java index c2e7684cad6da..b8cb321f61acc 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ARequesterPays.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.statistics.StreamStatisticNames; import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS; +import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT; import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -43,6 +44,7 @@ protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); S3ATestUtils.removeBaseAndBucketOverrides(conf, ALLOW_REQUESTER_PAYS, + ENDPOINT, S3A_BUCKET_PROBE); return conf; } From d5cba5c3d110bcc49d3eafaf9b78e19c93156ece Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=AB=A0=E9=94=A1=E5=B9=B3?= <40832063+zhangxiping1@users.noreply.github.com> Date: Fri, 15 Apr 2022 16:50:49 +0800 Subject: [PATCH 111/145] YARN-11107. Addendum. When NodeLabel is enabled for a YARN cluster, AM blacklist program does not work properly. (#4175) --- .../resourcemanager/TestApplicationMasterServiceCapacity.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java index ccbc375359aee..80f84c34503f9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterServiceCapacity.java @@ -45,7 +45,7 @@ import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import com.google.common.collect.ImmutableMap; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.junit.Assert; import org.junit.Test; From 8ea33583804b50dd748e815cc27181aa72330c42 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Fri, 15 Apr 2022 19:54:02 +0530 Subject: [PATCH 112/145] HADOOP-18196. Remove replace-guava from replacer plugin (#4152) --- hadoop-project/pom.xml | 21 --------------------- 1 file changed, 21 deletions(-) diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 871d7fc8082c8..eaa91c2c90001 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -2129,27 +2129,6 @@ - - replace-guava - process-sources - - replace - - - false - ${basedir} - - src/main/java/**/*.java - src/test/java/**/*.java - - - - ([^\.])com.google.common - $1${hadoop-thirdparty-shaded-guava-prefix} - - - - From f14f3050517ca09e5366210b3f1fca90c8feb81a Mon Sep 17 00:00:00 2001 From: Renukaprasad C <48682981+prasad-acit@users.noreply.github.com> Date: Fri, 15 Apr 2022 21:37:05 +0530 Subject: [PATCH 113/145] HDFS-16526. Add metrics for slow DataNode (#4162) --- .../src/site/markdown/Metrics.md | 3 +- .../hdfs/server/datanode/BlockReceiver.java | 4 ++ .../datanode/metrics/DataNodeMetrics.java | 10 ++++ .../server/datanode/TestDataNodeMetrics.java | 54 +++++++++++++++++++ 4 files changed, 70 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md index b0fc525776bbe..4b17e1504862e 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md @@ -481,7 +481,8 @@ Each metrics record contains tags such as SessionId and Hostname as additional i | `PacketsSlowWriteToMirror` | Total number of packets whose write to other Datanodes in the pipeline takes more than a certain time (300ms by default) | | `PacketsSlowWriteToDisk` | Total number of packets whose write to disk takes more than a certain time (300ms by default) | | `PacketsSlowWriteToOsCache` | Total number of packets whose write to os cache takes more than a certain time (300ms by default) | - +| `slowFlushOrSyncCount` | Total number of packets whose sync/flush takes more than a certain time (300ms by default) | +| `slowAckToUpstreamCount` | Total number of packets whose upstream ack takes more than a certain time (300ms by default) | FsVolume -------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index 62bc66080ce53..9b3a899323642 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -412,6 +412,7 @@ boolean packetSentInTime() { void flushOrSync(boolean isSync, long seqno) throws IOException { long flushTotalNanos = 0; long begin = Time.monotonicNow(); + DataNodeFaultInjector.get().delay(); if (checksumOut != null) { long flushStartNanos = System.nanoTime(); checksumOut.flush(); @@ -445,6 +446,7 @@ void flushOrSync(boolean isSync, long seqno) throws IOException { } long duration = Time.monotonicNow() - begin; if (duration > datanodeSlowLogThresholdMs && LOG.isWarnEnabled()) { + datanode.metrics.incrSlowFlushOrSyncCount(); LOG.warn("Slow flushOrSync took " + duration + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms), isSync:" + isSync + ", flushTotalNanos=" + flushTotalNanos + "ns, volume=" + getVolumeBaseUri() @@ -1656,6 +1658,7 @@ private void sendAckUpstreamUnprotected(PipelineAck ack, long seqno, } // send my ack back to upstream datanode long begin = Time.monotonicNow(); + DataNodeFaultInjector.get().delay(); /* for test only, no-op in production system */ DataNodeFaultInjector.get().delaySendingAckToUpstream(inAddr); replyAck.write(upstreamOut); @@ -1665,6 +1668,7 @@ private void sendAckUpstreamUnprotected(PipelineAck ack, long seqno, inAddr, duration); if (duration > datanodeSlowLogThresholdMs) { + datanode.metrics.incrSlowAckToUpstreamCount(); LOG.warn("Slow PacketResponder send ack to upstream took " + duration + "ms (threshold=" + datanodeSlowLogThresholdMs + "ms), " + myString + ", replyAck=" + replyAck diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java index 5203d7bf87f89..649d30e91e034 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodeMetrics.java @@ -191,6 +191,8 @@ public class DataNodeMetrics { @Metric MutableCounterLong packetsSlowWriteToMirror; @Metric MutableCounterLong packetsSlowWriteToDisk; @Metric MutableCounterLong packetsSlowWriteToOsCache; + @Metric private MutableCounterLong slowFlushOrSyncCount; + @Metric private MutableCounterLong slowAckToUpstreamCount; @Metric("Number of replaceBlock ops between" + " storage types on same host with local copy") @@ -440,6 +442,14 @@ public void incrVolumeFailures(int size) { volumeFailures.incr(size); } + public void incrSlowFlushOrSyncCount() { + slowFlushOrSyncCount.incr(); + } + + public void incrSlowAckToUpstreamCount() { + slowAckToUpstreamCount.incr(); + } + public void incrDatanodeNetworkErrors() { datanodeNetworkErrors.incr(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java index f8c89d4abe168..2bf7861287a2d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java @@ -603,6 +603,60 @@ public void testNNRpcMetricsWithNonHA() throws IOException { MetricsRecordBuilder rb = getMetrics(dn.getMetrics().name()); assertCounter("HeartbeatsNumOps", 1L, rb); } + @Test(timeout = 60000) + public void testSlowMetrics() throws Exception { + DataNodeFaultInjector dnFaultInjector = new DataNodeFaultInjector() { + @Override public void delay() { + try { + Thread.sleep(310); + } catch (InterruptedException e) { + } + } + }; + DataNodeFaultInjector oldDnInjector = DataNodeFaultInjector.get(); + DataNodeFaultInjector.set(dnFaultInjector); + + Configuration conf = new HdfsConfiguration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build(); + final FileSystem fs = cluster.getFileSystem(); + List datanodes = cluster.getDataNodes(); + assertEquals(datanodes.size(), 3); + final DataNode datanode = datanodes.get(0); + MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name()); + final long longFileLen = 10; + final long startFlushOrSyncValue = + getLongCounter("SlowFlushOrSyncCount", rb); + final long startAckToUpstreamValue = + getLongCounter("SlowAckToUpstreamCount", rb); + final AtomicInteger x = new AtomicInteger(0); + + GenericTestUtils.waitFor(new Supplier() { + @Override public Boolean get() { + x.getAndIncrement(); + try { + DFSTestUtil + .createFile(fs, new Path("/time.txt." + x.get()), longFileLen, + (short) 3, Time.monotonicNow()); + } catch (IOException ioe) { + LOG.error("Caught IOException while ingesting DN metrics", ioe); + return false; + } + MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name()); + final long endFlushOrSyncValue = getLongCounter("SlowFlushOrSyncCount", rbNew); + final long endAckToUpstreamValue = getLongCounter("SlowAckToUpstreamCount", rbNew); + return endFlushOrSyncValue > startFlushOrSyncValue + && endAckToUpstreamValue > startAckToUpstreamValue; + } + }, 30, 30000); + } finally { + DataNodeFaultInjector.set(oldDnInjector); + if (cluster != null) { + cluster.shutdown(); + } + } + } @Test public void testNNRpcMetricsWithHA() throws IOException { From cb975c3df6fcd4514d9413a21a3f36dc2748e8d8 Mon Sep 17 00:00:00 2001 From: He Xiaoqiao Date: Sun, 17 Apr 2022 19:21:27 +0800 Subject: [PATCH 114/145] HDFS-16534. Split FsDatasetImpl from block pool locks to volume grain locks. (#4141) Contributed by limingxiang. Signed-off-by: He Xiaoqiao --- .../fsdataset/impl/FsDatasetImpl.java | 109 +++++++++++------- .../server/datanode/TestBPOfferService.java | 8 ++ .../extdataset/ExternalVolumeImpl.java | 3 +- .../fsdataset/impl/TestFsDatasetImpl.java | 9 ++ .../impl/TestInterDatanodeProtocol.java | 7 +- 5 files changed, 95 insertions(+), 41 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index aaf37aa09c8c9..df24f9890db04 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -432,6 +432,9 @@ private synchronized void activateVolume( ReplicaMap replicaMap, Storage.StorageDirectory sd, StorageType storageType, FsVolumeReference ref) throws IOException { + for (String bp : volumeMap.getBlockPoolList()) { + lockManager.addLock(LockLevel.VOLUME, bp, ref.getVolume().getStorageID()); + } DatanodeStorage dnStorage = storageMap.get(sd.getStorageUuid()); if (dnStorage != null) { final String errorMsg = String.format( @@ -629,6 +632,9 @@ public void removeVolumes( synchronized (this) { for (String storageUuid : storageToRemove) { storageMap.remove(storageUuid); + for (String bp : volumeMap.getBlockPoolList()) { + lockManager.removeLock(LockLevel.VOLUME, bp, storageUuid); + } } } } @@ -906,8 +912,8 @@ ReplicaInfo getReplicaInfo(String bpid, long blkid) @Override // FsDatasetSpi public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkOffset, long metaOffset) throws IOException { - try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { ReplicaInfo info = getReplicaInfo(b); FsVolumeReference ref = info.getVolume().obtainReference(); try { @@ -1372,8 +1378,8 @@ static void computeChecksum(ReplicaInfo srcReplica, File dstMeta, @Override // FsDatasetSpi public ReplicaHandler append(ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException { - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { // If the block was successfully finalized because all packets // were successfully processed at the Datanode but the ack for // some of the packets were not received by the client. The client @@ -1425,7 +1431,8 @@ public ReplicaHandler append(ExtendedBlock b, private ReplicaInPipeline append(String bpid, ReplicaInfo replicaInfo, long newGS, long estimateBlockLen) throws IOException { - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + bpid, replicaInfo.getStorageUuid())) { // If the block is cached, start uncaching it. if (replicaInfo.getState() != ReplicaState.FINALIZED) { throw new IOException("Only a Finalized replica can be appended to; " @@ -1554,8 +1561,8 @@ public Replica recoverClose(ExtendedBlock b, long newGS, LOG.info("Recover failed close " + b); while (true) { try { - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { // check replica's state ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen); // bump the replica's GS @@ -1578,7 +1585,7 @@ public ReplicaHandler createRbw( StorageType storageType, String storageId, ExtendedBlock b, boolean allowLazyPersist) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, b.getBlockPoolId())) { ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getBlockId()); @@ -1626,20 +1633,20 @@ public ReplicaHandler createRbw( } ReplicaInPipeline newReplicaInfo; - try { + try (AutoCloseableLock l = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), v.getStorageID())) { newReplicaInfo = v.createRbw(b); if (newReplicaInfo.getReplicaInfo().getState() != ReplicaState.RBW) { throw new IOException("CreateRBW returned a replica of state " + newReplicaInfo.getReplicaInfo().getState() + " for block " + b.getBlockId()); } + volumeMap.add(b.getBlockPoolId(), newReplicaInfo.getReplicaInfo()); + return new ReplicaHandler(newReplicaInfo, ref); } catch (IOException e) { IOUtils.cleanupWithLogger(null, ref); throw e; } - - volumeMap.add(b.getBlockPoolId(), newReplicaInfo.getReplicaInfo()); - return new ReplicaHandler(newReplicaInfo, ref); } finally { if (dataNodeMetrics != null) { long createRbwMs = Time.monotonicNow() - startTimeMs; @@ -1657,8 +1664,8 @@ public ReplicaHandler recoverRbw( try { while (true) { try { - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId()); // check the replica's state @@ -1689,8 +1696,8 @@ public ReplicaHandler recoverRbw( private ReplicaHandler recoverRbwImpl(ReplicaInPipeline rbw, ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException { - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { // check generation stamp long replicaGenerationStamp = rbw.getGenerationStamp(); if (replicaGenerationStamp < b.getGenerationStamp() || @@ -1751,8 +1758,8 @@ private ReplicaHandler recoverRbwImpl(ReplicaInPipeline rbw, public ReplicaInPipeline convertTemporaryToRbw( final ExtendedBlock b) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { final long blockId = b.getBlockId(); final long expectedGs = b.getGenerationStamp(); final long visible = b.getNumBytes(); @@ -1887,12 +1894,12 @@ public ReplicaHandler createTemporary(StorageType storageType, false); } long startHoldLockTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { - FsVolumeReference ref = volumes.getNextVolume(storageType, storageId, b - .getNumBytes()); - FsVolumeImpl v = (FsVolumeImpl) ref.getVolume(); - ReplicaInPipeline newReplicaInfo; + FsVolumeReference ref = volumes.getNextVolume(storageType, storageId, b + .getNumBytes()); + FsVolumeImpl v = (FsVolumeImpl) ref.getVolume(); + ReplicaInPipeline newReplicaInfo; + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), v.getStorageID())) { try { newReplicaInfo = v.createTemporary(b); LOG.debug("creating temporary for block: {} on volume: {}", @@ -1949,8 +1956,8 @@ public void finalizeBlock(ExtendedBlock b, boolean fsyncDir) ReplicaInfo replicaInfo = null; ReplicaInfo finalizedReplicaInfo = null; long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { if (Thread.interrupted()) { // Don't allow data modifications from interrupted threads throw new IOException("Cannot finalize block from Interrupted Thread"); @@ -1986,7 +1993,8 @@ public void finalizeBlock(ExtendedBlock b, boolean fsyncDir) private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo) throws IOException { - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + bpid, replicaInfo.getStorageUuid())) { // Compare generation stamp of old and new replica before finalizing if (volumeMap.get(bpid, replicaInfo.getBlockId()).getGenerationStamp() > replicaInfo.getGenerationStamp()) { @@ -2032,8 +2040,8 @@ private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo) @Override // FsDatasetSpi public void unfinalizeBlock(ExtendedBlock b) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - b.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock()); if (replicaInfo != null && @@ -2423,10 +2431,17 @@ private void cacheBlock(String bpid, long blockId) { long length, genstamp; Executor volumeExecutor; - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { - ReplicaInfo info = volumeMap.get(bpid, blockId); + ReplicaInfo info = volumeMap.get(bpid, blockId); + if (info == null) { + LOG.warn("Failed to cache block with id " + blockId + ", pool " + + bpid + ": ReplicaInfo not found."); + return; + } + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, bpid, + info.getStorageUuid())) { boolean success = false; try { + info = volumeMap.get(bpid, blockId); if (info == null) { LOG.warn("Failed to cache block with id " + blockId + ", pool " + bpid + ": ReplicaInfo not found."); @@ -2619,7 +2634,8 @@ public void checkAndUpdate(String bpid, ScanInfo scanInfo) curDirScannerNotifyCount = 0; lastDirScannerNotifyTime = startTimeMs; } - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, bpid, + vol.getStorageID())) { memBlockInfo = volumeMap.get(bpid, blockId); if (memBlockInfo != null && memBlockInfo.getState() != ReplicaState.FINALIZED) { @@ -2860,7 +2876,14 @@ ReplicaRecoveryInfo initReplicaRecovery(String bpid, ReplicaMap map, Block block, long recoveryId, long xceiverStopTimeout) throws IOException { while (true) { try { - try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + ReplicaInfo replica = map.get(bpid, block.getBlockId()); + if (replica == null) { + return null; + } + LOG.info("initReplicaRecovery: " + block + ", recoveryId=" + recoveryId + + ", replica=" + replica); + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.VOLUME, bpid, + replica.getStorageUuid())) { return initReplicaRecoveryImpl(bpid, map, block, recoveryId); } } catch (MustStopExistingWriter e) { @@ -2875,7 +2898,14 @@ static ReplicaRecoveryInfo initReplicaRecovery(String bpid, ReplicaMap map, lockManager) throws IOException { while (true) { try { - try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + ReplicaInfo replica = map.get(bpid, block.getBlockId()); + if (replica == null) { + return null; + } + LOG.info("initReplicaRecovery: " + block + ", recoveryId=" + recoveryId + + ", replica=" + replica); + try (AutoCloseDataSetLock l = lockManager.writeLock(LockLevel.VOLUME, bpid, + replica.getStorageUuid())) { return initReplicaRecoveryImpl(bpid, map, block, recoveryId); } } catch (MustStopExistingWriter e) { @@ -2888,9 +2918,6 @@ static ReplicaRecoveryInfo initReplicaRecoveryImpl(String bpid, ReplicaMap map, Block block, long recoveryId) throws IOException, MustStopExistingWriter { final ReplicaInfo replica = map.get(bpid, block.getBlockId()); - LOG.info("initReplicaRecovery: " + block + ", recoveryId=" + recoveryId - + ", replica=" + replica); - //check replica if (replica == null) { return null; @@ -2964,8 +2991,8 @@ public Replica updateReplicaUnderRecovery( final long newBlockId, final long newlength) throws IOException { long startTimeMs = Time.monotonicNow(); - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, - oldBlock.getBlockPoolId())) { + try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, + oldBlock.getBlockPoolId(), getReplicaInfo(oldBlock).getStorageUuid())) { //get replica final String bpid = oldBlock.getBlockPoolId(); final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId()); @@ -3109,6 +3136,10 @@ public void addBlockPool(String bpid, Configuration conf) volumeExceptions.mergeException(e); } volumeMap.initBlockPool(bpid); + Set vols = storageMap.keySet(); + for (String v : vols) { + lockManager.addLock(LockLevel.VOLUME, bpid, v); + } } try { volumes.getAllVolumesMap(bpid, volumeMap, ramDiskReplicaTracker); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java index 0c0fe618174c4..d300eac4b69cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java @@ -88,6 +88,7 @@ import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.Time; import org.junit.Before; +import org.junit.After; import org.junit.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -157,6 +158,13 @@ public void setupMocks() throws Exception { Mockito.doReturn(dataSetLockManager).when(mockDn).getDataSetLockManager(); } + @After + public void checkDataSetLockManager() { + dataSetLockManager.lockLeakCheck(); + // make sure no lock Leak. + assertNull(dataSetLockManager.getLastException()); + } + /** * Set up a mock NN with the bare minimum for a DN to register to it. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java index 6c8e828f3689e..81cfa3e08b87b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalVolumeImpl.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; public class ExternalVolumeImpl implements FsVolumeSpi { + private final String defaultStroageId = "test"; @Override public FsVolumeReference obtainReference() throws ClosedChannelException { return null; @@ -54,7 +55,7 @@ public long getAvailable() throws IOException { @Override public String getStorageID() { - return null; + return defaultStroageId; } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java index f250eea2920d3..1e4de75148679 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java @@ -80,6 +80,7 @@ import org.apache.hadoop.util.StringUtils; import org.junit.Assert; import org.junit.Before; +import org.junit.After; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; @@ -236,6 +237,13 @@ public void setUp() throws IOException { assertEquals(0, dataset.getNumFailedVolumes()); } + @After + public void checkDataSetLockManager() { + manager.lockLeakCheck(); + // make sure no lock Leak. + assertNull(manager.getLastException()); + } + @Test public void testAddVolumes() throws IOException { final int numNewVolumes = 3; @@ -687,6 +695,7 @@ public void testAddVolumeFailureReleasesInUseLock() throws IOException { FsDatasetImpl spyDataset = spy(dataset); FsVolumeImpl mockVolume = mock(FsVolumeImpl.class); File badDir = new File(BASE_DIR, "bad"); + when(mockVolume.getStorageID()).thenReturn("test"); badDir.mkdirs(); doReturn(mockVolume).when(spyDataset) .createFsVolume(anyString(), any(StorageDirectory.class), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java index 5290337e2f270..659d53eda9b6e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestInterDatanodeProtocol.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.datanode.Replica; import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo; import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery; +import org.apache.hadoop.hdfs.server.datanode.extdataset.ExternalVolumeImpl; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol; @@ -218,7 +219,7 @@ private void checkBlockMetaDataInfo(boolean useDnHostname) throws Exception { } private static ReplicaInfo createReplicaInfo(Block b) { - return new FinalizedReplica(b, null, null); + return new FinalizedReplica(b, new ExternalVolumeImpl(), null); } private static void assertEquals(ReplicaInfo originalInfo, ReplicaRecoveryInfo recoveryInfo) { @@ -318,6 +319,10 @@ public void testInitReplicaRecovery() throws IOException { "replica.getGenerationStamp() < block.getGenerationStamp(), block="); } } + + manager.lockLeakCheck(); + // make sure no lock Leak. + assertNull(manager.getLastException()); } /** From dbeeee03639f41a022dd07d5fc04e3aa65a94b5f Mon Sep 17 00:00:00 2001 From: Stephen O'Donnell Date: Sun, 17 Apr 2022 13:05:11 +0100 Subject: [PATCH 115/145] HDFS-16531. Avoid setReplication writing an edit record if old replication equals the new value (#4148). Contributed by Stephen O'Donnell. --- .../hdfs/server/namenode/FSDirAttrOp.java | 31 +++++++++++++------ .../hdfs/server/namenode/FSNamesystem.java | 11 ++++--- .../hadoop/hdfs/TestSetrepIncreasing.java | 6 ++++ 3 files changed, 33 insertions(+), 15 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java index 04913d1a7cee0..a2c9f6bd76bdc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java @@ -48,6 +48,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY; public class FSDirAttrOp { + + protected enum SetRepStatus { + UNCHANGED, INVALID, SUCCESS + } + static FileStatus setPermission( FSDirectory fsd, FSPermissionChecker pc, final String src, FsPermission permission) throws IOException { @@ -134,11 +139,11 @@ static FileStatus setTimes( return fsd.getAuditFileInfo(iip); } - static boolean setReplication( + static SetRepStatus setReplication( FSDirectory fsd, FSPermissionChecker pc, BlockManager bm, String src, final short replication) throws IOException { bm.verifyReplication(src, replication, null); - final boolean isFile; + final SetRepStatus status; fsd.writeLock(); try { final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.WRITE); @@ -146,16 +151,14 @@ static boolean setReplication( fsd.checkPathAccess(pc, iip, FsAction.WRITE); } - final BlockInfo[] blocks = unprotectedSetReplication(fsd, iip, - replication); - isFile = blocks != null; - if (isFile) { + status = unprotectedSetReplication(fsd, iip, replication); + if (status == SetRepStatus.SUCCESS) { fsd.getEditLog().logSetReplication(iip.getPath(), replication); } } finally { fsd.writeUnlock(); } - return isFile; + return status; } static FileStatus unsetStoragePolicy(FSDirectory fsd, FSPermissionChecker pc, @@ -381,7 +384,7 @@ static INodeDirectory unprotectedSetQuota( return dirNode; } - static BlockInfo[] unprotectedSetReplication( + static SetRepStatus unprotectedSetReplication( FSDirectory fsd, INodesInPath iip, short replication) throws QuotaExceededException, UnresolvedLinkException, SnapshotAccessControlException, UnsupportedActionException { @@ -391,12 +394,20 @@ static BlockInfo[] unprotectedSetReplication( final INode inode = iip.getLastINode(); if (inode == null || !inode.isFile() || inode.asFile().isStriped()) { // TODO we do not support replication on stripe layout files yet - return null; + // We return invalid here, so we skip writing an edit, but also write an + // unsuccessful audit message. + return SetRepStatus.INVALID; } INodeFile file = inode.asFile(); // Make sure the directory has sufficient quotas short oldBR = file.getPreferredBlockReplication(); + if (oldBR == replication) { + // No need to do anything as the requested rep factor is the same as + // existing. Returning UNCHANGED to we can skip writing edits, but still + // log a successful audit message. + return SetRepStatus.UNCHANGED; + } long size = file.computeFileSize(true, true); // Ensure the quota does not exceed @@ -427,7 +438,7 @@ static BlockInfo[] unprotectedSetReplication( oldBR, iip.getPath()); } } - return file.getBlocks(); + return SetRepStatus.SUCCESS; } static void unprotectedSetStoragePolicy(FSDirectory fsd, BlockManager bm, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 0a00aa65c0bcb..6ab57ed880a42 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -2448,7 +2448,7 @@ void createSymlink(String target, String link, boolean setReplication(final String src, final short replication) throws IOException { final String operationName = "setReplication"; - boolean success = false; + FSDirAttrOp.SetRepStatus status; checkOperation(OperationCategory.WRITE); final FSPermissionChecker pc = getPermissionChecker(); FSPermissionChecker.setOperationType(operationName); @@ -2457,7 +2457,7 @@ boolean setReplication(final String src, final short replication) try { checkOperation(OperationCategory.WRITE); checkNameNodeSafeMode("Cannot set replication for " + src); - success = FSDirAttrOp.setReplication(dir, pc, blockManager, src, + status = FSDirAttrOp.setReplication(dir, pc, blockManager, src, replication); } finally { writeUnlock(operationName, getLockReportInfoSupplier(src)); @@ -2466,11 +2466,12 @@ boolean setReplication(final String src, final short replication) logAuditEvent(false, operationName, src); throw e; } - if (success) { + if (status == FSDirAttrOp.SetRepStatus.SUCCESS) { getEditLog().logSync(); - logAuditEvent(true, operationName, src); } - return success; + logAuditEvent(status != FSDirAttrOp.SetRepStatus.INVALID, + operationName, src); + return status != FSDirAttrOp.SetRepStatus.INVALID; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java index 497d450de2508..d89b0777633b2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java @@ -82,6 +82,12 @@ static void setrep(int fromREP, int toREP, boolean simulatedStorage) throws IOEx public void testSetrepIncreasing() throws IOException { setrep(3, 7, false); } + + @Test(timeout=120000) + public void testSetrepSameRepValue() throws IOException { + setrep(3, 3, false); + } + @Test(timeout=120000) public void testSetrepIncreasingSimulatedStorage() throws IOException { setrep(3, 7, true); From 35d4c02bccd303b5a2608a952fa1f8487001087b Mon Sep 17 00:00:00 2001 From: Quanlong Huang Date: Mon, 18 Apr 2022 10:32:29 +0800 Subject: [PATCH 116/145] HDFS-16535. SlotReleaser should reuse the domain socket based on socket paths (#4158) Reviewed-by: Lisheng Sun --- .../hdfs/shortcircuit/ShortCircuitCache.java | 25 ++++-- .../shortcircuit/TestShortCircuitCache.java | 78 +++++++++++++++++++ 2 files changed, 95 insertions(+), 8 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java index a950388a31239..df2a92c75c962 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java @@ -189,6 +189,7 @@ public void run() { final DfsClientShm shm = (DfsClientShm)slot.getShm(); final DomainSocket shmSock = shm.getPeer().getDomainSocket(); final String path = shmSock.getPath(); + DomainSocket domainSocket = pathToDomainSocket.get(path); DataOutputStream out = null; boolean success = false; int retries = 2; @@ -196,9 +197,10 @@ public void run() { while (retries > 0) { try { if (domainSocket == null || !domainSocket.isOpen()) { - // we are running in single thread mode, no protection needed for - // domainSocket domainSocket = DomainSocket.connect(path); + // we are running in single thread mode, no protection needed for + // pathToDomainSocket + pathToDomainSocket.put(path, domainSocket); } out = new DataOutputStream( @@ -221,13 +223,16 @@ public void run() { } catch (SocketException se) { // the domain socket on datanode may be timed out, we retry once retries--; - domainSocket.close(); - domainSocket = null; + if (domainSocket != null) { + domainSocket.close(); + domainSocket = null; + pathToDomainSocket.remove(path); + } if (retries == 0) { throw new SocketException("Create domain socket failed"); } } - } + } // end of while block } catch (IOException e) { LOG.warn(ShortCircuitCache.this + ": failed to release " + "short-circuit shared memory slot " + slot + " by sending " @@ -240,10 +245,10 @@ public void run() { } else { shm.getEndpointShmManager().shutdown(shm); IOUtilsClient.cleanupWithLogger(LOG, domainSocket, out); - domainSocket = null; + pathToDomainSocket.remove(path); } } - } + } // end of run() } public interface ShortCircuitReplicaCreator { @@ -354,7 +359,11 @@ public interface ShortCircuitReplicaCreator { */ private final DfsClientShmManager shmManager; - private DomainSocket domainSocket = null; + /** + * A map contains all DomainSockets used in SlotReleaser. Keys are the domain socket + * paths of short-circuit shared memory segments. + */ + private Map pathToDomainSocket = new HashMap<>(); public static ShortCircuitCache fromConf(ShortCircuitConf conf) { return new ShortCircuitCache( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java index 9754da3348359..965ac0ac98b0f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java @@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader; +import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector; import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry; import org.apache.hadoop.hdfs.server.datanode.ShortCircuitRegistry.RegisteredShm; @@ -957,6 +958,83 @@ public void testDomainSocketClosedByDN() throws Exception { } } + // Regression test for HDFS-16535 + @Test(timeout = 60000) + public void testDomainSocketClosedByMultipleDNs() throws Exception { + TemporarySocketDirectory sockDir = new TemporarySocketDirectory(); + String testName = "testDomainSocketClosedByMultipleDNs"; + Configuration conf = createShortCircuitConf(testName, sockDir); + conf.set(DFS_DOMAIN_SOCKET_PATH_KEY, new File(sockDir.getDir(), + testName + "._PORT").getAbsolutePath()); + MiniDFSCluster cluster = + new MiniDFSCluster.Builder(conf).numDataNodes(2).build(); + + try { + cluster.waitActive(); + DistributedFileSystem fs = cluster.getFileSystem(); + final ShortCircuitCache cache = + fs.getClient().getClientContext().getShortCircuitCache(); + + ExtendedBlockId blockId0 = new ExtendedBlockId(123, "xyz"); + ExtendedBlockId blockId1 = new ExtendedBlockId(456, "xyz"); + + DataNode dn0 = cluster.getDataNodes().get(0); + DataNode dn1 = cluster.getDataNodes().get(1); + + DomainPeer peer0 = new DomainPeer(DomainSocket.connect(new File( + sockDir.getDir(), testName + "." + dn0.getXferPort()).getAbsolutePath())); + DomainPeer peer1 = new DomainPeer(DomainSocket.connect(new File( + sockDir.getDir(), testName + "." + dn1.getXferPort()).getAbsolutePath())); + + final DatanodeInfo dnInfo0 = new DatanodeInfo.DatanodeInfoBuilder() + .setNodeID(dn0.getDatanodeId()).build(); + final DatanodeInfo dnInfo1 = new DatanodeInfo.DatanodeInfoBuilder() + .setNodeID(dn1.getDatanodeId()).build(); + + // Allocate 2 shm slots from DataNode-0 + MutableBoolean usedPeer = new MutableBoolean(false); + Slot slot1 = cache.allocShmSlot(dnInfo0, peer0, usedPeer, blockId0, + "testDomainSocketClosedByMultipleDNs_client"); + dn0.getShortCircuitRegistry() + .registerSlot(blockId0, slot1.getSlotId(), false); + + Slot slot2 = cache.allocShmSlot(dnInfo0, peer0, usedPeer, blockId0, + "testDomainSocketClosedByMultipleDNs_client"); + dn0.getShortCircuitRegistry() + .registerSlot(blockId0, slot2.getSlotId(), false); + + // Allocate 1 shm slot from DataNode-1 + Slot slot3 = cache.allocShmSlot(dnInfo1, peer1, usedPeer, blockId1, + "testDomainSocketClosedByMultipleDNs_client"); + dn1.getShortCircuitRegistry() + .registerSlot(blockId1, slot3.getSlotId(), false); + + Assert.assertEquals(2, cache.getDfsClientShmManager().getShmNum()); + Assert.assertEquals(1, dn0.getShortCircuitRegistry().getShmNum()); + Assert.assertEquals(1, dn1.getShortCircuitRegistry().getShmNum()); + + // Release the slot of DataNode-1 first. + cache.scheduleSlotReleaser(slot3); + Thread.sleep(2000); + Assert.assertEquals(1, cache.getDfsClientShmManager().getShmNum()); + + // Release the slots of DataNode-0. + cache.scheduleSlotReleaser(slot1); + Thread.sleep(2000); + Assert.assertEquals("0 ShmNum means the shm of DataNode-0 is shutdown" + + " due to slot release failures.", + 1, cache.getDfsClientShmManager().getShmNum()); + cache.scheduleSlotReleaser(slot2); + Thread.sleep(2000); + + Assert.assertEquals(0, dn0.getShortCircuitRegistry().getShmNum()); + Assert.assertEquals(0, dn1.getShortCircuitRegistry().getShmNum()); + Assert.assertEquals(0, cache.getDfsClientShmManager().getShmNum()); + } finally { + cluster.shutdown(); + } + } + @Test(timeout = 60000) public void testDNRestart() throws Exception { TemporarySocketDirectory sockDir = new TemporarySocketDirectory(); From 900682e7120e7af250ebf0b4bafb40b412d3e11c Mon Sep 17 00:00:00 2001 From: jianghuazhu <740087514@qq.com> Date: Mon, 18 Apr 2022 11:21:51 +0800 Subject: [PATCH 117/145] HDFS-16389.Improve NNThroughputBenchmark test mkdirs. (#3819) Reviewed-by: Viraj Jasani Reviewed-by: Akira Ajisaka Signed-off-by: Wei-Chiu Chuang --- .../server/namenode/FileNameGenerator.java | 4 ++ .../namenode/NNThroughputBenchmark.java | 40 +++++++++++++------ 2 files changed, 32 insertions(+), 12 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FileNameGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FileNameGenerator.java index 6f7087b921510..ed18696c2325e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FileNameGenerator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FileNameGenerator.java @@ -77,6 +77,10 @@ synchronized String getNextFileName(String fileNamePrefix) { return fn; } + public long getFileCount() { + return fileCount; + } + private synchronized void reset() { Arrays.fill(pathIndecies, -1); fileCount = 0L; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java index cfbcb40ffb76c..847f7dc0c12c7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java @@ -569,12 +569,20 @@ void generateInputs(int[] opsPerThread) throws IOException { // int generatedFileIdx = 0; LOG.info("Generate " + numOpsRequired + " intputs for " + getOpName()); fileNames = new String[numThreads][]; - for(int idx=0; idx < numThreads; idx++) { - int threadOps = opsPerThread[idx]; - fileNames[idx] = new String[threadOps]; - for(int jdx=0; jdx < threadOps; jdx++) - fileNames[idx][jdx] = nameGenerator. - getNextFileName("ThroughputBench"); + try { + for(int idx=0; idx < numThreads; idx++) { + int threadOps = opsPerThread[idx]; + fileNames[idx] = new String[threadOps]; + for(int jdx=0; jdx < threadOps; jdx++) { + fileNames[idx][jdx] = nameGenerator. + getNextFileName("ThroughputBench"); + } + } + } catch (ArrayIndexOutOfBoundsException e) { + LOG.error("The current environment allows {} files to be created. " + + "If you want to test more files, please update the -filesPerDir parameter.", + nameGenerator.getFileCount()); + throw e; } } @@ -669,12 +677,20 @@ void generateInputs(int[] opsPerThread) throws IOException { false); LOG.info("Generate " + numOpsRequired + " inputs for " + getOpName()); dirPaths = new String[numThreads][]; - for(int idx=0; idx < numThreads; idx++) { - int threadOps = opsPerThread[idx]; - dirPaths[idx] = new String[threadOps]; - for(int jdx=0; jdx < threadOps; jdx++) - dirPaths[idx][jdx] = nameGenerator. - getNextFileName("ThroughputBench"); + try { + for(int idx=0; idx < numThreads; idx++) { + int threadOps = opsPerThread[idx]; + dirPaths[idx] = new String[threadOps]; + for(int jdx=0; jdx < threadOps; jdx++) { + dirPaths[idx][jdx] = nameGenerator. + getNextFileName("ThroughputBench"); + } + } + } catch (ArrayIndexOutOfBoundsException e) { + LOG.error("The current environment allows {} directories to be created. " + + "If you want to test more directories, please update the -dirsPerDir parameter.", + nameGenerator.getFileCount()); + throw e; } } From f74f24163692087d7fa7a04e93ed24bca4569a7a Mon Sep 17 00:00:00 2001 From: Happy-shi <35164746+Happy-shi@users.noreply.github.com> Date: Tue, 19 Apr 2022 01:23:57 +0800 Subject: [PATCH 118/145] HDFS-16541. Fix a typo in NameNodeLayoutVersion (#4176) --- .../hdfs/server/namenode/NameNodeLayoutVersion.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java index f5e261dc78aae..8dfc3111a6ba0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java @@ -66,10 +66,10 @@ public static boolean supports(final LayoutFeature f, final int lv) { * layout version is the earliest prior version to which a downgrade is * possible after initiating rolling upgrade. If the feature cannot satisfy * compatibility with any prior version, then set its minimum compatible - * lqyout version to itself to indicate that downgrade is impossible. + * layout version to itself to indicate that downgrade is impossible. * Satisfying compatibility might require adding logic to the new feature to * reject operations or handle them differently while rolling upgrade is in - * progress. In general, it's possible to satisfy compatiblity for downgrade + * progress. In general, it's possible to satisfy compatibility for downgrade * if the new feature just involves adding new edit log ops. Deeper * structural changes, such as changing the way we place files in the metadata * directories, might be incompatible. Feature implementations should strive @@ -98,7 +98,7 @@ public enum Feature implements LayoutFeature { /** * Feature that is added at layout version {@code lv} - 1. * @param lv new layout version with the addition of this feature - * @param minCompatLV minimium compatible layout version + * @param minCompatLV minimum compatible layout version * @param description description of the feature */ Feature(final int lv, int minCompatLV, final String description) { @@ -106,7 +106,7 @@ public enum Feature implements LayoutFeature { } /** - * NameNode feature that is added at layout version {@code ancestoryLV}. + * NameNode feature that is added at layout version {@code ancestorLV}. * @param lv new layout version with the addition of this feature * @param ancestorLV layout version from which the new lv is derived from. * @param minCompatLV minimum compatible layout version From d7fd61d6163950a196a2bd66e1c3d03c9cff1009 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 18 Apr 2022 19:24:35 +0100 Subject: [PATCH 119/145] HADOOP-18202. create-release fails fatal: unsafe repository (#4188) Since April 2022/CVE-2022-24765, git refuses to work in directories whose owner != the current user, unless explicitly told to trust it. This patches the create-release script to trust the /build/source dir mounted from the hosting OS, whose userid is inevitably different from that of the account in the container running git. Contributed by: Steve Loughran, Ayush Saxena and the new git error messages --- dev-support/bin/create-release | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dev-support/bin/create-release b/dev-support/bin/create-release index 31ae6ee1b0659..0ba55e7e9573c 100755 --- a/dev-support/bin/create-release +++ b/dev-support/bin/create-release @@ -535,6 +535,10 @@ function makearelease big_console_header "Cleaning the Source Tree" + # Since CVE-2022-24765 in April 2022, git refuses to work in directories + # whose owner != the current user, unless explicitly told to trust it. + git config --global --add safe.directory /build/source + # git clean to clear any remnants from previous build run "${GIT}" clean -xdf -e /patchprocess From 52e152f8b0d5f522f3b799ea72c6c887d5d2c42d Mon Sep 17 00:00:00 2001 From: qinyuren <1476659627@qq.com> Date: Tue, 19 Apr 2022 12:37:28 +0800 Subject: [PATCH 120/145] HDFS-16538. EC decoding failed due to not enough valid inputs (#4167) Co-authored-by: liubingxing --- .../hadoop/hdfs/StatefulStripeReader.java | 4 +++- .../hdfs/TestReadStripedFileWithDecoding.java | 18 ++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java index 9069a558b67d7..9fb86e513ebc9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java @@ -52,7 +52,9 @@ void prepareDecodeInputs() { cur = dfsStripedInputStream.getCurStripeBuf().duplicate(); } - this.decodeInputs = new ECChunk[dataBlkNum + parityBlkNum]; + if (this.decodeInputs == null) { + this.decodeInputs = new ECChunk[dataBlkNum + parityBlkNum]; + } int bufLen = (int) alignedStripe.getSpanInBlock(); int bufOff = (int) alignedStripe.getOffsetInBlock(); for (int i = 0; i < dataBlkNum; i++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 132eb611a2dce..093138beb6981 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -44,6 +44,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.BLOCK_SIZE; import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.CELL_SIZE; import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_DATA_UNITS; import static org.apache.hadoop.hdfs.ReadStripedFileWithDecodingHelper.NUM_PARITY_UNITS; @@ -165,4 +166,21 @@ public void testInvalidateBlock() throws IOException, InterruptedException { DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false); } } + + @Test + public void testMoreThanOneCorruptedBlock() throws IOException { + final Path file = new Path("/corrupted"); + final int length = BLOCK_SIZE * NUM_DATA_UNITS; + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(dfs, file, bytes); + + // read the file with more than one corrupted data block + byte[] buffer = new byte[length + 100]; + for (int count = 2; count < NUM_PARITY_UNITS; ++count) { + ReadStripedFileWithDecodingHelper.corruptBlocks(cluster, dfs, file, count, 0, + false); + StripedFileTestUtil.verifyStatefulRead(dfs, file, length, bytes, + buffer); + } + } } From ec0ff1dc04b2ced199d71543a8260e9225d9e014 Mon Sep 17 00:00:00 2001 From: Ashutosh Gupta Date: Tue, 19 Apr 2022 06:35:23 +0100 Subject: [PATCH 121/145] HDFS-16035. Remove DummyGroupMapping as it is not longer used anywhere. (#4183) Signed-off-by: Akira Ajisaka --- .../service/security/DummyGroupMapping.java | 65 ------------------- 1 file changed, 65 deletions(-) delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/DummyGroupMapping.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/DummyGroupMapping.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/DummyGroupMapping.java deleted file mode 100644 index 4af93182ed341..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/security/DummyGroupMapping.java +++ /dev/null @@ -1,65 +0,0 @@ -/** - * 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.hadoop.lib.service.security; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; - -import org.apache.hadoop.security.GroupMappingServiceProvider; -import org.apache.hadoop.test.HadoopUsersConfTestHelper; -import org.apache.hadoop.util.Sets; - -public class DummyGroupMapping implements GroupMappingServiceProvider { - - @Override - public List getGroups(String user) throws IOException { - if (user.equals("root")) { - return Arrays.asList("admin"); - } - else if (user.equals("nobody")) { - return Arrays.asList("nobody"); - } else { - String[] groups = HadoopUsersConfTestHelper.getHadoopUserGroups(user); - return (groups != null) ? Arrays.asList(groups) : Collections.emptyList(); - } - } - - @Override - public void cacheGroupsRefresh() throws IOException { - } - - @Override - public void cacheGroupsAdd(List groups) throws IOException { - } - - @Override - public Set getGroupsSet(String user) throws IOException { - if (user.equals("root")) { - return Sets.newHashSet("admin"); - } else if (user.equals("nobody")) { - return Sets.newHashSet("nobody"); - } else { - String[] groups = HadoopUsersConfTestHelper.getHadoopUserGroups(user); - return (groups != null) ? Sets.newHashSet(groups) : - Collections.emptySet(); - } - } -} From 98b9c435f24c76e3629f6d8e9c14473b46fe6a0e Mon Sep 17 00:00:00 2001 From: Xing Lin Date: Tue, 19 Apr 2022 22:47:02 -0700 Subject: [PATCH 122/145] HADOOP-18172: Changed scope for isRootInternalDir/getRootFallbackLink for InodeTree (#4106) * HADOOP-18172: Change scope of InodeTree and its member methods to make them accessible from outside package. Co-authored-by: Xing Lin --- .../java/org/apache/hadoop/fs/viewfs/InodeTree.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index ef40f98cc02e0..23ad053a67d5c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -61,7 +61,7 @@ @InterfaceAudience.Private @InterfaceStability.Unstable -abstract class InodeTree { +public abstract class InodeTree { private static final Logger LOGGER = LoggerFactory.getLogger(InodeTree.class.getName()); @@ -458,11 +458,11 @@ private boolean hasFallbackLink() { * there will be root to root mapping. So, root does not represent as * internalDir. */ - protected boolean isRootInternalDir() { + public boolean isRootInternalDir() { return root.isInternalDir(); } - protected INodeLink getRootFallbackLink() { + public INodeLink getRootFallbackLink() { Preconditions.checkState(root.isInternalDir()); return rootFallbackLink; } @@ -742,7 +742,7 @@ private LinkEntry buildLinkRegexEntry( * If the input pathname leads to an internal mount-table entry then * the target file system is one that represents the internal inode. */ - static class ResolveResult { + public static class ResolveResult { final ResultKind kind; final T targetFileSystem; final String resolvedPath; @@ -777,7 +777,7 @@ boolean isLastInternalDirLink() { * @return ResolveResult which allows further resolution of the remaining path * @throws IOException */ - ResolveResult resolve(final String p, final boolean resolveLastComponent) + public ResolveResult resolve(final String p, final boolean resolveLastComponent) throws IOException { ResolveResult resolveResult = null; String[] path = breakIntoPathComponents(p); @@ -957,7 +957,7 @@ public List> getMountPoints() { * @return home dir value from mount table; null if no config value * was found. */ - String getHomeDirPrefixValue() { + public String getHomeDirPrefixValue() { return homedirPrefix; } } From 76bbd173749f2af4f17946fb37c4c72e2de26764 Mon Sep 17 00:00:00 2001 From: qinyuren <1476659627@qq.com> Date: Wed, 20 Apr 2022 14:04:27 +0800 Subject: [PATCH 123/145] HDFS-16544. EC decoding failed due to invalid buffer (#4179) --- .../hadoop/hdfs/StatefulStripeReader.java | 5 --- .../hdfs/TestReadStripedFileWithDecoding.java | 38 +++++++++++++++++++ 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java index 9fb86e513ebc9..730307b4434cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StatefulStripeReader.java @@ -74,11 +74,6 @@ void prepareDecodeInputs() { boolean prepareParityChunk(int index) { Preconditions.checkState(index >= dataBlkNum && alignedStripe.chunks[index] == null); - if (readerInfos[index] != null && readerInfos[index].shouldSkip) { - alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING); - // we have failed the block reader before - return false; - } final int parityIndex = index - dataBlkNum; ByteBuffer buf = dfsStripedInputStream.getParityBuffer().duplicate(); buf.position(cellSize * parityIndex); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java index 093138beb6981..f80cb01bab8a8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -19,7 +19,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; @@ -183,4 +185,40 @@ public void testMoreThanOneCorruptedBlock() throws IOException { buffer); } } + + @Test + public void testReadWithCorruptedDataBlockAndParityBlock() throws IOException { + final Path file = new Path("/corruptedDataBlockAndParityBlock"); + final int length = BLOCK_SIZE * NUM_DATA_UNITS; + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(dfs, file, bytes); + + // set one dataBlock and the first parityBlock corrupted + int dataBlkDelNum = 1; + int parityBlkDelNum = 1; + int recoverBlkNum = dataBlkDelNum + parityBlkDelNum; + int[] dataBlkIndices = {0}; + int[] parityBlkIndices = {6}; + + LocatedBlocks locatedBlocks = ReadStripedFileWithDecodingHelper.getLocatedBlocks(dfs, file); + LocatedStripedBlock lastBlock = + (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + + int[] delBlkIndices = new int[recoverBlkNum]; + System.arraycopy(dataBlkIndices, 0, + delBlkIndices, 0, dataBlkIndices.length); + System.arraycopy(parityBlkIndices, 0, + delBlkIndices, dataBlkIndices.length, parityBlkIndices.length); + ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum]; + for (int i = 0; i < recoverBlkNum; i++) { + delBlocks[i] = StripedBlockUtil + .constructInternalBlock(lastBlock.getBlock(), + CELL_SIZE, NUM_DATA_UNITS, delBlkIndices[i]); + cluster.corruptBlockOnDataNodes(delBlocks[i]); + } + + byte[] buffer = new byte[length + 100]; + StripedFileTestUtil.verifyStatefulRead(dfs, file, length, bytes, + buffer); + } } From 4ff8a5dc7386b059fa6de126551365d19a72073e Mon Sep 17 00:00:00 2001 From: Renukaprasad C <48682981+prasad-acit@users.noreply.github.com> Date: Wed, 20 Apr 2022 18:57:43 +0530 Subject: [PATCH 124/145] HDFS-16526. Addendum Add metrics for slow DataNode (#4191) --- hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md | 1 + 1 file changed, 1 insertion(+) diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md index 4b17e1504862e..47786e473a52f 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md @@ -483,6 +483,7 @@ Each metrics record contains tags such as SessionId and Hostname as additional i | `PacketsSlowWriteToOsCache` | Total number of packets whose write to os cache takes more than a certain time (300ms by default) | | `slowFlushOrSyncCount` | Total number of packets whose sync/flush takes more than a certain time (300ms by default) | | `slowAckToUpstreamCount` | Total number of packets whose upstream ack takes more than a certain time (300ms by default) | + FsVolume -------- From a4683be65ec315f1561ffa1639aed3c99be61f3f Mon Sep 17 00:00:00 2001 From: S O'Donnell Date: Wed, 20 Apr 2022 20:34:43 +0100 Subject: [PATCH 125/145] Revert "HDFS-16531. Avoid setReplication writing an edit record if old replication equals the new value (#4148). Contributed by Stephen O'Donnell." This reverts commit dbeeee03639f41a022dd07d5fc04e3aa65a94b5f. --- .../hdfs/server/namenode/FSDirAttrOp.java | 31 ++++++------------- .../hdfs/server/namenode/FSNamesystem.java | 11 +++---- .../hadoop/hdfs/TestSetrepIncreasing.java | 6 ---- 3 files changed, 15 insertions(+), 33 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java index a2c9f6bd76bdc..04913d1a7cee0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java @@ -48,11 +48,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_QUOTA_BY_STORAGETYPE_ENABLED_KEY; public class FSDirAttrOp { - - protected enum SetRepStatus { - UNCHANGED, INVALID, SUCCESS - } - static FileStatus setPermission( FSDirectory fsd, FSPermissionChecker pc, final String src, FsPermission permission) throws IOException { @@ -139,11 +134,11 @@ static FileStatus setTimes( return fsd.getAuditFileInfo(iip); } - static SetRepStatus setReplication( + static boolean setReplication( FSDirectory fsd, FSPermissionChecker pc, BlockManager bm, String src, final short replication) throws IOException { bm.verifyReplication(src, replication, null); - final SetRepStatus status; + final boolean isFile; fsd.writeLock(); try { final INodesInPath iip = fsd.resolvePath(pc, src, DirOp.WRITE); @@ -151,14 +146,16 @@ static SetRepStatus setReplication( fsd.checkPathAccess(pc, iip, FsAction.WRITE); } - status = unprotectedSetReplication(fsd, iip, replication); - if (status == SetRepStatus.SUCCESS) { + final BlockInfo[] blocks = unprotectedSetReplication(fsd, iip, + replication); + isFile = blocks != null; + if (isFile) { fsd.getEditLog().logSetReplication(iip.getPath(), replication); } } finally { fsd.writeUnlock(); } - return status; + return isFile; } static FileStatus unsetStoragePolicy(FSDirectory fsd, FSPermissionChecker pc, @@ -384,7 +381,7 @@ static INodeDirectory unprotectedSetQuota( return dirNode; } - static SetRepStatus unprotectedSetReplication( + static BlockInfo[] unprotectedSetReplication( FSDirectory fsd, INodesInPath iip, short replication) throws QuotaExceededException, UnresolvedLinkException, SnapshotAccessControlException, UnsupportedActionException { @@ -394,20 +391,12 @@ static SetRepStatus unprotectedSetReplication( final INode inode = iip.getLastINode(); if (inode == null || !inode.isFile() || inode.asFile().isStriped()) { // TODO we do not support replication on stripe layout files yet - // We return invalid here, so we skip writing an edit, but also write an - // unsuccessful audit message. - return SetRepStatus.INVALID; + return null; } INodeFile file = inode.asFile(); // Make sure the directory has sufficient quotas short oldBR = file.getPreferredBlockReplication(); - if (oldBR == replication) { - // No need to do anything as the requested rep factor is the same as - // existing. Returning UNCHANGED to we can skip writing edits, but still - // log a successful audit message. - return SetRepStatus.UNCHANGED; - } long size = file.computeFileSize(true, true); // Ensure the quota does not exceed @@ -438,7 +427,7 @@ static SetRepStatus unprotectedSetReplication( oldBR, iip.getPath()); } } - return SetRepStatus.SUCCESS; + return file.getBlocks(); } static void unprotectedSetStoragePolicy(FSDirectory fsd, BlockManager bm, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 6ab57ed880a42..0a00aa65c0bcb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -2448,7 +2448,7 @@ void createSymlink(String target, String link, boolean setReplication(final String src, final short replication) throws IOException { final String operationName = "setReplication"; - FSDirAttrOp.SetRepStatus status; + boolean success = false; checkOperation(OperationCategory.WRITE); final FSPermissionChecker pc = getPermissionChecker(); FSPermissionChecker.setOperationType(operationName); @@ -2457,7 +2457,7 @@ boolean setReplication(final String src, final short replication) try { checkOperation(OperationCategory.WRITE); checkNameNodeSafeMode("Cannot set replication for " + src); - status = FSDirAttrOp.setReplication(dir, pc, blockManager, src, + success = FSDirAttrOp.setReplication(dir, pc, blockManager, src, replication); } finally { writeUnlock(operationName, getLockReportInfoSupplier(src)); @@ -2466,12 +2466,11 @@ boolean setReplication(final String src, final short replication) logAuditEvent(false, operationName, src); throw e; } - if (status == FSDirAttrOp.SetRepStatus.SUCCESS) { + if (success) { getEditLog().logSync(); + logAuditEvent(true, operationName, src); } - logAuditEvent(status != FSDirAttrOp.SetRepStatus.INVALID, - operationName, src); - return status != FSDirAttrOp.SetRepStatus.INVALID; + return success; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java index d89b0777633b2..497d450de2508 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetrepIncreasing.java @@ -82,12 +82,6 @@ static void setrep(int fromREP, int toREP, boolean simulatedStorage) throws IOEx public void testSetrepIncreasing() throws IOException { setrep(3, 7, false); } - - @Test(timeout=120000) - public void testSetrepSameRepValue() throws IOException { - setrep(3, 3, false); - } - @Test(timeout=120000) public void testSetrepIncreasingSimulatedStorage() throws IOException { setrep(3, 7, true); From bf4730b4d69e796cb0d968aa5893bdb506effb2f Mon Sep 17 00:00:00 2001 From: smarthan <1139557635@qq.com> Date: Thu, 21 Apr 2022 12:32:12 +0800 Subject: [PATCH 126/145] HDFS-16500. Make asynchronous blocks deletion lock and unlock durtion threshold configurable. (#4061). Contributed by Chengwei Wang. --- .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 11 +++++++++++ .../server/blockmanagement/BlockManager.java | 10 ++++++++-- .../src/main/resources/hdfs-default.xml | 19 +++++++++++++++++++ 3 files changed, 38 insertions(+), 2 deletions(-) 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 cf1755cd9f9b8..9ed564ebd6fd1 100755 --- 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 @@ -481,6 +481,17 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.namenode.block.deletion.increment"; public static final int DFS_NAMENODE_BLOCK_DELETION_INCREMENT_DEFAULT = 1000; + /** The limit of single lock holding duration.*/ + public static final String DFS_NAMENODE_BLOCK_DELETION_LOCK_THRESHOLD_MS = + "dfs.namenode.block.deletion.lock.threshold.ms"; + public static final int DFS_NAMENODE_BLOCK_DELETION_LOCK_THRESHOLD_MS_DEFAULT = + 50; + /** The sleep interval for releasing lock.*/ + public static final String DFS_NAMENODE_BLOCK_DELETION_UNLOCK_INTERVAL_MS = + "dfs.namenode.block.deletion.unlock.interval.ms"; + public static final int DFS_NAMENODE_BLOCK_DELETION_UNLOCK_INTERVAL_MS_DEFAULT = + 10; + public static final String DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES = HdfsClientConfigKeys.DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES; public static final boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index 59b3fb32e38a5..f3f8f734cc5f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -192,8 +192,8 @@ public class BlockManager implements BlockStatsMXBean { private volatile long lowRedundancyBlocksCount = 0L; private volatile long scheduledReplicationBlocksCount = 0L; - private final long deleteBlockLockTimeMs = 500; - private final long deleteBlockUnlockIntervalTimeMs = 100; + private final long deleteBlockLockTimeMs; + private final long deleteBlockUnlockIntervalTimeMs; /** flag indicating whether replication queues have been initialized */ private boolean initializedReplQueues; @@ -495,6 +495,12 @@ public BlockManager(final Namesystem namesystem, boolean haEnabled, startupDelayBlockDeletionInMs = conf.getLong( DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_KEY, DFSConfigKeys.DFS_NAMENODE_STARTUP_DELAY_BLOCK_DELETION_SEC_DEFAULT) * 1000L; + deleteBlockLockTimeMs = conf.getLong( + DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_LOCK_THRESHOLD_MS, + DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_LOCK_THRESHOLD_MS_DEFAULT); + deleteBlockUnlockIntervalTimeMs = conf.getLong( + DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_UNLOCK_INTERVAL_MS, + DFSConfigKeys.DFS_NAMENODE_BLOCK_DELETION_UNLOCK_INTERVAL_MS_DEFAULT); invalidateBlocks = new InvalidateBlocks( datanodeManager.getBlockInvalidateLimit(), startupDelayBlockDeletionInMs, 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 a75a76d7af4ed..b21134cbae274 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -6136,6 +6136,25 @@ + + dfs.namenode.block.deletion.lock.threshold.ms + 50 + + The limit of single time lock holding duration for the block asynchronous + deletion thread. + + + + + dfs.namenode.block.deletion.unlock.interval.ms + 10 + + The sleep interval for yield lock. + When the single time lock holding duration of the block asynchronous deletion + thread exceeds limit, sleeping to yield lock. + + + dfs.namenode.rpc-address.auxiliary-ports From 56cfd6061770872ce35cf815544b0c0f49613209 Mon Sep 17 00:00:00 2001 From: Ashutosh Gupta Date: Thu, 21 Apr 2022 14:16:12 +0100 Subject: [PATCH 127/145] HADOOP-17551. Upgrade maven-site-plugin to 3.11.0 (#4196) Reviewed-by: Ayush Saxena Signed-off-by: Akira Ajisaka --- pom.xml | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index d3e95d990d020..1f6aa885666c4 100644 --- a/pom.xml +++ b/pom.xml @@ -95,7 +95,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x 2.8.1 - 3.6 + 3.11.0 1.5 1.7 2.4 @@ -313,11 +313,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x wagon-ssh ${wagon-ssh.version} - - org.apache.maven.doxia - doxia-module-markdown - 1.8 - - + -# class `org.apache.hadoop.fs.FSDataInputStreamBuilder` +# class `org.apache.hadoop.fs.FutureDataInputStreamBuilder` @@ -27,7 +27,7 @@ file for reading. ## Invariants -The `FSDataInputStreamBuilder` interface does not require parameters or +The `FutureDataInputStreamBuilder` interface does not require parameters or or the state of `FileSystem` until [`build()`](#build) is invoked and/or during the asynchronous open operation itself. @@ -39,11 +39,11 @@ path validation. ## Implementation-agnostic parameters. -### `FSDataInputStreamBuilder bufferSize(int bufSize)` +### `FutureDataInputStreamBuilder bufferSize(int bufSize)` Set the size of the buffer to be used. -### `FSDataInputStreamBuilder withFileStatus(FileStatus status)` +### `FutureDataInputStreamBuilder withFileStatus(FileStatus status)` A `FileStatus` instance which refers to the file being opened. @@ -53,7 +53,7 @@ So potentially saving on remote calls especially to object stores. Requirements: * `status != null` -* `status.getPath()` == the resolved path of the file being opened. +* `status.getPath().getName()` == the name of the file being opened. The path validation MUST take place if the store uses the `FileStatus` when it opens files, and MAY be performed otherwise. The validation @@ -65,27 +65,85 @@ If a filesystem implementation extends the `FileStatus` returned in its implementation MAY use this information when opening the file. This is relevant with those stores which return version/etag information, -including the S3A and ABFS connectors -they MAY use this to guarantee that -the file they opened is exactly the one returned in the listing. +-they MAY use this to guarantee that the file they opened +is exactly the one returned in the listing. + + +The final `status.getPath().getName()` element of the supplied status MUST equal +the name value of the path supplied to the `openFile(path)` call. + +Filesystems MUST NOT validate the rest of the path. +This is needed to support viewfs and other mount-point wrapper filesystems +where schemas and paths are different. These often create their own FileStatus results + +Preconditions + +```python +status == null or status.getPath().getName() == path.getName() + +``` + +Filesystems MUST NOT require the class of `status` to equal +that of any specific subclass their implementation returns in filestatus/list +operations. This is to support wrapper filesystems and serialization/deserialization +of the status. + ### Set optional or mandatory parameters - FSDataInputStreamBuilder opt(String key, ...) - FSDataInputStreamBuilder must(String key, ...) + FutureDataInputStreamBuilder opt(String key, ...) + FutureDataInputStreamBuilder must(String key, ...) Set optional or mandatory parameters to the builder. Using `opt()` or `must()`, client can specify FS-specific parameters without inspecting the concrete type of `FileSystem`. +Example: + ```java out = fs.openFile(path) - .opt("fs.s3a.experimental.input.fadvise", "random") - .must("fs.s3a.readahead.range", 256 * 1024) + .must("fs.option.openfile.read.policy", "random") + .opt("fs.http.connection.timeout", 30_000L) .withFileStatus(statusFromListing) .build() .get(); ``` +Here the read policy of `random` has been specified, +with the requirement that the filesystem implementation must understand the option. +An http-specific option has been supplied which may be interpreted by any store; +If the filesystem opening the file does not recognize the option, it can safely be +ignored. + +### When to use `opt()` versus `must()` + +The difference between `opt()` versus `must()` is how the FileSystem opening +the file must react to an option which it does not recognize. + +```python + +def must(name, value): + if not name in known_keys: + raise IllegalArgumentException + if not name in supported_keys: + raise UnsupportedException + + +def opt(name, value): + if not name in known_keys: + # ignore option + +``` + +For any known key, the validation of the `value` argument MUST be the same +irrespective of how the (key, value) pair was declared. + +1. For a filesystem-specific option, it is the choice of the implementation + how to validate the entry. +1. For standard options, the specification of what is a valid `value` is + defined in this filesystem specification, validated through contract + tests. + #### Implementation Notes Checking for supported options must be performed in the `build()` operation. @@ -93,9 +151,9 @@ Checking for supported options must be performed in the `build()` operation. 1. If a mandatory parameter declared via `must(key, value)`) is not recognized, `IllegalArgumentException` MUST be thrown. -1. If a mandatory parameter declared via `must(key, value)`) relies on +1. If a mandatory parameter declared via `must(key, value)` relies on a feature which is recognized but not supported in the specific -Filesystem/FileContext instance `UnsupportedException` MUST be thrown. +`FileSystem`/`FileContext` instance `UnsupportedException` MUST be thrown. The behavior of resolving the conflicts between the parameters set by builder methods (i.e., `bufferSize()`) and `opt()`/`must()` is as follows: @@ -110,13 +168,18 @@ custom subclasses. This is critical to ensure safe use of the feature: directory listing/ status serialization/deserialization can result result in the `withFileStatus()` -argumennt not being the custom subclass returned by the Filesystem instance's +argument not being the custom subclass returned by the Filesystem instance's own `getFileStatus()`, `listFiles()`, `listLocatedStatus()` calls, etc. In such a situation the implementations must: -1. Validate the path (always). -1. Use the status/convert to the custom type, *or* simply discard it. +1. Verify that `status.getPath().getName()` matches the current `path.getName()` + value. The rest of the path MUST NOT be validated. +1. Use any status fields as desired -for example the file length. + +Even if not values of the status are used, the presence of the argument +can be interpreted as the caller declaring that they believe the file +to be present and of the given size. ## Builder interface @@ -128,26 +191,499 @@ completed, returns an input stream which can read data from the filesystem. The `build()` operation MAY perform the validation of the file's existence, its kind, so rejecting attempts to read from a directory or non-existent -file. **Alternatively**, the `build()` operation may delay all checks -until an asynchronous operation whose outcome is provided by the `Future` +file. Alternatively +* file existence/status checks MAY be performed asynchronously within the returned + `CompletableFuture<>`. +* file existence/status checks MAY be postponed until the first byte is read in + any of the read such as `read()` or `PositionedRead`. That is, the precondition `exists(FS, path)` and `isFile(FS, path)` are -only guaranteed to have been met after the `get()` on the returned future is successful. +only guaranteed to have been met after the `get()` called on returned future +and an attempt has been made to read the stream. -Thus, if even a file does not exist, the following call will still succeed, returning -a future to be evaluated. +Thus, if even when file does not exist, or is a directory rather than a file, +the following call MUST succeed, returning a `CompletableFuture` to be evaluated. ```java Path p = new Path("file://tmp/file-which-does-not-exist"); CompletableFuture future = p.getFileSystem(conf) .openFile(p) - .build; + .build(); ``` -The preconditions for opening the file are checked during the asynchronous -evaluation, and so will surface when the future is completed: +The inability to access/read a file MUST raise an `IOException`or subclass +in either the future's `get()` call, or, for late binding operations, +when an operation to read data is invoked. + +Therefore the following sequence SHALL fail when invoked on the +`future` returned by the previous example. ```java -FSDataInputStream in = future.get(); + future.get().read(); ``` + +Access permission checks have the same visibility requirements: permission failures +MUST be delayed until the `get()` call and MAY be delayed into subsequent operations. + +Note: some operations on the input stream, such as `seek()` may not attempt any IO +at all. Such operations MAY NOT raise exceotions when interacting with +nonexistent/unreadable files. + +## Standard `openFile()` options since Hadoop 3.3.3 + +These are options which `FileSystem` and `FileContext` implementation +MUST recognise and MAY support by changing the behavior of +their input streams as appropriate. + +Hadoop 3.3.0 added the `openFile()` API; these standard options were defined in +a later release. Therefore, although they are "well known", unless confident that +the application will only be executed against releases of Hadoop which knows of +the options -applications SHOULD set the options via `opt()` calls rather than `must()`. + +When opening a file through the `openFile()` builder API, callers MAY use +both `.opt(key, value)` and `.must(key, value)` calls to set standard and +filesystem-specific options. + +If set as an `opt()` parameter, unsupported "standard" options MUST be ignored, +as MUST unrecognized standard options. + +If set as a `must()` parameter, unsupported "standard" options MUST be ignored. +unrecognized standard options MUST be rejected. + +The standard `openFile()` options are defined +in `org.apache.hadoop.fs.OpenFileOptions`; they all SHALL start +with `fs.option.openfile.`. + +Note that while all `FileSystem`/`FileContext` instances SHALL support these +options to the extent that `must()` declarations SHALL NOT fail, the +implementations MAY support them to the extent of interpreting the values. This +means that it is not a requirement for the stores to actually read the read +policy or file length values and use them when opening files. + +Unless otherwise stated, they SHOULD be viewed as hints. + +Note: if a standard option is added such that if set but not +supported would be an error, then implementations SHALL reject it. For example, +the S3A filesystem client supports the ability to push down SQL commands. If +something like that were ever standardized, then the use of the option, either +in `opt()` or `must()` argument MUST be rejected for filesystems which don't +support the feature. + +### Option: `fs.option.openfile.buffer.size` + +Read buffer size in bytes. + +This overrides the default value set in the configuration with the option +`io.file.buffer.size`. + +It is supported by all filesystem clients which allow for stream-specific buffer +sizes to be set via `FileSystem.open(path, buffersize)`. + +### Option: `fs.option.openfile.read.policy` + +Declare the read policy of the input stream. This is a hint as to what the +expected read pattern of an input stream will be. This MAY control readahead, +buffering and other optimizations. + +Sequential reads may be optimized with prefetching data and/or reading data in +larger blocks. Some applications (e.g. distCp) perform sequential IO even over +columnar data. + +In contrast, random IO reads data in different parts of the file using a +sequence of `seek()/read()` +or via the `PositionedReadable` or `ByteBufferPositionedReadable` APIs. + +Random IO performance may be best if little/no prefetching takes place, along +with other possible optimizations + +Queries over columnar formats such as Apache ORC and Apache Parquet perform such +random IO; other data formats may be best read with sequential or whole-file +policies. + +What is key is that optimizing reads for seqential reads may impair random +performance -and vice versa. + +1. The seek policy is a hint; even if declared as a `must()` option, the + filesystem MAY ignore it. +1. The interpretation/implementation of a policy is a filesystem specific + behavior -and it may change with Hadoop releases and/or specific storage + subsystems. +1. If a policy is not recognized, the filesystem client MUST ignore it. + +| Policy | Meaning | +|--------------|----------------------------------------------------------| +| `adaptive` | Any adaptive policy implemented by the store. | +| `default` | The default policy for this store. Generally "adaptive". | +| `random` | Optimize for random access. | +| `sequential` | Optimize for sequential access. | +| `vector` | The Vectored IO API is intended to be used. | +| `whole-file` | The whole file will be read. | + +Choosing the wrong read policy for an input source may be inefficient. + +A list of read policies MAY be supplied; the first one recognized/supported by +the filesystem SHALL be the one used. This allows for custom policies to be +supported, for example an `hbase-hfile` policy optimized for HBase HFiles. + +The S3A and ABFS input streams both implement +the [IOStatisticsSource](iostatistics.html) API, and can be queried for their IO +Performance. + +*Tip:* log the `toString()` value of input streams at `DEBUG`. The S3A and ABFS +Input Streams log read statistics, which can provide insight about whether reads +are being performed efficiently or not. + +_Futher reading_ + +* [Linux fadvise()](https://linux.die.net/man/2/fadvise). +* [Windows `CreateFile()`](https://docs.microsoft.com/en-us/windows/win32/api/fileapi/nf-fileapi-createfilea#caching-behavior) + +#### Read Policy `adaptive` + +Try to adapt the seek policy to the read pattern of the application. + +The `normal` policy of the S3A client and the sole policy supported by +the `wasb:` client are both adaptive -they assume sequential IO, but once a +backwards seek/positioned read call is made the stream switches to random IO. + +Other filesystem implementations may wish to adopt similar strategies, and/or +extend the algorithms to detect forward seeks and/or switch from random to +sequential IO if that is considered more efficient. + +Adaptive read policies are the absence of the ability to +declare the seek policy in the `open()` API, so requiring it to be declared, if +configurable, in the cluster/application configuration. However, the switch from +sequential to random seek policies may be exensive. + +When applications explicitly set the `fs.option.openfile.read.policy` option, if +they know their read plan, they SHOULD declare which policy is most appropriate. + +#### Read Policy `` + +The default policy for the filesystem instance. +Implementation/installation-specific. + +#### Read Policy `sequential` + +Expect sequential reads from the first byte read to the end of the file/until +the stream is closed. + +#### Read Policy `random` + +Expect `seek()/read()` sequences, or use of `PositionedReadable` +or `ByteBufferPositionedReadable` APIs. + + +#### Read Policy `vector` + +This declares that the caller intends to use the Vectored read API of +[HADOOP-11867](https://issues.apache.org/jira/browse/HADOOP-11867) +_Add a high-performance vectored read API_. + +This is a hint: it is not a requirement when using the API. +It does inform the implemenations that the stream should be +configured for optimal vectored IO performance, if such a +feature has been implemented. + +It is *not* exclusive: the same stream may still be used for +classic `InputStream` and `PositionedRead` API calls. +Implementations SHOULD use the `random` read policy +with these operations. + +#### Read Policy `whole-file` + + +This declares that the whole file is to be read end-to-end; the file system client is free to enable +whatever strategies maximise performance for this. In particular, larger ranged reads/GETs can +deliver high bandwidth by reducing socket/TLS setup costs and providing a connection long-lived +enough for TCP flow control to determine the optimal download rate. + +Strategies can include: + +* Initiate an HTTP GET of the entire file in `openFile()` operation. +* Prefech data in large blocks, possibly in parallel read operations. + +Applications which know that the entire file is to be read from an opened stream SHOULD declare this +read policy. + +### Option: `fs.option.openfile.length` + +Declare the length of a file. + +This can be used by clients to skip querying a remote store for the size +of/existence of a file when opening it, similar to declaring a file status +through the `withFileStatus()` option. + +If supported by a filesystem connector, this option MUST be interpreted as +declaring the minimum length of the file: + +1. If the value is negative, the option SHALL be considered unset. +2. It SHALL NOT be an error if the actual length of the file is greater than + this value. +3. `read()`, `seek()` and positioned read calls MAY use a position across/beyond + this length but below the actual length of the file. Implementations MAY + raise `EOFExceptions` in such cases, or they MAY return data. + +If this option is used by the FileSystem implementation + +*Implementor's Notes* + +* A value of `fs.option.openfile.length` < 0 MUST be rejected. +* If a file status is supplied along with a value in `fs.opt.openfile.length`; + the file status values take precedence. + +### Options: `fs.option.openfile.split.start` and `fs.option.openfile.split.end` + +Declare the start and end of the split when a file has been split for processing +in pieces. + +1. If a value is negative, the option SHALL be considered unset. +1. Filesystems MAY assume that the length of the file is greater than or equal + to the value of `fs.option.openfile.split.end`. +1. And that they MAY raise an exception if the client application reads past the + value set in `fs.option.openfile.split.end`. +1. The pair of options MAY be used to optimise the read plan, such as setting + the content range for GET requests, or using the split end as an implicit + declaration of the guaranteed minimum length of the file. +1. If both options are set, and the split start is declared as greater than the + split end, then the split start SHOULD just be reset to zero, rather than + rejecting the operation. + +The split end value can provide a hint as to the end of the input stream. The +split start can be used to optimize any initial read offset for filesystem +clients. + +*Note for implementors: applications will read past the end of a split when they +need to read to the end of a record/line which begins before the end of the +split. + +Therefore clients MUST be allowed to `seek()`/`read()` past the length +set in `fs.option.openfile.split.end` if the file is actually longer +than that value. + +## S3A-specific options + +The S3A Connector supports custom options for readahead and seek policy. + +| Name | Type | Meaning | +|--------------------------------------|----------|-------------------------------------------------------------| +| `fs.s3a.readahead.range` | `long` | readahead range in bytes | +| `fs.s3a.input.async.drain.threshold` | `long` | threshold to switch to asynchronous draining of the stream | +| `fs.s3a.experimental.input.fadvise` | `String` | seek policy. Superceded by `fs.option.openfile.read.policy` | + +If the option set contains a SQL statement in the `fs.s3a.select.sql` statement, +then the file is opened as an S3 Select query. +Consult the S3A documentation for more details. + +## ABFS-specific options + +The ABFS Connector supports custom input stream options. + +| Name | Type | Meaning | +|-----------------------------------|-----------|----------------------------------------------------| +| `fs.azure.buffered.pread.disable` | `boolean` | disable caching on the positioned read operations. | + + +Disables caching on data read through the [PositionedReadable](fsdatainputstream.html#PositionedReadable) +APIs. + +Consult the ABFS Documentation for more details. + +## Examples + +#### Declaring seek policy and split limits when opening a file. + +Here is an example from a proof of +concept `org.apache.parquet.hadoop.util.HadoopInputFile` +reader which uses a (nullable) file status and a split start/end. + +The `FileStatus` value is always passed in -but if it is null, then the split +end is used to declare the length of the file. + +```java +protected SeekableInputStream newStream(Path path, FileStatus stat, + long splitStart, long splitEnd) + throws IOException { + + FutureDataInputStreamBuilder builder = fs.openFile(path) + .opt("fs.option.openfile.read.policy", "vector, random") + .withFileStatus(stat); + + builder.opt("fs.option.openfile.split.start", splitStart); + builder.opt("fs.option.openfile.split.end", splitEnd); + CompletableFuture streamF = builder.build(); + return HadoopStreams.wrap(FutureIO.awaitFuture(streamF)); +} +``` + +As a result, whether driven directly by a file listing, or when opening a file +from a query plan of `(path, splitStart, splitEnd)`, there is no need to probe +the remote store for the length of the file. When working with remote object +stores, this can save tens to hundreds of milliseconds, even if such a probe is +done asynchronously. + +If both the file length and the split end is set, then the file length MUST be +considered "more" authoritative, that is it really SHOULD be defining the file +length. If the split end is set, the caller MAY ot read past it. + +The `CompressedSplitLineReader` can read past the end of a split if it is +partway through processing a compressed record. That is: it assumes an +incomplete record read means that the file length is greater than the split +length, and that it MUST read the entirety of the partially read record. Other +readers may behave similarly. + +Therefore + +1. File length as supplied in a `FileStatus` or in `fs.option.openfile.length` + SHALL set the strict upper limit on the length of a file +2. The split end as set in `fs.option.openfile.split.end` MUST be viewed as a + hint, rather than the strict end of the file. + +### Opening a file with both standard and non-standard options + +Standard and non-standard options MAY be combined in the same `openFile()` +operation. + +```java +Future f = openFile(path) + .must("fs.option.openfile.read.policy", "random, adaptive") + .opt("fs.s3a.readahead.range", 1024 * 1024) + .build(); + +FSDataInputStream is = f.get(); +``` + +The option set in `must()` MUST be understood, or at least recognized and +ignored by all filesystems. In this example, S3A-specific option MAY be +ignored by all other filesystem clients. + +### Opening a file with older releases + +Not all hadoop releases recognize the `fs.option.openfile.read.policy` option. + +The option can be safely used in application code if it is added via the `opt()` +builder argument, as it will be treated as an unknown optional key which can +then be discarded. + +```java +Future f = openFile(path) + .opt("fs.option.openfile.read.policy", "vector, random, adaptive") + .build(); + +FSDataInputStream is = f.get(); +``` + +*Note 1* if the option name is set by a reference to a constant in +`org.apache.hadoop.fs.Options.OpenFileOptions`, then the program will not link +against versions of Hadoop without the specific option. Therefore for resilient +linking against older releases -use a copy of the value. + +*Note 2* as option validation is performed in the FileSystem connector, +a third-party connector designed to work with multiple hadoop versions +MAY NOT support the option. + +### Passing options in to MapReduce + +Hadoop MapReduce will automatically read MR Job Options with the prefixes +`mapreduce.job.input.file.option.` and `mapreduce.job.input.file.must.` +prefixes, and apply these values as `.opt()` and `must()` respectively, after +remove the mapreduce-specific prefixes. + +This makes passing options in to MR jobs straightforward. For example, to +declare that a job should read its data using random IO: + +```java +JobConf jobConf = (JobConf) job.getConfiguration() +jobConf.set( + "mapreduce.job.input.file.option.fs.option.openfile.read.policy", + "random"); +``` + +### MapReduce input format propagating options + +An example of a record reader passing in options to the file it opens. + +```java + public void initialize(InputSplit genericSplit, + TaskAttemptContext context) throws IOException { + FileSplit split = (FileSplit)genericSplit; + Configuration job = context.getConfiguration(); + start = split.getStart(); + end = start + split.getLength(); + Path file = split.getPath(); + + // open the file and seek to the start of the split + FutureDataInputStreamBuilder builder = + file.getFileSystem(job).openFile(file); + // the start and end of the split may be used to build + // an input strategy. + builder.opt("fs.option.openfile.split.start", start); + builder.opt("fs.option.openfile.split.end", end); + FutureIO.propagateOptions(builder, job, + "mapreduce.job.input.file.option", + "mapreduce.job.input.file.must"); + + fileIn = FutureIO.awaitFuture(builder.build()); + fileIn.seek(start) + /* Rest of the operation on the opened stream */ + } +``` + +### `FileContext.openFile` + +From `org.apache.hadoop.fs.AvroFSInput`; a file is opened with sequential input. +Because the file length has already been probed for, the length is passd down + +```java + public AvroFSInput(FileContext fc, Path p) throws IOException { + FileStatus status = fc.getFileStatus(p); + this.len = status.getLen(); + this.stream = awaitFuture(fc.openFile(p) + .opt("fs.option.openfile.read.policy", + "sequential") + .opt("fs.option.openfile.length", + Long.toString(status.getLen())) + .build()); + fc.open(p); + } +``` + +In this example, the length is passed down as a string (via `Long.toString()`) +rather than directly as a long. This is to ensure that the input format will +link against versions of $Hadoop which do not have the +`opt(String, long)` and `must(String, long)` builder parameters. Similarly, the +values are passed as optional, so that if unrecognized the application will +still succeed. + +### Example: reading a whole file + +This is from `org.apache.hadoop.util.JsonSerialization`. + +Its `load(FileSystem, Path, FileStatus)` method +* declares the whole file is to be read end to end. +* passes down the file status + +```java +public T load(FileSystem fs, + Path path, + status) + throws IOException { + + try (FSDataInputStream dataInputStream = + awaitFuture(fs.openFile(path) + .opt("fs.option.openfile.read.policy", "whole-file") + .withFileStatus(status) + .build())) { + return fromJsonStream(dataInputStream); + } catch (JsonProcessingException e) { + throw new PathIOException(path.toString(), + "Failed to read JSON file " + e, e); + } +} +``` + +*Note:* : in Hadoop 3.3.2 and earlier, the `withFileStatus(status)` call +required a non-null parameter; this has since been relaxed. +For maximum compatibility across versions, only invoke the method +when the file status is known to be non-null. \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md index a4aa136033a0c..e18f4c3bf4ab2 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md @@ -41,3 +41,4 @@ HDFS as these are commonly expected by Hadoop client applications. 2. [Extending the specification and its tests](extending.html) 1. [Uploading a file using Multiple Parts](multipartuploader.html) 1. [IOStatistics](iostatistics.html) +1. [openFile()](openfile.html). diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md new file mode 100644 index 0000000000000..afb3245c5105f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/openfile.md @@ -0,0 +1,122 @@ + + +# `FileSystem.openFile()`/`FileContext.openFile()` + +This is a method provided by both FileSystem and FileContext for +advanced file opening options and, where implemented, +an asynchrounous/lazy opening of a file. + +Creates a builder to open a file, supporting options +both standard and filesystem specific. The return +value of the `build()` call is a `Future`, +which must be waited on. The file opening may be +asynchronous, and it may actually be postponed (including +permission/existence checks) until reads are actually +performed. + +This API call was added to `FileSystem` and `FileContext` in +Hadoop 3.3.0; it was tuned in Hadoop 3.3.1 as follows. + +* Added `opt(key, long)` and `must(key, long)`. +* Declared that `withFileStatus(null)` is allowed. +* Declared that `withFileStatus(status)` only checks + the filename of the path, not the full path. + This is needed to support passthrough/mounted filesystems. +* Added standard option keys. + +### `FutureDataInputStreamBuilder openFile(Path path)` + +Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html) +to construct a operation to open the file at `path` for reading. + +When `build()` is invoked on the returned `FutureDataInputStreamBuilder` instance, +the builder parameters are verified and +`FileSystem.openFileWithOptions(Path, OpenFileParameters)` or +`AbstractFileSystem.openFileWithOptions(Path, OpenFileParameters)` invoked. + +These protected methods returns a `CompletableFuture` +which, when its `get()` method is called, either returns an input +stream of the contents of opened file, or raises an exception. + +The base implementation of the `FileSystem.openFileWithOptions(PathHandle, OpenFileParameters)` +ultimately invokes `FileSystem.open(Path, int)`. + +Thus the chain `FileSystem.openFile(path).build().get()` has the same preconditions +and postconditions as `FileSystem.open(Path p, int bufferSize)` + +However, there is one difference which implementations are free to +take advantage of: + +The returned stream MAY implement a lazy open where file non-existence or +access permission failures may not surface until the first `read()` of the +actual data. + +This saves network IO on object stores. + +The `openFile()` operation MAY check the state of the filesystem during its +invocation, but as the state of the filesystem may change between this call and +the actual `build()` and `get()` operations, this file-specific +preconditions (file exists, file is readable, etc) MUST NOT be checked here. + +FileSystem implementations which do not implement `open(Path, int)` +MAY postpone raising an `UnsupportedOperationException` until either the +`FutureDataInputStreamBuilder.build()` or the subsequent `get()` call, +else they MAY fail fast in the `openFile()` call. + +Consult [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html) for details +on how to use the builder, and for standard options which may be passed in. + +### `FutureDataInputStreamBuilder openFile(PathHandle)` + +Creates a [`FutureDataInputStreamBuilder`](fsdatainputstreambuilder.html) +to construct a operation to open the file identified by the given `PathHandle` for reading. + +If implemented by a filesystem, the semantics of [`openFile(Path)`](#openfile_path_) +Thus the chain `openFile(pathhandle).build().get()` has the same preconditions and postconditions +as `open(Pathhandle, int)` + +FileSystem implementations which do not implement `open(PathHandle handle, int bufferSize)` +MAY postpone raising an `UnsupportedOperationException` until either the +`FutureDataInputStreamBuilder.build()` or the subsequent `get()` call, else they MAY fail fast in +the `openFile(PathHandle)` call. + +The base implementation raises this exception in the `build()` operation; other implementations +SHOULD copy this. + +### Implementors notes + +The base implementation of `openFileWithOptions()` actually executes +the `open(path)` operation synchronously, yet still returns the result +or any failures in the `CompletableFuture<>`, so as to provide a consistent +lifecycle across all filesystems. + +Any filesystem client where the time to open a file may be significant SHOULD +execute it asynchronously by submitting the operation in some executor/thread +pool. This is particularly recommended for object stores and other filesystems +likely to be accessed over long-haul connections. + +Arbitrary filesystem-specific options MAY be supported; these MUST +be prefixed with either the filesystem schema, e.g. `hdfs.` +or in the `fs.SCHEMA` format as normal configuration settings `fs.hdfs`. The +latter style allows the same configuration option to be used for both +filesystem configuration and file-specific configuration. + +It SHOULD be possible to always open a file without specifying any options, +so as to present a consistent model to users. However, an implementation MAY +opt to require one or more mandatory options to be set. + +The returned stream may perform "lazy" evaluation of file access. This is +relevant for object stores where the probes for existence are expensive, and, +even with an asynchronous open, may be considered needless. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java index 3e754e4578de8..c395afdb3779b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMultipartUploaderTest.java @@ -50,11 +50,11 @@ import org.apache.hadoop.util.DurationInfo; import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists; -import static org.apache.hadoop.fs.impl.FutureIOSupport.awaitFuture; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; import static org.apache.hadoop.test.LambdaTestUtils.eventually; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; /** * Tests of multipart uploads. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java index a43053180fbf8..25bfe082b01f6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java @@ -30,14 +30,18 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.IOUtils; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.fs.contract.ContractTestUtils.compareByteArrays; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; import static org.apache.hadoop.fs.contract.ContractTestUtils.touch; import static org.apache.hadoop.test.LambdaTestUtils.intercept; import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; import org.junit.Test; @@ -232,7 +236,7 @@ public void testAwaitFutureFailToFNFE() throws Throwable { getFileSystem().openFile(path("testAwaitFutureFailToFNFE")) .opt("fs.test.something", true); intercept(FileNotFoundException.class, - () -> FutureIOSupport.awaitFuture(builder.build())); + () -> awaitFuture(builder.build())); } @Test @@ -242,7 +246,7 @@ public void testAwaitFutureTimeoutFailToFNFE() throws Throwable { getFileSystem().openFile(path("testAwaitFutureFailToFNFE")) .opt("fs.test.something", true); intercept(FileNotFoundException.class, - () -> FutureIOSupport.awaitFuture(builder.build(), + () -> awaitFuture(builder.build(), 10, TimeUnit.DAYS)); } @@ -250,7 +254,7 @@ public void testAwaitFutureTimeoutFailToFNFE() throws Throwable { public void testOpenFileExceptionallyTranslating() throws Throwable { describe("openFile missing file chains into exceptionally()"); CompletableFuture f = getFileSystem() - .openFile(path("testOpenFileUnknownOption")).build(); + .openFile(path("testOpenFileExceptionallyTranslating")).build(); interceptFuture(RuntimeException.class, "exceptionally", f.exceptionally(ex -> { @@ -262,11 +266,12 @@ public void testOpenFileExceptionallyTranslating() throws Throwable { public void testChainedFailureAwaitFuture() throws Throwable { describe("await Future handles chained failures"); CompletableFuture f = getFileSystem() - .openFile(path("testOpenFileUnknownOption")) + .openFile(path("testChainedFailureAwaitFuture")) + .withFileStatus(null) .build(); intercept(RuntimeException.class, "exceptionally", - () -> FutureIOSupport.awaitFuture( + () -> awaitFuture( f.exceptionally(ex -> { throw new RuntimeException("exceptionally", ex); }))); @@ -280,13 +285,34 @@ public void testOpenFileApplyRead() throws Throwable { int len = 4096; createFile(fs, path, true, dataset(len, 0x40, 0x80)); + FileStatus st = fs.getFileStatus(path); CompletableFuture readAllBytes = fs.openFile(path) - .withFileStatus(fs.getFileStatus(path)) + .withFileStatus(st) .build() .thenApply(ContractTestUtils::readStream); assertEquals("Wrong number of bytes read value", len, (long) readAllBytes.get()); + // now reattempt with a new FileStatus and a different path + // other than the final name element + // implementations MUST use path in openFile() call + FileStatus st2 = new FileStatus( + len, false, + st.getReplication(), + st.getBlockSize(), + st.getModificationTime(), + st.getAccessTime(), + st.getPermission(), + st.getOwner(), + st.getGroup(), + new Path("gopher:///localhost:/" + path.getName())); + assertEquals("Wrong number of bytes read value", + len, + (long) fs.openFile(path) + .withFileStatus(st2) + .build() + .thenApply(ContractTestUtils::readStream) + .get()); } @Test @@ -298,17 +324,47 @@ public void testOpenFileApplyAsyncRead() throws Throwable { dataset(4, 0x40, 0x80)); CompletableFuture future = fs.openFile(path).build(); AtomicBoolean accepted = new AtomicBoolean(false); - future.thenAcceptAsync(i -> accepted.set(true)).get(); + future.thenApply(stream -> { + accepted.set(true); + return stream; + }).get().close(); assertTrue("async accept operation not invoked", accepted.get()); } + /** + * Open a file with a null status, and the length + * passed in as an opt() option (along with sequential IO). + * The file is opened, the data read, and it must match + * the source data. + * opt() is used so that integration testing with external + * filesystem connectors will downgrade if the option is not + * recognized. + */ @Test - public void testOpenFileNullStatus() throws Throwable { - describe("use openFile() with a null status"); + public void testOpenFileNullStatusButFileLength() throws Throwable { + describe("use openFile() with a null status and expect the status to be" + + " ignored. block size, fadvise and length are passed in as" + + " opt() options"); Path path = path("testOpenFileNullStatus"); - intercept(NullPointerException.class, - () -> getFileSystem().openFile(path).withFileStatus(null)); + FileSystem fs = getFileSystem(); + int len = 4; + byte[] result = new byte[len]; + byte[] dataset = dataset(len, 0x40, 0x80); + createFile(fs, path, true, + dataset); + CompletableFuture future = fs.openFile(path) + .withFileStatus(null) + .opt(FS_OPTION_OPENFILE_READ_POLICY, + "unknown, sequential, random") + .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, 32768) + .opt(FS_OPTION_OPENFILE_LENGTH, len) + .build(); + + try (FSDataInputStream in = future.get()) { + in.readFully(result); + } + compareByteArrays(dataset, result, len); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index e13a49ca10e70..eb56d957d9a1a 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -1642,17 +1642,22 @@ public static int read(InputStream in) { /** * Read a whole stream; downgrades an IOE to a runtime exception. + * Closes the stream afterwards. * @param in input * @return the number of bytes read. * @throws AssertionError on any IOException */ public static long readStream(InputStream in) { - long count = 0; + try { + long count = 0; - while (read(in) >= 0) { - count++; + while (read(in) >= 0) { + count++; + } + return count; + } finally { + IOUtils.cleanupWithLogger(LOG, in); } - return count; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java index 22f6c33d2e260..755599f0c390c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/IOStatisticAssertions.java @@ -36,6 +36,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX; +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN; import static org.assertj.core.api.Assertions.assertThat; /** @@ -347,6 +349,24 @@ public static AbstractLongAssert assertThatStatisticMaximum( verifyStatisticsNotNull(stats).maximums()); } + /** + * Assert that a duration is within a given minimum/maximum range. + * @param stats statistics source + * @param key statistic key without any suffix + * @param min minimum statistic must be equal to or greater than this. + * @param max maximum statistic must be equal to or less than this. + */ + public static void assertDurationRange( + final IOStatistics stats, + final String key, + final long min, + final long max) { + assertThatStatisticMinimum(stats, key + SUFFIX_MIN) + .isGreaterThanOrEqualTo(min); + assertThatStatisticMaximum(stats, key + SUFFIX_MAX) + .isLessThanOrEqualTo(max); + } + /** * Start an assertion chain on * a required mean statistic. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java index 8258b62c1f759..cfde1583e2c21 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/statistics/TestDurationTracking.java @@ -30,7 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.test.AbstractHadoopTestBase; import org.apache.hadoop.util.functional.FunctionRaisingIOE; @@ -276,7 +275,7 @@ public void testCallableIOEFailureDuration() throws Throwable { */ @Test public void testDurationThroughEval() throws Throwable { - CompletableFuture eval = FutureIOSupport.eval( + CompletableFuture eval = FutureIO.eval( trackDurationOfOperation(stats, REQUESTS, () -> { sleepf(100); throw new FileNotFoundException("oops"); From 6999acf520080def399b6d4b635a8ba0d0bcce94 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sun, 24 Apr 2022 17:10:34 +0100 Subject: [PATCH 136/145] HADOOP-16202. Enhanced openFile(): mapreduce and YARN changes. (#2584/2) These changes ensure that sequential files are opened with the right read policy, and split start/end is passed in. As well as offering opportunities for filesystem clients to choose fetch/cache/seek policies, the settings ensure that processing text files on an s3 bucket where the default policy is "random" will still be processed efficiently. This commit depends on the associated hadoop-common patch, which must be committed first. Contributed by Steve Loughran. Change-Id: Ic6713fd752441cf42ebe8739d05c2293a5db9f94 --- .../jobhistory/JobHistoryCopyService.java | 10 +++++++- .../hadoop/mapred/LineRecordReader.java | 13 +++++++--- .../lib/input/FixedLengthRecordReader.java | 7 +++--- .../mapreduce/lib/input/LineRecordReader.java | 14 ++++++++--- .../mapreduce/lib/input/NLineInputFormat.java | 6 ++--- .../examples/terasort/TeraInputFormat.java | 16 +++++++++++-- .../mapred/RetriableFileCopyCommand.java | 9 ++++++- .../mapreduce/StreamInputFormat.java | 6 ++--- .../logaggregation/AggregatedLogFormat.java | 17 +++++++++++-- .../apache/hadoop/yarn/util/FSDownload.java | 24 +++++++++++++------ 10 files changed, 94 insertions(+), 28 deletions(-) diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java index ee4ec2c86a1c4..ecae4f2fc061a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java @@ -35,6 +35,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; + /** * Reads in history events from the JobHistoryFile and sends them out again * to be recorded. @@ -118,7 +122,11 @@ public static FSDataInputStream getPreviousJobHistoryFileStream( fc.makeQualified(JobHistoryUtils.getStagingJobHistoryFile(histDirPath, jobId, (applicationAttemptId.getAttemptId() - 1))); LOG.info("History file is at " + historyFile); - in = fc.open(historyFile); + in = awaitFuture( + fc.openFile(historyFile) + .opt(FS_OPTION_OPENFILE_READ_POLICY, + FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE) + .build()); return in; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java index 1fcb118a100fc..5724e72931085 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/LineRecordReader.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Seekable; -import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CodecPool; @@ -41,9 +40,13 @@ import org.apache.hadoop.mapreduce.lib.input.CompressedSplitLineReader; import org.apache.hadoop.mapreduce.lib.input.SplitLineReader; import org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader; +import org.apache.hadoop.util.functional.FutureIO; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; + /** * Treats keys as offset in file and value as line. */ @@ -109,10 +112,14 @@ public LineRecordReader(Configuration job, FileSplit split, // open the file and seek to the start of the split final FutureDataInputStreamBuilder builder = file.getFileSystem(job).openFile(file); - FutureIOSupport.propagateOptions(builder, job, + // the start and end of the split may be used to build + // an input strategy. + builder.opt(FS_OPTION_OPENFILE_SPLIT_START, start) + .opt(FS_OPTION_OPENFILE_SPLIT_END, end); + FutureIO.propagateOptions(builder, job, MRJobConfig.INPUT_FILE_OPTION_PREFIX, MRJobConfig.INPUT_FILE_MANDATORY_PREFIX); - fileIn = FutureIOSupport.awaitFuture(builder.build()); + fileIn = FutureIO.awaitFuture(builder.build()); if (isCompressedInput()) { decompressor = CodecPool.getDecompressor(codec); if (codec instanceof SplittableCompressionCodec) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java index c0ae9a5cdac61..6969f61836fbc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FixedLengthRecordReader.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Seekable; -import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.compress.CodecPool; @@ -40,6 +39,8 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.functional.FutureIO; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,10 +95,10 @@ public void initialize(Configuration job, long splitStart, long splitLength, // open the file final FutureDataInputStreamBuilder builder = file.getFileSystem(job).openFile(file); - FutureIOSupport.propagateOptions(builder, job, + FutureIO.propagateOptions(builder, job, MRJobConfig.INPUT_FILE_OPTION_PREFIX, MRJobConfig.INPUT_FILE_MANDATORY_PREFIX); - fileIn = FutureIOSupport.awaitFuture(builder.build()); + fileIn = FutureIO.awaitFuture(builder.build()); CompressionCodec codec = new CompressionCodecFactory(job).getCodec(file); if (null != codec) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java index 160c7635658a4..617abaacae065 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/LineRecordReader.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Seekable; -import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.CodecPool; @@ -40,9 +39,14 @@ import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.functional.FutureIO; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; + /** * Treats keys as offset in file and value as line. */ @@ -86,10 +90,14 @@ public void initialize(InputSplit genericSplit, // open the file and seek to the start of the split final FutureDataInputStreamBuilder builder = file.getFileSystem(job).openFile(file); - FutureIOSupport.propagateOptions(builder, job, + // the start and end of the split may be used to build + // an input strategy. + builder.opt(FS_OPTION_OPENFILE_SPLIT_START, start); + builder.opt(FS_OPTION_OPENFILE_SPLIT_END, end); + FutureIO.propagateOptions(builder, job, MRJobConfig.INPUT_FILE_OPTION_PREFIX, MRJobConfig.INPUT_FILE_MANDATORY_PREFIX); - fileIn = FutureIOSupport.awaitFuture(builder.build()); + fileIn = FutureIO.awaitFuture(builder.build()); CompressionCodec codec = new CompressionCodecFactory(job).getCodec(file); if (null!=codec) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java index dfff9ad0d2b73..5161a96c3459d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/NLineInputFormat.java @@ -29,7 +29,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.InputSplit; @@ -39,6 +38,7 @@ import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.LineReader; +import org.apache.hadoop.util.functional.FutureIO; /** * NLineInputFormat which splits N lines of input as one split. @@ -99,10 +99,10 @@ public static List getSplitsForFile(FileStatus status, try { final FutureDataInputStreamBuilder builder = fileName.getFileSystem(conf).openFile(fileName); - FutureIOSupport.propagateOptions(builder, conf, + FutureIO.propagateOptions(builder, conf, MRJobConfig.INPUT_FILE_OPTION_PREFIX, MRJobConfig.INPUT_FILE_MANDATORY_PREFIX); - FSDataInputStream in = FutureIOSupport.awaitFuture(builder.build()); + FSDataInputStream in = FutureIO.awaitFuture(builder.build()); lr = new LineReader(in, conf); Text line = new Text(); int numLines = 0; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java index 20ce8ef2b60de..f284a9c380756 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraInputFormat.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.InputSplit; @@ -41,6 +42,12 @@ import org.apache.hadoop.util.IndexedSortable; import org.apache.hadoop.util.QuickSort; import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.functional.FutureIO; + +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START; /** * An input format that reads the first 10 characters of each line as the key @@ -224,12 +231,17 @@ public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException { Path p = ((FileSplit)split).getPath(); FileSystem fs = p.getFileSystem(context.getConfiguration()); - in = fs.open(p); long start = ((FileSplit)split).getStart(); // find the offset to start at a record boundary offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH; - in.seek(start + offset); length = ((FileSplit)split).getLength(); + final FutureDataInputStreamBuilder builder = fs.openFile(p) + .opt(FS_OPTION_OPENFILE_SPLIT_START, start) + .opt(FS_OPTION_OPENFILE_SPLIT_END, start + length) + .opt(FS_OPTION_OPENFILE_READ_POLICY, + FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL); + in = FutureIO.awaitFuture(builder.build()); + in.seek(start + offset); } public void close() throws IOException { diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java index 544dbfbc4c0f0..ae6b734f48603 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java @@ -52,7 +52,10 @@ import org.apache.hadoop.classification.VisibleForTesting; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL; import static org.apache.hadoop.tools.mapred.CopyMapper.getFileAttributeSettings; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; /** * This class extends RetriableCommand to implement the copy of files, @@ -362,7 +365,11 @@ private static ThrottledInputStream getInputStream(Path path, FileSystem fs = path.getFileSystem(conf); float bandwidthMB = conf.getFloat(DistCpConstants.CONF_LABEL_BANDWIDTH_MB, DistCpConstants.DEFAULT_BANDWIDTH_MB); - FSDataInputStream in = fs.open(path); + // open with sequential read, but not whole-file + FSDataInputStream in = awaitFuture(fs.openFile(path) + .opt(FS_OPTION_OPENFILE_READ_POLICY, + FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL) + .build()); return new ThrottledInputStream(in, bandwidthMB * 1024 * 1024); } catch (IOException e) { diff --git a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java index 77f4e041d5f09..f44488c7c0202 100644 --- a/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java +++ b/hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/mapreduce/StreamInputFormat.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.impl.FutureIOSupport; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.MRJobConfig; @@ -35,6 +34,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit; import org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat; import org.apache.hadoop.streaming.StreamUtil; +import org.apache.hadoop.util.functional.FutureIO; /** * An input format that selects a RecordReader based on a JobConf property. This @@ -66,10 +66,10 @@ public RecordReader createRecordReader(InputSplit genericSplit, FileSystem fs = path.getFileSystem(conf); // open the file final FutureDataInputStreamBuilder builder = fs.openFile(path); - FutureIOSupport.propagateOptions(builder, conf, + FutureIO.propagateOptions(builder, conf, MRJobConfig.INPUT_FILE_OPTION_PREFIX, MRJobConfig.INPUT_FILE_MANDATORY_PREFIX); - FSDataInputStream in = FutureIOSupport.awaitFuture(builder.build()); + FSDataInputStream in = FutureIO.awaitFuture(builder.build()); // Factory dispatch based on available params.. Class readerClass; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java index 3f251c74bea62..5a49f9ff5012d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java @@ -56,6 +56,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; @@ -77,6 +78,11 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.collect.Iterables; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; + @Public @Evolving public class AggregatedLogFormat { @@ -576,9 +582,16 @@ public LogReader(Configuration conf, Path remoteAppLogFile) try { FileContext fileContext = FileContext.getFileContext(remoteAppLogFile.toUri(), conf); - this.fsDataIStream = fileContext.open(remoteAppLogFile); + FileStatus status = fileContext.getFileStatus(remoteAppLogFile); + this.fsDataIStream = awaitFuture( + fileContext.openFile(remoteAppLogFile) + .opt(FS_OPTION_OPENFILE_READ_POLICY, + FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL) + .opt(FS_OPTION_OPENFILE_LENGTH, + status.getLen()) // file length hint for object stores + .build()); reader = new TFile.Reader(this.fsDataIStream, - fileContext.getFileStatus(remoteAppLogFile).getLen(), conf); + status.getLen(), conf); this.scanner = reader.createScanner(); } catch (IOException ioe) { close(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java index e24b96cd32f78..56808c75ff62f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/FSDownload.java @@ -60,7 +60,11 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; import org.apache.hadoop.yarn.exceptions.YarnException; -/** +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; +import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE; +import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; + + /** * Download a single URL to the local disk. * */ @@ -285,23 +289,25 @@ private void verifyAndCopy(Path destination) } } - downloadAndUnpack(sCopy, destination); + downloadAndUnpack(sCopy, sStat, destination); } /** * Copy source path to destination with localization rules. - * @param source source path to copy. Typically HDFS + * @param source source path to copy. Typically HDFS or an object store. + * @param sourceStatus status of source * @param destination destination path. Typically local filesystem * @exception YarnException Any error has occurred */ - private void downloadAndUnpack(Path source, Path destination) + private void downloadAndUnpack(Path source, + FileStatus sourceStatus, Path destination) throws YarnException { try { FileSystem sourceFileSystem = source.getFileSystem(conf); FileSystem destinationFileSystem = destination.getFileSystem(conf); - if (sourceFileSystem.getFileStatus(source).isDirectory()) { + if (sourceStatus.isDirectory()) { FileUtil.copy( - sourceFileSystem, source, + sourceFileSystem, sourceStatus, destinationFileSystem, destination, false, true, conf); } else { @@ -329,7 +335,11 @@ private void unpack(Path source, Path destination, FileSystem sourceFileSystem, FileSystem destinationFileSystem) throws IOException, InterruptedException, ExecutionException { - try (InputStream inputStream = sourceFileSystem.open(source)) { + try (InputStream inputStream = awaitFuture( + sourceFileSystem.openFile(source) + .opt(FS_OPTION_OPENFILE_READ_POLICY, + FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE) + .build())) { File dst = new File(destination.toUri()); String lowerDst = StringUtils.toLowerCase(dst.getName()); switch (resource.getType()) { From e0cd0a82e032b926774dcea69edd4fa20aae2603 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Sun, 24 Apr 2022 17:23:19 +0100 Subject: [PATCH 137/145] HADOOP-16202. Enhanced openFile(): hadoop-aws changes. (#2584/3) S3A input stream support for the few fs.option.openfile settings. As well as supporting the read policy option and values, if the file length is declared in fs.option.openfile.length then no HEAD request will be issued when opening a file. This can cut a few tens of milliseconds off the operation. The patch adds a new openfile parameter/FS configuration option fs.s3a.input.async.drain.threshold (default: 16000). It declares the number of bytes remaining in the http input stream above which any operation to read and discard the rest of the stream, "draining", is executed asynchronously. This asynchronous draining offers some performance benefit on seek-heavy file IO. Contributed by Steve Loughran. Change-Id: I9b0626bbe635e9fd97ac0f463f5e7167e0111e39 --- .../dev-support/findbugs-exclude.xml | 5 + .../org/apache/hadoop/fs/s3a/Constants.java | 45 +- .../org/apache/hadoop/fs/s3a/Invoker.java | 28 + .../apache/hadoop/fs/s3a/S3AFileSystem.java | 283 ++++----- .../apache/hadoop/fs/s3a/S3AInputPolicy.java | 93 ++- .../apache/hadoop/fs/s3a/S3AInputStream.java | 263 +++++--- .../hadoop/fs/s3a/S3AInstrumentation.java | 12 +- .../hadoop/fs/s3a/S3AReadOpContext.java | 101 ++- .../hadoop/fs/s3a/S3ObjectAttributes.java | 15 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 17 + .../fs/s3a/commit/CommitOperations.java | 8 +- .../fs/s3a/commit/files/PendingSet.java | 26 +- .../s3a/commit/files/SinglePendingCommit.java | 20 +- .../fs/s3a/impl/AbstractStoreOperation.java | 26 +- .../hadoop/fs/s3a/impl/CallableSupplier.java | 2 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 21 +- .../hadoop/fs/s3a/impl/OpenFileSupport.java | 600 ++++++++++++++++++ .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 4 +- .../s3a/select/InternalSelectConstants.java | 2 +- .../hadoop/fs/s3a/select/SelectTool.java | 4 +- .../statistics/S3AInputStreamStatistics.java | 7 + .../impl/EmptyS3AStatisticsContext.java | 4 + .../site/markdown/tools/hadoop-aws/index.md | 8 + .../fs/contract/s3a/ITestS3AContractOpen.java | 67 ++ .../fs/contract/s3a/ITestS3AContractSeek.java | 15 +- .../hadoop/fs/s3a/AbstractS3AMockTest.java | 4 + .../hadoop/fs/s3a/ITestS3AConfiguration.java | 11 - .../apache/hadoop/fs/s3a/S3ATestUtils.java | 4 +- .../fs/s3a/TestS3AInputStreamRetry.java | 17 +- .../apache/hadoop/fs/s3a/TestS3AUnbuffer.java | 5 +- .../fs/s3a/TestStreamChangeTracker.java | 2 +- .../fs/s3a/impl/TestOpenFileSupport.java | 429 +++++++++++++ .../fs/s3a/performance/ITestS3AOpenCost.java | 209 ++++++ .../fs/s3a/performance/OperationCost.java | 6 + .../scale/ITestS3AInputStreamPerformance.java | 58 +- .../fs/s3a/select/AbstractS3SelectTest.java | 2 +- .../hadoop/fs/s3a/select/ITestS3Select.java | 7 +- .../fs/s3a/select/ITestS3SelectMRJob.java | 4 +- 38 files changed, 2062 insertions(+), 372 deletions(-) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestOpenFileSupport.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java diff --git a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml index b4568b69de463..324369076b84b 100644 --- a/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml +++ b/hadoop-tools/hadoop-aws/dev-support/findbugs-exclude.xml @@ -28,6 +28,11 @@ + + + + +