diff --git a/LICENSE-binary b/LICENSE-binary
index 60fdcb45e84..aa7f9a42e96 100644
--- a/LICENSE-binary
+++ b/LICENSE-binary
@@ -210,9 +210,9 @@ hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/nvd3-1.8.5.* (css and js
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/TimeoutFuture.java
-com.aliyun:aliyun-java-sdk-core:3.4.0
-com.aliyun:aliyun-java-sdk-ecs:4.2.0
-com.aliyun:aliyun-java-sdk-ram:3.0.0
+com.aliyun:aliyun-java-sdk-core:4.5.10
+com.aliyun:aliyun-java-sdk-kms:2.11.0
+com.aliyun:aliyun-java-sdk-ram:3.1.0
com.aliyun:aliyun-java-sdk-sts:3.0.0
com.aliyun.oss:aliyun-sdk-oss:3.13.2
com.amazonaws:aws-java-sdk-bundle:1.12.316
@@ -289,8 +289,12 @@ io.netty:netty-resolver-dns-classes-macos:4.1.77.Final
io.netty:netty-transport-native-epoll:4.1.77.Final
io.netty:netty-transport-native-kqueue:4.1.77.Final
io.netty:netty-resolver-dns-native-macos:4.1.77.Final
-io.opencensus:opencensus-api:0.12.3
-io.opencensus:opencensus-contrib-grpc-metrics:0.12.3
+io.opencensus:opencensus-api:0.24.0
+io.opencensus:opencensus-contrib-grpc-metrics:0.24.0
+io.opentracing:opentracing-api:0.33.0
+io.opentracing:opentracing-noop:0.33.0
+io.opentracing:opentracing-util:0.33.0
+io.perfmark:perfmark-api:0.19.0
io.reactivex:rxjava:1.3.8
io.reactivex:rxjava-string:1.1.1
io.reactivex:rxnetty:0.4.20
@@ -357,6 +361,9 @@ org.eclipse.jetty:jetty-xml:9.4.48.v20220622
org.eclipse.jetty.websocket:javax-websocket-client-impl:9.4.48.v20220622
org.eclipse.jetty.websocket:javax-websocket-server-impl:9.4.48.v20220622
org.ehcache:ehcache:3.3.1
+org.ini4j:ini4j:0.5.4
+org.jetbrains.kotlin:kotlin-stdlib:1.4.10
+org.jetbrains.kotlin:kotlin-stdlib-common:1.4.10
org.lz4:lz4-java:1.7.1
org.objenesis:objenesis:2.6
org.xerial.snappy:snappy-java:1.0.5
@@ -516,6 +523,8 @@ Eclipse Public License 1.0
--------------------------
junit:junit:4.13.2
+org.jacoco:org.jacoco.agent:0.8.5
+
HSQL License
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
index 774e015b373..d8ab16f41d3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java
@@ -414,7 +414,14 @@ public class LocalDirAllocator {
//build the "roulette wheel"
for(int i =0; i < ctx.dirDF.length; ++i) {
- availableOnDisk[i] = ctx.dirDF[i].getAvailable();
+ final DF target = ctx.dirDF[i];
+ // attempt to recreate the dir so that getAvailable() is valid
+ // if it fails, getAvailable() will return 0, so the dir will
+ // be declared unavailable.
+ // return value is logged at debug to keep spotbugs quiet.
+ final boolean b = new File(target.getDirPath()).mkdirs();
+ LOG.debug("mkdirs of {}={}", target, b);
+ availableOnDisk[i] = target.getAvailable();
totalAvailable += availableOnDisk[i];
}
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 b6e6f0c57a8..fec2e36f85b 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
@@ -49,6 +49,7 @@ public final class CallerContext {
public static final String CLIENT_PORT_STR = "clientPort";
public static final String CLIENT_ID_STR = "clientId";
public static final String CLIENT_CALL_ID_STR = "clientCallId";
+ public static final String REAL_USER_STR = "realUser";
/** The caller context.
*
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
index 3369869bde2..d045a7f6fc4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
@@ -314,7 +314,8 @@ public final class SecurityUtil {
String keytabFilename = conf.get(keytabFileKey);
if (keytabFilename == null || keytabFilename.length() == 0) {
- throw new IOException("Running in secure mode, but config doesn't have a keytab");
+ throw new IOException(
+ "Running in secure mode, but config doesn't have a keytab for key: " + keytabFileKey);
}
String principalConfig = conf.get(userNameKey, System
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
index 8aaf9bbd8de..cde4cf48413 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
@@ -19,7 +19,9 @@
package org.apache.hadoop.security.token.delegation;
import java.io.ByteArrayInputStream;
+import java.io.DataInput;
import java.io.DataInputStream;
+import java.io.DataOutput;
import java.io.IOException;
import java.security.MessageDigest;
import java.util.ArrayList;
@@ -41,6 +43,8 @@ import org.apache.hadoop.fs.statistics.DurationTrackerFactory;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding;
import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@@ -441,8 +445,9 @@ extends AbstractDelegationTokenIdentifier>
/**
* Update the current master key for generating delegation tokens
* It should be called only by tokenRemoverThread.
+ * @throws IOException raised on errors performing I/O.
*/
- void rollMasterKey() throws IOException {
+ protected void rollMasterKey() throws IOException {
synchronized (this) {
removeExpiredKeys();
/* set final expiry date for retiring currentKey */
@@ -677,11 +682,15 @@ extends AbstractDelegationTokenIdentifier>
/** Class to encapsulate a token's renew date and password. */
@InterfaceStability.Evolving
- public static class DelegationTokenInformation {
+ public static class DelegationTokenInformation implements Writable {
long renewDate;
byte[] password;
String trackingId;
+ public DelegationTokenInformation() {
+ this(0, null);
+ }
+
public DelegationTokenInformation(long renewDate, byte[] password) {
this(renewDate, password, null);
}
@@ -711,6 +720,29 @@ extends AbstractDelegationTokenIdentifier>
public String getTrackingId() {
return trackingId;
}
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ WritableUtils.writeVLong(out, renewDate);
+ if (password == null) {
+ WritableUtils.writeVInt(out, -1);
+ } else {
+ WritableUtils.writeVInt(out, password.length);
+ out.write(password);
+ }
+ WritableUtils.writeString(out, trackingId);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ renewDate = WritableUtils.readVLong(in);
+ int len = WritableUtils.readVInt(in);
+ if (len > -1) {
+ password = new byte[len];
+ in.readFully(password);
+ }
+ trackingId = WritableUtils.readString(in);
+ }
}
/** Remove expired delegation tokens from cache */
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/SQLDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/SQLDelegationTokenSecretManager.java
new file mode 100644
index 00000000000..4b6ae21d7a9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/SQLDelegationTokenSecretManager.java
@@ -0,0 +1,400 @@
+/*
+ * 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.security.token.delegation;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.sql.SQLException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * An implementation of {@link AbstractDelegationTokenSecretManager} that
+ * persists TokenIdentifiers and DelegationKeys in an existing SQL database.
+ */
+public abstract class SQLDelegationTokenSecretManager
+ extends AbstractDelegationTokenSecretManager {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(SQLDelegationTokenSecretManager.class);
+
+ public static final String SQL_DTSM_CONF_PREFIX = "sql-dt-secret-manager.";
+ private static final String SQL_DTSM_TOKEN_SEQNUM_BATCH_SIZE = SQL_DTSM_CONF_PREFIX
+ + "token.seqnum.batch.size";
+ public static final int DEFAULT_SEQ_NUM_BATCH_SIZE = 10;
+
+ // Batch of sequence numbers that will be requested by the sequenceNumCounter.
+ // A new batch is requested once the sequenceNums available to a secret manager are
+ // exhausted, including during initialization.
+ private final int seqNumBatchSize;
+
+ // Last sequenceNum in the current batch that has been allocated to a token.
+ private int currentSeqNum;
+
+ // Max sequenceNum in the current batch that can be allocated to a token.
+ // Unused sequenceNums in the current batch cannot be reused by other routers.
+ private int currentMaxSeqNum;
+
+ public SQLDelegationTokenSecretManager(Configuration conf) {
+ super(conf.getLong(DelegationTokenManager.UPDATE_INTERVAL,
+ DelegationTokenManager.UPDATE_INTERVAL_DEFAULT) * 1000,
+ conf.getLong(DelegationTokenManager.MAX_LIFETIME,
+ DelegationTokenManager.MAX_LIFETIME_DEFAULT) * 1000,
+ conf.getLong(DelegationTokenManager.RENEW_INTERVAL,
+ DelegationTokenManager.RENEW_INTERVAL_DEFAULT) * 1000,
+ conf.getLong(DelegationTokenManager.REMOVAL_SCAN_INTERVAL,
+ DelegationTokenManager.REMOVAL_SCAN_INTERVAL_DEFAULT) * 1000);
+
+ this.seqNumBatchSize = conf.getInt(SQL_DTSM_TOKEN_SEQNUM_BATCH_SIZE,
+ DEFAULT_SEQ_NUM_BATCH_SIZE);
+ }
+
+ /**
+ * Persists a TokenIdentifier and its corresponding TokenInformation into
+ * the SQL database. The TokenIdentifier is expected to be unique and any
+ * duplicate token attempts will result in an IOException.
+ * @param ident TokenIdentifier to persist.
+ * @param tokenInfo DelegationTokenInformation associated with the TokenIdentifier.
+ */
+ @Override
+ protected void storeToken(TokenIdent ident,
+ DelegationTokenInformation tokenInfo) throws IOException {
+ try (ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos)) {
+ tokenInfo.write(dos);
+ // Add token to SQL database
+ insertToken(ident.getSequenceNumber(), ident.getBytes(), bos.toByteArray());
+ // Add token to local cache
+ super.storeToken(ident, tokenInfo);
+ } catch (SQLException e) {
+ throw new IOException("Failed to store token in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Updates the TokenInformation of an existing TokenIdentifier in
+ * the SQL database.
+ * @param ident Existing TokenIdentifier in the SQL database.
+ * @param tokenInfo Updated DelegationTokenInformation associated with the TokenIdentifier.
+ */
+ @Override
+ protected void updateToken(TokenIdent ident,
+ DelegationTokenInformation tokenInfo) throws IOException {
+ try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) {
+ try (DataOutputStream dos = new DataOutputStream(bos)) {
+ tokenInfo.write(dos);
+ // Update token in SQL database
+ updateToken(ident.getSequenceNumber(), ident.getBytes(), bos.toByteArray());
+ // Update token in local cache
+ super.updateToken(ident, tokenInfo);
+ }
+ } catch (SQLException e) {
+ throw new IOException("Failed to update token in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Cancels a token by removing it from the SQL database. This will
+ * call the corresponding method in {@link AbstractDelegationTokenSecretManager}
+ * to perform validation and remove the token from the cache.
+ * @return Identifier of the canceled token
+ */
+ @Override
+ public synchronized TokenIdent cancelToken(Token token,
+ String canceller) throws IOException {
+ try (ByteArrayInputStream bis = new ByteArrayInputStream(token.getIdentifier());
+ DataInputStream din = new DataInputStream(bis)) {
+ TokenIdent id = createIdentifier();
+ id.readFields(din);
+
+ // Calling getTokenInfo to load token into local cache if not present.
+ // super.cancelToken() requires token to be present in local cache.
+ getTokenInfo(id);
+ }
+
+ return super.cancelToken(token, canceller);
+ }
+
+ /**
+ * Removes the existing TokenInformation from the SQL database to
+ * invalidate it.
+ * @param ident TokenInformation to remove from the SQL database.
+ */
+ @Override
+ protected void removeStoredToken(TokenIdent ident) throws IOException {
+ try {
+ deleteToken(ident.getSequenceNumber(), ident.getBytes());
+ } catch (SQLException e) {
+ LOG.warn("Failed to remove token in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Obtains the DelegationTokenInformation associated with the given
+ * TokenIdentifier in the SQL database.
+ * @param ident Existing TokenIdentifier in the SQL database.
+ * @return DelegationTokenInformation that matches the given TokenIdentifier or
+ * null if it doesn't exist in the database.
+ */
+ @Override
+ protected DelegationTokenInformation getTokenInfo(TokenIdent ident) {
+ // Look for token in local cache
+ DelegationTokenInformation tokenInfo = super.getTokenInfo(ident);
+
+ if (tokenInfo == null) {
+ try {
+ // Look for token in SQL database
+ byte[] tokenInfoBytes = selectTokenInfo(ident.getSequenceNumber(), ident.getBytes());
+
+ if (tokenInfoBytes != null) {
+ tokenInfo = new DelegationTokenInformation();
+ try (ByteArrayInputStream bis = new ByteArrayInputStream(tokenInfoBytes)) {
+ try (DataInputStream dis = new DataInputStream(bis)) {
+ tokenInfo.readFields(dis);
+ }
+ }
+
+ // Update token in local cache
+ currentTokens.put(ident, tokenInfo);
+ }
+ } catch (IOException | SQLException e) {
+ LOG.error("Failed to get token in SQL secret manager", e);
+ }
+ }
+
+ return tokenInfo;
+ }
+
+ /**
+ * Obtains the value of the last reserved sequence number.
+ * @return Last reserved sequence number.
+ */
+ @Override
+ public int getDelegationTokenSeqNum() {
+ try {
+ return selectSequenceNum();
+ } catch (SQLException e) {
+ throw new RuntimeException(
+ "Failed to get token sequence number in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Updates the value of the last reserved sequence number.
+ * @param seqNum Value to update the sequence number to.
+ */
+ @Override
+ public void setDelegationTokenSeqNum(int seqNum) {
+ try {
+ updateSequenceNum(seqNum);
+ } catch (SQLException e) {
+ throw new RuntimeException(
+ "Failed to update token sequence number in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Obtains the next available sequence number that can be allocated to a Token.
+ * Sequence numbers need to be reserved using the shared sequenceNumberCounter once
+ * the local batch has been exhausted, which handles sequenceNumber allocation
+ * concurrently with other secret managers.
+ * This method ensures that sequence numbers are incremental in a single secret manager,
+ * but not across secret managers.
+ * @return Next available sequence number.
+ */
+ @Override
+ public synchronized int incrementDelegationTokenSeqNum() {
+ if (currentSeqNum >= currentMaxSeqNum) {
+ try {
+ // Request a new batch of sequence numbers and use the
+ // lowest one available.
+ currentSeqNum = incrementSequenceNum(seqNumBatchSize);
+ currentMaxSeqNum = currentSeqNum + seqNumBatchSize;
+ } catch (SQLException e) {
+ throw new RuntimeException(
+ "Failed to increment token sequence number in SQL secret manager", e);
+ }
+ }
+
+ return ++currentSeqNum;
+ }
+
+ /**
+ * Persists a DelegationKey into the SQL database. The delegation keyId
+ * is expected to be unique and any duplicate key attempts will result
+ * in an IOException.
+ * @param key DelegationKey to persist into the SQL database.
+ */
+ @Override
+ protected void storeDelegationKey(DelegationKey key) throws IOException {
+ try (ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos)) {
+ key.write(dos);
+ // Add delegation key to SQL database
+ insertDelegationKey(key.getKeyId(), bos.toByteArray());
+ // Add delegation key to local cache
+ super.storeDelegationKey(key);
+ } catch (SQLException e) {
+ throw new IOException("Failed to store delegation key in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Updates an existing DelegationKey in the SQL database.
+ * @param key Updated DelegationKey.
+ */
+ @Override
+ protected void updateDelegationKey(DelegationKey key) throws IOException {
+ try (ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos)) {
+ key.write(dos);
+ // Update delegation key in SQL database
+ updateDelegationKey(key.getKeyId(), bos.toByteArray());
+ // Update delegation key in local cache
+ super.updateDelegationKey(key);
+ } catch (SQLException e) {
+ throw new IOException("Failed to update delegation key in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Removes the existing DelegationKey from the SQL database to
+ * invalidate it.
+ * @param key DelegationKey to remove from the SQL database.
+ */
+ @Override
+ protected void removeStoredMasterKey(DelegationKey key) {
+ try {
+ deleteDelegationKey(key.getKeyId());
+ } catch (SQLException e) {
+ LOG.warn("Failed to remove delegation key in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Obtains the DelegationKey from the SQL database.
+ * @param keyId KeyId of the DelegationKey to obtain.
+ * @return DelegationKey that matches the given keyId or null
+ * if it doesn't exist in the database.
+ */
+ @Override
+ protected DelegationKey getDelegationKey(int keyId) {
+ // Look for delegation key in local cache
+ DelegationKey delegationKey = super.getDelegationKey(keyId);
+
+ if (delegationKey == null) {
+ try {
+ // Look for delegation key in SQL database
+ byte[] delegationKeyBytes = selectDelegationKey(keyId);
+
+ if (delegationKeyBytes != null) {
+ delegationKey = new DelegationKey();
+ try (ByteArrayInputStream bis = new ByteArrayInputStream(delegationKeyBytes)) {
+ try (DataInputStream dis = new DataInputStream(bis)) {
+ delegationKey.readFields(dis);
+ }
+ }
+
+ // Update delegation key in local cache
+ allKeys.put(keyId, delegationKey);
+ }
+ } catch (IOException | SQLException e) {
+ LOG.error("Failed to get delegation key in SQL secret manager", e);
+ }
+ }
+
+ return delegationKey;
+ }
+
+ /**
+ * Obtains the value of the last delegation key id.
+ * @return Last delegation key id.
+ */
+ @Override
+ public int getCurrentKeyId() {
+ try {
+ return selectKeyId();
+ } catch (SQLException e) {
+ throw new RuntimeException(
+ "Failed to get delegation key id in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Updates the value of the last delegation key id.
+ * @param keyId Value to update the delegation key id to.
+ */
+ @Override
+ public void setCurrentKeyId(int keyId) {
+ try {
+ updateKeyId(keyId);
+ } catch (SQLException e) {
+ throw new RuntimeException(
+ "Failed to set delegation key id in SQL secret manager", e);
+ }
+ }
+
+ /**
+ * Obtains the next available delegation key id that can be allocated to a DelegationKey.
+ * Delegation key id need to be reserved using the shared delegationKeyIdCounter,
+ * which handles keyId allocation concurrently with other secret managers.
+ * @return Next available delegation key id.
+ */
+ @Override
+ public int incrementCurrentKeyId() {
+ try {
+ return incrementKeyId(1) + 1;
+ } catch (SQLException e) {
+ throw new RuntimeException(
+ "Failed to increment delegation key id in SQL secret manager", e);
+ }
+ }
+
+ // Token operations in SQL database
+ protected abstract byte[] selectTokenInfo(int sequenceNum, byte[] tokenIdentifier)
+ throws SQLException;
+ protected abstract void insertToken(int sequenceNum, byte[] tokenIdentifier, byte[] tokenInfo)
+ throws SQLException;
+ protected abstract void updateToken(int sequenceNum, byte[] tokenIdentifier, byte[] tokenInfo)
+ throws SQLException;
+ protected abstract void deleteToken(int sequenceNum, byte[] tokenIdentifier)
+ throws SQLException;
+ // Delegation key operations in SQL database
+ protected abstract byte[] selectDelegationKey(int keyId) throws SQLException;
+ protected abstract void insertDelegationKey(int keyId, byte[] delegationKey)
+ throws SQLException;
+ protected abstract void updateDelegationKey(int keyId, byte[] delegationKey)
+ throws SQLException;
+ protected abstract void deleteDelegationKey(int keyId) throws SQLException;
+ // Counter operations in SQL database
+ protected abstract int selectSequenceNum() throws SQLException;
+ protected abstract void updateSequenceNum(int value) throws SQLException;
+ protected abstract int incrementSequenceNum(int amount) throws SQLException;
+ protected abstract int selectKeyId() throws SQLException;
+ protected abstract void updateKeyId(int value) throws SQLException;
+ protected abstract int incrementKeyId(int amount) throws SQLException;
+}
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 e7d387b1131..a551e3ae15f 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md
@@ -370,6 +370,9 @@ Each metrics record contains tags such as SessionId and Hostname as additional i
|:---- |:---- |
| `BytesWritten` | Total number of bytes written to DataNode |
| `BytesRead` | Total number of bytes read from DataNode |
+| `ReadTransferRateNumOps` | Total number of data read transfers |
+| `ReadTransferRateAvgTime` | Average transfer rate of bytes read from DataNode, measured in bytes per second. |
+| `ReadTransferRate`*num*`s(50/75/90/95/99)thPercentileRate` | The 50/75/90/95/99th percentile of the transfer rate of bytes read from DataNode, measured in bytes per second. |
| `BlocksWritten` | Total number of blocks written to DataNode |
| `BlocksRead` | Total number of blocks read from DataNode |
| `BlocksReplicated` | Total number of blocks replicated |
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
index 98c3dd2bbb9..4ef2cd9291a 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/SecureMode.md
@@ -20,7 +20,9 @@ Hadoop in Secure Mode
Introduction
------------
-This document describes how to configure authentication for Hadoop in secure mode. When Hadoop is configured to run in secure mode, each Hadoop service and each user must be authenticated by Kerberos.
+In its default configuration, we expect you to make sure attackers don't have access to your Hadoop cluster by restricting all network access. If you want any restrictions on who can remotely access data or submit work, you MUST secure authentication and access for your Hadoop cluster as described in this document.
+
+When Hadoop is configured to run in secure mode, each Hadoop service and each user must be authenticated by Kerberos.
Forward and reverse host lookup for all service hosts must be configured correctly to allow services to authenticate with each other. Host lookups may be configured using either DNS or `/etc/hosts` files. Working knowledge of Kerberos and DNS is recommended before attempting to configure Hadoop services in Secure Mode.
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java
index 939881f39df..3693b4f0acd 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalDirAllocator.java
@@ -548,5 +548,24 @@ public class TestLocalDirAllocator {
"p1/x", Long.MAX_VALUE - 1), "Expect a DiskErrorException.",
() -> dirAllocator.getLocalPathForWrite("p1/x", Long.MAX_VALUE - 1, conf));
}
+
+ /**
+ * Test for HADOOP-18636 LocalDirAllocator cannot recover from directory tree deletion.
+ */
+ @Test(timeout = 30000)
+ public void testDirectoryRecovery() throws Throwable {
+ String dir0 = buildBufferDir(ROOT, 0);
+ String subdir = dir0 + "/subdir1/subdir2";
+
+ conf.set(CONTEXT, subdir);
+ // get local path and an ancestor
+ final Path pathForWrite = dirAllocator.getLocalPathForWrite("file", -1, conf);
+ final Path ancestor = pathForWrite.getParent().getParent();
+
+ // delete that ancestor
+ localFs.delete(ancestor, true);
+ // and expect to get a new file back
+ dirAllocator.getLocalPathForWrite("file2", -1, conf);
+ }
}
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
index 35ffb429816..353260f0f9b 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSConfiguration.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.crypto.key.kms.server;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.log4j.PropertyConfigurator;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -104,8 +104,6 @@ public class KMSConfiguration {
public static final boolean KEY_AUTHORIZATION_ENABLE_DEFAULT = true;
- private static final String LOG4J_PROPERTIES = "kms-log4j.properties";
-
static {
Configuration.addDefaultResource(KMS_DEFAULT_XML);
Configuration.addDefaultResource(KMS_SITE_XML);
@@ -163,31 +161,20 @@ public class KMSConfiguration {
return newer;
}
- public static void initLogging() {
- String confDir = System.getProperty(KMS_CONFIG_DIR);
- if (confDir == null) {
- throw new RuntimeException("System property '" +
- KMSConfiguration.KMS_CONFIG_DIR + "' not defined");
+ /**
+ * Validate whether "kms.config.dir" and "log4j.configuration" are defined in the System
+ * properties. If not, abort the KMS WebServer.
+ */
+ public static void validateSystemProps() {
+ if (System.getProperty(KMS_CONFIG_DIR) == null) {
+ String errorMsg = "System property '" + KMS_CONFIG_DIR + "' not defined";
+ System.err.println("Aborting KMSWebServer because " + errorMsg);
+ throw new RuntimeException(errorMsg);
}
if (System.getProperty("log4j.configuration") == null) {
- System.setProperty("log4j.defaultInitOverride", "true");
- boolean fromClasspath = true;
- File log4jConf = new File(confDir, LOG4J_PROPERTIES).getAbsoluteFile();
- if (log4jConf.exists()) {
- PropertyConfigurator.configureAndWatch(log4jConf.getPath(), 1000);
- fromClasspath = false;
- } else {
- ClassLoader cl = Thread.currentThread().getContextClassLoader();
- URL log4jUrl = cl.getResource(LOG4J_PROPERTIES);
- if (log4jUrl != null) {
- PropertyConfigurator.configure(log4jUrl);
- }
- }
- LOG.debug("KMS log starting");
- if (fromClasspath) {
- LOG.warn("Log4j configuration file '{}' not found", LOG4J_PROPERTIES);
- LOG.warn("Logging with INFO level to standard output");
- }
+ String errorMsg = "System property 'log4j.configuration' not defined";
+ System.err.println("Aborting KMSWebServer because " + errorMsg);
+ throw new RuntimeException(errorMsg);
}
}
}
diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java
index a6cab81eb8e..5c9f23e9a0c 100644
--- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java
+++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSWebServer.java
@@ -185,7 +185,7 @@ public class KMSWebServer {
}
public static void main(String[] args) throws Exception {
- KMSConfiguration.initLogging();
+ KMSConfiguration.validateSystemProps();
StringUtils.startupShutdownMessage(KMSWebServer.class, args, LOG);
Configuration conf = KMSConfiguration.getKMSConf();
Configuration sslConf = SSLFactory.readSSLConfiguration(conf, SSLFactory.Mode.SERVER);
diff --git a/hadoop-common-project/hadoop-kms/src/main/libexec/shellprofile.d/hadoop-kms.sh b/hadoop-common-project/hadoop-kms/src/main/libexec/shellprofile.d/hadoop-kms.sh
index 0d084bb36e6..b54bf811d6b 100755
--- a/hadoop-common-project/hadoop-kms/src/main/libexec/shellprofile.d/hadoop-kms.sh
+++ b/hadoop-common-project/hadoop-kms/src/main/libexec/shellprofile.d/hadoop-kms.sh
@@ -49,6 +49,8 @@ function hadoop_subcommand_kms
"-Dkms.config.dir=${HADOOP_CONF_DIR}"
hadoop_add_param HADOOP_OPTS "-Dkms.log.dir=" \
"-Dkms.log.dir=${HADOOP_LOG_DIR}"
+ hadoop_add_param HADOOP_OPTS "-Dlog4j.configuration=" \
+ "-Dlog4j.configuration=file:${HADOOP_CONF_DIR}/kms-log4j.properties"
if [[ "${HADOOP_DAEMON_MODE}" == "default" ]] ||
[[ "${HADOOP_DAEMON_MODE}" == "start" ]]; then
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 7b664e4f311..a8d80016072 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -224,7 +224,7 @@ public class DFSInputStream extends FSInputStream
}
/**
- * Grab the open-file info from namenode
+ * Grab the open-file info from namenode.
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
*/
void openInfo(boolean refreshLocatedBlocks) throws IOException {
@@ -940,7 +940,8 @@ public class DFSInputStream extends FSInputStream
* @return Returns chosen DNAddrPair; Can be null if refetchIfRequired is
* false.
*/
- private DNAddrPair chooseDataNode(LocatedBlock block,
+ @VisibleForTesting
+ DNAddrPair chooseDataNode(LocatedBlock block,
Collection ignoredNodes, boolean refetchIfRequired)
throws IOException {
while (true) {
@@ -955,6 +956,14 @@ public class DFSInputStream extends FSInputStream
}
}
+ /**
+ * RefetchLocations should only be called when there are no active requests
+ * to datanodes. In the hedged read case this means futures should be empty.
+ * @param block The locatedBlock to get new datanode locations for.
+ * @param ignoredNodes A list of ignored nodes. This list can be null and can be cleared.
+ * @return the locatedBlock with updated datanode locations.
+ * @throws IOException
+ */
private LocatedBlock refetchLocations(LocatedBlock block,
Collection ignoredNodes) throws IOException {
String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
@@ -999,13 +1008,24 @@ public class DFSInputStream extends FSInputStream
throw new InterruptedIOException(
"Interrupted while choosing DataNode for read.");
}
- clearLocalDeadNodes(); //2nd option is to remove only nodes[blockId]
+ clearCachedNodeState(ignoredNodes);
openInfo(true);
block = refreshLocatedBlock(block);
failures++;
return block;
}
+ /**
+ * Clear both the dead nodes and the ignored nodes
+ * @param ignoredNodes is cleared
+ */
+ private void clearCachedNodeState(Collection ignoredNodes) {
+ clearLocalDeadNodes(); //2nd option is to remove only nodes[blockId]
+ if (ignoredNodes != null) {
+ ignoredNodes.clear();
+ }
+ }
+
/**
* Get the best node from which to stream the data.
* @param block LocatedBlock, containing nodes in priority order.
@@ -1337,8 +1357,12 @@ public class DFSInputStream extends FSInputStream
} catch (InterruptedException ie) {
// Ignore and retry
}
- if (refetch) {
- refetchLocations(block, ignored);
+ // If refetch is true, then all nodes are in deadNodes or ignoredNodes.
+ // We should loop through all futures and remove them, so we do not
+ // have concurrent requests to the same node.
+ // Once all futures are cleared, we can clear the ignoredNodes and retry.
+ if (refetch && futures.isEmpty()) {
+ block = refetchLocations(block, ignored);
}
// We got here if exception. Ignore this node on next go around IFF
// we found a chosenNode to hedge read against.
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/fs/TestXAttr.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/fs/TestXAttr.java
index e47658dd6a8..af602477d1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/fs/TestXAttr.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/fs/TestXAttr.java
@@ -19,8 +19,8 @@
package org.apache.hadoop.fs;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNotEquals;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -77,18 +77,18 @@ public class TestXAttr {
assertEquals(XATTR3, XATTR3);
assertEquals(XATTR4, XATTR4);
assertEquals(XATTR5, XATTR5);
- assertFalse(XATTR1.equals(XATTR2));
- assertFalse(XATTR2.equals(XATTR3));
- assertFalse(XATTR3.equals(XATTR4));
- assertFalse(XATTR4.equals(XATTR5));
+ assertNotEquals(XATTR1, XATTR2);
+ assertNotEquals(XATTR2, XATTR3);
+ assertNotEquals(XATTR3, XATTR4);
+ assertNotEquals(XATTR4, XATTR5);
}
@Test
public void testXAttrHashCode() {
assertEquals(XATTR.hashCode(), XATTR1.hashCode());
- assertFalse(XATTR1.hashCode() == XATTR2.hashCode());
- assertFalse(XATTR2.hashCode() == XATTR3.hashCode());
- assertFalse(XATTR3.hashCode() == XATTR4.hashCode());
- assertFalse(XATTR4.hashCode() == XATTR5.hashCode());
+ assertNotEquals(XATTR1.hashCode(), XATTR2.hashCode());
+ assertNotEquals(XATTR2.hashCode(), XATTR3.hashCode());
+ assertNotEquals(XATTR3.hashCode(), XATTR4.hashCode());
+ assertNotEquals(XATTR4.hashCode(), XATTR5.hashCode());
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml
index 9fb868f79f3..9e6f12ba8a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/pom.xml
@@ -117,6 +117,15 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
com.fasterxml.jackson.corejackson-databind
+
+ com.zaxxer
+ HikariCP
+
+
+ mysql
+ mysql-connector-java
+ provided
+ junitjunit
@@ -153,6 +162,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
curator-testtest
+
+ org.apache.derby
+ derby
+ test
+ org.mockitomockito-core
@@ -170,6 +184,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
org.apache.maven.pluginsmaven-surefire-plugin
+
+
+
+ derby.stream.error.file
+ ${project.build.directory}/derby.log
+
+
+ org.apache.maven.plugins
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreDatabase.sql b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreDatabase.sql
new file mode 100644
index 00000000000..07fea4c24bc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreDatabase.sql
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+
+-- Script to create a new Database in MySQL for the TokenStore
+
+CREATE DATABASE IF NOT EXISTS TokenStore;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreTables.sql b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreTables.sql
new file mode 100644
index 00000000000..d377c4e15f2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreTables.sql
@@ -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.
+ */
+
+-- Script to generate all the tables for the TokenStore in MySQL
+
+USE TokenStore
+
+CREATE TABLE IF NOT EXISTS Tokens(
+ sequenceNum int NOT NULL,
+ tokenIdentifier varbinary(255) NOT NULL,
+ tokenInfo varbinary(255) NOT NULL,
+ modifiedTime timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
+ PRIMARY KEY(sequenceNum, tokenIdentifier)
+);
+
+CREATE TABLE IF NOT EXISTS DelegationKeys(
+ keyId int NOT NULL,
+ delegationKey varbinary(255) NOT NULL,
+ modifiedTime timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
+ PRIMARY KEY(keyId)
+);
+
+CREATE TABLE IF NOT EXISTS LastSequenceNum(
+ sequenceNum int NOT NULL
+);
+
+-- Initialize the LastSequenceNum table with a single entry
+INSERT INTO LastSequenceNum (sequenceNum)
+SELECT 0 WHERE NOT EXISTS (SELECT * FROM LastSequenceNum);
+
+CREATE TABLE IF NOT EXISTS LastDelegationKeyId(
+ keyId int NOT NULL
+);
+
+-- Initialize the LastDelegationKeyId table with a single entry
+INSERT INTO LastDelegationKeyId (keyId)
+SELECT 0 WHERE NOT EXISTS (SELECT * FROM LastDelegationKeyId);
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreUser.sql b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreUser.sql
new file mode 100644
index 00000000000..844d7a2f944
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/MySQL/TokenStoreUser.sql
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+-- Script to create a new User in MySQL for the TokenStore
+
+-- Update TokenStore user and password on this script
+CREATE USER IF NOT EXISTS 'TokenStoreUser'@'%' IDENTIFIED BY 'TokenStorePassword';
+
+GRANT ALL PRIVILEGES ON TokenStore.* TO 'TokenStoreUser'@'%';
+
+FLUSH PRIVILEGES;
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/README b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/README
new file mode 100644
index 00000000000..72425315319
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/scripts/TokenStore/README
@@ -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.
+#
+These scripts must be executed to create the TokenStore database, tables and users needed to use the
+SQLDelegationTokenSecretManagerImpl as the delegation token secret manager:
+1. TokenStoreDatabase.sql
+2. TokenStoreTables.sql
+3. TokenStoreUser.sql
+
+Note: The TokenStoreUser.sql defines a default user/password. You are highly encouraged to set
+this to a proper strong password.
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
index 7e07d7b6549..c0ee9504597 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java
@@ -201,6 +201,10 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
FEDERATION_ROUTER_PREFIX + "observer.federated.state.propagation.maxsize";
public static final int DFS_ROUTER_OBSERVER_FEDERATED_STATE_PROPAGATION_MAXSIZE_DEFAULT = 5;
+ public static final String DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_KEY =
+ FEDERATION_ROUTER_PREFIX + "observer.state.id.refresh.period";
+ public static final String DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_DEFAULT = "15s";
+
public static final String FEDERATION_STORE_SERIALIZER_CLASS =
FEDERATION_STORE_PREFIX + "serializer";
public static final Class
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 62ae4b0b95d..92f1fc06a81 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
@@ -57,6 +57,7 @@ import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAccumulator;
import java.util.concurrent.atomic.LongAdder;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@@ -86,6 +87,7 @@ import org.apache.hadoop.net.ConnectTimeoutException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
import org.eclipse.jetty.util.ajax.JSON;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -136,6 +138,14 @@ public class RouterRpcClient {
private final boolean observerReadEnabledDefault;
/** Nameservice specific overrides of the default setting for enabling observer reads. */
private HashSet observerReadEnabledOverrides = new HashSet<>();
+ /**
+ * Period to refresh namespace stateID using active namenode.
+ * This ensures the namespace stateID is fresh even when an
+ * observer is trailing behind.
+ */
+ private long activeNNStateIdRefreshPeriodMs;
+ /** Last msync times for each namespace. */
+ private final ConcurrentHashMap lastActiveNNRefreshTimes;
/** Pattern to parse a stack trace line. */
private static final Pattern STACK_TRACE_PATTERN =
@@ -211,13 +221,25 @@ public class RouterRpcClient {
this.observerReadEnabledDefault = conf.getBoolean(
RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_KEY,
RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_DEFAULT_VALUE);
- String[] observerReadOverrides = conf.getStrings(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_OVERRIDES);
+ String[] observerReadOverrides =
+ conf.getStrings(RBFConfigKeys.DFS_ROUTER_OBSERVER_READ_OVERRIDES);
if (observerReadOverrides != null) {
observerReadEnabledOverrides.addAll(Arrays.asList(observerReadOverrides));
}
if (this.observerReadEnabledDefault) {
LOG.info("Observer read is enabled for router.");
}
+ this.activeNNStateIdRefreshPeriodMs = conf.getTimeDuration(
+ RBFConfigKeys.DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_KEY,
+ RBFConfigKeys.DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_DEFAULT,
+ TimeUnit.SECONDS, TimeUnit.MILLISECONDS);
+ if (activeNNStateIdRefreshPeriodMs < 0) {
+ LOG.info("Periodic stateId freshness check is disabled"
+ + " since '{}' is {}ms, which is less than 0.",
+ RBFConfigKeys.DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_KEY,
+ activeNNStateIdRefreshPeriodMs);
+ }
+ this.lastActiveNNRefreshTimes = new ConcurrentHashMap<>();
}
/**
@@ -491,7 +513,7 @@ public class RouterRpcClient {
+ router.getRouterId());
}
- addClientInfoToCallerContext();
+ addClientInfoToCallerContext(ugi);
Object ret = null;
if (rpcMonitor != null) {
@@ -627,14 +649,18 @@ public class RouterRpcClient {
/**
* For tracking some information about the actual client.
* It adds trace info "clientIp:ip", "clientPort:port",
- * "clientId:id" and "clientCallId:callId"
+ * "clientId:id", "clientCallId:callId" and "realUser:userName"
* in the caller context, removing the old values if they were
* already present.
*/
- private void addClientInfoToCallerContext() {
+ private void addClientInfoToCallerContext(UserGroupInformation ugi) {
CallerContext ctx = CallerContext.getCurrent();
String origContext = ctx == null ? null : ctx.getContext();
byte[] origSignature = ctx == null ? null : ctx.getSignature();
+ String realUser = null;
+ if (ugi.getRealUser() != null) {
+ realUser = ugi.getRealUser().getUserName();
+ }
CallerContext.Builder builder =
new CallerContext.Builder("", contextFieldSeparator)
.append(CallerContext.CLIENT_IP_STR, Server.getRemoteAddress())
@@ -644,6 +670,7 @@ public class RouterRpcClient {
StringUtils.byteToHexString(Server.getClientId()))
.append(CallerContext.CLIENT_CALL_ID_STR,
Integer.toString(Server.getCallId()))
+ .append(CallerContext.REAL_USER_STR, realUser)
.setSignature(origSignature);
// Append the original caller context
if (origContext != null) {
@@ -1702,10 +1729,13 @@ public class RouterRpcClient {
boolean isObserverRead) throws IOException {
final List extends FederationNamenodeContext> namenodes;
- if (RouterStateIdContext.getClientStateIdFromCurrentCall(nsId) > Long.MIN_VALUE) {
- namenodes = namenodeResolver.getNamenodesForNameserviceId(nsId, isObserverRead);
- } else {
- namenodes = namenodeResolver.getNamenodesForNameserviceId(nsId, false);
+ boolean listObserverNamenodesFirst = isObserverRead
+ && isNamespaceStateIdFresh(nsId)
+ && (RouterStateIdContext.getClientStateIdFromCurrentCall(nsId) > Long.MIN_VALUE);
+ namenodes = namenodeResolver.getNamenodesForNameserviceId(nsId, listObserverNamenodesFirst);
+ if (!listObserverNamenodesFirst) {
+ // Refresh time of last call to active NameNode.
+ getTimeOfLastCallToActive(nsId).accumulate(Time.monotonicNow());
}
if (namenodes == null || namenodes.isEmpty()) {
@@ -1716,7 +1746,8 @@ public class RouterRpcClient {
}
private boolean isObserverReadEligible(String nsId, Method method) {
- boolean isReadEnabledForNamespace = observerReadEnabledDefault != observerReadEnabledOverrides.contains(nsId);
+ boolean isReadEnabledForNamespace =
+ observerReadEnabledDefault != observerReadEnabledOverrides.contains(nsId);
return isReadEnabledForNamespace && isReadCall(method);
}
@@ -1730,4 +1761,24 @@ public class RouterRpcClient {
}
return !method.getAnnotationsByType(ReadOnly.class)[0].activeOnly();
}
+
+ /**
+ * Checks and sets last refresh time for a namespace's stateId.
+ * Returns true if refresh time is newer than threshold.
+ * Otherwise, return false and call should be handled by active namenode.
+ * @param nsId namespaceID
+ */
+ @VisibleForTesting
+ boolean isNamespaceStateIdFresh(String nsId) {
+ if (activeNNStateIdRefreshPeriodMs < 0) {
+ return true;
+ }
+ long timeSinceRefreshMs = Time.monotonicNow() - getTimeOfLastCallToActive(nsId).get();
+ return (timeSinceRefreshMs <= activeNNStateIdRefreshPeriodMs);
+ }
+
+ private LongAccumulator getTimeOfLastCallToActive(String namespaceId) {
+ return lastActiveNNRefreshTimes
+ .computeIfAbsent(namespaceId, key -> new LongAccumulator(Math::max, 0));
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/DistributedSQLCounter.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/DistributedSQLCounter.java
new file mode 100644
index 00000000000..14b232783f5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/DistributedSQLCounter.java
@@ -0,0 +1,138 @@
+/*
+ * 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.federation.router.security.token;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Distributed counter that relies on a SQL database to synchronize
+ * between multiple clients. This expects a table with a single int field
+ * to exist in the database. One record must exist on the table at all times,
+ * representing the last used value reserved by a client.
+ */
+public class DistributedSQLCounter {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(DistributedSQLCounter.class);
+
+ private final String field;
+ private final String table;
+ private final SQLConnectionFactory connectionFactory;
+
+ public DistributedSQLCounter(String field, String table,
+ SQLConnectionFactory connectionFactory) {
+ this.field = field;
+ this.table = table;
+ this.connectionFactory = connectionFactory;
+ }
+
+ /**
+ * Obtains the value of the counter.
+ * @return counter value.
+ */
+ public int selectCounterValue() throws SQLException {
+ try (Connection connection = connectionFactory.getConnection()) {
+ return selectCounterValue(false, connection);
+ }
+ }
+
+ private int selectCounterValue(boolean forUpdate, Connection connection) throws SQLException {
+ String query = String.format("SELECT %s FROM %s %s", field, table,
+ forUpdate ? "FOR UPDATE" : "");
+ LOG.debug("Select counter statement: " + query);
+ try (Statement statement = connection.createStatement();
+ ResultSet result = statement.executeQuery(query)) {
+ if (result.next()) {
+ return result.getInt(field);
+ } else {
+ throw new IllegalStateException("Counter table not initialized: " + table);
+ }
+ }
+ }
+
+ /**
+ * Sets the counter to the given value.
+ * @param value Value to assign to counter.
+ */
+ public void updateCounterValue(int value) throws SQLException {
+ try (Connection connection = connectionFactory.getConnection(true)) {
+ updateCounterValue(value, connection);
+ }
+ }
+
+ /**
+ * Sets the counter to the given value.
+ * @param connection Connection to database hosting the counter table.
+ * @param value Value to assign to counter.
+ */
+ public void updateCounterValue(int value, Connection connection) throws SQLException {
+ String queryText = String.format("UPDATE %s SET %s = ?", table, field);
+ LOG.debug("Update counter statement: " + queryText + ". Value: " + value);
+ try (PreparedStatement statement = connection.prepareStatement(queryText)) {
+ statement.setInt(1, value);
+ statement.execute();
+ }
+ }
+
+ /**
+ * Increments the counter by the given amount and
+ * returns the previous counter value.
+ * @param amount Amount to increase the counter.
+ * @return Previous counter value.
+ */
+ public int incrementCounterValue(int amount) throws SQLException {
+ // Disabling auto-commit to ensure that all statements on this transaction
+ // are committed at once.
+ try (Connection connection = connectionFactory.getConnection(false)) {
+ // Preventing dirty reads and non-repeatable reads to ensure that the
+ // value read will not be updated by a different connection.
+ if (connection.getTransactionIsolation() < Connection.TRANSACTION_REPEATABLE_READ) {
+ connection.setTransactionIsolation(Connection.TRANSACTION_REPEATABLE_READ);
+ }
+
+ try {
+ // Reading the counter value "FOR UPDATE" to lock the value record,
+ // forcing other connections to wait until this transaction is committed.
+ int lastValue = selectCounterValue(true, connection);
+
+ // Calculate the new counter value and handling overflow by
+ // resetting the counter to 0.
+ int newValue = lastValue + amount;
+ if (newValue < 0) {
+ lastValue = 0;
+ newValue = amount;
+ }
+
+ updateCounterValue(newValue, connection);
+ connection.commit();
+ return lastValue;
+ } catch (Exception e) {
+ // Rollback transaction to release table locks
+ connection.rollback();
+ throw e;
+ }
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/HikariDataSourceConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/HikariDataSourceConnectionFactory.java
new file mode 100644
index 00000000000..5510e9f54b9
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/HikariDataSourceConnectionFactory.java
@@ -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.
+ */
+
+package org.apache.hadoop.hdfs.server.federation.router.security.token;
+
+import com.zaxxer.hikari.HikariConfig;
+import com.zaxxer.hikari.HikariDataSource;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.delegation.SQLDelegationTokenSecretManager;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Properties;
+
+/**
+ * Class that relies on a HikariDataSource to provide SQL connections.
+ */
+class HikariDataSourceConnectionFactory implements SQLConnectionFactory {
+ protected final static String HIKARI_PROPS = SQLDelegationTokenSecretManager.SQL_DTSM_CONF_PREFIX
+ + "connection.hikari.";
+ private final HikariDataSource dataSource;
+
+ HikariDataSourceConnectionFactory(Configuration conf) {
+ Properties properties = new Properties();
+ properties.setProperty("jdbcUrl", conf.get(CONNECTION_URL));
+ properties.setProperty("username", conf.get(CONNECTION_USERNAME));
+ properties.setProperty("password", conf.get(CONNECTION_PASSWORD));
+ properties.setProperty("driverClassName", conf.get(CONNECTION_DRIVER));
+
+ // Include hikari connection properties
+ properties.putAll(conf.getPropsWithPrefix(HIKARI_PROPS));
+
+ HikariConfig hikariConfig = new HikariConfig(properties);
+ this.dataSource = new HikariDataSource(hikariConfig);
+ }
+
+ @Override
+ public Connection getConnection() throws SQLException {
+ return dataSource.getConnection();
+ }
+
+ @Override
+ public void shutdown() {
+ // Close database connections
+ dataSource.close();
+ }
+
+ @VisibleForTesting
+ HikariDataSource getDataSource() {
+ return dataSource;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLConnectionFactory.java
new file mode 100644
index 00000000000..a464cc81968
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLConnectionFactory.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.hdfs.server.federation.router.security.token;
+
+import com.mysql.cj.jdbc.MysqlDataSource;
+import com.zaxxer.hikari.HikariConfig;
+import com.zaxxer.hikari.HikariDataSource;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Properties;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.delegation.SQLDelegationTokenSecretManager;
+
+
+/**
+ * Interface to provide SQL connections to the {@link SQLDelegationTokenSecretManagerImpl}.
+ */
+public interface SQLConnectionFactory {
+ String CONNECTION_URL = SQLDelegationTokenSecretManager.SQL_DTSM_CONF_PREFIX
+ + "connection.url";
+ String CONNECTION_USERNAME = SQLDelegationTokenSecretManager.SQL_DTSM_CONF_PREFIX
+ + "connection.username";
+ String CONNECTION_PASSWORD = SQLDelegationTokenSecretManager.SQL_DTSM_CONF_PREFIX
+ + "connection.password";
+ String CONNECTION_DRIVER = SQLDelegationTokenSecretManager.SQL_DTSM_CONF_PREFIX
+ + "connection.driver";
+
+ Connection getConnection() throws SQLException;
+ void shutdown();
+
+ default Connection getConnection(boolean autocommit) throws SQLException {
+ Connection connection = getConnection();
+ connection.setAutoCommit(autocommit);
+ return connection;
+ }
+}
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLDelegationTokenSecretManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLDelegationTokenSecretManagerImpl.java
new file mode 100644
index 00000000000..7da54778f31
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLDelegationTokenSecretManagerImpl.java
@@ -0,0 +1,242 @@
+/*
+ * 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.federation.router.security.token;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.security.token.delegation.SQLDelegationTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * An implementation of {@link SQLDelegationTokenSecretManager} that
+ * persists TokenIdentifiers and DelegationKeys in a SQL database.
+ * This implementation relies on the Datanucleus JDO PersistenceManager, which
+ * can be configured with datanucleus.* configuration properties.
+ */
+public class SQLDelegationTokenSecretManagerImpl
+ extends SQLDelegationTokenSecretManager {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(SQLDelegationTokenSecretManagerImpl.class);
+ private static final String SEQ_NUM_COUNTER_FIELD = "sequenceNum";
+ private static final String SEQ_NUM_COUNTER_TABLE = "LastSequenceNum";
+ private static final String KEY_ID_COUNTER_FIELD = "keyId";
+ private static final String KEY_ID_COUNTER_TABLE = "LastDelegationKeyId";
+
+ private final SQLConnectionFactory connectionFactory;
+ private final DistributedSQLCounter sequenceNumCounter;
+ private final DistributedSQLCounter delegationKeyIdCounter;
+ private final SQLSecretManagerRetriableHandler retryHandler;
+
+ public SQLDelegationTokenSecretManagerImpl(Configuration conf) {
+ this(conf, new HikariDataSourceConnectionFactory(conf),
+ SQLSecretManagerRetriableHandlerImpl.getInstance(conf));
+ }
+
+ public SQLDelegationTokenSecretManagerImpl(Configuration conf,
+ SQLConnectionFactory connectionFactory, SQLSecretManagerRetriableHandler retryHandler) {
+ super(conf);
+
+ this.connectionFactory = connectionFactory;
+ this.sequenceNumCounter = new DistributedSQLCounter(SEQ_NUM_COUNTER_FIELD,
+ SEQ_NUM_COUNTER_TABLE, connectionFactory);
+ this.delegationKeyIdCounter = new DistributedSQLCounter(KEY_ID_COUNTER_FIELD,
+ KEY_ID_COUNTER_TABLE, connectionFactory);
+ this.retryHandler = retryHandler;
+
+ try {
+ super.startThreads();
+ } catch (IOException e) {
+ throw new RuntimeException("Error starting threads for MySQL secret manager", e);
+ }
+
+ LOG.info("MySQL delegation token secret manager instantiated");
+ }
+
+ @Override
+ public DelegationTokenIdentifier createIdentifier() {
+ return new DelegationTokenIdentifier();
+ }
+
+ @Override
+ public void stopThreads() {
+ super.stopThreads();
+ connectionFactory.shutdown();
+ }
+
+ @Override
+ protected void insertToken(int sequenceNum, byte[] tokenIdentifier, byte[] tokenInfo)
+ throws SQLException {
+ retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection(true);
+ PreparedStatement statement = connection.prepareStatement(
+ "INSERT INTO Tokens (sequenceNum, tokenIdentifier, tokenInfo) VALUES (?, ?, ?)")) {
+ statement.setInt(1, sequenceNum);
+ statement.setBytes(2, tokenIdentifier);
+ statement.setBytes(3, tokenInfo);
+ statement.execute();
+ }
+ });
+ }
+
+ @Override
+ protected void updateToken(int sequenceNum, byte[] tokenIdentifier, byte[] tokenInfo)
+ throws SQLException {
+ retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection(true);
+ PreparedStatement statement = connection.prepareStatement(
+ "UPDATE Tokens SET tokenInfo = ? WHERE sequenceNum = ? AND tokenIdentifier = ?")) {
+ statement.setBytes(1, tokenInfo);
+ statement.setInt(2, sequenceNum);
+ statement.setBytes(3, tokenIdentifier);
+ statement.execute();
+ }
+ });
+ }
+
+ @Override
+ protected void deleteToken(int sequenceNum, byte[] tokenIdentifier) throws SQLException {
+ retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection(true);
+ PreparedStatement statement = connection.prepareStatement(
+ "DELETE FROM Tokens WHERE sequenceNum = ? AND tokenIdentifier = ?")) {
+ statement.setInt(1, sequenceNum);
+ statement.setBytes(2, tokenIdentifier);
+ statement.execute();
+ }
+ });
+ }
+
+ @Override
+ protected byte[] selectTokenInfo(int sequenceNum, byte[] tokenIdentifier) throws SQLException {
+ return retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection();
+ PreparedStatement statement = connection.prepareStatement(
+ "SELECT tokenInfo FROM Tokens WHERE sequenceNum = ? AND tokenIdentifier = ?")) {
+ statement.setInt(1, sequenceNum);
+ statement.setBytes(2, tokenIdentifier);
+ try (ResultSet result = statement.executeQuery()) {
+ if (result.next()) {
+ return result.getBytes("tokenInfo");
+ }
+ }
+ }
+ return null;
+ });
+ }
+
+ @Override
+ protected void insertDelegationKey(int keyId, byte[] delegationKey) throws SQLException {
+ retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection(true);
+ PreparedStatement statement = connection.prepareStatement(
+ "INSERT INTO DelegationKeys (keyId, delegationKey) VALUES (?, ?)")) {
+ statement.setInt(1, keyId);
+ statement.setBytes(2, delegationKey);
+ statement.execute();
+ }
+ });
+ }
+
+ @Override
+ protected void updateDelegationKey(int keyId, byte[] delegationKey) throws SQLException {
+ retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection(true);
+ PreparedStatement statement = connection.prepareStatement(
+ "UPDATE DelegationKeys SET delegationKey = ? WHERE keyId = ?")) {
+ statement.setBytes(1, delegationKey);
+ statement.setInt(2, keyId);
+ statement.execute();
+ }
+ });
+ }
+
+ @Override
+ protected void deleteDelegationKey(int keyId) throws SQLException {
+ retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection(true);
+ PreparedStatement statement = connection.prepareStatement(
+ "DELETE FROM DelegationKeys WHERE keyId = ?")) {
+ statement.setInt(1, keyId);
+ statement.execute();
+ }
+ });
+ }
+
+ @Override
+ protected byte[] selectDelegationKey(int keyId) throws SQLException {
+ return retryHandler.execute(() -> {
+ try (Connection connection = connectionFactory.getConnection();
+ PreparedStatement statement = connection.prepareStatement(
+ "SELECT delegationKey FROM DelegationKeys WHERE keyId = ?")) {
+ statement.setInt(1, keyId);
+ try (ResultSet result = statement.executeQuery()) {
+ if (result.next()) {
+ return result.getBytes("delegationKey");
+ }
+ }
+ }
+ return null;
+ });
+ }
+
+ @Override
+ protected int selectSequenceNum() throws SQLException {
+ return retryHandler.execute(() -> sequenceNumCounter.selectCounterValue());
+ }
+
+ @Override
+ protected void updateSequenceNum(int value) throws SQLException {
+ retryHandler.execute(() -> sequenceNumCounter.updateCounterValue(value));
+ }
+
+ @Override
+ protected int incrementSequenceNum(int amount) throws SQLException {
+ return retryHandler.execute(() -> sequenceNumCounter.incrementCounterValue(amount));
+ }
+
+ @Override
+ protected int selectKeyId() throws SQLException {
+ return retryHandler.execute(delegationKeyIdCounter::selectCounterValue);
+ }
+
+ @Override
+ protected void updateKeyId(int value) throws SQLException {
+ retryHandler.execute(() -> delegationKeyIdCounter.updateCounterValue(value));
+ }
+
+ @Override
+ protected int incrementKeyId(int amount) throws SQLException {
+ return retryHandler.execute(() -> delegationKeyIdCounter.incrementCounterValue(amount));
+ }
+
+ @VisibleForTesting
+ protected SQLConnectionFactory getConnectionFactory() {
+ return connectionFactory;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLSecretManagerRetriableHandler.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLSecretManagerRetriableHandler.java
new file mode 100644
index 00000000000..16151226217
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/SQLSecretManagerRetriableHandler.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.hdfs.server.federation.router.security.token;
+
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.io.retry.RetryProxy;
+import org.apache.hadoop.security.token.delegation.SQLDelegationTokenSecretManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Interface to handle retries when {@link SQLDelegationTokenSecretManagerImpl}
+ * throws expected errors.
+ */
+public interface SQLSecretManagerRetriableHandler {
+ void execute(SQLCommandVoid command) throws SQLException;
+ T execute(SQLCommand command) throws SQLException;
+
+ @FunctionalInterface
+ interface SQLCommandVoid {
+ void doCall() throws SQLException;
+ }
+
+ @FunctionalInterface
+ interface SQLCommand {
+ T doCall() throws SQLException;
+ }
+}
+
+/**
+ * Implementation of {@link SQLSecretManagerRetriableHandler} that uses a
+ * {@link RetryProxy} to simplify the retryable operations.
+ */
+class SQLSecretManagerRetriableHandlerImpl implements SQLSecretManagerRetriableHandler {
+ public final static String MAX_RETRIES =
+ SQLDelegationTokenSecretManager.SQL_DTSM_CONF_PREFIX + "max-retries";
+ public final static int MAX_RETRIES_DEFAULT = 0;
+ public final static String RETRY_SLEEP_TIME_MS =
+ SQLDelegationTokenSecretManager.SQL_DTSM_CONF_PREFIX + "retry-sleep-time-ms";
+ public final static long RETRY_SLEEP_TIME_MS_DEFAULT = 100;
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(SQLSecretManagerRetriableHandlerImpl.class);
+
+ static SQLSecretManagerRetriableHandler getInstance(Configuration conf) {
+ return getInstance(conf, new SQLSecretManagerRetriableHandlerImpl());
+ }
+
+ static SQLSecretManagerRetriableHandler getInstance(Configuration conf,
+ SQLSecretManagerRetriableHandlerImpl retryHandler) {
+ RetryPolicy basePolicy = RetryPolicies.exponentialBackoffRetry(
+ conf.getInt(MAX_RETRIES, MAX_RETRIES_DEFAULT),
+ conf.getLong(RETRY_SLEEP_TIME_MS, RETRY_SLEEP_TIME_MS_DEFAULT),
+ TimeUnit.MILLISECONDS);
+
+ // Configure SQLSecretManagerRetriableException to retry with exponential backoff
+ Map, RetryPolicy> exceptionToPolicyMap = new HashMap<>();
+ exceptionToPolicyMap.put(SQLSecretManagerRetriableException.class, basePolicy);
+
+ // Configure all other exceptions to fail after one attempt
+ RetryPolicy retryPolicy = RetryPolicies.retryByException(
+ RetryPolicies.TRY_ONCE_THEN_FAIL, exceptionToPolicyMap);
+
+ return (SQLSecretManagerRetriableHandler) RetryProxy.create(
+ SQLSecretManagerRetriableHandler.class, retryHandler, retryPolicy);
+ }
+
+ /**
+ * Executes a SQL command and raises retryable errors as
+ * {@link SQLSecretManagerRetriableException}s so they are recognized by the
+ * {@link RetryProxy}.
+ * @param command SQL command to execute
+ * @throws SQLException When SQL connection errors occur
+ */
+ @Override
+ public void execute(SQLCommandVoid command) throws SQLException {
+ try {
+ command.doCall();
+ } catch (SQLException e) {
+ LOG.warn("Failed to execute SQL command", e);
+ throw new SQLSecretManagerRetriableException(e);
+ }
+ }
+
+ /**
+ * Executes a SQL command and raises retryable errors as
+ * {@link SQLSecretManagerRetriableException}s so they are recognized by the
+ * {@link RetryProxy}.
+ * @param command SQL command to execute
+ * @throws SQLException When SQL connection errors occur
+ */
+ @Override
+ public T execute(SQLCommand command) throws SQLException {
+ try {
+ return command.doCall();
+ } catch (SQLException e) {
+ LOG.warn("Failed to execute SQL command", e);
+ throw new SQLSecretManagerRetriableException(e);
+ }
+ }
+
+ /**
+ * Class used to identify errors that can be retried.
+ */
+ static class SQLSecretManagerRetriableException extends SQLException {
+ SQLSecretManagerRetriableException(Throwable cause) {
+ super(cause);
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
index b5096cd253d..79a16cc2022 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/resources/hdfs-rbf-default.xml
@@ -884,4 +884,14 @@
of namespaces in use and the latency of the msync requests.
+
+
+ dfs.federation.router.observer.state.id.refresh.period
+ 15s
+
+ Period to refresh namespace stateID using active namenode. This ensures the
+ namespace stateID is refresh even when an observer is trailing behind.
+ If this is below 0, the auto-refresh is disabled.
+
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java
index 45001b461ba..72e8f8f66d5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestObserverWithRouter.java
@@ -34,9 +34,11 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.LongAccumulator;
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.hdfs.ClientGSIContext;
+import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RouterFederatedStateProto;
@@ -50,6 +52,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServi
import org.apache.hadoop.hdfs.server.federation.resolver.MembershipNamenodeResolver;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos;
+import org.apache.hadoop.test.GenericTestUtils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.AfterEach;
@@ -95,7 +98,9 @@ public class TestObserverWithRouter {
conf.set(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, "0ms");
conf.setBoolean(DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY, true);
if (confOverrides != null) {
- conf.addResource(confOverrides);
+ confOverrides
+ .iterator()
+ .forEachRemaining(entry -> conf.set(entry.getKey(), entry.getValue()));
}
cluster = new MiniRouterDFSCluster(true, 2, numberOfNamenode);
cluster.addNamenodeOverrides(conf);
@@ -639,4 +644,42 @@ public class TestObserverWithRouter {
assertEquals("ns0", namespace1.get(0));
assertTrue(namespace2.isEmpty());
}
+
+ @Test
+ @Tag(SKIP_BEFORE_EACH_CLUSTER_STARTUP)
+ public void testPeriodicStateRefreshUsingActiveNamenode() throws Exception {
+ Path rootPath = new Path("/");
+
+ Configuration confOverride = new Configuration(false);
+ confOverride.set(RBFConfigKeys.DFS_ROUTER_OBSERVER_STATE_ID_REFRESH_PERIOD_KEY, "500ms");
+ confOverride.set(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, "3s");
+ startUpCluster(1, confOverride);
+
+ fileSystem = routerContext.getFileSystem(getConfToEnableObserverReads());
+ fileSystem.listStatus(rootPath);
+ int initialLengthOfRootListing = fileSystem.listStatus(rootPath).length;
+
+ DFSClient activeClient = cluster.getNamenodes("ns0")
+ .stream()
+ .filter(nnContext -> nnContext.getNamenode().isActiveState())
+ .findFirst().orElseThrow(() -> new IllegalStateException("No active namenode."))
+ .getClient();
+
+ for (int i = 0; i < 10; i++) {
+ activeClient.mkdirs("/dir" + i, null, false);
+ }
+ activeClient.close();
+
+ // Wait long enough for state in router to be considered stale.
+ GenericTestUtils.waitFor(
+ () -> !routerContext
+ .getRouterRpcClient()
+ .isNamespaceStateIdFresh("ns0"),
+ 100,
+ 10000,
+ "Timeout: Namespace state was never considered stale.");
+ FileStatus[] rootFolderAfterMkdir = fileSystem.listStatus(rootPath);
+ assertEquals("List-status should show newly created directories.",
+ initialLengthOfRootListing + 10, rootFolderAfterMkdir.length);
+ }
}
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 cd98b635b50..d5e7f7173fc 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
@@ -39,6 +39,7 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.lang.reflect.Method;
import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
@@ -218,6 +219,14 @@ public class TestRouterRpc {
cluster.setIndependentDNs();
Configuration conf = new Configuration();
+ // Setup proxy users.
+ conf.set("hadoop.proxyuser.testRealUser.groups", "*");
+ conf.set("hadoop.proxyuser.testRealUser.hosts", "*");
+ String loginUser = UserGroupInformation.getLoginUser().getUserName();
+ conf.set(String.format("hadoop.proxyuser.%s.groups", loginUser), "*");
+ conf.set(String.format("hadoop.proxyuser.%s.hosts", loginUser), "*");
+ // Enable IP proxy users.
+ conf.set(DFSConfigKeys.DFS_NAMENODE_IP_PROXY_USERS, "placeholder");
conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, 5);
cluster.addNamenodeOverrides(conf);
// Start NNs and DNs and wait until ready
@@ -2077,6 +2086,38 @@ public class TestRouterRpc {
assertTrue(verifyFileExists(routerFS, dirPath));
}
+ @Test
+ public void testRealUserPropagationInCallerContext()
+ throws IOException, InterruptedException {
+ GenericTestUtils.LogCapturer auditlog =
+ GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.AUDIT_LOG);
+
+ // Current callerContext is null
+ assertNull(CallerContext.getCurrent());
+
+ UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
+ UserGroupInformation realUser = UserGroupInformation
+ .createUserForTesting("testRealUser", new String[]{"group"});
+ UserGroupInformation proxyUser = UserGroupInformation
+ .createProxyUser("testProxyUser", realUser);
+ FileSystem proxyFs = proxyUser.doAs(
+ (PrivilegedExceptionAction) () -> router.getFileSystem());
+ proxyFs.listStatus(new Path("/"));
+
+
+ final String logOutput = auditlog.getOutput();
+ // Login user, which is used as the router's user, is different from the realUser.
+ assertNotEquals(loginUser.getUserName(), realUser.getUserName());
+ // Login user is used in the audit log's ugi field.
+ assertTrue("The login user is the proxyUser in the UGI field",
+ logOutput.contains(String.format("ugi=%s (auth:PROXY) via %s (auth:SIMPLE)",
+ proxyUser.getUserName(),
+ loginUser.getUserName())));
+ // Real user is added to the caller context.
+ assertTrue("The audit log should contain the real user.",
+ logOutput.contains(String.format("realUser:%s", realUser.getUserName())));
+ }
+
@Test
public void testSetBalancerBandwidth() throws Exception {
long defaultBandwidth =
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWithSecureStartup.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWithSecureStartup.java
index b660b4bcbb2..b0c0c050bf0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWithSecureStartup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterWithSecureStartup.java
@@ -56,7 +56,8 @@ public class TestRouterWithSecureStartup {
@Test
public void testStartupWithoutKeytab() throws Exception {
testCluster(DFS_ROUTER_KEYTAB_FILE_KEY,
- "Running in secure mode, but config doesn't have a keytab");
+ "Running in secure mode, but config doesn't have a keytab for "
+ + "key: dfs.federation.router.keytab.file");
}
@Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/security/token/TestSQLDelegationTokenSecretManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/security/token/TestSQLDelegationTokenSecretManagerImpl.java
new file mode 100644
index 00000000000..569a274042b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/security/token/TestSQLDelegationTokenSecretManagerImpl.java
@@ -0,0 +1,471 @@
+/*
+ * 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.federation.router.security.token;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+public class TestSQLDelegationTokenSecretManagerImpl {
+ private static final String CONNECTION_URL = "jdbc:derby:memory:TokenStore";
+ private static final int TEST_MAX_RETRIES = 3;
+ private static Configuration conf;
+
+ @Before
+ public void init() throws SQLException {
+ createTestDBTables();
+ }
+
+ @After
+ public void cleanup() throws SQLException {
+ dropTestDBTables();
+ }
+
+ @BeforeClass
+ public static void initDatabase() throws SQLException {
+ DriverManager.getConnection(CONNECTION_URL + ";create=true");
+
+ conf = new Configuration();
+ conf.set(SQLConnectionFactory.CONNECTION_URL, CONNECTION_URL);
+ conf.set(SQLConnectionFactory.CONNECTION_USERNAME, "testuser");
+ conf.set(SQLConnectionFactory.CONNECTION_PASSWORD, "testpassword");
+ conf.set(SQLConnectionFactory.CONNECTION_DRIVER, "org.apache.derby.jdbc.EmbeddedDriver");
+ conf.setInt(SQLSecretManagerRetriableHandlerImpl.MAX_RETRIES, TEST_MAX_RETRIES);
+ conf.setInt(SQLSecretManagerRetriableHandlerImpl.RETRY_SLEEP_TIME_MS, 10);
+ }
+
+ @AfterClass
+ public static void cleanupDatabase() {
+ try {
+ DriverManager.getConnection(CONNECTION_URL + ";drop=true");
+ } catch (SQLException e) {
+ // SQLException expected when database is dropped
+ if (!e.getMessage().contains("dropped")) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ @Test
+ public void testSingleSecretManager() throws Exception {
+ DelegationTokenManager tokenManager = createTokenManager();
+ try {
+ Token extends AbstractDelegationTokenIdentifier> token =
+ tokenManager.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ validateToken(tokenManager, token);
+ } finally {
+ stopTokenManager(tokenManager);
+ }
+ }
+
+ @Test
+ public void testMultipleSecretManagers() throws Exception {
+ DelegationTokenManager tokenManager1 = createTokenManager();
+ DelegationTokenManager tokenManager2 = createTokenManager();
+
+ try {
+ Token extends AbstractDelegationTokenIdentifier> token1 =
+ tokenManager1.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ Token extends AbstractDelegationTokenIdentifier> token2 =
+ tokenManager2.createToken(UserGroupInformation.getCurrentUser(), "foo");
+
+ validateToken(tokenManager1, token2);
+ validateToken(tokenManager2, token1);
+ } finally {
+ stopTokenManager(tokenManager1);
+ stopTokenManager(tokenManager2);
+ }
+ }
+
+ @Test
+ public void testSequenceNumAllocation() throws Exception {
+ int tokensPerManager = SQLDelegationTokenSecretManagerImpl.DEFAULT_SEQ_NUM_BATCH_SIZE * 5;
+ Set sequenceNums1 = new HashSet<>();
+ Set sequenceNums2 = new HashSet<>();
+ Set sequenceNums3 = new HashSet<>();
+ Set sequenceNums = new HashSet<>();
+ DelegationTokenManager tokenManager1 = createTokenManager();
+ DelegationTokenManager tokenManager2 = createTokenManager();
+ DelegationTokenManager tokenManager3 = createTokenManager();
+
+ try {
+ for (int i = 0; i < tokensPerManager; i++) {
+ allocateSequenceNum(tokenManager1, sequenceNums1);
+ allocateSequenceNum(tokenManager2, sequenceNums2);
+ allocateSequenceNum(tokenManager3, sequenceNums3);
+ sequenceNums.addAll(sequenceNums1);
+ sequenceNums.addAll(sequenceNums2);
+ sequenceNums.addAll(sequenceNums3);
+ }
+
+ Assert.assertEquals("Verify that all tokens were created with unique sequence numbers",
+ tokensPerManager * 3, sequenceNums.size());
+ Assert.assertEquals("Verify that tokenManager1 generated unique sequence numbers",
+ tokensPerManager, sequenceNums1.size());
+ Assert.assertEquals("Verify that tokenManager2 generated unique sequence number",
+ tokensPerManager, sequenceNums2.size());
+ Assert.assertEquals("Verify that tokenManager3 generated unique sequence numbers",
+ tokensPerManager, sequenceNums3.size());
+
+ // Validate sequence number batches allocated in order to each token manager
+ int batchSize = SQLDelegationTokenSecretManagerImpl.DEFAULT_SEQ_NUM_BATCH_SIZE;
+ for (int seqNum = 1; seqNum < tokensPerManager;) {
+ // First batch allocated tokenManager1
+ for (int i = 0; i < batchSize; i++, seqNum++) {
+ Assert.assertTrue(sequenceNums1.contains(seqNum));
+ }
+ // Second batch allocated tokenManager2
+ for (int i = 0; i < batchSize; i++, seqNum++) {
+ Assert.assertTrue(sequenceNums2.contains(seqNum));
+ }
+ // Third batch allocated tokenManager3
+ for (int i = 0; i < batchSize; i++, seqNum++) {
+ Assert.assertTrue(sequenceNums3.contains(seqNum));
+ }
+ }
+
+ SQLDelegationTokenSecretManagerImpl secretManager =
+ (SQLDelegationTokenSecretManagerImpl) tokenManager1.getDelegationTokenSecretManager();
+ Assert.assertEquals("Verify that the counter is set to the highest sequence number",
+ tokensPerManager * 3, secretManager.getDelegationTokenSeqNum());
+ } finally {
+ stopTokenManager(tokenManager1);
+ stopTokenManager(tokenManager2);
+ stopTokenManager(tokenManager3);
+ }
+ }
+
+ @Test
+ public void testSequenceNumRollover() throws Exception {
+ int tokenBatch = SQLDelegationTokenSecretManagerImpl.DEFAULT_SEQ_NUM_BATCH_SIZE;
+ Set sequenceNums = new HashSet<>();
+
+ DelegationTokenManager tokenManager = createTokenManager();
+
+ try {
+ SQLDelegationTokenSecretManagerImpl secretManager =
+ (SQLDelegationTokenSecretManagerImpl) tokenManager.getDelegationTokenSecretManager();
+ secretManager.setDelegationTokenSeqNum(Integer.MAX_VALUE - tokenBatch);
+
+ // Allocate sequence numbers before they are rolled over
+ for (int seqNum = Integer.MAX_VALUE - tokenBatch; seqNum < Integer.MAX_VALUE; seqNum++) {
+ allocateSequenceNum(tokenManager, sequenceNums);
+ Assert.assertTrue(sequenceNums.contains(seqNum + 1));
+ }
+
+ // Allocate sequence numbers after they are rolled over
+ for (int seqNum = 0; seqNum < tokenBatch; seqNum++) {
+ allocateSequenceNum(tokenManager, sequenceNums);
+ Assert.assertTrue(sequenceNums.contains(seqNum + 1));
+ }
+ } finally {
+ stopTokenManager(tokenManager);
+ }
+ }
+
+ @Test
+ public void testDelegationKeyAllocation() throws Exception {
+ DelegationTokenManager tokenManager1 = createTokenManager();
+
+ try {
+ SQLDelegationTokenSecretManagerImpl secretManager1 =
+ (SQLDelegationTokenSecretManagerImpl) tokenManager1.getDelegationTokenSecretManager();
+ // Prevent delegation keys to roll for the rest of the test to avoid race conditions
+ // between the keys generated and the active keys in the database.
+ ((TestDelegationTokenSecretManager) secretManager1).lockKeyRoll();
+ int keyId1 = secretManager1.getCurrentKeyId();
+
+ // Validate that latest key1 is assigned to tokenManager1 tokens
+ Token extends AbstractDelegationTokenIdentifier> token1 =
+ tokenManager1.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ validateKeyId(token1, keyId1);
+
+ DelegationTokenManager tokenManager2 = createTokenManager();
+ try {
+ SQLDelegationTokenSecretManagerImpl secretManager2 =
+ (SQLDelegationTokenSecretManagerImpl) tokenManager2.getDelegationTokenSecretManager();
+ // Prevent delegation keys to roll for the rest of the test
+ ((TestDelegationTokenSecretManager) secretManager2).lockKeyRoll();
+ int keyId2 = secretManager2.getCurrentKeyId();
+
+ Assert.assertNotEquals("Each secret manager has its own key", keyId1, keyId2);
+
+ // Validate that latest key2 is assigned to tokenManager2 tokens
+ Token extends AbstractDelegationTokenIdentifier> token2 =
+ tokenManager2.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ validateKeyId(token2, keyId2);
+
+ // Validate that key1 is still assigned to tokenManager1 tokens
+ token1 = tokenManager1.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ validateKeyId(token1, keyId1);
+
+ // Validate that key2 is still assigned to tokenManager2 tokens
+ token2 = tokenManager2.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ validateKeyId(token2, keyId2);
+ } finally {
+ stopTokenManager(tokenManager2);
+ }
+ } finally {
+ stopTokenManager(tokenManager1);
+ }
+ }
+
+ @Test
+ public void testHikariConfigs() {
+ HikariDataSourceConnectionFactory factory1 = new HikariDataSourceConnectionFactory(conf);
+ int defaultMaximumPoolSize = factory1.getDataSource().getMaximumPoolSize();
+ factory1.shutdown();
+
+ // Changing default maximumPoolSize
+ Configuration hikariConf = new Configuration(conf);
+ hikariConf.setInt(HikariDataSourceConnectionFactory.HIKARI_PROPS + "maximumPoolSize",
+ defaultMaximumPoolSize + 1);
+
+ // Verifying property is correctly set in datasource
+ HikariDataSourceConnectionFactory factory2 = new HikariDataSourceConnectionFactory(hikariConf);
+ Assert.assertEquals(factory2.getDataSource().getMaximumPoolSize(),
+ defaultMaximumPoolSize + 1);
+ factory2.shutdown();
+ }
+
+ @Test
+ public void testRetries() throws Exception {
+ DelegationTokenManager tokenManager = createTokenManager();
+ TestDelegationTokenSecretManager secretManager =
+ (TestDelegationTokenSecretManager) tokenManager.getDelegationTokenSecretManager();
+
+ try {
+ // Prevent delegation keys to roll for the rest of the test
+ secretManager.lockKeyRoll();
+
+ // Reset counter and expect a single request when inserting a token
+ TestRetryHandler.resetExecutionAttemptCounter();
+ tokenManager.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ Assert.assertEquals(1, TestRetryHandler.getExecutionAttempts());
+
+ // Breaking database connections to cause retries
+ secretManager.setReadOnly(true);
+
+ // Reset counter and expect a multiple retries when failing to insert a token
+ TestRetryHandler.resetExecutionAttemptCounter();
+ tokenManager.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ Assert.assertEquals(TEST_MAX_RETRIES + 1, TestRetryHandler.getExecutionAttempts());
+ } finally {
+ // Fix database connections
+ secretManager.setReadOnly(false);
+ stopTokenManager(tokenManager);
+ }
+ }
+
+ private DelegationTokenManager createTokenManager() {
+ DelegationTokenManager tokenManager = new DelegationTokenManager(new Configuration(), null);
+ tokenManager.setExternalDelegationTokenSecretManager(new TestDelegationTokenSecretManager());
+ return tokenManager;
+ }
+
+ private void allocateSequenceNum(DelegationTokenManager tokenManager, Set sequenceNums)
+ throws IOException {
+ Token extends AbstractDelegationTokenIdentifier> token =
+ tokenManager.createToken(UserGroupInformation.getCurrentUser(), "foo");
+ AbstractDelegationTokenIdentifier tokenIdentifier = token.decodeIdentifier();
+ Assert.assertFalse("Verify sequence number is unique",
+ sequenceNums.contains(tokenIdentifier.getSequenceNumber()));
+
+ sequenceNums.add(tokenIdentifier.getSequenceNumber());
+ }
+
+ private void validateToken(DelegationTokenManager tokenManager,
+ Token extends AbstractDelegationTokenIdentifier> token)
+ throws Exception {
+ SQLDelegationTokenSecretManagerImpl secretManager =
+ (SQLDelegationTokenSecretManagerImpl) tokenManager.getDelegationTokenSecretManager();
+ AbstractDelegationTokenIdentifier tokenIdentifier = token.decodeIdentifier();
+
+ // Verify token using token manager
+ tokenManager.verifyToken(token);
+
+ byte[] tokenInfo1 = secretManager.selectTokenInfo(tokenIdentifier.getSequenceNumber(),
+ tokenIdentifier.getBytes());
+ Assert.assertNotNull("Verify token exists in database", tokenInfo1);
+
+ // Renew token using token manager
+ tokenManager.renewToken(token, "foo");
+
+ byte[] tokenInfo2 = secretManager.selectTokenInfo(tokenIdentifier.getSequenceNumber(),
+ tokenIdentifier.getBytes());
+ Assert.assertNotNull("Verify token exists in database", tokenInfo2);
+ Assert.assertFalse("Verify token has been updated in database",
+ Arrays.equals(tokenInfo1, tokenInfo2));
+
+ // Cancel token using token manager
+ tokenManager.cancelToken(token, "foo");
+ byte[] tokenInfo3 = secretManager.selectTokenInfo(tokenIdentifier.getSequenceNumber(),
+ tokenIdentifier.getBytes());
+ Assert.assertNull("Verify token was removed from database", tokenInfo3);
+ }
+
+ private void validateKeyId(Token extends AbstractDelegationTokenIdentifier> token,
+ int expectedKeyiD) throws IOException {
+ AbstractDelegationTokenIdentifier tokenIdentifier = token.decodeIdentifier();
+ Assert.assertEquals("Verify that keyId is assigned to token",
+ tokenIdentifier.getMasterKeyId(), expectedKeyiD);
+ }
+
+ private static Connection getTestDBConnection() {
+ try {
+ return DriverManager.getConnection(CONNECTION_URL);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static void createTestDBTables() throws SQLException {
+ execute("CREATE TABLE Tokens (sequenceNum INT NOT NULL, "
+ + "tokenIdentifier VARCHAR (255) FOR BIT DATA NOT NULL, "
+ + "tokenInfo VARCHAR (255) FOR BIT DATA NOT NULL, "
+ + "modifiedTime TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP, "
+ + "PRIMARY KEY(sequenceNum))");
+ execute("CREATE TABLE DelegationKeys (keyId INT NOT NULL, "
+ + "delegationKey VARCHAR (255) FOR BIT DATA NOT NULL, "
+ + "modifiedTime TIMESTAMP NOT NULL DEFAULT CURRENT_TIMESTAMP, "
+ + "PRIMARY KEY(keyId))");
+ execute("CREATE TABLE LastSequenceNum (sequenceNum INT NOT NULL)");
+ execute("INSERT INTO LastSequenceNum VALUES (0)");
+ execute("CREATE TABLE LastDelegationKeyId (keyId INT NOT NULL)");
+ execute("INSERT INTO LastDelegationKeyId VALUES (0)");
+ }
+
+ private static void dropTestDBTables() throws SQLException {
+ execute("DROP TABLE Tokens");
+ execute("DROP TABLE DelegationKeys");
+ execute("DROP TABLE LastSequenceNum");
+ execute("DROP TABLE LastDelegationKeyId");
+ }
+
+ private static void execute(String statement) throws SQLException {
+ try (Connection connection = getTestDBConnection()) {
+ connection.createStatement().execute(statement);
+ }
+ }
+
+ private void stopTokenManager(DelegationTokenManager tokenManager) {
+ TestDelegationTokenSecretManager secretManager =
+ (TestDelegationTokenSecretManager) tokenManager.getDelegationTokenSecretManager();
+ // Release any locks on tables
+ secretManager.unlockKeyRoll();
+ // Stop threads to close database connections
+ secretManager.stopThreads();
+ }
+
+ static class TestDelegationTokenSecretManager extends SQLDelegationTokenSecretManagerImpl {
+ private ReentrantLock keyRollLock;
+
+ private synchronized ReentrantLock getKeyRollLock() {
+ if (keyRollLock == null) {
+ keyRollLock = new ReentrantLock();
+ }
+ return keyRollLock;
+ }
+
+ TestDelegationTokenSecretManager() {
+ super(conf, new TestConnectionFactory(conf),
+ SQLSecretManagerRetriableHandlerImpl.getInstance(conf, new TestRetryHandler()));
+ }
+
+ // Tests can call this method to prevent delegation keys from
+ // being rolled in the middle of a test to prevent race conditions
+ public void lockKeyRoll() {
+ getKeyRollLock().lock();
+ }
+
+ public void unlockKeyRoll() {
+ if (getKeyRollLock().isHeldByCurrentThread()) {
+ getKeyRollLock().unlock();
+ }
+ }
+
+ @Override
+ protected void rollMasterKey() throws IOException {
+ try {
+ lockKeyRoll();
+ super.rollMasterKey();
+ } finally {
+ unlockKeyRoll();
+ }
+ }
+
+ public void setReadOnly(boolean readOnly) {
+ ((TestConnectionFactory) getConnectionFactory()).readOnly = readOnly;
+ }
+ }
+
+ static class TestConnectionFactory extends HikariDataSourceConnectionFactory {
+ private boolean readOnly = false;
+ TestConnectionFactory(Configuration conf) {
+ super(conf);
+ }
+
+ @Override
+ public Connection getConnection() throws SQLException {
+ Connection connection = super.getConnection();
+ // Change to default schema as derby driver looks for user schema
+ connection.setSchema("APP");
+ connection.setReadOnly(readOnly);
+ return connection;
+ }
+ }
+
+ static class TestRetryHandler extends SQLSecretManagerRetriableHandlerImpl {
+ // Tracks the amount of times that a SQL command is executed, regardless of
+ // whether it completed successfully or not.
+ private static AtomicInteger executionAttemptCounter = new AtomicInteger();
+
+ static void resetExecutionAttemptCounter() {
+ executionAttemptCounter = new AtomicInteger();
+ }
+
+ static int getExecutionAttempts() {
+ return executionAttemptCounter.get();
+ }
+
+ @Override
+ public void execute(SQLCommandVoid command) throws SQLException {
+ executionAttemptCounter.incrementAndGet();
+ super.execute(command);
+ }
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 5c2df9acf4e..8632c567aa1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -310,4 +310,14 @@
+
+
+
+
+
+
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 1ab7edd6adc..e5e21e4307a 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
@@ -733,12 +733,43 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME = "default";
public static final String DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY = "dfs.namenode.audit.log.token.tracking.id";
public static final boolean DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT = false;
- public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY = "dfs.namenode.audit.log.async";
+ /**
+ * Deprecated. Use log4j properties instead.
+ * Set system env variable HDFS_AUDIT_LOGGER, which in tern assigns the value to
+ * "hdfs.audit.logger" for log4j properties to determine log level and appender.
+ */
+ @Deprecated
+ public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY = "dfs.namenode.audit.log.async";
+ @Deprecated
public static final boolean DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT = false;
- public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_BLOCKING_KEY = "dfs.namenode.audit.log.async.blocking";
+
+ /**
+ * Deprecated. Use log4j properties instead.
+ * Set value to Async appender "blocking" property as part of log4j properties configuration.
+ *
+ * For example,
+ * log4j.appender.ASYNCAPPENDER=org.apache.log4j.AsyncAppender
+ * log4j.appender.ASYNCAPPENDER.blocking=false
+ */
+ @Deprecated
+ public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_BLOCKING_KEY =
+ "dfs.namenode.audit.log.async.blocking";
+ @Deprecated
public static final boolean DFS_NAMENODE_AUDIT_LOG_ASYNC_BLOCKING_DEFAULT = true;
- public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_BUFFER_SIZE_KEY = "dfs.namenode.audit.log.async.buffer.size";
- public static final int DFS_NAMENODE_AUDIT_LOG_ASYNC_BUFFER_SIZE_DEFAULT = 128;
+
+ /**
+ * Deprecated. Use log4j properties instead.
+ * Set value to Async appender "bufferSize" property as part of log4j properties configuration.
+ *
+ * For example,
+ * log4j.appender.ASYNCAPPENDER=org.apache.log4j.AsyncAppender
+ * log4j.appender.ASYNCAPPENDER.bufferSize=128
+ */
+ @Deprecated
+ public static final String DFS_NAMENODE_AUDIT_LOG_ASYNC_BUFFER_SIZE_KEY =
+ "dfs.namenode.audit.log.async.buffer.size";
+ @Deprecated
+ public static final int DFS_NAMENODE_AUDIT_LOG_ASYNC_BUFFER_SIZE_DEFAULT = 128;
public static final String DFS_NAMENODE_AUDIT_LOG_DEBUG_CMDLIST = "dfs.namenode.audit.log.debug.cmdlist";
public static final String DFS_NAMENODE_METRICS_LOGGER_PERIOD_SECONDS_KEY =
"dfs.namenode.metrics.logger.period.seconds";
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index 7237489e7bf..25726cee510 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -69,6 +69,7 @@ import org.apache.commons.cli.PosixParser;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.namenode.INodesInPath;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
@@ -1936,4 +1937,18 @@ public class DFSUtil {
return path.charAt(parent.length()) == Path.SEPARATOR_CHAR
|| parent.equals(Path.SEPARATOR);
}
+
+ /**
+ * Add transfer rate metrics for valid data read and duration values.
+ * @param metrics metrics for datanodes
+ * @param read bytes read
+ * @param duration read duration
+ */
+ public static void addTransferRateMetric(final DataNodeMetrics metrics, final long read, final long duration) {
+ if (read >= 0 && duration > 0) {
+ metrics.addReadTransferRate(read * 1000 / duration);
+ } else {
+ LOG.warn("Unexpected value for data transfer bytes={} duration={}", read, duration);
+ }
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java
index 66685f6cc18..21c01cebd40 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/MetricsLoggerTask.java
@@ -18,9 +18,7 @@
package org.apache.hadoop.hdfs.server.common;
import java.lang.management.ManagementFactory;
-import java.util.Collections;
import java.util.HashSet;
-import java.util.List;
import java.util.Set;
import javax.management.Attribute;
@@ -34,8 +32,6 @@ import javax.management.ObjectName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.metrics2.util.MBeans;
-import org.apache.log4j.Appender;
-import org.apache.log4j.AsyncAppender;
/**
* MetricsLoggerTask can be used as utility to dump metrics to log.
@@ -56,12 +52,12 @@ public class MetricsLoggerTask implements Runnable {
}
}
- private org.apache.log4j.Logger metricsLog;
+ private Logger metricsLog;
private String nodeName;
private short maxLogLineLength;
public MetricsLoggerTask(String metricsLog, String nodeName, short maxLogLineLength) {
- this.metricsLog = org.apache.log4j.Logger.getLogger(metricsLog);
+ this.metricsLog = LoggerFactory.getLogger(metricsLog);
this.nodeName = nodeName;
this.maxLogLineLength = maxLogLineLength;
}
@@ -115,8 +111,11 @@ public class MetricsLoggerTask implements Runnable {
.substring(0, maxLogLineLength) + "...");
}
- private static boolean hasAppenders(org.apache.log4j.Logger logger) {
- return logger.getAllAppenders().hasMoreElements();
+ // TODO : hadoop-logging module to hide log4j implementation details, this method
+ // can directly call utility from hadoop-logging.
+ private static boolean hasAppenders(Logger logger) {
+ return org.apache.log4j.Logger.getLogger(logger.getName()).getAllAppenders()
+ .hasMoreElements();
}
/**
@@ -138,26 +137,4 @@ public class MetricsLoggerTask implements Runnable {
return attributeNames;
}
- /**
- * Make the metrics logger async and add all pre-existing appenders to the
- * async appender.
- */
- public static void makeMetricsLoggerAsync(String metricsLog) {
- org.apache.log4j.Logger logger = org.apache.log4j.Logger.getLogger(metricsLog);
- logger.setAdditivity(false); // Don't pollute actual logs with metrics dump
-
- @SuppressWarnings("unchecked")
- List appenders = Collections.list(logger.getAllAppenders());
- // failsafe against trying to async it more than once
- if (!appenders.isEmpty() && !(appenders.get(0) instanceof AsyncAppender)) {
- AsyncAppender asyncAppender = new AsyncAppender();
- // change logger to have an async appender containing all the
- // previously configured appenders
- for (Appender appender : appenders) {
- logger.removeAppender(appender);
- asyncAppender.addAppender(appender);
- }
- logger.addAppender(asyncAppender);
- }
- }
}
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 ce566885983..b781053a767 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
@@ -4058,8 +4058,6 @@ public class DataNode extends ReconfigurableBase
return;
}
- MetricsLoggerTask.makeMetricsLoggerAsync(METRICS_LOG_NAME);
-
// Schedule the periodic logging.
metricsLoggerTimer = new ScheduledThreadPoolExecutor(1);
metricsLoggerTimer.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 3cf4bde3d45..ab706fb1731 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.datanode;
import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.util.Preconditions;
import org.apache.hadoop.thirdparty.protobuf.ByteString;
import javax.crypto.SecretKey;
@@ -632,6 +633,7 @@ class DataXceiver extends Receiver implements Runnable {
datanode.metrics.incrBytesRead((int) read);
datanode.metrics.incrBlocksRead();
datanode.metrics.incrTotalReadTime(duration);
+ DFSUtil.addTransferRateMetric(datanode.metrics, read, duration);
} catch ( SocketException ignored ) {
LOG.trace("{}:Ignoring exception while serving {} to {}",
dnR, block, remoteAddress, ignored);
@@ -1122,6 +1124,7 @@ class DataXceiver extends Receiver implements Runnable {
datanode.metrics.incrBytesRead((int) read);
datanode.metrics.incrBlocksRead();
datanode.metrics.incrTotalReadTime(duration);
+ DFSUtil.addTransferRateMetric(datanode.metrics, read, duration);
LOG.info("Copied {} to {}", block, peer.getRemoteAddressString());
} catch (IOException ioe) {
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 649d30e91e0..675dbbff4c3 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
@@ -61,6 +61,8 @@ public class DataNodeMetrics {
@Metric MutableCounterLong bytesRead;
@Metric("Milliseconds spent reading")
MutableCounterLong totalReadTime;
+ @Metric private MutableRate readTransferRate;
+ final private MutableQuantiles[] readTransferRateQuantiles;
@Metric MutableCounterLong blocksWritten;
@Metric MutableCounterLong blocksRead;
@Metric MutableCounterLong blocksReplicated;
@@ -227,6 +229,7 @@ public class DataNodeMetrics {
sendDataPacketTransferNanosQuantiles = new MutableQuantiles[len];
ramDiskBlocksEvictionWindowMsQuantiles = new MutableQuantiles[len];
ramDiskBlocksLazyPersistWindowMsQuantiles = new MutableQuantiles[len];
+ readTransferRateQuantiles = new MutableQuantiles[len];
for (int i = 0; i < len; i++) {
int interval = intervals[i];
@@ -255,6 +258,10 @@ public class DataNodeMetrics {
"ramDiskBlocksLazyPersistWindows" + interval + "s",
"Time between the RamDisk block write and disk persist in ms",
"ops", "latency", interval);
+ readTransferRateQuantiles[i] = registry.newQuantiles(
+ "readTransferRate" + interval + "s",
+ "Rate at which bytes are read from datanode calculated in bytes per second",
+ "ops", "rate", interval);
}
}
@@ -316,6 +323,13 @@ public class DataNodeMetrics {
}
}
+ public void addReadTransferRate(long readTransferRate) {
+ this.readTransferRate.add(readTransferRate);
+ for (MutableQuantiles q : readTransferRateQuantiles) {
+ q.add(readTransferRate);
+ }
+ }
+
public void addCacheReport(long latency) {
cacheReports.add(latency);
}
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 44b2a21779f..0e46dca9dff 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
@@ -338,10 +338,9 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.util.Lists;
-import org.apache.log4j.Logger;
-import org.apache.log4j.Appender;
-import org.apache.log4j.AsyncAppender;
import org.eclipse.jetty.util.ajax.JSON;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.thirdparty.com.google.common.base.Charsets;
@@ -349,8 +348,6 @@ import org.apache.hadoop.util.Preconditions;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.slf4j.LoggerFactory;
-
/**
* FSNamesystem is a container of both transient
* and persisted name-space state, and does all the book-keeping
@@ -384,8 +381,7 @@ import org.slf4j.LoggerFactory;
public class FSNamesystem implements Namesystem, FSNamesystemMBean,
NameNodeMXBean, ReplicatedBlocksMBean, ECBlockGroupsMBean {
- public static final org.slf4j.Logger LOG = LoggerFactory
- .getLogger(FSNamesystem.class.getName());
+ public static final Logger LOG = LoggerFactory.getLogger(FSNamesystem.class);
// The following are private configurations
public static final String DFS_NAMENODE_SNAPSHOT_TRASHROOT_ENABLED =
@@ -488,7 +484,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
* perm=<permissions (optional)>
*
*/
- public static final Logger AUDIT_LOG = Logger.getLogger(FSNamesystem.class.getName() + ".audit");
+ public static final Logger AUDIT_LOG =
+ LoggerFactory.getLogger(FSNamesystem.class.getName() + ".audit");
private final int maxCorruptFileBlocksReturn;
private final boolean isPermissionEnabled;
@@ -858,11 +855,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
throws IOException {
provider = DFSUtil.createKeyProviderCryptoExtension(conf);
LOG.info("KeyProvider: " + provider);
- if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY,
- DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) {
- LOG.info("Enabling async auditlog");
- enableAsyncAuditLog(conf);
- }
+ checkForAsyncLogEnabledByOldConfigs(conf);
auditLogWithRemotePort =
conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_WITH_REMOTE_PORT_KEY,
DFS_NAMENODE_AUDIT_LOG_WITH_REMOTE_PORT_DEFAULT);
@@ -1076,6 +1069,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
}
+ @SuppressWarnings("deprecation")
+ private static void checkForAsyncLogEnabledByOldConfigs(Configuration conf) {
+ if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY, DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) {
+ LOG.warn("Use log4j properties to enable async log for audit logs. {} is deprecated",
+ DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY);
+ }
+ }
+
@VisibleForTesting
public List getAuditLoggers() {
return auditLoggers;
@@ -4173,7 +4174,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
logAuditEvent(false, operationName, src);
throw e;
}
- if (needLocation && isObserver()) {
+ if (dl != null && needLocation && isObserver()) {
for (HdfsFileStatus fs : dl.getPartialListing()) {
if (fs instanceof HdfsLocatedFileStatus) {
LocatedBlocks lbs = ((HdfsLocatedFileStatus) fs).getLocatedBlocks();
@@ -8787,9 +8788,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
src = escapeJava(src);
dst = escapeJava(dst);
sb.setLength(0);
+ String ipAddr = addr != null ? "/" + addr.getHostAddress() : "null";
sb.append("allowed=").append(succeeded).append("\t")
.append("ugi=").append(userName).append("\t")
- .append("ip=").append(addr).append("\t")
+ .append("ip=").append(ipAddr).append("\t")
.append("cmd=").append(cmd).append("\t")
.append("src=").append(src).append("\t")
.append("dst=").append(dst).append("\t");
@@ -8855,30 +8857,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
}
- private static void enableAsyncAuditLog(Configuration conf) {
- Logger logger = AUDIT_LOG;
- @SuppressWarnings("unchecked")
- List appenders = Collections.list(logger.getAllAppenders());
- // failsafe against trying to async it more than once
- if (!appenders.isEmpty() && !(appenders.get(0) instanceof AsyncAppender)) {
- AsyncAppender asyncAppender = new AsyncAppender();
- asyncAppender.setBlocking(conf.getBoolean(
- DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_BLOCKING_KEY,
- DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_BLOCKING_DEFAULT
- ));
- asyncAppender.setBufferSize(conf.getInt(
- DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_BUFFER_SIZE_KEY,
- DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_BUFFER_SIZE_DEFAULT
- ));
- // change logger to have an async appender containing all the
- // previously configured appenders
- for (Appender appender : appenders) {
- logger.removeAppender(appender);
- asyncAppender.addAppender(appender);
- }
- logger.addAppender(asyncAppender);
- }
- }
/**
* Return total number of Sync Operations on FSEditLog.
*/
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 ddd9fd8087f..ff25eedea0e 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
@@ -946,8 +946,6 @@ public class NameNode extends ReconfigurableBase implements
return;
}
- MetricsLoggerTask.makeMetricsLoggerAsync(METRICS_LOG_NAME);
-
// Schedule the periodic logging.
metricsLoggerTimer = new ScheduledThreadPoolExecutor(1);
metricsLoggerTimer.setExecuteExistingDelayedTasksAfterShutdownPolicy(
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/AsyncRFAAppender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/AsyncRFAAppender.java
new file mode 100644
index 00000000000..276e5b0987a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/AsyncRFAAppender.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.hdfs.util;
+
+import java.io.IOException;
+
+import org.apache.log4j.AsyncAppender;
+import org.apache.log4j.PatternLayout;
+import org.apache.log4j.RollingFileAppender;
+import org.apache.log4j.spi.LoggingEvent;
+
+/**
+ * Until we migrate to log4j2, use this appender for namenode audit logger as well as
+ * datanode and namenode metric loggers with log4j properties, if async logging is required with
+ * RFA.
+ * This appender will take parameters necessary to supply RollingFileAppender to AsyncAppender.
+ * While migrating to log4j2, we can directly wrap RFA appender to Async appender as part of
+ * log4j2 properties. However, same is not possible with log4j1 properties.
+ */
+public class AsyncRFAAppender extends AsyncAppender {
+
+ /**
+ * The default maximum file size is 10MB.
+ */
+ private String maxFileSize = String.valueOf(10*1024*1024);
+
+ /**
+ * There is one backup file by default.
+ */
+ private int maxBackupIndex = 1;
+
+ /**
+ * The name of the log file.
+ */
+ private String fileName = null;
+
+ private String conversionPattern = null;
+
+ /**
+ * Does appender block when buffer is full.
+ */
+ private boolean blocking = true;
+
+ /**
+ * Buffer size.
+ */
+ private int bufferSize = DEFAULT_BUFFER_SIZE;
+
+ private RollingFileAppender rollingFileAppender = null;
+
+ private volatile boolean isRollingFileAppenderAssigned = false;
+
+ @Override
+ public void append(LoggingEvent event) {
+ if (rollingFileAppender == null) {
+ appendRFAToAsyncAppender();
+ }
+ super.append(event);
+ }
+
+ private synchronized void appendRFAToAsyncAppender() {
+ if (!isRollingFileAppenderAssigned) {
+ PatternLayout patternLayout;
+ if (conversionPattern != null) {
+ patternLayout = new PatternLayout(conversionPattern);
+ } else {
+ patternLayout = new PatternLayout();
+ }
+ try {
+ rollingFileAppender = new RollingFileAppender(patternLayout, fileName, true);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ rollingFileAppender.setMaxBackupIndex(maxBackupIndex);
+ rollingFileAppender.setMaxFileSize(maxFileSize);
+ this.addAppender(rollingFileAppender);
+ isRollingFileAppenderAssigned = true;
+ super.setBlocking(blocking);
+ super.setBufferSize(bufferSize);
+ }
+ }
+
+ public String getMaxFileSize() {
+ return maxFileSize;
+ }
+
+ public void setMaxFileSize(String maxFileSize) {
+ this.maxFileSize = maxFileSize;
+ }
+
+ public int getMaxBackupIndex() {
+ return maxBackupIndex;
+ }
+
+ public void setMaxBackupIndex(int maxBackupIndex) {
+ this.maxBackupIndex = maxBackupIndex;
+ }
+
+ public String getFileName() {
+ return fileName;
+ }
+
+ public void setFileName(String fileName) {
+ this.fileName = fileName;
+ }
+
+ public String getConversionPattern() {
+ return conversionPattern;
+ }
+
+ public void setConversionPattern(String conversionPattern) {
+ this.conversionPattern = conversionPattern;
+ }
+
+ public boolean isBlocking() {
+ return blocking;
+ }
+
+ public void setBlocking(boolean blocking) {
+ this.blocking = blocking;
+ }
+
+ public int getBufferSize() {
+ return bufferSize;
+ }
+
+ public void setBufferSize(int bufferSize) {
+ this.bufferSize = bufferSize;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStreamBlockLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStreamBlockLocations.java
index 50378f60381..2e4e496bc3d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStreamBlockLocations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSInputStreamBlockLocations.java
@@ -27,6 +27,7 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.net.InetSocketAddress;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
@@ -35,11 +36,14 @@ import java.util.Map;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.util.Time;
import org.junit.After;
+import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -200,6 +204,25 @@ public class TestDFSInputStreamBlockLocations {
testWithRegistrationMethod(DFSInputStream::getAllBlocks);
}
+ /**
+ * If the ignoreList contains all datanodes, the ignoredList should be cleared to take advantage
+ * of retries built into chooseDataNode. This is needed for hedged reads
+ * @throws IOException
+ */
+ @Test
+ public void testClearIgnoreListChooseDataNode() throws IOException {
+ final String fileName = "/test_cache_locations";
+ filePath = createFile(fileName);
+
+ try (DFSInputStream fin = dfsClient.open(fileName)) {
+ LocatedBlocks existing = fin.locatedBlocks;
+ LocatedBlock block = existing.getLastLocatedBlock();
+ ArrayList ignoreList = new ArrayList<>(Arrays.asList(block.getLocations()));
+ Assert.assertNotNull(fin.chooseDataNode(block, ignoreList, true));
+ Assert.assertEquals(0, ignoreList.size());
+ }
+ }
+
@FunctionalInterface
interface ThrowingConsumer {
void accept(DFSInputStream fin) throws IOException;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index e6ce29316c5..f8e8e4120c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -45,6 +45,7 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
+import static org.mockito.Mockito.*;
import java.io.File;
import java.io.IOException;
@@ -71,6 +72,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.http.HttpConfig;
@@ -1108,4 +1110,18 @@ public class TestDFSUtil {
LambdaTestUtils.intercept(IOException.class, expectedErrorMessage,
()->DFSUtil.getNNServiceRpcAddressesForCluster(conf));
}
+
+ @Test
+ public void testAddTransferRateMetricForValidValues() {
+ DataNodeMetrics mockMetrics = mock(DataNodeMetrics.class);
+ DFSUtil.addTransferRateMetric(mockMetrics, 100, 10);
+ verify(mockMetrics).addReadTransferRate(10000);
+ }
+
+ @Test
+ public void testAddTransferRateMetricForInvalidValue() {
+ DataNodeMetrics mockMetrics = mock(DataNodeMetrics.class);
+ DFSUtil.addTransferRateMetric(mockMetrics, 100, 0);
+ verify(mockMetrics, times(0)).addReadTransferRate(anyLong());
+ }
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index c1e0dbb8e63..729a7941605 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -603,7 +603,9 @@ public class TestPread {
input.read(0, buffer, 0, 1024);
Assert.fail("Reading the block should have thrown BlockMissingException");
} catch (BlockMissingException e) {
- assertEquals(3, input.getHedgedReadOpsLoopNumForTesting());
+ // The result of 9 is due to 2 blocks by 4 iterations plus one because
+ // hedgedReadOpsLoopNumForTesting is incremented at start of the loop.
+ assertEquals(9, input.getHedgedReadOpsLoopNumForTesting());
assertTrue(metrics.getHedgedReadOps() == 0);
} finally {
Mockito.reset(injector);
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 2bf7861287a..de5c985a4f0 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
@@ -380,6 +380,7 @@ public class TestDataNodeMetrics {
@Test(timeout=120000)
public void testDataNodeTimeSpend() throws Exception {
Configuration conf = new HdfsConfiguration();
+ conf.set(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY, "" + 60);
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
try {
final FileSystem fs = cluster.getFileSystem();
@@ -391,6 +392,7 @@ public class TestDataNodeMetrics {
final long startWriteValue = getLongCounter("TotalWriteTime", rb);
final long startReadValue = getLongCounter("TotalReadTime", rb);
+ assertCounter("ReadTransferRateNumOps", 0L, rb);
final AtomicInteger x = new AtomicInteger(0);
// Lets Metric system update latest metrics
@@ -410,6 +412,8 @@ public class TestDataNodeMetrics {
MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name());
final long endWriteValue = getLongCounter("TotalWriteTime", rbNew);
final long endReadValue = getLongCounter("TotalReadTime", rbNew);
+ assertCounter("ReadTransferRateNumOps", 1L, rbNew);
+ assertQuantileGauges("ReadTransferRate" + "60s", rbNew, "Rate");
return endWriteValue > startWriteValue
&& endReadValue > startReadValue;
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
index 29619cc4e31..73201ba6054 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetricsLogger.java
@@ -30,7 +30,6 @@ import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.TimeoutException;
-import java.util.regex.Pattern;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -40,12 +39,11 @@ import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.PatternMatchingAppender;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Appender;
-import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.AsyncAppender;
-import org.apache.log4j.spi.LoggingEvent;
import org.junit.After;
import org.junit.Assert;
import org.junit.Rule;
@@ -151,9 +149,9 @@ public class TestDataNodeMetricsLogger {
metricsProvider);
startDNForTest(true);
assertNotNull(dn);
- final PatternMatchingAppender appender = new PatternMatchingAppender(
- "^.*FakeMetric.*$");
- addAppender(org.apache.log4j.Logger.getLogger(DataNode.METRICS_LOG_NAME), appender);
+ final PatternMatchingAppender appender =
+ (PatternMatchingAppender) org.apache.log4j.Logger.getLogger(DataNode.METRICS_LOG_NAME)
+ .getAppender("PATTERNMATCHERAPPENDER");
// Ensure that the supplied pattern was matched.
GenericTestUtils.waitFor(new Supplier() {
@@ -186,37 +184,4 @@ public class TestDataNodeMetricsLogger {
}
}
- /**
- * An appender that matches logged messages against the given regular
- * expression.
- */
- public static class PatternMatchingAppender extends AppenderSkeleton {
- private final Pattern pattern;
- private volatile boolean matched;
-
- public PatternMatchingAppender(String pattern) {
- this.pattern = Pattern.compile(pattern);
- this.matched = false;
- }
-
- public boolean isMatched() {
- return matched;
- }
-
- @Override
- protected void append(LoggingEvent event) {
- if (pattern.matcher(event.getMessage().toString()).matches()) {
- matched = true;
- }
- }
-
- @Override
- public void close() {
- }
-
- @Override
- public boolean requiresLayout() {
- return false;
- }
- }
}
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 d6f42f3d020..b744a6fa586 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
@@ -1075,16 +1075,14 @@ public class TestFsDatasetImpl {
@Test(timeout = 30000)
public void testReportBadBlocks() throws Exception {
boolean threwException = false;
- MiniDFSCluster cluster = null;
- try {
- Configuration config = new HdfsConfiguration();
- cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
+ final Configuration config = new HdfsConfiguration();
+ try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(config)
+ .numDataNodes(1).build()) {
cluster.waitActive();
Assert.assertEquals(0, cluster.getNamesystem().getCorruptReplicaBlocks());
DataNode dataNode = cluster.getDataNodes().get(0);
- ExtendedBlock block =
- new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(), 0);
+ ExtendedBlock block = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(), 0);
try {
// Test the reportBadBlocks when the volume is null
dataNode.reportBadBlocks(block);
@@ -1101,15 +1099,11 @@ public class TestFsDatasetImpl {
block = DFSTestUtil.getFirstBlock(fs, filePath);
// Test for the overloaded method reportBadBlocks
- dataNode.reportBadBlocks(block, dataNode.getFSDataset()
- .getFsVolumeReferences().get(0));
- Thread.sleep(3000);
- BlockManagerTestUtil.updateState(cluster.getNamesystem()
- .getBlockManager());
- // Verify the bad block has been reported to namenode
- Assert.assertEquals(1, cluster.getNamesystem().getCorruptReplicaBlocks());
- } finally {
- cluster.shutdown();
+ dataNode.reportBadBlocks(block, dataNode.getFSDataset().getFsVolumeReferences().get(0));
+ DataNodeTestUtils.triggerHeartbeat(dataNode);
+ BlockManagerTestUtil.updateState(cluster.getNamesystem().getBlockManager());
+ assertEquals("Corrupt replica blocks could not be reflected with the heartbeat", 1,
+ cluster.getNamesystem().getCorruptReplicaBlocks());
}
}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/PatternMatchingAppender.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/PatternMatchingAppender.java
new file mode 100644
index 00000000000..f099dfae733
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/PatternMatchingAppender.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.hdfs.server.namenode;
+
+import java.util.regex.Pattern;
+
+import org.apache.log4j.AppenderSkeleton;
+import org.apache.log4j.spi.LoggingEvent;
+
+/**
+ * An appender that matches logged messages against the given
+ * regular expression.
+ */
+public class PatternMatchingAppender extends AppenderSkeleton {
+ private final Pattern pattern;
+ private volatile boolean matched;
+
+ public PatternMatchingAppender() {
+ this.pattern = Pattern.compile("^.*FakeMetric.*$");
+ this.matched = false;
+ }
+
+ public boolean isMatched() {
+ return matched;
+ }
+
+ @Override
+ protected void append(LoggingEvent event) {
+ if (pattern.matcher(event.getMessage().toString()).matches()) {
+ matched = true;
+ }
+ }
+
+ @Override
+ public void close() {
+ }
+
+ @Override
+ public boolean requiresLayout() {
+ return false;
+ }
+}
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java
index dad4fa306c7..a6eba0ea051 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogAtDebug.java
@@ -26,11 +26,11 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.FSNamesystemAuditLogger;
import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
+import org.slf4j.event.Level;
import java.net.Inet4Address;
import java.util.Arrays;
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
index 54fcc17cdcf..698178e4e96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
@@ -25,10 +25,10 @@ import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
+import java.io.PrintWriter;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
-import java.util.Enumeration;
import java.util.List;
import java.util.regex.Pattern;
@@ -46,15 +46,10 @@ import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.PathUtils;
import org.apache.log4j.Appender;
import org.apache.log4j.AsyncAppender;
-import org.apache.log4j.Level;
-import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
-import org.apache.log4j.PatternLayout;
-import org.apache.log4j.RollingFileAppender;
+
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -68,36 +63,39 @@ import org.slf4j.LoggerFactory;
*/
@RunWith(Parameterized.class)
public class TestAuditLogs {
- static final String auditLogFile = PathUtils.getTestDirName(TestAuditLogs.class) + "/TestAuditLogs-audit.log";
- final boolean useAsyncLog;
+
+ private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(TestAuditLogs.class);
+
+ private static final File AUDIT_LOG_FILE =
+ new File(System.getProperty("hadoop.log.dir"), "hdfs-audit.log");
+
final boolean useAsyncEdits;
@Parameters
public static Collection
-
- org.junit.jupiter
- junit-jupiter-api
- test
-
-
- org.junit.jupiter
- junit-jupiter-engine
- test
-
-
- org.junit.jupiter
- junit-jupiter-params
- test
-
-
- org.mockito
- mockito-junit-jupiter
- 4.11.0
- test
-
-
- uk.org.webcompere
- system-stubs-core
- 1.1.0
- test
-
-
- uk.org.webcompere
- system-stubs-jupiter
- 1.1.0
- test
- com.fasterxml.jackson.corejackson-databind
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java
index 3a99760aab9..94cd5182a58 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestLocalContainerLauncher.java
@@ -53,11 +53,10 @@ import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
@@ -76,7 +75,7 @@ public class TestLocalContainerLauncher {
fs.delete(p, true);
}
- @BeforeAll
+ @BeforeClass
public static void setupTestDirs() throws IOException {
testWorkDir = new File("target",
TestLocalContainerLauncher.class.getCanonicalName());
@@ -90,7 +89,7 @@ public class TestLocalContainerLauncher {
}
}
- @AfterAll
+ @AfterClass
public static void cleanupTestDirs() throws IOException {
if (testWorkDir != null) {
delete(testWorkDir);
@@ -98,8 +97,7 @@ public class TestLocalContainerLauncher {
}
@SuppressWarnings("rawtypes")
- @Test
- @Timeout(10000)
+ @Test(timeout=10000)
public void testKillJob() throws Exception {
JobConf conf = new JobConf();
AppContext context = mock(AppContext.class);
@@ -200,8 +198,8 @@ public class TestLocalContainerLauncher {
final Path mapOut = mrOutputFiles.getOutputFileForWrite(1);
conf.set(MRConfig.LOCAL_DIR, localDirs[1].toString());
final Path mapOutIdx = mrOutputFiles.getOutputIndexFileForWrite(1);
- Assertions.assertNotEquals(mapOut.getParent(), mapOutIdx.getParent(),
- "Paths must be different!");
+ Assert.assertNotEquals("Paths must be different!",
+ mapOut.getParent(), mapOutIdx.getParent());
// make both dirs part of LOCAL_DIR
conf.setStrings(MRConfig.LOCAL_DIR, localDirs);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java
index 7389aebbd30..49b986e2259 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptFinishingMonitor.java
@@ -37,8 +37,8 @@ import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.Test;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import org.junit.Test;
+import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -87,7 +87,7 @@ public class TestTaskAttemptFinishingMonitor {
}
taskAttemptFinishingMonitor.stop();
- assertTrue(eventHandler.timedOut, "Finishing attempt didn't time out.");
+ assertTrue("Finishing attempt didn't time out.", eventHandler.timedOut);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
index f57ac802fe5..b5a7694e4cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java
@@ -19,18 +19,19 @@ package org.apache.hadoop.mapred;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
-import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
import org.mockito.ArgumentCaptor;
import org.mockito.Captor;
import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -66,15 +67,14 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.mockito.junit.jupiter.MockitoExtension;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.eq;
@@ -87,7 +87,7 @@ import static org.mockito.Mockito.when;
/**
* Tests the behavior of TaskAttemptListenerImpl.
*/
-@ExtendWith(MockitoExtension.class)
+@RunWith(MockitoJUnitRunner.class)
public class TestTaskAttemptListenerImpl {
private static final String ATTEMPT1_ID =
"attempt_123456789012_0001_m_000001_0";
@@ -172,7 +172,7 @@ public class TestTaskAttemptListenerImpl {
}
}
- @AfterEach
+ @After
public void after() throws IOException {
if (listener != null) {
listener.close();
@@ -180,8 +180,7 @@ public class TestTaskAttemptListenerImpl {
}
}
- @Test
- @Timeout(5000)
+ @Test (timeout=5000)
public void testGetTask() throws IOException {
configureMocks();
startListener(false);
@@ -190,12 +189,12 @@ public class TestTaskAttemptListenerImpl {
//The JVM ID has not been registered yet so we should kill it.
JvmContext context = new JvmContext();
- context.jvmId = id;
+ context.jvmId = id;
JvmTask result = listener.getTask(context);
assertNotNull(result);
assertTrue(result.shouldDie);
- // Verify ask after registration but before launch.
+ // Verify ask after registration but before launch.
// Don't kill, should be null.
//Now put a task with the ID
listener.registerPendingTask(task, wid);
@@ -239,8 +238,7 @@ public class TestTaskAttemptListenerImpl {
}
- @Test
- @Timeout(5000)
+ @Test (timeout=5000)
public void testJVMId() {
JVMId jvmid = new JVMId("test", 1, true, 2);
@@ -249,8 +247,7 @@ public class TestTaskAttemptListenerImpl {
assertEquals(0, jvmid.compareTo(jvmid1));
}
- @Test
- @Timeout(10000)
+ @Test (timeout=10000)
public void testGetMapCompletionEvents() throws IOException {
TaskAttemptCompletionEvent[] empty = {};
TaskAttemptCompletionEvent[] taskEvents = {
@@ -260,6 +257,12 @@ public class TestTaskAttemptListenerImpl {
createTce(3, false, TaskAttemptCompletionEventStatus.FAILED) };
TaskAttemptCompletionEvent[] mapEvents = { taskEvents[0], taskEvents[2] };
Job mockJob = mock(Job.class);
+ when(mockJob.getTaskAttemptCompletionEvents(0, 100))
+ .thenReturn(taskEvents);
+ when(mockJob.getTaskAttemptCompletionEvents(0, 2))
+ .thenReturn(Arrays.copyOfRange(taskEvents, 0, 2));
+ when(mockJob.getTaskAttemptCompletionEvents(2, 100))
+ .thenReturn(Arrays.copyOfRange(taskEvents, 2, 4));
when(mockJob.getMapAttemptCompletionEvents(0, 100)).thenReturn(
TypeConverter.fromYarn(mapEvents));
when(mockJob.getMapAttemptCompletionEvents(0, 2)).thenReturn(
@@ -309,8 +312,7 @@ public class TestTaskAttemptListenerImpl {
return tce;
}
- @Test
- @Timeout(10000)
+ @Test (timeout=10000)
public void testCommitWindow() throws IOException {
SystemClock clock = SystemClock.getInstance();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java
index daaabf3e863..8ad62065fa1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestYarnChild.java
@@ -21,8 +21,8 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ClusterStorageCapacityExceededException;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
import static org.mockito.Mockito.*;
@@ -36,7 +36,7 @@ public class TestYarnChild {
final static private String KILL_LIMIT_EXCEED_CONF_NAME =
"mapreduce.job.dfs.storage.capacity.kill-limit-exceed";
- @BeforeEach
+ @Before
public void setUp() throws Exception {
task = mock(Task.class);
umbilical = mock(TaskUmbilicalProtocol.class);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
index 43d3dd89cb9..08896b7b2cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestEvents.java
@@ -19,8 +19,8 @@
package org.apache.hadoop.mapreduce.jobhistory;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -40,8 +40,7 @@ import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Test;
public class TestEvents {
@@ -51,9 +50,9 @@ public class TestEvents {
*
* @throws Exception
*/
- @Test
- @Timeout(10000)
+ @Test(timeout = 10000)
public void testTaskAttemptFinishedEvent() throws Exception {
+
JobID jid = new JobID("001", 1);
TaskID tid = new TaskID(jid, TaskType.REDUCE, 2);
TaskAttemptID taskAttemptId = new TaskAttemptID(tid, 3);
@@ -80,18 +79,17 @@ public class TestEvents {
* @throws Exception
*/
- @Test
- @Timeout(10000)
+ @Test(timeout = 10000)
public void testJobPriorityChange() throws Exception {
org.apache.hadoop.mapreduce.JobID jid = new JobID("001", 1);
JobPriorityChangeEvent test = new JobPriorityChangeEvent(jid,
JobPriority.LOW);
assertThat(test.getJobId().toString()).isEqualTo(jid.toString());
assertThat(test.getPriority()).isEqualTo(JobPriority.LOW);
- }
- @Test
- @Timeout(10000)
+ }
+
+ @Test(timeout = 10000)
public void testJobQueueChange() throws Exception {
org.apache.hadoop.mapreduce.JobID jid = new JobID("001", 1);
JobQueueChangeEvent test = new JobQueueChangeEvent(jid,
@@ -105,14 +103,14 @@ public class TestEvents {
*
* @throws Exception
*/
- @Test
- @Timeout(10000)
+ @Test(timeout = 10000)
public void testTaskUpdated() throws Exception {
JobID jid = new JobID("001", 1);
TaskID tid = new TaskID(jid, TaskType.REDUCE, 2);
TaskUpdatedEvent test = new TaskUpdatedEvent(tid, 1234L);
assertThat(test.getTaskId().toString()).isEqualTo(tid.toString());
assertThat(test.getFinishTime()).isEqualTo(1234L);
+
}
/*
@@ -120,9 +118,9 @@ public class TestEvents {
* instance of HistoryEvent Different HistoryEvent should have a different
* datum.
*/
- @Test
- @Timeout(10000)
+ @Test(timeout = 10000)
public void testEvents() throws Exception {
+
EventReader reader = new EventReader(new DataInputStream(
new ByteArrayInputStream(getEvents())));
HistoryEvent e = reader.getNextEvent();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index ccaf3531034..8159bc2456c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -19,9 +19,9 @@
package org.apache.hadoop.mapreduce.jobhistory;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
@@ -81,12 +81,11 @@ import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.server.MiniYARNCluster;
import org.apache.hadoop.yarn.server.timeline.TimelineStore;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
import org.mockito.Mockito;
import com.fasterxml.jackson.databind.JsonNode;
@@ -102,7 +101,7 @@ public class TestJobHistoryEventHandler {
private static MiniDFSCluster dfsCluster = null;
private static String coreSitePath;
- @BeforeAll
+ @BeforeClass
public static void setUpClass() throws Exception {
coreSitePath = "." + File.separator + "target" + File.separator +
"test-classes" + File.separator + "core-site.xml";
@@ -110,18 +109,17 @@ public class TestJobHistoryEventHandler {
dfsCluster = new MiniDFSCluster.Builder(conf).build();
}
- @AfterAll
+ @AfterClass
public static void cleanUpClass() throws Exception {
dfsCluster.shutdown();
}
- @AfterEach
+ @After
public void cleanTest() throws Exception {
new File(coreSitePath).delete();
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testFirstFlushOnCompletionEvent() throws Exception {
TestParams t = new TestParams();
Configuration conf = new Configuration();
@@ -164,8 +162,7 @@ public class TestJobHistoryEventHandler {
}
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testMaxUnflushedCompletionEvents() throws Exception {
TestParams t = new TestParams();
Configuration conf = new Configuration();
@@ -210,8 +207,7 @@ public class TestJobHistoryEventHandler {
}
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testUnflushedTimer() throws Exception {
TestParams t = new TestParams();
Configuration conf = new Configuration();
@@ -236,26 +232,25 @@ public class TestJobHistoryEventHandler {
mockWriter = jheh.getEventWriter();
verify(mockWriter).write(any(HistoryEvent.class));
- for (int i = 0; i < 100; i++) {
+ for (int i = 0 ; i < 100 ; i++) {
queueEvent(jheh, new JobHistoryEvent(t.jobId, new TaskFinishedEvent(
t.taskID, t.taskAttemptID, 0, TaskType.MAP, "", null, 0)));
}
handleNextNEvents(jheh, 9);
- Assertions.assertTrue(jheh.getFlushTimerStatus());
+ Assert.assertTrue(jheh.getFlushTimerStatus());
verify(mockWriter, times(0)).flush();
Thread.sleep(2 * 4 * 1000l); // 4 seconds should be enough. Just be safe.
verify(mockWriter).flush();
- Assertions.assertFalse(jheh.getFlushTimerStatus());
+ Assert.assertFalse(jheh.getFlushTimerStatus());
} finally {
jheh.stop();
verify(mockWriter).close();
}
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testBatchedFlushJobEndMultiplier() throws Exception {
TestParams t = new TestParams();
Configuration conf = new Configuration();
@@ -300,8 +295,7 @@ public class TestJobHistoryEventHandler {
}
// In case of all types of events, process Done files if it's last AM retry
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testProcessDoneFilesOnLastAMRetry() throws Exception {
TestParams t = new TestParams(true);
Configuration conf = new Configuration();
@@ -315,12 +309,12 @@ public class TestJobHistoryEventHandler {
try {
jheh.start();
handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
- t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
+ t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
verify(jheh, times(0)).processDoneFiles(any(JobId.class));
handleEvent(jheh, new JobHistoryEvent(t.jobId,
- new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
- 0, 0, 0, 0, 0, 0, JobStateInternal.ERROR.toString())));
+ new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+ 0, 0, 0, 0, 0, 0, JobStateInternal.ERROR.toString())));
verify(jheh, times(1)).processDoneFiles(any(JobId.class));
handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
@@ -329,13 +323,13 @@ public class TestJobHistoryEventHandler {
verify(jheh, times(2)).processDoneFiles(any(JobId.class));
handleEvent(jheh, new JobHistoryEvent(t.jobId,
- new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
- 0, 0, 0, 0, 0, 0, JobStateInternal.FAILED.toString())));
+ new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+ 0, 0, 0, 0, 0, 0, JobStateInternal.FAILED.toString())));
verify(jheh, times(3)).processDoneFiles(any(JobId.class));
handleEvent(jheh, new JobHistoryEvent(t.jobId,
- new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
- 0, 0, 0, 0, 0, 0, JobStateInternal.KILLED.toString())));
+ new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+ 0, 0, 0, 0, 0, 0, JobStateInternal.KILLED.toString())));
verify(jheh, times(4)).processDoneFiles(any(JobId.class));
mockWriter = jheh.getEventWriter();
@@ -347,8 +341,7 @@ public class TestJobHistoryEventHandler {
}
// Skip processing Done files in case of ERROR, if it's not last AM retry
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testProcessDoneFilesNotLastAMRetry() throws Exception {
TestParams t = new TestParams(false);
Configuration conf = new Configuration();
@@ -361,13 +354,13 @@ public class TestJobHistoryEventHandler {
try {
jheh.start();
handleEvent(jheh, new JobHistoryEvent(t.jobId, new AMStartedEvent(
- t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
+ t.appAttemptId, 200, t.containerId, "nmhost", 3000, 4000, -1)));
verify(jheh, times(0)).processDoneFiles(t.jobId);
// skip processing done files
handleEvent(jheh, new JobHistoryEvent(t.jobId,
- new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
- 0, 0, 0, 0, 0, 0, JobStateInternal.ERROR.toString())));
+ new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+ 0, 0, 0, 0, 0, 0, JobStateInternal.ERROR.toString())));
verify(jheh, times(0)).processDoneFiles(t.jobId);
handleEvent(jheh, new JobHistoryEvent(t.jobId, new JobFinishedEvent(
@@ -376,13 +369,13 @@ public class TestJobHistoryEventHandler {
verify(jheh, times(1)).processDoneFiles(t.jobId);
handleEvent(jheh, new JobHistoryEvent(t.jobId,
- new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
- 0, 0, 0, 0, 0, 0, JobStateInternal.FAILED.toString())));
+ new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+ 0, 0, 0, 0, 0, 0, JobStateInternal.FAILED.toString())));
verify(jheh, times(2)).processDoneFiles(t.jobId);
handleEvent(jheh, new JobHistoryEvent(t.jobId,
- new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
- 0, 0, 0, 0, 0, 0, JobStateInternal.KILLED.toString())));
+ new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId), 0,
+ 0, 0, 0, 0, 0, 0, JobStateInternal.KILLED.toString())));
verify(jheh, times(3)).processDoneFiles(t.jobId);
mockWriter = jheh.getEventWriter();
@@ -428,15 +421,16 @@ public class TestJobHistoryEventHandler {
// load the job_conf.xml in JHS directory and verify property redaction.
Path jhsJobConfFile = getJobConfInIntermediateDoneDir(conf, params.jobId);
- Assertions.assertTrue(FileContext.getFileContext(conf).util().exists(jhsJobConfFile),
- "The job_conf.xml file is not in the JHS directory");
+ Assert.assertTrue("The job_conf.xml file is not in the JHS directory",
+ FileContext.getFileContext(conf).util().exists(jhsJobConfFile));
Configuration jhsJobConf = new Configuration();
try (InputStream input = FileSystem.get(conf).open(jhsJobConfFile)) {
jhsJobConf.addResource(input);
- Assertions.assertEquals(MRJobConfUtil.REDACTION_REPLACEMENT_VAL,
- jhsJobConf.get(sensitivePropertyName),
- sensitivePropertyName + " is not redacted in HDFS.");
+ Assert.assertEquals(
+ sensitivePropertyName + " is not redacted in HDFS.",
+ MRJobConfUtil.REDACTION_REPLACEMENT_VAL,
+ jhsJobConf.get(sensitivePropertyName));
}
} finally {
jheh.stop();
@@ -462,20 +456,19 @@ public class TestJobHistoryEventHandler {
fs.delete(new Path(intermDoneDirPrefix), true);
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testDefaultFsIsUsedForHistory() throws Exception {
// Create default configuration pointing to the minicluster
Configuration conf = new Configuration();
conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
- dfsCluster.getURI().toString());
+ dfsCluster.getURI().toString());
FileOutputStream os = new FileOutputStream(coreSitePath);
conf.writeXml(os);
os.close();
// simulate execution under a non-default namenode
conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
- "file:///");
+ "file:///");
TestParams t = new TestParams();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, t.dfsWorkDir);
@@ -497,11 +490,11 @@ public class TestJobHistoryEventHandler {
// If we got here then event handler worked but we don't know with which
// file system. Now we check that history stuff was written to minicluster
FileSystem dfsFileSystem = dfsCluster.getFileSystem();
- assertTrue(dfsFileSystem.globStatus(new Path(t.dfsWorkDir + "/*")).length != 0,
- "Minicluster contains some history files");
+ assertTrue("Minicluster contains some history files",
+ dfsFileSystem.globStatus(new Path(t.dfsWorkDir + "/*")).length != 0);
FileSystem localFileSystem = LocalFileSystem.get(conf);
- assertFalse(localFileSystem.exists(new Path(t.dfsWorkDir)),
- "No history directory on non-default file system");
+ assertFalse("No history directory on non-default file system",
+ localFileSystem.exists(new Path(t.dfsWorkDir)));
} finally {
jheh.stop();
purgeHdfsHistoryIntermediateDoneDirectory(conf);
@@ -516,7 +509,7 @@ public class TestJobHistoryEventHandler {
"/mapred/history/done_intermediate");
conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
String pathStr = JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf);
- Assertions.assertEquals("/mapred/history/done_intermediate/" +
+ Assert.assertEquals("/mapred/history/done_intermediate/" +
System.getProperty("user.name"), pathStr);
// Test fully qualified path
@@ -530,14 +523,13 @@ public class TestJobHistoryEventHandler {
conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY,
"file:///");
pathStr = JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf);
- Assertions.assertEquals(dfsCluster.getURI().toString() +
+ Assert.assertEquals(dfsCluster.getURI().toString() +
"/mapred/history/done_intermediate/" + System.getProperty("user.name"),
pathStr);
}
// test AMStartedEvent for submitTime and startTime
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testAMStartedEvent() throws Exception {
TestParams t = new TestParams();
Configuration conf = new Configuration();
@@ -579,8 +571,7 @@ public class TestJobHistoryEventHandler {
// Have JobHistoryEventHandler handle some events and make sure they get
// stored to the Timeline store
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testTimelineEventHandling() throws Exception {
TestParams t = new TestParams(RunningAppContext.class, false);
Configuration conf = new YarnConfiguration();
@@ -607,13 +598,13 @@ public class TestJobHistoryEventHandler {
jheh.getDispatcher().await();
TimelineEntities entities = ts.getEntities("MAPREDUCE_JOB", null, null,
null, null, null, null, null, null, null);
- Assertions.assertEquals(1, entities.getEntities().size());
+ Assert.assertEquals(1, entities.getEntities().size());
TimelineEntity tEntity = entities.getEntities().get(0);
- Assertions.assertEquals(t.jobId.toString(), tEntity.getEntityId());
- Assertions.assertEquals(1, tEntity.getEvents().size());
- Assertions.assertEquals(EventType.AM_STARTED.toString(),
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
+ Assert.assertEquals(1, tEntity.getEvents().size());
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
tEntity.getEvents().get(0).getEventType());
- Assertions.assertEquals(currentTime - 10,
+ Assert.assertEquals(currentTime - 10,
tEntity.getEvents().get(0).getTimestamp());
handleEvent(jheh, new JobHistoryEvent(t.jobId,
@@ -624,17 +615,17 @@ public class TestJobHistoryEventHandler {
jheh.getDispatcher().await();
entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
null, null, null, null, null, null);
- Assertions.assertEquals(1, entities.getEntities().size());
+ Assert.assertEquals(1, entities.getEntities().size());
tEntity = entities.getEntities().get(0);
- Assertions.assertEquals(t.jobId.toString(), tEntity.getEntityId());
- Assertions.assertEquals(2, tEntity.getEvents().size());
- Assertions.assertEquals(EventType.JOB_SUBMITTED.toString(),
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
+ Assert.assertEquals(2, tEntity.getEvents().size());
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
tEntity.getEvents().get(0).getEventType());
- Assertions.assertEquals(EventType.AM_STARTED.toString(),
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
tEntity.getEvents().get(1).getEventType());
- Assertions.assertEquals(currentTime + 10,
+ Assert.assertEquals(currentTime + 10,
tEntity.getEvents().get(0).getTimestamp());
- Assertions.assertEquals(currentTime - 10,
+ Assert.assertEquals(currentTime - 10,
tEntity.getEvents().get(1).getTimestamp());
handleEvent(jheh, new JobHistoryEvent(t.jobId,
@@ -643,80 +634,80 @@ public class TestJobHistoryEventHandler {
jheh.getDispatcher().await();
entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
null, null, null, null, null, null);
- Assertions.assertEquals(1, entities.getEntities().size());
+ Assert.assertEquals(1, entities.getEntities().size());
tEntity = entities.getEntities().get(0);
- Assertions.assertEquals(t.jobId.toString(), tEntity.getEntityId());
- Assertions.assertEquals(3, tEntity.getEvents().size());
- Assertions.assertEquals(EventType.JOB_SUBMITTED.toString(),
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
+ Assert.assertEquals(3, tEntity.getEvents().size());
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
tEntity.getEvents().get(0).getEventType());
- Assertions.assertEquals(EventType.AM_STARTED.toString(),
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
tEntity.getEvents().get(1).getEventType());
- Assertions.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
+ Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
tEntity.getEvents().get(2).getEventType());
- Assertions.assertEquals(currentTime + 10,
+ Assert.assertEquals(currentTime + 10,
tEntity.getEvents().get(0).getTimestamp());
- Assertions.assertEquals(currentTime - 10,
+ Assert.assertEquals(currentTime - 10,
tEntity.getEvents().get(1).getTimestamp());
- Assertions.assertEquals(currentTime - 20,
+ Assert.assertEquals(currentTime - 20,
tEntity.getEvents().get(2).getTimestamp());
handleEvent(jheh, new JobHistoryEvent(t.jobId,
- new JobFinishedEvent(TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0,
+ new JobFinishedEvent(TypeConverter.fromYarn(t.jobId), 0, 0, 0, 0,
0, 0, 0, new Counters(), new Counters(), new Counters()), currentTime));
jheh.getDispatcher().await();
entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
- null, null, null, null, null, null);
- Assertions.assertEquals(1, entities.getEntities().size());
+ null, null, null, null, null, null);
+ Assert.assertEquals(1, entities.getEntities().size());
tEntity = entities.getEntities().get(0);
- Assertions.assertEquals(t.jobId.toString(), tEntity.getEntityId());
- Assertions.assertEquals(4, tEntity.getEvents().size());
- Assertions.assertEquals(EventType.JOB_SUBMITTED.toString(),
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
+ Assert.assertEquals(4, tEntity.getEvents().size());
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
tEntity.getEvents().get(0).getEventType());
- Assertions.assertEquals(EventType.JOB_FINISHED.toString(),
+ Assert.assertEquals(EventType.JOB_FINISHED.toString(),
tEntity.getEvents().get(1).getEventType());
- Assertions.assertEquals(EventType.AM_STARTED.toString(),
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
tEntity.getEvents().get(2).getEventType());
- Assertions.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
+ Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
tEntity.getEvents().get(3).getEventType());
- Assertions.assertEquals(currentTime + 10,
+ Assert.assertEquals(currentTime + 10,
tEntity.getEvents().get(0).getTimestamp());
- Assertions.assertEquals(currentTime,
+ Assert.assertEquals(currentTime,
tEntity.getEvents().get(1).getTimestamp());
- Assertions.assertEquals(currentTime - 10,
+ Assert.assertEquals(currentTime - 10,
tEntity.getEvents().get(2).getTimestamp());
- Assertions.assertEquals(currentTime - 20,
+ Assert.assertEquals(currentTime - 20,
tEntity.getEvents().get(3).getTimestamp());
handleEvent(jheh, new JobHistoryEvent(t.jobId,
new JobUnsuccessfulCompletionEvent(TypeConverter.fromYarn(t.jobId),
0, 0, 0, 0, 0, 0, 0, JobStateInternal.KILLED.toString()),
- currentTime + 20));
+ currentTime + 20));
jheh.getDispatcher().await();
entities = ts.getEntities("MAPREDUCE_JOB", null, null, null,
null, null, null, null, null, null);
- Assertions.assertEquals(1, entities.getEntities().size());
+ Assert.assertEquals(1, entities.getEntities().size());
tEntity = entities.getEntities().get(0);
- Assertions.assertEquals(t.jobId.toString(), tEntity.getEntityId());
- Assertions.assertEquals(5, tEntity.getEvents().size());
- Assertions.assertEquals(EventType.JOB_KILLED.toString(),
+ Assert.assertEquals(t.jobId.toString(), tEntity.getEntityId());
+ Assert.assertEquals(5, tEntity.getEvents().size());
+ Assert.assertEquals(EventType.JOB_KILLED.toString(),
tEntity.getEvents().get(0).getEventType());
- Assertions.assertEquals(EventType.JOB_SUBMITTED.toString(),
+ Assert.assertEquals(EventType.JOB_SUBMITTED.toString(),
tEntity.getEvents().get(1).getEventType());
- Assertions.assertEquals(EventType.JOB_FINISHED.toString(),
+ Assert.assertEquals(EventType.JOB_FINISHED.toString(),
tEntity.getEvents().get(2).getEventType());
- Assertions.assertEquals(EventType.AM_STARTED.toString(),
+ Assert.assertEquals(EventType.AM_STARTED.toString(),
tEntity.getEvents().get(3).getEventType());
- Assertions.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
+ Assert.assertEquals(EventType.JOB_QUEUE_CHANGED.toString(),
tEntity.getEvents().get(4).getEventType());
- Assertions.assertEquals(currentTime + 20,
+ Assert.assertEquals(currentTime + 20,
tEntity.getEvents().get(0).getTimestamp());
- Assertions.assertEquals(currentTime + 10,
+ Assert.assertEquals(currentTime + 10,
tEntity.getEvents().get(1).getTimestamp());
- Assertions.assertEquals(currentTime,
+ Assert.assertEquals(currentTime,
tEntity.getEvents().get(2).getTimestamp());
- Assertions.assertEquals(currentTime - 10,
+ Assert.assertEquals(currentTime - 10,
tEntity.getEvents().get(3).getTimestamp());
- Assertions.assertEquals(currentTime - 20,
+ Assert.assertEquals(currentTime - 20,
tEntity.getEvents().get(4).getTimestamp());
handleEvent(jheh, new JobHistoryEvent(t.jobId,
@@ -724,13 +715,13 @@ public class TestJobHistoryEventHandler {
jheh.getDispatcher().await();
entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
null, null, null, null, null, null);
- Assertions.assertEquals(1, entities.getEntities().size());
+ Assert.assertEquals(1, entities.getEntities().size());
tEntity = entities.getEntities().get(0);
- Assertions.assertEquals(t.taskID.toString(), tEntity.getEntityId());
- Assertions.assertEquals(1, tEntity.getEvents().size());
- Assertions.assertEquals(EventType.TASK_STARTED.toString(),
+ Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
+ Assert.assertEquals(1, tEntity.getEvents().size());
+ Assert.assertEquals(EventType.TASK_STARTED.toString(),
tEntity.getEvents().get(0).getEventType());
- Assertions.assertEquals(TaskType.MAP.toString(),
+ Assert.assertEquals(TaskType.MAP.toString(),
tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
handleEvent(jheh, new JobHistoryEvent(t.jobId,
@@ -738,31 +729,30 @@ public class TestJobHistoryEventHandler {
jheh.getDispatcher().await();
entities = ts.getEntities("MAPREDUCE_TASK", null, null, null,
null, null, null, null, null, null);
- Assertions.assertEquals(1, entities.getEntities().size());
+ Assert.assertEquals(1, entities.getEntities().size());
tEntity = entities.getEntities().get(0);
- Assertions.assertEquals(t.taskID.toString(), tEntity.getEntityId());
- Assertions.assertEquals(2, tEntity.getEvents().size());
- Assertions.assertEquals(EventType.TASK_STARTED.toString(),
+ Assert.assertEquals(t.taskID.toString(), tEntity.getEntityId());
+ Assert.assertEquals(2, tEntity.getEvents().size());
+ Assert.assertEquals(EventType.TASK_STARTED.toString(),
tEntity.getEvents().get(1).getEventType());
- Assertions.assertEquals(TaskType.REDUCE.toString(),
+ Assert.assertEquals(TaskType.REDUCE.toString(),
tEntity.getEvents().get(0).getEventInfo().get("TASK_TYPE"));
- Assertions.assertEquals(TaskType.MAP.toString(),
+ Assert.assertEquals(TaskType.MAP.toString(),
tEntity.getEvents().get(1).getEventInfo().get("TASK_TYPE"));
}
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testCountersToJSON() throws Exception {
JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0);
Counters counters = new Counters();
CounterGroup group1 = counters.addGroup("DOCTORS",
- "Incarnations of the Doctor");
+ "Incarnations of the Doctor");
group1.addCounter("PETER_CAPALDI", "Peter Capaldi", 12);
group1.addCounter("MATT_SMITH", "Matt Smith", 11);
group1.addCounter("DAVID_TENNANT", "David Tennant", 10);
CounterGroup group2 = counters.addGroup("COMPANIONS",
- "Companions of the Doctor");
+ "Companions of the Doctor");
group2.addCounter("CLARA_OSWALD", "Clara Oswald", 6);
group2.addCounter("RORY_WILLIAMS", "Rory Williams", 5);
group2.addCounter("AMY_POND", "Amy Pond", 4);
@@ -785,31 +775,30 @@ public class TestJobHistoryEventHandler {
+ "{\"NAME\":\"MATT_SMITH\",\"DISPLAY_NAME\":\"Matt Smith\",\"VALUE\":"
+ "11},{\"NAME\":\"PETER_CAPALDI\",\"DISPLAY_NAME\":\"Peter Capaldi\","
+ "\"VALUE\":12}]}]";
- Assertions.assertEquals(expected, jsonStr);
+ Assert.assertEquals(expected, jsonStr);
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testCountersToJSONEmpty() throws Exception {
JobHistoryEventHandler jheh = new JobHistoryEventHandler(null, 0);
Counters counters = null;
JsonNode jsonNode = JobHistoryEventUtils.countersToJSON(counters);
String jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
String expected = "[]";
- Assertions.assertEquals(expected, jsonStr);
+ Assert.assertEquals(expected, jsonStr);
counters = new Counters();
jsonNode = JobHistoryEventUtils.countersToJSON(counters);
jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
expected = "[]";
- Assertions.assertEquals(expected, jsonStr);
+ Assert.assertEquals(expected, jsonStr);
counters.addGroup("DOCTORS", "Incarnations of the Doctor");
jsonNode = JobHistoryEventUtils.countersToJSON(counters);
jsonStr = new ObjectMapper().writeValueAsString(jsonNode);
expected = "[{\"NAME\":\"DOCTORS\",\"DISPLAY_NAME\":\"Incarnations of the "
+ "Doctor\",\"COUNTERS\":[]}]";
- Assertions.assertEquals(expected, jsonStr);
+ Assert.assertEquals(expected, jsonStr);
}
private void queueEvent(JHEvenHandlerForTest jheh, JobHistoryEvent event) {
@@ -923,9 +912,8 @@ public class TestJobHistoryEventHandler {
}
jheh.stop();
//Make sure events were handled
- assertTrue(jheh.eventsHandled == 4,
- "handleEvent should've been called only 4 times but was "
- + jheh.eventsHandled);
+ assertTrue("handleEvent should've been called only 4 times but was "
+ + jheh.eventsHandled, jheh.eventsHandled == 4);
//Create a new jheh because the last stop closed the eventWriter etc.
jheh = new JHEventHandlerForSigtermTest(mockedContext, 0);
@@ -946,15 +934,14 @@ public class TestJobHistoryEventHandler {
}
jheh.stop();
//Make sure events were handled, 4 + 1 finish event
- assertTrue(jheh.eventsHandled == 5, "handleEvent should've been called only 5 times but was "
- + jheh.eventsHandled);
- assertTrue(jheh.lastEventHandled.getHistoryEvent()
- instanceof JobUnsuccessfulCompletionEvent,
- "Last event handled wasn't JobUnsuccessfulCompletionEvent");
+ assertTrue("handleEvent should've been called only 5 times but was "
+ + jheh.eventsHandled, jheh.eventsHandled == 5);
+ assertTrue("Last event handled wasn't JobUnsuccessfulCompletionEvent",
+ jheh.lastEventHandled.getHistoryEvent()
+ instanceof JobUnsuccessfulCompletionEvent);
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testSetTrackingURLAfterHistoryIsWritten() throws Exception {
TestParams t = new TestParams(true);
Configuration conf = new Configuration();
@@ -985,8 +972,7 @@ public class TestJobHistoryEventHandler {
}
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testDontSetTrackingURLIfHistoryWriteFailed() throws Exception {
TestParams t = new TestParams(true);
Configuration conf = new Configuration();
@@ -1017,8 +1003,7 @@ public class TestJobHistoryEventHandler {
jheh.stop();
}
}
- @Test
- @Timeout(50000)
+ @Test (timeout=50000)
public void testDontSetTrackingURLIfHistoryWriteThrows() throws Exception {
TestParams t = new TestParams(true);
Configuration conf = new Configuration();
@@ -1054,8 +1039,7 @@ public class TestJobHistoryEventHandler {
}
}
- @Test
- @Timeout(50000)
+ @Test(timeout = 50000)
public void testJobHistoryFilePermissions() throws Exception {
TestParams t = new TestParams(true);
Configuration conf = new Configuration();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java
index 41835d4f3b7..b81f716ebc7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobSummary.java
@@ -19,9 +19,9 @@
package org.apache.hadoop.mapreduce.jobhistory;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -34,7 +34,7 @@ public class TestJobSummary {
LoggerFactory.getLogger(TestJobSummary.class);
private JobSummary summary = new JobSummary();
- @BeforeEach
+ @Before
public void before() {
JobId mockJobId = mock(JobId.class);
when(mockJobId.toString()).thenReturn("testJobId");
@@ -64,8 +64,8 @@ public class TestJobSummary {
summary.setJobName("aa\rbb\ncc\r\ndd");
String out = summary.getJobSummaryString();
LOG.info("summary: " + out);
- Assertions.assertFalse(out.contains("\r"));
- Assertions.assertFalse(out.contains("\n"));
- Assertions.assertTrue(out.contains("aa\\rbb\\ncc\\r\\ndd"));
+ Assert.assertFalse(out.contains("\r"));
+ Assert.assertFalse(out.contains("\n"));
+ Assert.assertTrue(out.contains("aa\\rbb\\ncc\\r\\ndd"));
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java
index 4d4be84a74b..c8d81aea99b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskAttemptReport.java
@@ -24,12 +24,12 @@ import org.apache.hadoop.mapreduce.v2.app.MockJobs;
import org.apache.hadoop.mapreduce.v2.proto.MRProtos;
import org.apache.hadoop.yarn.util.Records;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
public class TestTaskAttemptReport {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java
index bc25ac4e9cd..a9b34eea7cf 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestTaskReport.java
@@ -24,12 +24,12 @@ import org.apache.hadoop.mapreduce.v2.app.MockJobs;
import org.apache.hadoop.mapreduce.v2.proto.MRProtos;
import org.apache.hadoop.yarn.util.Records;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
public class TestTaskReport {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
index 39cf27ae441..4be80c44a3e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
@@ -98,7 +98,7 @@ import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -326,8 +326,8 @@ public class MRApp extends MRAppMaster {
iState = job.getInternalState();
}
LOG.info("Job {} Internal State is : {}", job.getID(), iState);
- Assertions.assertEquals(finalState, iState,
- "Task Internal state is not correct (timedout)");
+ Assert.assertEquals("Task Internal state is not correct (timedout)",
+ finalState, iState);
}
public void waitForInternalState(TaskImpl task,
@@ -339,8 +339,8 @@ public class MRApp extends MRAppMaster {
iState = task.getInternalState();
}
LOG.info("Task {} Internal State is : {}", task.getID(), iState);
- Assertions.assertEquals(finalState, iState,
- "Task Internal state is not correct (timedout)");
+ Assert.assertEquals("Task Internal state is not correct (timedout)",
+ finalState, iState);
}
public void waitForInternalState(TaskAttemptImpl attempt,
@@ -352,8 +352,8 @@ public class MRApp extends MRAppMaster {
iState = attempt.getInternalState();
}
LOG.info("TaskAttempt {} Internal State is : {}", attempt.getID(), iState);
- Assertions.assertEquals(finalState, iState,
- "TaskAttempt Internal state is not correct (timedout)");
+ Assert.assertEquals("TaskAttempt Internal state is not correct (timedout)",
+ finalState, iState);
}
public void waitForState(TaskAttempt attempt,
@@ -367,8 +367,9 @@ public class MRApp extends MRAppMaster {
}
LOG.info("TaskAttempt {} State is : {}", attempt.getID(),
report.getTaskAttemptState());
- Assertions.assertEquals(finalState, report.getTaskAttemptState(),
- "TaskAttempt state is not correct (timedout)");
+ Assert.assertEquals("TaskAttempt state is not correct (timedout)",
+ finalState,
+ report.getTaskAttemptState());
}
public void waitForState(Task task, TaskState finalState) throws Exception {
@@ -380,8 +381,8 @@ public class MRApp extends MRAppMaster {
report = task.getReport();
}
LOG.info("Task {} State is : {}", task.getID(), report.getTaskState());
- Assertions.assertEquals(finalState, report.getTaskState(),
- "Task state is not correct (timedout)");
+ Assert.assertEquals("Task state is not correct (timedout)", finalState,
+ report.getTaskState());
}
public void waitForState(Job job, JobState finalState) throws Exception {
@@ -393,14 +394,14 @@ public class MRApp extends MRAppMaster {
Thread.sleep(WAIT_FOR_STATE_INTERVAL);
}
LOG.info("Job {} State is : {}", job.getID(), report.getJobState());
- Assertions.assertEquals(finalState, job.getState(),
- "Job state is not correct (timedout)");
+ Assert.assertEquals("Job state is not correct (timedout)", finalState,
+ job.getState());
}
public void waitForState(Service.STATE finalState) throws Exception {
if (finalState == Service.STATE.STOPPED) {
- Assertions.assertTrue(waitForServiceToStop(20 * 1000),
- "Timeout while waiting for MRApp to stop");
+ Assert.assertTrue("Timeout while waiting for MRApp to stop",
+ waitForServiceToStop(20 * 1000));
} else {
int timeoutSecs = 0;
while (!finalState.equals(getServiceState())
@@ -408,8 +409,8 @@ public class MRApp extends MRAppMaster {
Thread.sleep(WAIT_FOR_STATE_INTERVAL);
}
LOG.info("MRApp State is : {}", getServiceState());
- Assertions.assertEquals(finalState, getServiceState(),
- "MRApp state is not correct (timedout)");
+ Assert.assertEquals("MRApp state is not correct (timedout)", finalState,
+ getServiceState());
}
}
@@ -418,23 +419,22 @@ public class MRApp extends MRAppMaster {
JobReport jobReport = job.getReport();
LOG.info("Job start time :{}", jobReport.getStartTime());
LOG.info("Job finish time :", jobReport.getFinishTime());
- Assertions.assertTrue(jobReport.getStartTime() <= jobReport.getFinishTime(),
- "Job start time is not less than finish time");
- Assertions.assertTrue(jobReport.getFinishTime() <= System.currentTimeMillis(),
- "Job finish time is in future");
+ Assert.assertTrue("Job start time is not less than finish time",
+ jobReport.getStartTime() <= jobReport.getFinishTime());
+ Assert.assertTrue("Job finish time is in future",
+ jobReport.getFinishTime() <= System.currentTimeMillis());
for (Task task : job.getTasks().values()) {
TaskReport taskReport = task.getReport();
LOG.info("Task {} start time : {}", task.getID(),
taskReport.getStartTime());
LOG.info("Task {} finish time : {}", task.getID(),
taskReport.getFinishTime());
- Assertions.assertTrue(taskReport.getStartTime() <= taskReport.getFinishTime(),
- "Task start time is not less than finish time");
+ Assert.assertTrue("Task start time is not less than finish time",
+ taskReport.getStartTime() <= taskReport.getFinishTime());
for (TaskAttempt attempt : task.getAttempts().values()) {
TaskAttemptReport attemptReport = attempt.getReport();
- Assertions.assertTrue(attemptReport.getStartTime() <=
- attemptReport.getFinishTime(),
- "Attempt start time is not less than finish time");
+ Assert.assertTrue("Attempt start time is not less than finish time",
+ attemptReport.getStartTime() <= attemptReport.getFinishTime());
}
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
index 20e1a836f04..efe150fad19 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
@@ -56,8 +56,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.Records;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Test;
import org.slf4j.event.Level;
public class MRAppBenchmark {
@@ -197,8 +196,7 @@ public class MRAppBenchmark {
}
}
- @Test
- @Timeout(60000)
+ @Test(timeout = 60000)
public void benchmark1() throws Exception {
int maps = 100; // Adjust for benchmarking. Start with thousands.
int reduces = 0;
@@ -277,8 +275,7 @@ public class MRAppBenchmark {
});
}
- @Test
- @Timeout(60000)
+ @Test(timeout = 60000)
public void benchmark2() throws Exception {
int maps = 100; // Adjust for benchmarking, start with a couple of thousands
int reduces = 50;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java
index 085013b774a..4b9015f10c5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestAMInfos.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app;
import java.util.Iterator;
import java.util.List;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -33,7 +33,7 @@ import org.apache.hadoop.mapreduce.v2.app.TestRecovery.MRAppWithHistory;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
public class TestAMInfos {
@@ -50,7 +50,7 @@ public class TestAMInfos {
long am1StartTime = app.getAllAMInfos().get(0).getStartTime();
- Assertions.assertEquals(1, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 1, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask = it.next();
app.waitForState(mapTask, TaskState.RUNNING);
@@ -71,14 +71,14 @@ public class TestAMInfos {
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(1, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 1, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask = it.next();
// There should be two AMInfos
List amInfos = app.getAllAMInfos();
- Assertions.assertEquals(2, amInfos.size());
+ Assert.assertEquals(2, amInfos.size());
AMInfo amInfoOne = amInfos.get(0);
- Assertions.assertEquals(am1StartTime, amInfoOne.getStartTime());
+ Assert.assertEquals(am1StartTime, amInfoOne.getStartTime());
app.stop();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java
index fbe8cb18248..59778161f20 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.yarn.api.records.PreemptionMessage;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.util.resource.Resources;
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
import java.util.ArrayList;
@@ -58,8 +58,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
public class TestCheckpointPreemptionPolicy {
@@ -77,7 +77,7 @@ public class TestCheckpointPreemptionPolicy {
private int minAlloc = 1024;
- @BeforeEach
+ @Before
@SuppressWarnings("rawtypes") // mocked generics
public void setup() {
ApplicationId appId = ApplicationId.newInstance(200, 1);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
index 170e39f53eb..3b5cfe221ed 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
@@ -24,7 +24,7 @@ import java.util.Iterator;
import java.util.Map;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
@@ -48,7 +48,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
/**
* Tests the state machine with respect to Job/Task/TaskAttempt failure
@@ -68,20 +68,20 @@ public class TestFail {
Job job = app.submit(conf);
app.waitForState(job, JobState.SUCCEEDED);
Map tasks = job.getTasks();
- Assertions.assertEquals(1, tasks.size(), "Num tasks is not correct");
+ Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
Task task = tasks.values().iterator().next();
- Assertions.assertEquals(TaskState.SUCCEEDED, task.getReport().getTaskState(),
- "Task state not correct");
+ Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED,
+ task.getReport().getTaskState());
Map attempts =
tasks.values().iterator().next().getAttempts();
- Assertions.assertEquals(2, attempts.size(), "Num attempts is not correct");
+ Assert.assertEquals("Num attempts is not correct", 2, attempts.size());
//one attempt must be failed
//and another must have succeeded
Iterator it = attempts.values().iterator();
- Assertions.assertEquals(TaskAttemptState.FAILED,
- it.next().getReport().getTaskAttemptState(), "Attempt state not correct");
- Assertions.assertEquals(TaskAttemptState.SUCCEEDED,
- it.next().getReport().getTaskAttemptState(), "Attempt state not correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
+ it.next().getReport().getTaskAttemptState());
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED,
+ it.next().getReport().getTaskAttemptState());
}
@Test
@@ -159,17 +159,17 @@ public class TestFail {
Job job = app.submit(conf);
app.waitForState(job, JobState.FAILED);
Map tasks = job.getTasks();
- Assertions.assertEquals(1, tasks.size(), "Num tasks is not correct");
+ Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
Task task = tasks.values().iterator().next();
- Assertions.assertEquals(TaskState.FAILED,
- task.getReport().getTaskState(), "Task state not correct");
+ Assert.assertEquals("Task state not correct", TaskState.FAILED,
+ task.getReport().getTaskState());
Map attempts =
tasks.values().iterator().next().getAttempts();
- Assertions.assertEquals(maxAttempts,
- attempts.size(), "Num attempts is not correct");
+ Assert.assertEquals("Num attempts is not correct", maxAttempts,
+ attempts.size());
for (TaskAttempt attempt : attempts.values()) {
- Assertions.assertEquals(TaskAttemptState.FAILED,
- attempt.getReport().getTaskAttemptState(), "Attempt state not correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
+ attempt.getReport().getTaskAttemptState());
}
}
@@ -185,14 +185,13 @@ public class TestFail {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
Map tasks = job.getTasks();
- Assertions.assertEquals(1, tasks.size(),
- "Num tasks is not correct");
+ Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
Task task = tasks.values().iterator().next();
app.waitForState(task, TaskState.SCHEDULED);
Map attempts = tasks.values().iterator()
.next().getAttempts();
- Assertions.assertEquals(maxAttempts, attempts.size(),
- "Num attempts is not correct");
+ Assert.assertEquals("Num attempts is not correct", maxAttempts, attempts
+ .size());
TaskAttempt attempt = attempts.values().iterator().next();
app.waitForInternalState((TaskAttemptImpl) attempt,
TaskAttemptStateInternal.ASSIGNED);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
index 4fe2237bcf7..d2bd0104fff 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFetchFailure.java
@@ -19,7 +19,7 @@
package org.apache.hadoop.mapreduce.v2.app;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
import java.util.Arrays;
@@ -50,8 +50,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.event.EventHandler;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Test;
public class TestFetchFailure {
@@ -65,8 +65,8 @@ public class TestFetchFailure {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(2, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct",
+ 2, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask = it.next();
Task reduceTask = it.next();
@@ -97,10 +97,10 @@ public class TestFetchFailure {
TaskAttemptCompletionEvent[] events =
job.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(1, events.length,
- "Num completion events not correct");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED,
- events[0].getStatus(), "Event status not correct");
+ Assert.assertEquals("Num completion events not correct",
+ 1, events.length);
+ Assert.assertEquals("Event status not correct",
+ TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus());
// wait for reduce to start running
app.waitForState(reduceTask, TaskState.RUNNING);
@@ -117,11 +117,11 @@ public class TestFetchFailure {
app.waitForState(mapTask, TaskState.RUNNING);
//map attempt must have become FAILED
- Assertions.assertEquals(TaskAttemptState.FAILED, mapAttempt1.getState(),
- "Map TaskAttempt state not correct");
+ Assert.assertEquals("Map TaskAttempt state not correct",
+ TaskAttemptState.FAILED, mapAttempt1.getState());
- Assertions.assertEquals(2, mapTask.getAttempts().size(),
- "Num attempts in Map Task not correct");
+ Assert.assertEquals("Num attempts in Map Task not correct",
+ 2, mapTask.getAttempts().size());
Iterator atIt = mapTask.getAttempts().values().iterator();
atIt.next();
@@ -144,41 +144,39 @@ public class TestFetchFailure {
app.waitForState(job, JobState.SUCCEEDED);
//previous completion event now becomes obsolete
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE,
- events[0].getStatus(), "Event status not correct");
+ Assert.assertEquals("Event status not correct",
+ TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
events = job.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(4, events.length,
- "Num completion events not correct");
- Assertions.assertEquals(mapAttempt1.getID(), events[0].getAttemptId(),
- "Event map attempt id not correct");
- Assertions.assertEquals(mapAttempt1.getID(), events[1].getAttemptId(),
- "Event map attempt id not correct");
- Assertions.assertEquals(mapAttempt2.getID(), events[2].getAttemptId(),
- "Event map attempt id not correct");
- Assertions.assertEquals(reduceAttempt.getID(), events[3].getAttemptId(),
- "Event redude attempt id not correct");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE,
- events[0].getStatus(), "Event status not correct for map attempt1");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.FAILED,
- events[1].getStatus(), "Event status not correct for map attempt1");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED,
- events[2].getStatus(), "Event status not correct for map attempt2");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED,
- events[3].getStatus(), "Event status not correct for reduce attempt1");
+ Assert.assertEquals("Num completion events not correct",
+ 4, events.length);
+ Assert.assertEquals("Event map attempt id not correct",
+ mapAttempt1.getID(), events[0].getAttemptId());
+ Assert.assertEquals("Event map attempt id not correct",
+ mapAttempt1.getID(), events[1].getAttemptId());
+ Assert.assertEquals("Event map attempt id not correct",
+ mapAttempt2.getID(), events[2].getAttemptId());
+ Assert.assertEquals("Event redude attempt id not correct",
+ reduceAttempt.getID(), events[3].getAttemptId());
+ Assert.assertEquals("Event status not correct for map attempt1",
+ TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
+ Assert.assertEquals("Event status not correct for map attempt1",
+ TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus());
+ Assert.assertEquals("Event status not correct for map attempt2",
+ TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus());
+ Assert.assertEquals("Event status not correct for reduce attempt1",
+ TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus());
TaskCompletionEvent mapEvents[] =
job.getMapAttemptCompletionEvents(0, 2);
TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events);
- Assertions.assertEquals(2, mapEvents.length,
- "Incorrect number of map events");
- Assertions.assertArrayEquals(Arrays.copyOfRange(convertedEvents, 0, 2),
- mapEvents, "Unexpected map events");
+ Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length);
+ Assert.assertArrayEquals("Unexpected map events",
+ Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents);
mapEvents = job.getMapAttemptCompletionEvents(2, 200);
- Assertions.assertEquals(1, mapEvents.length,
- "Incorrect number of map events");
- Assertions.assertEquals(convertedEvents[2], mapEvents[0],
- "Unexpected map event");
+ Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length);
+ Assert.assertEquals("Unexpected map event", convertedEvents[2],
+ mapEvents[0]);
}
/**
@@ -199,8 +197,8 @@ public class TestFetchFailure {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(2, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct",
+ 2, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask = it.next();
Task reduceTask = it.next();
@@ -220,10 +218,10 @@ public class TestFetchFailure {
TaskAttemptCompletionEvent[] events =
job.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(1, events.length,
- "Num completion events not correct");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED,
- events[0].getStatus(), "Event status not correct");
+ Assert.assertEquals("Num completion events not correct",
+ 1, events.length);
+ Assert.assertEquals("Event status not correct",
+ TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus());
// wait for reduce to start running
app.waitForState(reduceTask, TaskState.RUNNING);
@@ -252,8 +250,8 @@ public class TestFetchFailure {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(2, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct",
+ 2, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask = it.next();
reduceTask = it.next();
@@ -279,8 +277,7 @@ public class TestFetchFailure {
app.waitForState(job, JobState.SUCCEEDED);
events = job.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(2, events.length,
- "Num completion events not correct");
+ Assert.assertEquals("Num completion events not correct", 2, events.length);
}
@Test
@@ -293,8 +290,8 @@ public class TestFetchFailure {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(4, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct",
+ 4, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask = it.next();
Task reduceTask = it.next();
@@ -316,10 +313,10 @@ public class TestFetchFailure {
TaskAttemptCompletionEvent[] events =
job.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(1, events.length,
- "Num completion events not correct");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus(),
- "Event status not correct");
+ Assert.assertEquals("Num completion events not correct",
+ 1, events.length);
+ Assert.assertEquals("Event status not correct",
+ TaskAttemptCompletionEventStatus.SUCCEEDED, events[0].getStatus());
// wait for reduce to start running
app.waitForState(reduceTask, TaskState.RUNNING);
@@ -357,16 +354,16 @@ public class TestFetchFailure {
app.waitForState(mapTask, TaskState.RUNNING);
//map attempt must have become FAILED
- Assertions.assertEquals(TaskAttemptState.FAILED, mapAttempt1.getState(),
- "Map TaskAttempt state not correct");
+ Assert.assertEquals("Map TaskAttempt state not correct",
+ TaskAttemptState.FAILED, mapAttempt1.getState());
assertThat(mapAttempt1.getDiagnostics().get(0))
.isEqualTo("Too many fetch failures. Failing the attempt. "
+ "Last failure reported by "
+ reduceAttempt3.getID().toString() + " from host host3");
- Assertions.assertEquals(2, mapTask.getAttempts().size(),
- "Num attempts in Map Task not correct");
+ Assert.assertEquals("Num attempts in Map Task not correct",
+ 2, mapTask.getAttempts().size());
Iterator atIt = mapTask.getAttempts().values().iterator();
atIt.next();
@@ -399,40 +396,39 @@ public class TestFetchFailure {
app.waitForState(job, JobState.SUCCEEDED);
//previous completion event now becomes obsolete
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus(),
- "Event status not correct");
+ Assert.assertEquals("Event status not correct",
+ TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
events = job.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(6, events.length,
- "Num completion events not correct");
- Assertions.assertEquals(mapAttempt1.getID(), events[0].getAttemptId(),
- "Event map attempt id not correct");
- Assertions.assertEquals(mapAttempt1.getID(), events[1].getAttemptId(),
- "Event map attempt id not correct");
- Assertions.assertEquals(mapAttempt2.getID(), events[2].getAttemptId(),
- "Event map attempt id not correct");
- Assertions.assertEquals(reduceAttempt.getID(), events[3].getAttemptId(),
- "Event reduce attempt id not correct");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.OBSOLETE,
- events[0].getStatus(), "Event status not correct for map attempt1");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.FAILED,
- events[1].getStatus(), "Event status not correct for map attempt1");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED,
- events[2].getStatus(), "Event status not correct for map attempt2");
- Assertions.assertEquals(TaskAttemptCompletionEventStatus.SUCCEEDED,
- events[3].getStatus(), "Event status not correct for reduce attempt1");
+ Assert.assertEquals("Num completion events not correct",
+ 6, events.length);
+ Assert.assertEquals("Event map attempt id not correct",
+ mapAttempt1.getID(), events[0].getAttemptId());
+ Assert.assertEquals("Event map attempt id not correct",
+ mapAttempt1.getID(), events[1].getAttemptId());
+ Assert.assertEquals("Event map attempt id not correct",
+ mapAttempt2.getID(), events[2].getAttemptId());
+ Assert.assertEquals("Event reduce attempt id not correct",
+ reduceAttempt.getID(), events[3].getAttemptId());
+ Assert.assertEquals("Event status not correct for map attempt1",
+ TaskAttemptCompletionEventStatus.OBSOLETE, events[0].getStatus());
+ Assert.assertEquals("Event status not correct for map attempt1",
+ TaskAttemptCompletionEventStatus.FAILED, events[1].getStatus());
+ Assert.assertEquals("Event status not correct for map attempt2",
+ TaskAttemptCompletionEventStatus.SUCCEEDED, events[2].getStatus());
+ Assert.assertEquals("Event status not correct for reduce attempt1",
+ TaskAttemptCompletionEventStatus.SUCCEEDED, events[3].getStatus());
TaskCompletionEvent mapEvents[] =
job.getMapAttemptCompletionEvents(0, 2);
TaskCompletionEvent convertedEvents[] = TypeConverter.fromYarn(events);
- Assertions.assertEquals(2, mapEvents.length,
- "Incorrect number of map events");
- Assertions.assertArrayEquals(Arrays.copyOfRange(convertedEvents, 0, 2),
- mapEvents, "Unexpected map events");
+ Assert.assertEquals("Incorrect number of map events", 2, mapEvents.length);
+ Assert.assertArrayEquals("Unexpected map events",
+ Arrays.copyOfRange(convertedEvents, 0, 2), mapEvents);
mapEvents = job.getMapAttemptCompletionEvents(2, 200);
- Assertions.assertEquals(1, mapEvents.length, "Incorrect number of map events");
- Assertions.assertEquals(convertedEvents[2], mapEvents[0],
- "Unexpected map event");
+ Assert.assertEquals("Incorrect number of map events", 1, mapEvents.length);
+ Assert.assertEquals("Unexpected map event", convertedEvents[2],
+ mapEvents[0]);
}
private void updateStatus(MRApp app, TaskAttempt attempt, Phase phase) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
index e7fe432d45b..1cd625551a6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestJobEndNotifier.java
@@ -59,8 +59,8 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Test;
/**
* Tests job end notification
@@ -74,18 +74,18 @@ public class TestJobEndNotifier extends JobEndNotifier {
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "0");
conf.set(MRJobConfig.MR_JOB_END_RETRY_ATTEMPTS, "10");
setConf(conf);
- Assertions.assertTrue(numTries == 0,
- "Expected numTries to be 0, but was " + numTries);
+ Assert.assertTrue("Expected numTries to be 0, but was " + numTries,
+ numTries == 0 );
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "1");
setConf(conf);
- Assertions.assertTrue(numTries == 1,
- "Expected numTries to be 1, but was " + numTries);
+ Assert.assertTrue("Expected numTries to be 1, but was " + numTries,
+ numTries == 1 );
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "20");
setConf(conf);
- Assertions.assertTrue(numTries == 11 , "Expected numTries to be 11, but was "
- + numTries); //11 because number of _retries_ is 10
+ Assert.assertTrue("Expected numTries to be 11, but was " + numTries,
+ numTries == 11 ); //11 because number of _retries_ is 10
}
//Test maximum retry interval is capped by
@@ -94,53 +94,53 @@ public class TestJobEndNotifier extends JobEndNotifier {
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_RETRY_INTERVAL, "5000");
conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "1000");
setConf(conf);
- Assertions.assertTrue(waitInterval == 1000,
- "Expected waitInterval to be 1000, but was " + waitInterval);
+ Assert.assertTrue("Expected waitInterval to be 1000, but was "
+ + waitInterval, waitInterval == 1000);
conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "10000");
setConf(conf);
- Assertions.assertTrue(waitInterval == 5000,
- "Expected waitInterval to be 5000, but was " + waitInterval);
+ Assert.assertTrue("Expected waitInterval to be 5000, but was "
+ + waitInterval, waitInterval == 5000);
//Test negative numbers are set to default
conf.set(MRJobConfig.MR_JOB_END_RETRY_INTERVAL, "-10");
setConf(conf);
- Assertions.assertTrue(waitInterval == 5000,
- "Expected waitInterval to be 5000, but was " + waitInterval);
+ Assert.assertTrue("Expected waitInterval to be 5000, but was "
+ + waitInterval, waitInterval == 5000);
}
private void testTimeout(Configuration conf) {
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_TIMEOUT, "1000");
setConf(conf);
- Assertions.assertTrue(timeout == 1000,
- "Expected timeout to be 1000, but was " + timeout);
+ Assert.assertTrue("Expected timeout to be 1000, but was "
+ + timeout, timeout == 1000);
}
private void testProxyConfiguration(Configuration conf) {
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost");
setConf(conf);
- Assertions.assertTrue(proxyToUse.type() == Proxy.Type.DIRECT,
- "Proxy shouldn't be set because port wasn't specified");
+ Assert.assertTrue("Proxy shouldn't be set because port wasn't specified",
+ proxyToUse.type() == Proxy.Type.DIRECT);
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost:someport");
setConf(conf);
- Assertions.assertTrue(proxyToUse.type() == Proxy.Type.DIRECT,
- "Proxy shouldn't be set because port wasn't numeric");
+ Assert.assertTrue("Proxy shouldn't be set because port wasn't numeric",
+ proxyToUse.type() == Proxy.Type.DIRECT);
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "somehost:1000");
setConf(conf);
- Assertions.assertEquals("HTTP @ somehost:1000", proxyToUse.toString(),
- "Proxy should have been set but wasn't ");
+ Assert.assertEquals("Proxy should have been set but wasn't ",
+ "HTTP @ somehost:1000", proxyToUse.toString());
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "socks@somehost:1000");
setConf(conf);
- Assertions.assertEquals("SOCKS @ somehost:1000", proxyToUse.toString(),
- "Proxy should have been socks but wasn't ");
+ Assert.assertEquals("Proxy should have been socks but wasn't ",
+ "SOCKS @ somehost:1000", proxyToUse.toString());
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "SOCKS@somehost:1000");
setConf(conf);
- Assertions.assertEquals("SOCKS @ somehost:1000", proxyToUse.toString(),
- "Proxy should have been socks but wasn't ");
+ Assert.assertEquals("Proxy should have been socks but wasn't ",
+ "SOCKS @ somehost:1000", proxyToUse.toString());
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_PROXY, "sfafn@somehost:1000");
setConf(conf);
- Assertions.assertEquals("HTTP @ somehost:1000", proxyToUse.toString(),
- "Proxy should have been http but wasn't ");
+ Assert.assertEquals("Proxy should have been http but wasn't ",
+ "HTTP @ somehost:1000", proxyToUse.toString());
}
@@ -181,10 +181,10 @@ public class TestJobEndNotifier extends JobEndNotifier {
this.setConf(conf);
this.notify(jobReport);
long endTime = System.currentTimeMillis();
- Assertions.assertEquals(1, this.notificationCount,
- "Only 1 try was expected but was : " + this.notificationCount);
- Assertions.assertTrue(endTime - startTime > 5000,
- "Should have taken more than 5 seconds it took " + (endTime - startTime));
+ Assert.assertEquals("Only 1 try was expected but was : "
+ + this.notificationCount, 1, this.notificationCount);
+ Assert.assertTrue("Should have taken more than 5 seconds it took "
+ + (endTime - startTime), endTime - startTime > 5000);
conf.set(MRJobConfig.MR_JOB_END_NOTIFICATION_MAX_ATTEMPTS, "3");
conf.set(MRJobConfig.MR_JOB_END_RETRY_ATTEMPTS, "3");
@@ -196,10 +196,10 @@ public class TestJobEndNotifier extends JobEndNotifier {
this.setConf(conf);
this.notify(jobReport);
endTime = System.currentTimeMillis();
- Assertions.assertEquals(3, this.notificationCount,
- "Only 3 retries were expected but was : " + this.notificationCount);
- Assertions.assertTrue(endTime - startTime > 9000,
- "Should have taken more than 9 seconds it took " + (endTime - startTime));
+ Assert.assertEquals("Only 3 retries were expected but was : "
+ + this.notificationCount, 3, this.notificationCount);
+ Assert.assertTrue("Should have taken more than 9 seconds it took "
+ + (endTime - startTime), endTime - startTime > 9000);
}
@@ -222,11 +222,11 @@ public class TestJobEndNotifier extends JobEndNotifier {
doThrow(runtimeException).when(app).stop();
}
app.shutDownJob();
- Assertions.assertTrue(app.isLastAMRetry());
- Assertions.assertEquals(1, JobEndServlet.calledTimes);
- Assertions.assertEquals("jobid=" + job.getID() + "&status=SUCCEEDED",
+ Assert.assertTrue(app.isLastAMRetry());
+ Assert.assertEquals(1, JobEndServlet.calledTimes);
+ Assert.assertEquals("jobid=" + job.getID() + "&status=SUCCEEDED",
JobEndServlet.requestUri.getQuery());
- Assertions.assertEquals(JobState.SUCCEEDED.toString(),
+ Assert.assertEquals(JobState.SUCCEEDED.toString(),
JobEndServlet.foundJobState);
server.stop();
}
@@ -262,10 +262,10 @@ public class TestJobEndNotifier extends JobEndNotifier {
app.shutDownJob();
// Not the last AM attempt. So user should that the job is still running.
app.waitForState(job, JobState.RUNNING);
- Assertions.assertFalse(app.isLastAMRetry());
- Assertions.assertEquals(0, JobEndServlet.calledTimes);
- Assertions.assertNull(JobEndServlet.requestUri);
- Assertions.assertNull(JobEndServlet.foundJobState);
+ Assert.assertFalse(app.isLastAMRetry());
+ Assert.assertEquals(0, JobEndServlet.calledTimes);
+ Assert.assertNull(JobEndServlet.requestUri);
+ Assert.assertNull(JobEndServlet.foundJobState);
server.stop();
}
@@ -294,11 +294,11 @@ public class TestJobEndNotifier extends JobEndNotifier {
// Unregistration fails: isLastAMRetry is recalculated, this is
///reboot will stop service internally, we don't need to shutdown twice
app.waitForServiceToStop(10000);
- Assertions.assertFalse(app.isLastAMRetry());
+ Assert.assertFalse(app.isLastAMRetry());
// Since it's not last retry, JobEndServlet didn't called
- Assertions.assertEquals(0, JobEndServlet.calledTimes);
- Assertions.assertNull(JobEndServlet.requestUri);
- Assertions.assertNull(JobEndServlet.foundJobState);
+ Assert.assertEquals(0, JobEndServlet.calledTimes);
+ Assert.assertNull(JobEndServlet.requestUri);
+ Assert.assertNull(JobEndServlet.foundJobState);
server.stop();
}
@@ -321,7 +321,7 @@ public class TestJobEndNotifier extends JobEndNotifier {
this.notify(jobReport);
final URL urlToNotify = CustomNotifier.urlToNotify;
- Assertions.assertEquals("http://example.com?jobId=mock-Id&jobStatus=SUCCEEDED",
+ Assert.assertEquals("http://example.com?jobId=mock-Id&jobStatus=SUCCEEDED",
urlToNotify.toString());
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
index 63dc2f88067..f681cf81650 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKill.java
@@ -23,7 +23,7 @@ import java.util.Map;
import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.service.Service;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -48,7 +48,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.Event;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
/**
* Tests the state machine with respect to Job/Task/TaskAttempt kill scenarios.
@@ -83,17 +83,18 @@ public class TestKill {
app.waitForState(Service.STATE.STOPPED);
Map tasks = job.getTasks();
- Assertions.assertEquals(1, tasks.size(),
- "No of tasks is not correct");
+ Assert.assertEquals("No of tasks is not correct", 1,
+ tasks.size());
Task task = tasks.values().iterator().next();
- Assertions.assertEquals(TaskState.KILLED,
- task.getReport().getTaskState(), "Task state not correct");
+ Assert.assertEquals("Task state not correct", TaskState.KILLED,
+ task.getReport().getTaskState());
Map attempts =
tasks.values().iterator().next().getAttempts();
- Assertions.assertEquals(1, attempts.size(), "No of attempts is not correct");
+ Assert.assertEquals("No of attempts is not correct", 1,
+ attempts.size());
Iterator it = attempts.values().iterator();
- Assertions.assertEquals(TaskAttemptState.KILLED,
- it.next().getReport().getTaskAttemptState(), "Attempt state not correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED,
+ it.next().getReport().getTaskAttemptState());
}
@Test
@@ -106,8 +107,8 @@ public class TestKill {
//wait and vailidate for Job to become RUNNING
app.waitForInternalState((JobImpl) job, JobStateInternal.RUNNING);
Map tasks = job.getTasks();
- Assertions.assertEquals(2, tasks.size(),
- "No of tasks is not correct");
+ Assert.assertEquals("No of tasks is not correct", 2,
+ tasks.size());
Iterator it = tasks.values().iterator();
Task task1 = it.next();
Task task2 = it.next();
@@ -124,24 +125,24 @@ public class TestKill {
//first Task is killed and second is Succeeded
//Job is succeeded
-
- Assertions.assertEquals(TaskState.KILLED, task1.getReport().getTaskState(),
- "Task state not correct");
- Assertions.assertEquals(TaskState.SUCCEEDED, task2.getReport().getTaskState(),
- "Task state not correct");
+
+ Assert.assertEquals("Task state not correct", TaskState.KILLED,
+ task1.getReport().getTaskState());
+ Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED,
+ task2.getReport().getTaskState());
Map attempts = task1.getAttempts();
- Assertions.assertEquals(1, attempts.size(),
- "No of attempts is not correct");
+ Assert.assertEquals("No of attempts is not correct", 1,
+ attempts.size());
Iterator iter = attempts.values().iterator();
- Assertions.assertEquals(TaskAttemptState.KILLED,
- iter.next().getReport().getTaskAttemptState(), "Attempt state not correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED,
+ iter.next().getReport().getTaskAttemptState());
attempts = task2.getAttempts();
- Assertions.assertEquals(1, attempts.size(),
- "No of attempts is not correct");
+ Assert.assertEquals("No of attempts is not correct", 1,
+ attempts.size());
iter = attempts.values().iterator();
- Assertions.assertEquals(TaskAttemptState.SUCCEEDED,
- iter.next().getReport().getTaskAttemptState(), "Attempt state not correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED,
+ iter.next().getReport().getTaskAttemptState());
}
@Test
@@ -193,8 +194,7 @@ public class TestKill {
Job job = app.submit(new Configuration());
JobId jobId = app.getJobId();
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(2, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask = it.next();
Task reduceTask = it.next();
@@ -232,8 +232,7 @@ public class TestKill {
Job job = app.submit(new Configuration());
JobId jobId = app.getJobId();
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(2, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask = it.next();
Task reduceTask = it.next();
@@ -281,8 +280,7 @@ public class TestKill {
Job job = app.submit(new Configuration());
JobId jobId = app.getJobId();
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(2, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 2, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask = it.next();
Task reduceTask = it.next();
@@ -372,8 +370,8 @@ public class TestKill {
//wait and vailidate for Job to become RUNNING
app.waitForState(job, JobState.RUNNING);
Map tasks = job.getTasks();
- Assertions.assertEquals(2, tasks.size(),
- "No of tasks is not correct");
+ Assert.assertEquals("No of tasks is not correct", 2,
+ tasks.size());
Iterator it = tasks.values().iterator();
Task task1 = it.next();
Task task2 = it.next();
@@ -396,26 +394,26 @@ public class TestKill {
//first Task will have two attempts 1st is killed, 2nd Succeeds
//both Tasks and Job succeeds
- Assertions.assertEquals(TaskState.SUCCEEDED,
- task1.getReport().getTaskState(), "Task state not correct");
- Assertions.assertEquals(TaskState.SUCCEEDED,
- task2.getReport().getTaskState(), "Task state not correct");
+ Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED,
+ task1.getReport().getTaskState());
+ Assert.assertEquals("Task state not correct", TaskState.SUCCEEDED,
+ task2.getReport().getTaskState());
Map attempts = task1.getAttempts();
- Assertions.assertEquals(2, attempts.size(),
- "No of attempts is not correct");
+ Assert.assertEquals("No of attempts is not correct", 2,
+ attempts.size());
Iterator iter = attempts.values().iterator();
- Assertions.assertEquals(TaskAttemptState.KILLED,
- iter.next().getReport().getTaskAttemptState(), "Attempt state not correct");
- Assertions.assertEquals(TaskAttemptState.SUCCEEDED,
- iter.next().getReport().getTaskAttemptState(), "Attempt state not correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.KILLED,
+ iter.next().getReport().getTaskAttemptState());
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED,
+ iter.next().getReport().getTaskAttemptState());
attempts = task2.getAttempts();
- Assertions.assertEquals(1, attempts.size(),
- "No of attempts is not correct");
+ Assert.assertEquals("No of attempts is not correct", 1,
+ attempts.size());
iter = attempts.values().iterator();
- Assertions.assertEquals(TaskAttemptState.SUCCEEDED,
- iter.next().getReport().getTaskAttemptState(), "Attempt state not correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.SUCCEEDED,
+ iter.next().getReport().getTaskAttemptState());
}
static class BlockingMRApp extends MRApp {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java
index 62e016a734b..3c3c4c90625 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestKillAMPreemptionPolicy.java
@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
public class TestKillAMPreemptionPolicy {
private final RecordFactory recordFactory = RecordFactoryProvider
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java
index f4a68a34e74..534bcd09408 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java
@@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
@@ -68,7 +68,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
import org.mockito.Mockito;
/**
@@ -83,7 +83,7 @@ public class TestMRApp {
Job job = app.submit(new Configuration());
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertEquals(System.getProperty("user.name"),job.getUserName());
+ Assert.assertEquals(System.getProperty("user.name"),job.getUserName());
}
@Test
@@ -106,7 +106,7 @@ public class TestMRApp {
MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true);
Job job = app.submit(new Configuration());
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(1, job.getTasks().size(), "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task task = it.next();
app.waitForState(task, TaskState.RUNNING);
@@ -151,7 +151,7 @@ public class TestMRApp {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(), "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -170,8 +170,8 @@ public class TestMRApp {
app.waitForState(task2Attempt, TaskAttemptState.RUNNING);
// reduces must be in NEW state
- Assertions.assertEquals(TaskState.NEW,
- reduceTask.getReport().getTaskState(), "Reduce Task state not correct");
+ Assert.assertEquals("Reduce Task state not correct",
+ TaskState.NEW, reduceTask.getReport().getTaskState());
//send the done signal to the 1st map task
app.getContext().getEventHandler().handle(
@@ -224,8 +224,7 @@ public class TestMRApp {
final Job job1 = app.submit(conf);
app.waitForState(job1, JobState.RUNNING);
- Assertions.assertEquals(4, job1.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 4, job1.getTasks().size());
Iterator it = job1.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -240,7 +239,7 @@ public class TestMRApp {
.next();
NodeId node1 = task1Attempt.getNodeId();
NodeId node2 = task2Attempt.getNodeId();
- Assertions.assertEquals(node1, node2);
+ Assert.assertEquals(node1, node2);
// send the done signal to the task
app.getContext()
@@ -272,8 +271,8 @@ public class TestMRApp {
TaskAttemptCompletionEvent[] events = job1.getTaskAttemptCompletionEvents
(0, 100);
- Assertions.assertEquals(2, events.length,
- "Expecting 2 completion events for success");
+ Assert.assertEquals("Expecting 2 completion events for success", 2,
+ events.length);
// send updated nodes info
ArrayList updatedNodes = new ArrayList();
@@ -298,8 +297,8 @@ public class TestMRApp {
}, checkIntervalMillis, waitForMillis);
events = job1.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(4, events.length,
- "Expecting 2 more completion events for killed");
+ Assert.assertEquals("Expecting 2 more completion events for killed", 4,
+ events.length);
// 2 map task attempts which were killed above should be requested from
// container allocator with the previous map task marked as failed. If
// this happens allocator will request the container for this mapper from
@@ -336,8 +335,8 @@ public class TestMRApp {
}, checkIntervalMillis, waitForMillis);
events = job1.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(5, events.length,
- "Expecting 1 more completion events for success");
+ Assert.assertEquals("Expecting 1 more completion events for success", 5,
+ events.length);
// Crash the app again.
app.stop();
@@ -352,8 +351,7 @@ public class TestMRApp {
final Job job2 = app.submit(conf);
app.waitForState(job2, JobState.RUNNING);
- Assertions.assertEquals(4, job2.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 4, job2.getTasks().size());
it = job2.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -374,8 +372,9 @@ public class TestMRApp {
}, checkIntervalMillis, waitForMillis);
events = job2.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(2, events.length,
- "Expecting 2 completion events for killed & success of map1");
+ Assert.assertEquals(
+ "Expecting 2 completion events for killed & success of map1", 2,
+ events.length);
task2Attempt = mapTask2.getAttempts().values().iterator().next();
app.getContext()
@@ -395,8 +394,8 @@ public class TestMRApp {
}, checkIntervalMillis, waitForMillis);
events = job2.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(3, events.length,
- "Expecting 1 more completion events for success");
+ Assert.assertEquals("Expecting 1 more completion events for success", 3,
+ events.length);
app.waitForState(reduceTask1, TaskState.RUNNING);
app.waitForState(reduceTask2, TaskState.RUNNING);
@@ -434,8 +433,8 @@ public class TestMRApp {
}
}, checkIntervalMillis, waitForMillis);
events = job2.getTaskAttemptCompletionEvents(0, 100);
- Assertions.assertEquals(5, events.length,
- "Expecting 2 more completion events for reduce success");
+ Assert.assertEquals("Expecting 2 more completion events for reduce success",
+ 5, events.length);
// job succeeds
app.waitForState(job2, JobState.SUCCEEDED);
@@ -473,8 +472,7 @@ public class TestMRApp {
MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true);
Job job = app.submit(new Configuration());
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(1, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task task = it.next();
app.waitForState(task, TaskState.RUNNING);
@@ -495,7 +493,7 @@ public class TestMRApp {
JobImpl job = (JobImpl) app.submit(new Configuration());
app.waitForInternalState(job, JobStateInternal.SUCCEEDED);
// AM is not unregistered
- Assertions.assertEquals(JobState.RUNNING, job.getState());
+ Assert.assertEquals(JobState.RUNNING, job.getState());
// imitate that AM is unregistered
app.successfullyUnregistered.set(true);
app.waitForState(job, JobState.SUCCEEDED);
@@ -507,8 +505,7 @@ public class TestMRApp {
MRApp app = new MRApp(1, 0, false, this.getClass().getName(), true);
Job job = app.submit(new Configuration());
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(1, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task task = it.next();
app.waitForState(task, TaskState.RUNNING);
@@ -533,8 +530,7 @@ public class TestMRApp {
Configuration conf = new Configuration();
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(1, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task task = it.next();
app.waitForState(task, TaskState.RUNNING);
@@ -628,7 +624,7 @@ public class TestMRApp {
(TaskAttemptImpl) taskAttempts.iterator().next();
// Container from RM should pass through to the launcher. Container object
// should be the same.
- Assertions.assertTrue(taskAttempt.container
+ Assert.assertTrue(taskAttempt.container
== containerObtainedByContainerLauncher);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java
index 7e47ec1a49a..9710ec94a69 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppComponentDependencies.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.mapreduce.v2.app;
import java.io.IOException;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
@@ -35,13 +35,11 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Test;
public class TestMRAppComponentDependencies {
- @Test
- @Timeout(20000)
+ @Test(timeout = 20000)
public void testComponentStopOrder() throws Exception {
@SuppressWarnings("resource")
TestMRApp app = new TestMRApp(1, 1, true, this.getClass().getName(), true);
@@ -56,8 +54,8 @@ public class TestMRAppComponentDependencies {
}
// assert JobHistoryEventHandlerStopped and then clientServiceStopped
- Assertions.assertEquals(1, app.JobHistoryEventHandlerStopped);
- Assertions.assertEquals(2, app.clientServiceStopped);
+ Assert.assertEquals(1, app.JobHistoryEventHandlerStopped);
+ Assert.assertEquals(2, app.clientServiceStopped);
}
private final class TestMRApp extends MRApp {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
index b8e55d9ca06..06550378ba9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
@@ -18,9 +18,9 @@
package org.apache.hadoop.mapreduce.v2.app;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
@@ -40,7 +40,7 @@ import java.util.concurrent.atomic.AtomicLong;
import java.util.HashMap;
import java.util.Map;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
@@ -84,11 +84,10 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import org.slf4j.event.Level;
@@ -105,7 +104,7 @@ public class TestMRAppMaster {
static String stagingDir = new Path(testDir, "staging").toString();
private static FileContext localFS = null;
- @BeforeAll
+ @BeforeClass
public static void setup() throws AccessControlException,
FileNotFoundException, IllegalArgumentException, IOException {
//Do not error out if metrics are inited multiple times
@@ -117,7 +116,7 @@ public class TestMRAppMaster {
new File(testDir.toString()).mkdir();
}
- @BeforeEach
+ @Before
public void prepare() throws IOException {
File dir = new File(stagingDir);
if(dir.exists()) {
@@ -126,7 +125,7 @@ public class TestMRAppMaster {
dir.mkdirs();
}
- @AfterAll
+ @AfterClass
public static void cleanup() throws IOException {
localFS.delete(testDir, true);
}
@@ -227,8 +226,8 @@ public class TestMRAppMaster {
"host", -1, -1, System.currentTimeMillis());
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
appMaster.stop();
- assertTrue(appMaster.jobLaunchTime.get() >= 0,
- "Job launch time should not be negative.");
+ assertTrue("Job launch time should not be negative.",
+ appMaster.jobLaunchTime.get() >= 0);
}
@Test
@@ -344,8 +343,7 @@ public class TestMRAppMaster {
appMaster.stop();
}
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testMRAppMasterMaxAppAttempts() throws IOException,
InterruptedException {
// No matter what's the maxAppAttempt or attempt id, the isLastRetry always
@@ -370,8 +368,8 @@ public class TestMRAppMaster {
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
System.currentTimeMillis(), false, true);
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
- assertEquals(expectedBools[i], appMaster.isLastAMRetry(),
- "isLastAMRetry is correctly computed.");
+ assertEquals("isLastAMRetry is correctly computed.", expectedBools[i],
+ appMaster.isLastAMRetry());
}
}
@@ -467,37 +465,37 @@ public class TestMRAppMaster {
// Now validate the task credentials
Credentials appMasterCreds = appMaster.getCredentials();
- Assertions.assertNotNull(appMasterCreds);
- Assertions.assertEquals(1, appMasterCreds.numberOfSecretKeys());
- Assertions.assertEquals(1, appMasterCreds.numberOfTokens());
+ Assert.assertNotNull(appMasterCreds);
+ Assert.assertEquals(1, appMasterCreds.numberOfSecretKeys());
+ Assert.assertEquals(1, appMasterCreds.numberOfTokens());
// Validate the tokens - app token should not be present
Token extends TokenIdentifier> usedToken =
appMasterCreds.getToken(tokenAlias);
- Assertions.assertNotNull(usedToken);
- Assertions.assertEquals(storedToken, usedToken);
+ Assert.assertNotNull(usedToken);
+ Assert.assertEquals(storedToken, usedToken);
// Validate the keys
byte[] usedKey = appMasterCreds.getSecretKey(keyAlias);
- Assertions.assertNotNull(usedKey);
- Assertions.assertEquals("mySecretKey", new String(usedKey));
+ Assert.assertNotNull(usedKey);
+ Assert.assertEquals("mySecretKey", new String(usedKey));
// The credentials should also be added to conf so that OuputCommitter can
// access it - app token should not be present
Credentials confCredentials = conf.getCredentials();
- Assertions.assertEquals(1, confCredentials.numberOfSecretKeys());
- Assertions.assertEquals(1, confCredentials.numberOfTokens());
- Assertions.assertEquals(storedToken, confCredentials.getToken(tokenAlias));
- Assertions.assertEquals("mySecretKey",
+ Assert.assertEquals(1, confCredentials.numberOfSecretKeys());
+ Assert.assertEquals(1, confCredentials.numberOfTokens());
+ Assert.assertEquals(storedToken, confCredentials.getToken(tokenAlias));
+ Assert.assertEquals("mySecretKey",
new String(confCredentials.getSecretKey(keyAlias)));
// Verify the AM's ugi - app token should be present
Credentials ugiCredentials = appMaster.getUgi().getCredentials();
- Assertions.assertEquals(1, ugiCredentials.numberOfSecretKeys());
- Assertions.assertEquals(2, ugiCredentials.numberOfTokens());
- Assertions.assertEquals(storedToken, ugiCredentials.getToken(tokenAlias));
- Assertions.assertEquals(appToken, ugiCredentials.getToken(appTokenService));
- Assertions.assertEquals("mySecretKey",
+ Assert.assertEquals(1, ugiCredentials.numberOfSecretKeys());
+ Assert.assertEquals(2, ugiCredentials.numberOfTokens());
+ Assert.assertEquals(storedToken, ugiCredentials.getToken(tokenAlias));
+ Assert.assertEquals(appToken, ugiCredentials.getToken(appTokenService));
+ Assert.assertEquals("mySecretKey",
new String(ugiCredentials.getSecretKey(keyAlias)));
@@ -527,10 +525,10 @@ public class TestMRAppMaster {
doNothing().when(appMaster).serviceStop();
// Test normal shutdown.
appMaster.shutDownJob();
- Assertions.assertTrue(ExitUtil.terminateCalled(),
- "Expected shutDownJob to terminate.");
- Assertions.assertEquals(0, ExitUtil.getFirstExitException().status,
- "Expected shutDownJob to exit with status code of 0.");
+ Assert.assertTrue("Expected shutDownJob to terminate.",
+ ExitUtil.terminateCalled());
+ Assert.assertEquals("Expected shutDownJob to exit with status code of 0.",
+ 0, ExitUtil.getFirstExitException().status);
// Test shutdown with exception.
ExitUtil.resetFirstExitException();
@@ -538,10 +536,10 @@ public class TestMRAppMaster {
doThrow(new RuntimeException(msg))
.when(appMaster).notifyIsLastAMRetry(anyBoolean());
appMaster.shutDownJob();
- assertTrue(ExitUtil.getFirstExitException().getMessage().contains(msg),
- "Expected message from ExitUtil.ExitException to be " + msg);
- Assertions.assertEquals(1, ExitUtil.getFirstExitException().status,
- "Expected shutDownJob to exit with status code of 1.");
+ assertTrue("Expected message from ExitUtil.ExitException to be " + msg,
+ ExitUtil.getFirstExitException().getMessage().contains(msg));
+ Assert.assertEquals("Expected shutDownJob to exit with status code of 1.",
+ 1, ExitUtil.getFirstExitException().status);
}
private void verifyFailedStatus(MRAppMasterTest appMaster,
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
index 4057ed5a46b..9906def3ac9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.mapreduce.v2.app;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.fail;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
@@ -26,7 +26,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobACL;
@@ -70,7 +70,7 @@ import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
public class TestMRClientService {
@@ -82,8 +82,7 @@ public class TestMRClientService {
Configuration conf = new Configuration();
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(1, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task task = it.next();
app.waitForState(task, TaskState.RUNNING);
@@ -117,8 +116,8 @@ public class TestMRClientService {
GetCountersRequest gcRequest =
recordFactory.newRecordInstance(GetCountersRequest.class);
gcRequest.setJobId(job.getID());
- Assertions.assertNotNull(proxy.getCounters(gcRequest).getCounters(),
- "Counters is null");
+ Assert.assertNotNull("Counters is null",
+ proxy.getCounters(gcRequest).getCounters());
GetJobReportRequest gjrRequest =
recordFactory.newRecordInstance(GetJobReportRequest.class);
@@ -132,14 +131,14 @@ public class TestMRClientService {
gtaceRequest.setJobId(job.getID());
gtaceRequest.setFromEventId(0);
gtaceRequest.setMaxEvents(10);
- Assertions.assertNotNull(proxy.getTaskAttemptCompletionEvents(gtaceRequest).
- getCompletionEventList(), "TaskCompletionEvents is null");
+ Assert.assertNotNull("TaskCompletionEvents is null",
+ proxy.getTaskAttemptCompletionEvents(gtaceRequest).getCompletionEventList());
GetDiagnosticsRequest gdRequest =
recordFactory.newRecordInstance(GetDiagnosticsRequest.class);
gdRequest.setTaskAttemptId(attempt.getID());
- Assertions.assertNotNull(proxy.getDiagnostics(gdRequest).
- getDiagnosticsList(), "Diagnostics is null");
+ Assert.assertNotNull("Diagnostics is null",
+ proxy.getDiagnostics(gdRequest).getDiagnosticsList());
GetTaskAttemptReportRequest gtarRequest =
recordFactory.newRecordInstance(GetTaskAttemptReportRequest.class);
@@ -152,32 +151,31 @@ public class TestMRClientService {
GetTaskReportRequest gtrRequest =
recordFactory.newRecordInstance(GetTaskReportRequest.class);
gtrRequest.setTaskId(task.getID());
- Assertions.assertNotNull(proxy.getTaskReport(gtrRequest).getTaskReport(),
- "TaskReport is null");
+ Assert.assertNotNull("TaskReport is null",
+ proxy.getTaskReport(gtrRequest).getTaskReport());
GetTaskReportsRequest gtreportsRequest =
recordFactory.newRecordInstance(GetTaskReportsRequest.class);
gtreportsRequest.setJobId(job.getID());
gtreportsRequest.setTaskType(TaskType.MAP);
- Assertions.assertNotNull(proxy.getTaskReports(gtreportsRequest)
- .getTaskReportList(), "TaskReports for map is null");
+ Assert.assertNotNull("TaskReports for map is null",
+ proxy.getTaskReports(gtreportsRequest).getTaskReportList());
gtreportsRequest =
recordFactory.newRecordInstance(GetTaskReportsRequest.class);
gtreportsRequest.setJobId(job.getID());
gtreportsRequest.setTaskType(TaskType.REDUCE);
- Assertions.assertNotNull(proxy.getTaskReports(gtreportsRequest).getTaskReportList(),
- "TaskReports for reduce is null");
+ Assert.assertNotNull("TaskReports for reduce is null",
+ proxy.getTaskReports(gtreportsRequest).getTaskReportList());
List diag = proxy.getDiagnostics(gdRequest).getDiagnosticsList();
- Assertions.assertEquals(1 , diag.size(),
- "Num diagnostics not correct");
- Assertions.assertEquals(diagnostic1, diag.get(0).toString(),
- "Diag 1 not correct");
+ Assert.assertEquals("Num diagnostics not correct", 1 , diag.size());
+ Assert.assertEquals("Diag 1 not correct",
+ diagnostic1, diag.get(0).toString());
TaskReport taskReport = proxy.getTaskReport(gtrRequest).getTaskReport();
- Assertions.assertEquals(1, taskReport.getDiagnosticsCount(),
- "Num diagnostics not correct");
+ Assert.assertEquals("Num diagnostics not correct", 1,
+ taskReport.getDiagnosticsCount());
//send the done signal to the task
app.getContext().getEventHandler().handle(
@@ -209,8 +207,7 @@ public class TestMRClientService {
conf.set(MRJobConfig.JOB_ACL_VIEW_JOB, "viewonlyuser");
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(1, job.getTasks().size(),
- "Num tasks not correct");
+ Assert.assertEquals("Num tasks not correct", 1, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task task = it.next();
app.waitForState(task, TaskState.RUNNING);
@@ -220,10 +217,10 @@ public class TestMRClientService {
UserGroupInformation viewOnlyUser =
UserGroupInformation.createUserForTesting(
"viewonlyuser", new String[] {});
- Assertions.assertTrue(job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB),
- "viewonlyuser cannot view job");
- Assertions.assertFalse(job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB),
- "viewonlyuser can modify job");
+ Assert.assertTrue("viewonlyuser cannot view job",
+ job.checkAccess(viewOnlyUser, JobACL.VIEW_JOB));
+ Assert.assertFalse("viewonlyuser can modify job",
+ job.checkAccess(viewOnlyUser, JobACL.MODIFY_JOB));
MRClientProtocol client = viewOnlyUser.doAs(
new PrivilegedExceptionAction() {
@Override
@@ -276,28 +273,28 @@ public class TestMRClientService {
}
private void verifyJobReport(JobReport jr) {
- Assertions.assertNotNull(jr, "JobReport is null");
+ Assert.assertNotNull("JobReport is null", jr);
List amInfos = jr.getAMInfos();
- Assertions.assertEquals(1, amInfos.size());
- Assertions.assertEquals(JobState.RUNNING, jr.getJobState());
+ Assert.assertEquals(1, amInfos.size());
+ Assert.assertEquals(JobState.RUNNING, jr.getJobState());
AMInfo amInfo = amInfos.get(0);
- Assertions.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
- Assertions.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
- Assertions.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
- Assertions.assertEquals(1, amInfo.getAppAttemptId().getAttemptId());
- Assertions.assertEquals(1, amInfo.getContainerId().getApplicationAttemptId()
+ Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
+ Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
+ Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
+ Assert.assertEquals(1, amInfo.getAppAttemptId().getAttemptId());
+ Assert.assertEquals(1, amInfo.getContainerId().getApplicationAttemptId()
.getAttemptId());
- Assertions.assertTrue(amInfo.getStartTime() > 0);
- Assertions.assertFalse(jr.isUber());
+ Assert.assertTrue(amInfo.getStartTime() > 0);
+ Assert.assertFalse(jr.isUber());
}
private void verifyTaskAttemptReport(TaskAttemptReport tar) {
- Assertions.assertEquals(TaskAttemptState.RUNNING, tar.getTaskAttemptState());
- Assertions.assertNotNull(tar, "TaskAttemptReport is null");
- Assertions.assertEquals(MRApp.NM_HOST, tar.getNodeManagerHost());
- Assertions.assertEquals(MRApp.NM_PORT, tar.getNodeManagerPort());
- Assertions.assertEquals(MRApp.NM_HTTP_PORT, tar.getNodeManagerHttpPort());
- Assertions.assertEquals(1, tar.getContainerId().getApplicationAttemptId()
+ Assert.assertEquals(TaskAttemptState.RUNNING, tar.getTaskAttemptState());
+ Assert.assertNotNull("TaskAttemptReport is null", tar);
+ Assert.assertEquals(MRApp.NM_HOST, tar.getNodeManagerHost());
+ Assert.assertEquals(MRApp.NM_PORT, tar.getNodeManagerPort());
+ Assert.assertEquals(MRApp.NM_HTTP_PORT, tar.getNodeManagerHttpPort());
+ Assert.assertEquals(1, tar.getContainerId().getApplicationAttemptId()
.getAttemptId());
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
index ce8e1e1573e..5a23b58875a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRecovery.java
@@ -19,9 +19,9 @@
package org.apache.hadoop.mapreduce.v2.app;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.atLeast;
import static org.mockito.Mockito.mock;
@@ -42,7 +42,7 @@ import java.util.concurrent.TimeoutException;
import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -107,9 +107,8 @@ import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.BeforeClass;
+import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -127,7 +126,7 @@ public class TestRecovery {
private Text val1 = new Text("val1");
private Text val2 = new Text("val2");
- @BeforeAll
+ @BeforeClass
public static void setupClass() throws Exception {
// setup the test root directory
testRootDir =
@@ -159,8 +158,8 @@ public class TestRecovery {
app.waitForState(job, JobState.RUNNING);
long jobStartTime = job.getReport().getStartTime();
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -193,7 +192,7 @@ public class TestRecovery {
Thread.sleep(2000);
LOG.info("Waiting for next attempt to start");
}
- Assertions.assertEquals(2, mapTask1.getAttempts().size());
+ Assert.assertEquals(2, mapTask1.getAttempts().size());
Iterator itr = mapTask1.getAttempts().values().iterator();
itr.next();
TaskAttempt task1Attempt2 = itr.next();
@@ -214,7 +213,7 @@ public class TestRecovery {
Thread.sleep(2000);
LOG.info("Waiting for next attempt to start");
}
- Assertions.assertEquals(3, mapTask1.getAttempts().size());
+ Assert.assertEquals(3, mapTask1.getAttempts().size());
itr = mapTask1.getAttempts().values().iterator();
itr.next();
itr.next();
@@ -235,7 +234,7 @@ public class TestRecovery {
Thread.sleep(2000);
LOG.info("Waiting for next attempt to start");
}
- Assertions.assertEquals(4, mapTask1.getAttempts().size());
+ Assert.assertEquals(4, mapTask1.getAttempts().size());
itr = mapTask1.getAttempts().values().iterator();
itr.next();
itr.next();
@@ -273,8 +272,8 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -309,29 +308,29 @@ public class TestRecovery {
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertEquals(jobStartTime, job.getReport().getStartTime(),
- "Job Start time not correct");
- Assertions.assertEquals(task1StartTime, mapTask1.getReport().getStartTime(),
- "Task Start time not correct");
- Assertions.assertEquals(task1FinishTime, mapTask1.getReport().getFinishTime(),
- "Task Finish time not correct");
- Assertions.assertEquals(2, job.getAMInfos().size());
+ Assert.assertEquals("Job Start time not correct",
+ jobStartTime, job.getReport().getStartTime());
+ Assert.assertEquals("Task Start time not correct",
+ task1StartTime, mapTask1.getReport().getStartTime());
+ Assert.assertEquals("Task Finish time not correct",
+ task1FinishTime, mapTask1.getReport().getFinishTime());
+ Assert.assertEquals(2, job.getAMInfos().size());
int attemptNum = 1;
// Verify AMInfo
for (AMInfo amInfo : job.getAMInfos()) {
- Assertions.assertEquals(attemptNum++, amInfo.getAppAttemptId()
+ Assert.assertEquals(attemptNum++, amInfo.getAppAttemptId()
.getAttemptId());
- Assertions.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
+ Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
.getApplicationAttemptId());
- Assertions.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
- Assertions.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
- Assertions.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
+ Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
+ Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
+ Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
}
long am1StartTimeReal = job.getAMInfos().get(0).getStartTime();
long am2StartTimeReal = job.getAMInfos().get(1).getStartTime();
- Assertions.assertTrue(am1StartTimeReal >= am1StartTimeEst
+ Assert.assertTrue(am1StartTimeReal >= am1StartTimeEst
&& am1StartTimeReal <= am2StartTimeEst);
- Assertions.assertTrue(am2StartTimeReal >= am2StartTimeEst
+ Assert.assertTrue(am2StartTimeReal >= am2StartTimeEst
&& am2StartTimeReal <= System.currentTimeMillis());
// TODO Add verification of additional data from jobHistory - whatever was
// available in the failed attempt should be available here
@@ -372,7 +371,7 @@ public class TestRecovery {
app.waitForState(job, JobState.RUNNING);
// all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -430,7 +429,7 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -517,7 +516,7 @@ public class TestRecovery {
app.waitForState(job, JobState.RUNNING);
// all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -576,7 +575,7 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -642,9 +641,8 @@ public class TestRecovery {
app = new MRAppWithHistory(1, 1, false, this.getClass().getName(), false,
++runCount);
Job jobAttempt2 = app.submit(conf);
- Assertions.assertTrue(!app.recovered(),
- "Recovery from previous job attempt is processed even " +
- "though intermediate data encryption is enabled.");
+ Assert.assertTrue("Recovery from previous job attempt is processed even " +
+ "though intermediate data encryption is enabled.", !app.recovered());
// The map task succeeded from previous job attempt will not be recovered
// because the data spill encryption is enabled.
@@ -696,7 +694,7 @@ public class TestRecovery {
app.waitForState(job, JobState.RUNNING);
// all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -755,7 +753,7 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -815,8 +813,8 @@ public class TestRecovery {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -835,8 +833,8 @@ public class TestRecovery {
app.waitForState(task2Attempt, TaskAttemptState.RUNNING);
// reduces must be in NEW state
- Assertions.assertEquals(TaskState.RUNNING, reduceTask.getReport().getTaskState(),
- "Reduce Task state not correct");
+ Assert.assertEquals("Reduce Task state not correct",
+ TaskState.RUNNING, reduceTask.getReport().getTaskState());
//send the done signal to the 1st map
app.getContext().getEventHandler().handle(
@@ -864,8 +862,8 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -907,8 +905,8 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -942,8 +940,8 @@ public class TestRecovery {
conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task reduceTask1 = it.next();
@@ -968,7 +966,7 @@ public class TestRecovery {
app.waitForState(mapTask1, TaskState.SUCCEEDED);
// Verify the shuffle-port
- Assertions.assertEquals(5467, task1Attempt1.getShufflePort());
+ Assert.assertEquals(5467, task1Attempt1.getShufflePort());
app.waitForState(reduceTask1, TaskState.RUNNING);
TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next();
@@ -1000,8 +998,8 @@ public class TestRecovery {
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
reduceTask1 = it.next();
@@ -1012,7 +1010,7 @@ public class TestRecovery {
// Verify the shuffle-port after recovery
task1Attempt1 = mapTask1.getAttempts().values().iterator().next();
- Assertions.assertEquals(5467, task1Attempt1.getShufflePort());
+ Assert.assertEquals(5467, task1Attempt1.getShufflePort());
// first reduce will be recovered, no need to send done
app.waitForState(reduceTask1, TaskState.SUCCEEDED);
@@ -1053,7 +1051,7 @@ public class TestRecovery {
conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
//stop the app before the job completes.
app.stop();
app.close();
@@ -1063,11 +1061,11 @@ public class TestRecovery {
++runCount);
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(), "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
TestFileOutputCommitter committer = (
TestFileOutputCommitter) app.getCommitter();
- assertTrue(committer.isAbortJobCalled(),
- "commiter.abortJob() has not been called");
+ assertTrue("commiter.abortJob() has not been called",
+ committer.isAbortJobCalled());
app.close();
}
@@ -1088,8 +1086,7 @@ public class TestRecovery {
conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
//stop the app before the job completes.
app.stop();
app.close();
@@ -1099,12 +1096,11 @@ public class TestRecovery {
++runCount);
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct", 3, job.getTasks().size());
TestFileOutputCommitter committer = (
TestFileOutputCommitter) app.getCommitter();
- assertFalse(committer.isAbortJobCalled(),
- "commiter.abortJob() has been called");
+ assertFalse("commiter.abortJob() has been called",
+ committer.isAbortJobCalled());
app.close();
}
@@ -1120,8 +1116,8 @@ public class TestRecovery {
conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -1151,7 +1147,7 @@ public class TestRecovery {
app.waitForState(mapTask1, TaskState.SUCCEEDED);
// Verify the shuffle-port
- Assertions.assertEquals(5467, task1Attempt1.getShufflePort());
+ Assert.assertEquals(5467, task1Attempt1.getShufflePort());
//stop the app before the job completes.
app.stop();
@@ -1168,8 +1164,8 @@ public class TestRecovery {
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -1180,7 +1176,7 @@ public class TestRecovery {
// Verify the shuffle-port after recovery
task1Attempt1 = mapTask1.getAttempts().values().iterator().next();
- Assertions.assertEquals(5467, task1Attempt1.getShufflePort());
+ Assert.assertEquals(5467, task1Attempt1.getShufflePort());
app.waitForState(mapTask2, TaskState.RUNNING);
@@ -1201,7 +1197,7 @@ public class TestRecovery {
app.waitForState(mapTask2, TaskState.SUCCEEDED);
// Verify the shuffle-port
- Assertions.assertEquals(5467, task2Attempt1.getShufflePort());
+ Assert.assertEquals(5467, task2Attempt1.getShufflePort());
app.waitForState(reduceTask1, TaskState.RUNNING);
TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next();
@@ -1235,8 +1231,8 @@ public class TestRecovery {
conf.set(FileOutputFormat.OUTDIR, outputDir.toString());
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task reduceTask1 = it.next();
@@ -1261,7 +1257,7 @@ public class TestRecovery {
app.waitForState(mapTask1, TaskState.SUCCEEDED);
// Verify the shuffle-port
- Assertions.assertEquals(5467, task1Attempt1.getShufflePort());
+ Assert.assertEquals(5467, task1Attempt1.getShufflePort());
app.waitForState(reduceTask1, TaskState.RUNNING);
TaskAttempt reduce1Attempt1 = reduceTask1.getAttempts().values().iterator().next();
@@ -1293,8 +1289,8 @@ public class TestRecovery {
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
reduceTask1 = it.next();
@@ -1305,7 +1301,7 @@ public class TestRecovery {
// Verify the shuffle-port after recovery
task1Attempt1 = mapTask1.getAttempts().values().iterator().next();
- Assertions.assertEquals(5467, task1Attempt1.getShufflePort());
+ Assert.assertEquals(5467, task1Attempt1.getShufflePort());
// first reduce will be recovered, no need to send done
app.waitForState(reduceTask1, TaskState.SUCCEEDED);
@@ -1355,8 +1351,8 @@ public class TestRecovery {
app.waitForState(job, JobState.RUNNING);
long jobStartTime = job.getReport().getStartTime();
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
@@ -1429,8 +1425,8 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -1466,36 +1462,36 @@ public class TestRecovery {
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertEquals(jobStartTime, job.getReport().getStartTime(),
- "Job Start time not correct");
- Assertions.assertEquals(task1StartTime, mapTask1.getReport().getStartTime(),
- "Task Start time not correct");
- Assertions.assertEquals(task1FinishTime, mapTask1.getReport().getFinishTime(),
- "Task Finish time not correct");
- Assertions.assertEquals(2, job.getAMInfos().size());
+ Assert.assertEquals("Job Start time not correct",
+ jobStartTime, job.getReport().getStartTime());
+ Assert.assertEquals("Task Start time not correct",
+ task1StartTime, mapTask1.getReport().getStartTime());
+ Assert.assertEquals("Task Finish time not correct",
+ task1FinishTime, mapTask1.getReport().getFinishTime());
+ Assert.assertEquals(2, job.getAMInfos().size());
int attemptNum = 1;
// Verify AMInfo
for (AMInfo amInfo : job.getAMInfos()) {
- Assertions.assertEquals(attemptNum++, amInfo.getAppAttemptId()
+ Assert.assertEquals(attemptNum++, amInfo.getAppAttemptId()
.getAttemptId());
- Assertions.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
+ Assert.assertEquals(amInfo.getAppAttemptId(), amInfo.getContainerId()
.getApplicationAttemptId());
- Assertions.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
- Assertions.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
- Assertions.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
+ Assert.assertEquals(MRApp.NM_HOST, amInfo.getNodeManagerHost());
+ Assert.assertEquals(MRApp.NM_PORT, amInfo.getNodeManagerPort());
+ Assert.assertEquals(MRApp.NM_HTTP_PORT, amInfo.getNodeManagerHttpPort());
}
long am1StartTimeReal = job.getAMInfos().get(0).getStartTime();
long am2StartTimeReal = job.getAMInfos().get(1).getStartTime();
- Assertions.assertTrue(am1StartTimeReal >= am1StartTimeEst
+ Assert.assertTrue(am1StartTimeReal >= am1StartTimeEst
&& am1StartTimeReal <= am2StartTimeEst);
- Assertions.assertTrue(am2StartTimeReal >= am2StartTimeEst
+ Assert.assertTrue(am2StartTimeReal >= am2StartTimeEst
&& am2StartTimeReal <= System.currentTimeMillis());
}
- @Test
- @Timeout(30000)
+ @Test(timeout=30000)
public void testRecoveryWithoutShuffleSecret() throws Exception {
+
int runCount = 0;
MRApp app = new MRAppNoShuffleSecret(2, 1, false,
this.getClass().getName(), true, ++runCount);
@@ -1507,8 +1503,8 @@ public class TestRecovery {
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
Iterator it = job.getTasks().values().iterator();
Task mapTask1 = it.next();
Task mapTask2 = it.next();
@@ -1554,8 +1550,8 @@ public class TestRecovery {
job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
//all maps would be running
- Assertions.assertEquals(3, job.getTasks().size(),
- "No of tasks not correct");
+ Assert.assertEquals("No of tasks not correct",
+ 3, job.getTasks().size());
it = job.getTasks().values().iterator();
mapTask1 = it.next();
mapTask2 = it.next();
@@ -1894,16 +1890,16 @@ public class TestRecovery {
ArgumentCaptor arg, List expectedJobHistoryEvents,
long expectedMapLaunches, long expectedFailedMaps) {
- assertEquals(finalState, checkTask.getState(), "Final State of Task");
+ assertEquals("Final State of Task", finalState, checkTask.getState());
Map recoveredAttempts =
checkTask.getAttempts();
- assertEquals(finalAttemptStates.size(), recoveredAttempts.size(),
- "Expected Number of Task Attempts");
+ assertEquals("Expected Number of Task Attempts",
+ finalAttemptStates.size(), recoveredAttempts.size());
for (TaskAttemptID taID : finalAttemptStates.keySet()) {
- assertEquals(finalAttemptStates.get(taID),
- recoveredAttempts.get(TypeConverter.toYarn(taID)).getState(),
- "Expected Task Attempt State");
+ assertEquals("Expected Task Attempt State",
+ finalAttemptStates.get(taID),
+ recoveredAttempts.get(TypeConverter.toYarn(taID)).getState());
}
Iterator ie = arg.getAllValues().iterator();
@@ -1951,12 +1947,12 @@ public class TestRecovery {
}
}
assertTrue(jobTaskEventReceived || (finalState == TaskState.RUNNING));
- assertEquals(0, expectedJobHistoryEvents.size(),
- "Did not process all expected JobHistoryEvents");
- assertEquals(expectedMapLaunches, totalLaunchedMaps,
- "Expected Map Launches");
- assertEquals(expectedFailedMaps, totalFailedMaps,
- "Expected Failed Maps");
+ assertEquals("Did not process all expected JobHistoryEvents",
+ 0, expectedJobHistoryEvents.size());
+ assertEquals("Expected Map Launches",
+ expectedMapLaunches, totalLaunchedMaps);
+ assertEquals("Expected Failed Maps",
+ expectedFailedMaps, totalFailedMaps);
}
private MapTaskImpl getMockMapTask(long clusterTimestamp, EventHandler eh) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
index b45b674bf50..0031598da5b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
@@ -78,8 +78,8 @@ import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -152,16 +152,16 @@ public class TestRuntimeEstimators {
conf.setDouble(MRJobConfig.SPECULATIVECAP_TOTAL_TASKS, 0.001);
conf.setInt(MRJobConfig.SPECULATIVE_MINIMUM_ALLOWED_TASKS, 5);
speculator = new DefaultSpeculator(conf, myAppContext, estimator, clock);
- Assertions.assertEquals(500L, speculator.getSoonestRetryAfterNoSpeculate(),
- "wrong SPECULATIVE_RETRY_AFTER_NO_SPECULATE value");
- Assertions.assertEquals(5000L, speculator.getSoonestRetryAfterSpeculate(),
- "wrong SPECULATIVE_RETRY_AFTER_SPECULATE value");
+ Assert.assertEquals("wrong SPECULATIVE_RETRY_AFTER_NO_SPECULATE value",
+ 500L, speculator.getSoonestRetryAfterNoSpeculate());
+ Assert.assertEquals("wrong SPECULATIVE_RETRY_AFTER_SPECULATE value",
+ 5000L, speculator.getSoonestRetryAfterSpeculate());
assertThat(speculator.getProportionRunningTasksSpeculatable())
.isCloseTo(0.1, offset(0.00001));
assertThat(speculator.getProportionTotalTasksSpeculatable())
.isCloseTo(0.001, offset(0.00001));
- Assertions.assertEquals(5, speculator.getMinimumAllowedSpeculativeTasks(),
- "wrong SPECULATIVE_MINIMUM_ALLOWED_TASKS value");
+ Assert.assertEquals("wrong SPECULATIVE_MINIMUM_ALLOWED_TASKS value",
+ 5, speculator.getMinimumAllowedSpeculativeTasks());
dispatcher.register(Speculator.EventType.class, speculator);
@@ -244,8 +244,8 @@ public class TestRuntimeEstimators {
}
}
- Assertions.assertEquals(expectedSpeculations, successfulSpeculations.get(),
- "We got the wrong number of successful speculations.");
+ Assert.assertEquals("We got the wrong number of successful speculations.",
+ expectedSpeculations, successfulSpeculations.get());
}
@Test
@@ -279,8 +279,8 @@ public class TestRuntimeEstimators {
TaskId taskID = event.getTaskID();
Task task = myJob.getTask(taskID);
- Assertions.assertEquals
- (TaskEventType.T_ADD_SPEC_ATTEMPT, event.getType(), "Wrong type event");
+ Assert.assertEquals
+ ("Wrong type event", TaskEventType.T_ADD_SPEC_ATTEMPT, event.getType());
System.out.println("SpeculationRequestEventHandler.handle adds a speculation task for " + taskID);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
index 81314704d1f..1f0ce2309e2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
@@ -18,8 +18,8 @@
package org.apache.hadoop.mapreduce.v2.app;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.Mockito.mock;
@@ -61,10 +61,9 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
/**
@@ -79,7 +78,7 @@ import org.junit.jupiter.api.Timeout;
private final static RecordFactory recordFactory = RecordFactoryProvider.
getRecordFactory(null);
- @AfterEach
+ @After
public void tearDown() {
conf.setBoolean(MRJobConfig.PRESERVE_FAILED_TASK_FILES, false);
}
@@ -136,7 +135,7 @@ import org.junit.jupiter.api.Timeout;
JobId jobid = recordFactory.newRecordInstance(JobId.class);
jobid.setAppId(appId);
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
- Assertions.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+ Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
appMaster.init(conf);
@@ -147,8 +146,7 @@ import org.junit.jupiter.api.Timeout;
verify(fs).delete(stagingJobPath, true);
}
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testNoDeletionofStagingOnReboot() throws IOException {
conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
fs = mock(FileSystem.class);
@@ -160,7 +158,7 @@ import org.junit.jupiter.api.Timeout;
0);
ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
- Assertions.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+ Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
JobStateInternal.REBOOT, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
appMaster.init(conf);
@@ -199,8 +197,7 @@ import org.junit.jupiter.api.Timeout;
verify(fs).delete(stagingJobPath, true);
}
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testDeletionofStagingOnKill() throws IOException {
conf.set(MRJobConfig.MAPREDUCE_JOB_DIR, stagingJobDir);
fs = mock(FileSystem.class);
@@ -218,7 +215,7 @@ import org.junit.jupiter.api.Timeout;
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc);
appMaster.init(conf);
//simulate the process being killed
- MRAppMaster.MRAppMasterShutdownHook hook =
+ MRAppMaster.MRAppMasterShutdownHook hook =
new MRAppMaster.MRAppMasterShutdownHook(appMaster);
hook.run();
verify(fs, times(0)).delete(stagingJobPath, true);
@@ -245,14 +242,13 @@ import org.junit.jupiter.api.Timeout;
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc); //no retry
appMaster.init(conf);
- assertTrue(appMaster.isLastAMRetry(),
- "appMaster.isLastAMRetry() is false");
+ assertTrue("appMaster.isLastAMRetry() is false", appMaster.isLastAMRetry());
//simulate the process being killed
MRAppMaster.MRAppMasterShutdownHook hook =
new MRAppMaster.MRAppMasterShutdownHook(appMaster);
hook.run();
- assertTrue(appMaster.isInState(Service.STATE.STOPPED),
- "MRAppMaster isn't stopped");
+ assertTrue("MRAppMaster isn't stopped",
+ appMaster.isInState(Service.STATE.STOPPED));
verify(fs).delete(stagingJobPath, true);
}
@@ -274,7 +270,7 @@ import org.junit.jupiter.api.Timeout;
JobId jobid = recordFactory.newRecordInstance(JobId.class);
jobid.setAppId(appId);
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
- Assertions.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+ Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
JobStateInternal.FAILED, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
appMaster.init(conf);
@@ -302,7 +298,7 @@ import org.junit.jupiter.api.Timeout;
JobId jobid = recordFactory.newRecordInstance(JobId.class);
jobid.setAppId(appId);
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
- Assertions.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+ Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
appMaster.init(conf);
@@ -328,7 +324,7 @@ import org.junit.jupiter.api.Timeout;
JobId jobid = recordFactory.newRecordInstance(JobId.class);
jobid.setAppId(appId);
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
- Assertions.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+ Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
appMaster.init(conf);
@@ -359,7 +355,7 @@ import org.junit.jupiter.api.Timeout;
JobId jobid = recordFactory.newRecordInstance(JobId.class);
jobid.setAppId(appId);
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
- Assertions.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
+ Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
JobStateInternal.RUNNING, MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
appMaster.init(conf);
@@ -587,8 +583,7 @@ import org.junit.jupiter.api.Timeout;
};
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testStagingCleanupOrder() throws Exception {
MRAppTestCleanup app = new MRAppTestCleanup(1, 1, true,
this.getClass().getName(), true);
@@ -603,7 +598,7 @@ import org.junit.jupiter.api.Timeout;
}
// assert ContainerAllocatorStopped and then tagingDirCleanedup
- Assertions.assertEquals(1, app.ContainerAllocatorStopped);
- Assertions.assertEquals(2, app.stagingDirCleanedup);
+ Assert.assertEquals(1, app.ContainerAllocatorStopped);
+ Assert.assertEquals(2, app.stagingDirCleanedup);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
index c0ba8d6c265..f5c30c2a8db 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestTaskHeartbeatHandler.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.mapreduce.v2.app;
-import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.Assert.assertFalse;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
@@ -40,8 +40,8 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Test;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
@@ -214,11 +214,11 @@ public class TestTaskHeartbeatHandler {
JobId jobId = MRBuilderUtils.newJobId(appId, 4);
TaskId tid = MRBuilderUtils.newTaskId(jobId, 3, TaskType.MAP);
final TaskAttemptId taid = MRBuilderUtils.newTaskAttemptId(tid, 2);
- Assertions.assertFalse(hb.hasRecentlyUnregistered(taid));
+ Assert.assertFalse(hb.hasRecentlyUnregistered(taid));
hb.register(taid);
- Assertions.assertFalse(hb.hasRecentlyUnregistered(taid));
+ Assert.assertFalse(hb.hasRecentlyUnregistered(taid));
hb.unregister(taid);
- Assertions.assertTrue(hb.hasRecentlyUnregistered(taid));
+ Assert.assertTrue(hb.hasRecentlyUnregistered(taid));
long unregisterTimeout = conf.getLong(MRJobConfig.TASK_EXIT_TIMEOUT,
MRJobConfig.TASK_EXIT_TIMEOUT_DEFAULT);
clock.setTime(unregisterTimeout + 1);
@@ -260,7 +260,7 @@ public class TestTaskHeartbeatHandler {
new TaskHeartbeatHandler(null, SystemClock.getInstance(), 1);
hb.init(conf);
- Assertions.assertTrue(hb.getTaskTimeOut() == expectedTimeout,
- "The value of the task timeout is incorrect.");
+ Assert.assertTrue("The value of the task timeout is incorrect.",
+ hb.getTaskTimeOut() == expectedTimeout);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
index c051504b322..a3e85aad841 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/commit/TestCommitterEventHandler.java
@@ -27,7 +27,7 @@ import static org.mockito.Mockito.when;
import java.util.concurrent.ConcurrentLinkedQueue;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobContext;
@@ -39,7 +39,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
import java.io.File;
@@ -62,9 +62,9 @@ import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
public class TestCommitterEventHandler {
public static class WaitForItHandler implements EventHandler {
@@ -95,13 +95,13 @@ public class TestCommitterEventHandler {
static String stagingDir = "target/test-staging/";
- @BeforeAll
+ @BeforeClass
public static void setup() {
File dir = new File(stagingDir);
stagingDir = dir.getAbsolutePath();
}
- @BeforeEach
+ @Before
public void cleanup() throws IOException {
File dir = new File(stagingDir);
if(dir.exists()) {
@@ -146,11 +146,11 @@ public class TestCommitterEventHandler {
Thread.sleep(10);
timeToWaitMs -= 10;
}
- Assertions.assertEquals(1, rmhh.getNumCallbacks(),
- "committer did not register a heartbeat callback");
+ Assert.assertEquals("committer did not register a heartbeat callback",
+ 1, rmhh.getNumCallbacks());
verify(committer, never()).commitJob(any(JobContext.class));
- Assertions.assertEquals(0, jeh.numCommitCompletedEvents,
- "committer should not have committed");
+ Assert.assertEquals("committer should not have committed",
+ 0, jeh.numCommitCompletedEvents);
// set a fresh heartbeat and verify commit completes
rmhh.setLastHeartbeatTime(clock.getTime());
@@ -159,8 +159,8 @@ public class TestCommitterEventHandler {
Thread.sleep(10);
timeToWaitMs -= 10;
}
- Assertions.assertEquals(1, jeh.numCommitCompletedEvents,
- "committer did not complete commit after RM hearbeat");
+ Assert.assertEquals("committer did not complete commit after RM hearbeat",
+ 1, jeh.numCommitCompletedEvents);
verify(committer, times(1)).commitJob(any());
//Clean up so we can try to commit again (Don't do this at home)
@@ -174,8 +174,8 @@ public class TestCommitterEventHandler {
Thread.sleep(10);
timeToWaitMs -= 10;
}
- Assertions.assertEquals(2, jeh.numCommitCompletedEvents,
- "committer did not commit");
+ Assert.assertEquals("committer did not commit",
+ 2, jeh.numCommitCompletedEvents);
verify(committer, times(2)).commitJob(any());
ceh.stop();
@@ -262,9 +262,9 @@ public class TestCommitterEventHandler {
assertNotNull(e);
assertTrue(e instanceof JobCommitCompletedEvent);
FileSystem fs = FileSystem.get(conf);
- assertTrue(fs.exists(startCommitFile), startCommitFile.toString());
- assertTrue(fs.exists(endCommitSuccessFile), endCommitSuccessFile.toString());
- assertFalse(fs.exists(endCommitFailureFile), endCommitFailureFile.toString());
+ assertTrue(startCommitFile.toString(), fs.exists(startCommitFile));
+ assertTrue(endCommitSuccessFile.toString(), fs.exists(endCommitSuccessFile));
+ assertFalse(endCommitFailureFile.toString(), fs.exists(endCommitFailureFile));
verify(mockCommitter).commitJob(any(JobContext.class));
} finally {
handler.stop();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
index 5f827e46d95..5f378e4f9c3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestJobImpl.java
@@ -105,11 +105,10 @@ import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
import org.mockito.Mockito;
@@ -121,13 +120,13 @@ public class TestJobImpl {
static String stagingDir = "target/test-staging/";
- @BeforeAll
+ @BeforeClass
public static void setup() {
File dir = new File(stagingDir);
stagingDir = dir.getAbsolutePath();
}
- @BeforeEach
+ @Before
public void cleanup() throws IOException {
File dir = new File(stagingDir);
if(dir.exists()) {
@@ -170,14 +169,13 @@ public class TestJobImpl {
dispatcher.stop();
commitHandler.stop();
try {
- Assertions.assertTrue(jseHandler.getAssertValue());
+ Assert.assertTrue(jseHandler.getAssertValue());
} catch (InterruptedException e) {
- Assertions.fail("Workflow related attributes are not tested properly");
+ Assert.fail("Workflow related attributes are not tested properly");
}
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testCommitJobFailsJob() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -202,8 +200,7 @@ public class TestJobImpl {
commitHandler.stop();
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testCheckJobCompleteSuccess() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -242,7 +239,7 @@ public class TestJobImpl {
JobEventType.JOB_TASK_ATTEMPT_COMPLETED));
assertJobState(job, JobStateInternal.SUCCEEDED);
- job.handle(new JobEvent(job.getID(),
+ job.handle(new JobEvent(job.getID(),
JobEventType.JOB_MAP_TASK_RESCHEDULED));
assertJobState(job, JobStateInternal.SUCCEEDED);
@@ -250,14 +247,13 @@ public class TestJobImpl {
JobEventType.JOB_TASK_COMPLETED));
dispatcher.await();
assertJobState(job, JobStateInternal.SUCCEEDED);
-
+
dispatcher.stop();
commitHandler.stop();
}
- @Test
- @Timeout(20000)
- public void testRebootedDuringSetup() throws Exception {
+ @Test(timeout=20000)
+ public void testRebootedDuringSetup() throws Exception{
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
AsyncDispatcher dispatcher = new AsyncDispatcher();
@@ -293,14 +289,13 @@ public class TestJobImpl {
assertJobState(job, JobStateInternal.REBOOT);
// return the external state as RUNNING since otherwise JobClient will
// exit when it polls the AM for job state
- Assertions.assertEquals(JobState.RUNNING, job.getState());
+ Assert.assertEquals(JobState.RUNNING, job.getState());
dispatcher.stop();
commitHandler.stop();
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testRebootedDuringCommit() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -326,16 +321,15 @@ public class TestJobImpl {
job.handle(new JobEvent(job.getID(), JobEventType.JOB_AM_REBOOT));
assertJobState(job, JobStateInternal.REBOOT);
// return the external state as ERROR since this is last retry.
- Assertions.assertEquals(JobState.RUNNING, job.getState());
+ Assert.assertEquals(JobState.RUNNING, job.getState());
when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true);
- Assertions.assertEquals(JobState.ERROR, job.getState());
+ Assert.assertEquals(JobState.ERROR, job.getState());
dispatcher.stop();
commitHandler.stop();
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testKilledDuringSetup() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -372,8 +366,7 @@ public class TestJobImpl {
commitHandler.stop();
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testKilledDuringCommit() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -430,8 +423,7 @@ public class TestJobImpl {
dispatcher.stop();
}
- @Test
- @Timeout(10000)
+ @Test (timeout=10000)
public void testFailAbortDoesntHang() throws IOException {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -469,8 +461,7 @@ public class TestJobImpl {
dispatcher.stop();
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testKilledDuringFailAbort() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -512,8 +503,7 @@ public class TestJobImpl {
commitHandler.stop();
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testKilledDuringKillAbort() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -556,8 +546,7 @@ public class TestJobImpl {
commitHandler.stop();
}
- @Test
- @Timeout(20000)
+ @Test(timeout=20000)
public void testUnusableNodeTransition() throws Exception {
Configuration conf = new Configuration();
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
@@ -610,7 +599,7 @@ public class TestJobImpl {
job.handle(new JobTaskAttemptCompletedEvent(tce));
// complete the task itself
job.handle(new JobTaskEvent(taskId, TaskState.SUCCEEDED));
- Assertions.assertEquals(JobState.RUNNING, job.getState());
+ Assert.assertEquals(JobState.RUNNING, job.getState());
}
}
@@ -710,13 +699,13 @@ public class TestJobImpl {
* much value. Instead, we validate the T_KILL events.
*/
if (killMappers) {
- Assertions.assertEquals(2, killedEvents.size(), "Number of killed events");
- Assertions.assertEquals("task_1234567890000_0001_m_000000",
- killedEvents.get(0).getTaskID().toString(), "AttemptID");
- Assertions.assertEquals("task_1234567890000_0001_m_000001",
- killedEvents.get(1).getTaskID().toString(), "AttemptID");
+ Assert.assertEquals("Number of killed events", 2, killedEvents.size());
+ Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000000",
+ killedEvents.get(0).getTaskID().toString());
+ Assert.assertEquals("AttemptID", "task_1234567890000_0001_m_000001",
+ killedEvents.get(1).getTaskID().toString());
} else {
- Assertions.assertEquals(0, killedEvents.size(), "Number of killed events");
+ Assert.assertEquals("Number of killed events", 0, killedEvents.size());
}
}
@@ -749,8 +738,8 @@ public class TestJobImpl {
// Verify access
JobImpl job1 = new JobImpl(jobId, null, conf1, null, null, null, null, null,
null, null, null, true, user1, 0, null, null, null, null);
- Assertions.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB));
- Assertions.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB));
+ Assert.assertTrue(job1.checkAccess(ugi1, JobACL.VIEW_JOB));
+ Assert.assertFalse(job1.checkAccess(ugi2, JobACL.VIEW_JOB));
// Setup configuration access to the user1 (owner) and user2
Configuration conf2 = new Configuration();
@@ -760,8 +749,8 @@ public class TestJobImpl {
// Verify access
JobImpl job2 = new JobImpl(jobId, null, conf2, null, null, null, null, null,
null, null, null, true, user1, 0, null, null, null, null);
- Assertions.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB));
- Assertions.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB));
+ Assert.assertTrue(job2.checkAccess(ugi1, JobACL.VIEW_JOB));
+ Assert.assertTrue(job2.checkAccess(ugi2, JobACL.VIEW_JOB));
// Setup configuration access with security enabled and access to all
Configuration conf3 = new Configuration();
@@ -771,8 +760,8 @@ public class TestJobImpl {
// Verify access
JobImpl job3 = new JobImpl(jobId, null, conf3, null, null, null, null, null,
null, null, null, true, user1, 0, null, null, null, null);
- Assertions.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB));
- Assertions.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB));
+ Assert.assertTrue(job3.checkAccess(ugi1, JobACL.VIEW_JOB));
+ Assert.assertTrue(job3.checkAccess(ugi2, JobACL.VIEW_JOB));
// Setup configuration access without security enabled
Configuration conf4 = new Configuration();
@@ -782,8 +771,8 @@ public class TestJobImpl {
// Verify access
JobImpl job4 = new JobImpl(jobId, null, conf4, null, null, null, null, null,
null, null, null, true, user1, 0, null, null, null, null);
- Assertions.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB));
- Assertions.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB));
+ Assert.assertTrue(job4.checkAccess(ugi1, JobACL.VIEW_JOB));
+ Assert.assertTrue(job4.checkAccess(ugi2, JobACL.VIEW_JOB));
// Setup configuration access without security enabled
Configuration conf5 = new Configuration();
@@ -793,8 +782,8 @@ public class TestJobImpl {
// Verify access
JobImpl job5 = new JobImpl(jobId, null, conf5, null, null, null, null, null,
null, null, null, true, user1, 0, null, null, null, null);
- Assertions.assertTrue(job5.checkAccess(ugi1, null));
- Assertions.assertTrue(job5.checkAccess(ugi2, null));
+ Assert.assertTrue(job5.checkAccess(ugi1, null));
+ Assert.assertTrue(job5.checkAccess(ugi2, null));
}
@Test
@@ -815,8 +804,8 @@ public class TestJobImpl {
mrAppMetrics, null, true, null, 0, null, mockContext, null, null);
job.handle(diagUpdateEvent);
String diagnostics = job.getReport().getDiagnostics();
- Assertions.assertNotNull(diagnostics);
- Assertions.assertTrue(diagnostics.contains(diagMsg));
+ Assert.assertNotNull(diagnostics);
+ Assert.assertTrue(diagnostics.contains(diagMsg));
job = new JobImpl(jobId, Records
.newRecord(ApplicationAttemptId.class), new Configuration(),
@@ -827,8 +816,8 @@ public class TestJobImpl {
job.handle(new JobEvent(jobId, JobEventType.JOB_KILL));
job.handle(diagUpdateEvent);
diagnostics = job.getReport().getDiagnostics();
- Assertions.assertNotNull(diagnostics);
- Assertions.assertTrue(diagnostics.contains(diagMsg));
+ Assert.assertNotNull(diagnostics);
+ Assert.assertTrue(diagnostics.contains(diagMsg));
}
@Test
@@ -837,13 +826,13 @@ public class TestJobImpl {
// with default values, no of maps is 2
Configuration conf = new Configuration();
boolean isUber = testUberDecision(conf);
- Assertions.assertFalse(isUber);
+ Assert.assertFalse(isUber);
// enable uber mode, no of maps is 2
conf = new Configuration();
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
isUber = testUberDecision(conf);
- Assertions.assertTrue(isUber);
+ Assert.assertTrue(isUber);
// enable uber mode, no of maps is 2, no of reduces is 1 and uber task max
// reduces is 0
@@ -852,7 +841,7 @@ public class TestJobImpl {
conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 0);
conf.setInt(MRJobConfig.NUM_REDUCES, 1);
isUber = testUberDecision(conf);
- Assertions.assertFalse(isUber);
+ Assert.assertFalse(isUber);
// enable uber mode, no of maps is 2, no of reduces is 1 and uber task max
// reduces is 1
@@ -861,14 +850,14 @@ public class TestJobImpl {
conf.setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
conf.setInt(MRJobConfig.NUM_REDUCES, 1);
isUber = testUberDecision(conf);
- Assertions.assertTrue(isUber);
+ Assert.assertTrue(isUber);
// enable uber mode, no of maps is 2 and uber task max maps is 0
conf = new Configuration();
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
conf.setInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 1);
isUber = testUberDecision(conf);
- Assertions.assertFalse(isUber);
+ Assert.assertFalse(isUber);
// enable uber mode of 0 reducer no matter how much memory assigned to reducer
conf = new Configuration();
@@ -877,7 +866,7 @@ public class TestJobImpl {
conf.setInt(MRJobConfig.REDUCE_MEMORY_MB, 2048);
conf.setInt(MRJobConfig.REDUCE_CPU_VCORES, 10);
isUber = testUberDecision(conf);
- Assertions.assertTrue(isUber);
+ Assert.assertTrue(isUber);
}
private boolean testUberDecision(Configuration conf) {
@@ -942,9 +931,9 @@ public class TestJobImpl {
assertJobState(job, JobStateInternal.FAILED);
job.handle(new JobEvent(jobId, JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE));
assertJobState(job, JobStateInternal.FAILED);
- Assertions.assertEquals(JobState.RUNNING, job.getState());
+ Assert.assertEquals(JobState.RUNNING, job.getState());
when(mockContext.hasSuccessfullyUnregistered()).thenReturn(true);
- Assertions.assertEquals(JobState.FAILED, job.getState());
+ Assert.assertEquals(JobState.FAILED, job.getState());
dispatcher.stop();
commitHandler.stop();
@@ -971,12 +960,12 @@ public class TestJobImpl {
JobEvent mockJobEvent = mock(JobEvent.class);
JobStateInternal jobSI = initTransition.transition(job, mockJobEvent);
- Assertions.assertTrue(jobSI.equals(JobStateInternal.NEW),
- "When init fails, return value from InitTransition.transition should equal NEW.");
- Assertions.assertTrue(job.getDiagnostics().toString().contains("YarnRuntimeException"),
- "Job diagnostics should contain YarnRuntimeException");
- Assertions.assertTrue(job.getDiagnostics().toString().contains(EXCEPTIONMSG),
- "Job diagnostics should contain " + EXCEPTIONMSG);
+ Assert.assertTrue("When init fails, return value from InitTransition.transition should equal NEW.",
+ jobSI.equals(JobStateInternal.NEW));
+ Assert.assertTrue("Job diagnostics should contain YarnRuntimeException",
+ job.getDiagnostics().toString().contains("YarnRuntimeException"));
+ Assert.assertTrue("Job diagnostics should contain " + EXCEPTIONMSG,
+ job.getDiagnostics().toString().contains(EXCEPTIONMSG));
}
@Test
@@ -997,7 +986,7 @@ public class TestJobImpl {
assertJobState(job, JobStateInternal.SETUP);
// Update priority of job to 5, and it will be updated
job.setJobPriority(submittedPriority);
- Assertions.assertEquals(submittedPriority, job.getReport().getJobPriority());
+ Assert.assertEquals(submittedPriority, job.getReport().getJobPriority());
job.handle(new JobSetupCompletedEvent(jobId));
assertJobState(job, JobStateInternal.RUNNING);
@@ -1007,10 +996,10 @@ public class TestJobImpl {
job.setJobPriority(updatedPriority);
assertJobState(job, JobStateInternal.RUNNING);
Priority jobPriority = job.getReport().getJobPriority();
- Assertions.assertNotNull(jobPriority);
+ Assert.assertNotNull(jobPriority);
// Verify whether changed priority is same as what is set in Job.
- Assertions.assertEquals(updatedPriority, jobPriority);
+ Assert.assertEquals(updatedPriority, jobPriority);
}
@Test
@@ -1024,14 +1013,14 @@ public class TestJobImpl {
filePolicies.put("file1", true);
filePolicies.put("jar1", true);
Job.setFileSharedCacheUploadPolicies(config, filePolicies);
- Assertions.assertEquals(
+ Assert.assertEquals(
2, Job.getArchiveSharedCacheUploadPolicies(config).size());
- Assertions.assertEquals(
+ Assert.assertEquals(
2, Job.getFileSharedCacheUploadPolicies(config).size());
JobImpl.cleanupSharedCacheUploadPolicies(config);
- Assertions.assertEquals(
+ Assert.assertEquals(
0, Job.getArchiveSharedCacheUploadPolicies(config).size());
- Assertions.assertEquals(
+ Assert.assertEquals(
0, Job.getFileSharedCacheUploadPolicies(config).size());
}
@@ -1099,14 +1088,14 @@ public class TestJobImpl {
job.handle(new JobTaskEvent(
MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP),
TaskState.SUCCEEDED));
- Assertions.assertEquals(JobState.RUNNING, job.getState());
+ Assert.assertEquals(JobState.RUNNING, job.getState());
}
int numReduces = job.getTotalReduces();
for (int i = 0; i < numReduces; ++i) {
job.handle(new JobTaskEvent(
MRBuilderUtils.newTaskId(job.getID(), 1, TaskType.MAP),
TaskState.SUCCEEDED));
- Assertions.assertEquals(JobState.RUNNING, job.getState());
+ Assert.assertEquals(JobState.RUNNING, job.getState());
}
}
@@ -1120,7 +1109,7 @@ public class TestJobImpl {
break;
}
}
- Assertions.assertEquals(state, job.getInternalState());
+ Assert.assertEquals(state, job.getInternalState());
}
private void createSpiedMapTasks(Map
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
index 5e3dfcca7cb..f00ff281f30 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestMapReduceChildJVM.java
@@ -22,7 +22,7 @@ import java.util.ArrayList;
import java.util.Map;
import org.apache.hadoop.mapreduce.TaskType;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
@@ -37,8 +37,7 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -47,9 +46,9 @@ public class TestMapReduceChildJVM {
private static final Logger LOG =
LoggerFactory.getLogger(TestMapReduceChildJVM.class);
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testCommandLine() throws Exception {
+
MyMRApp app = new MyMRApp(1, 0, true, this.getClass().getName(), true);
Configuration conf = new Configuration();
conf.setBoolean(MRConfig.MAPREDUCE_APP_SUBMISSION_CROSS_PLATFORM, true);
@@ -57,7 +56,7 @@ public class TestMapReduceChildJVM {
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertEquals(
+ Assert.assertEquals(
"[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
" -Djava.net.preferIPv4Stack=true" +
" -Dhadoop.metrics.log.level=WARN " +
@@ -72,26 +71,24 @@ public class TestMapReduceChildJVM {
" 0" +
" 1>/stdout" +
" 2>/stderr ]", app.launchCmdList.get(0));
-
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
- "HADOOP_ROOT_LOGGER not set for job");
- Assertions.assertEquals("INFO,console",
+
+ Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
+ Assert.assertEquals("INFO,console",
app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"),
- "HADOOP_CLIENT_OPTS not set for job");
- Assertions.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
+ Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
+ Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
}
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testReduceCommandLineWithSeparateShuffle() throws Exception {
final Configuration conf = new Configuration();
conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true);
testReduceCommandLine(conf);
}
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testReduceCommandLineWithSeparateCRLAShuffle() throws Exception {
final Configuration conf = new Configuration();
conf.setBoolean(MRJobConfig.REDUCE_SEPARATE_SHUFFLE_LOG, true);
@@ -100,8 +97,7 @@ public class TestMapReduceChildJVM {
testReduceCommandLine(conf);
}
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testReduceCommandLine() throws Exception {
final Configuration conf = new Configuration();
testReduceCommandLine(conf);
@@ -123,7 +119,7 @@ public class TestMapReduceChildJVM {
? "shuffleCRLA"
: "shuffleCLA";
- Assertions.assertEquals(
+ Assert.assertEquals(
"[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
" -Djava.net.preferIPv4Stack=true" +
" -Dhadoop.metrics.log.level=WARN " +
@@ -143,17 +139,16 @@ public class TestMapReduceChildJVM {
" 1>/stdout" +
" 2>/stderr ]", app.launchCmdList.get(0));
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
- "HADOOP_ROOT_LOGGER not set for job");
- Assertions.assertEquals("INFO,console",
+ Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
+ Assert.assertEquals("INFO,console",
app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"),
- "HADOOP_CLIENT_OPTS not set for job");
- Assertions.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
+ Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
+ Assert.assertEquals("", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
}
- @Test
- @Timeout(30000)
+ @Test (timeout = 30000)
public void testCommandLineWithLog4JConifg() throws Exception {
MyMRApp app = new MyMRApp(1, 0, true, this.getClass().getName(), true);
@@ -166,7 +161,7 @@ public class TestMapReduceChildJVM {
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertEquals(
+ Assert.assertEquals(
"[" + MRApps.crossPlatformify("JAVA_HOME") + "/bin/java" +
" -Djava.net.preferIPv4Stack=true" +
" -Dhadoop.metrics.log.level=WARN " +
@@ -208,10 +203,10 @@ public class TestMapReduceChildJVM {
MRJobConfig.DEFAULT_HEAP_MEMORY_MB_RATIO);
// Verify map and reduce java opts are not set by default
- Assertions.assertNull(conf.get(MRJobConfig.MAP_JAVA_OPTS),
- "Default map java opts!");
- Assertions.assertNull(conf.get(MRJobConfig.REDUCE_JAVA_OPTS),
- "Default reduce java opts!");
+ Assert.assertNull("Default map java opts!",
+ conf.get(MRJobConfig.MAP_JAVA_OPTS));
+ Assert.assertNull("Default reduce java opts!",
+ conf.get(MRJobConfig.REDUCE_JAVA_OPTS));
// Set the memory-mbs and java-opts
if (mapMb > 0) {
conf.setInt(MRJobConfig.MAP_MEMORY_MB, mapMb);
@@ -247,8 +242,8 @@ public class TestMapReduceChildJVM {
: MRJobConfig.REDUCE_JAVA_OPTS);
heapMb = JobConf.parseMaximumHeapSizeMB(javaOpts);
}
- Assertions.assertEquals(heapMb, JobConf.parseMaximumHeapSizeMB(cmd),
- "Incorrect heapsize in the command opts");
+ Assert.assertEquals("Incorrect heapsize in the command opts",
+ heapMb, JobConf.parseMaximumHeapSizeMB(cmd));
}
}
@@ -293,13 +288,13 @@ public class TestMapReduceChildJVM {
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
- "HADOOP_ROOT_LOGGER not set for job");
- Assertions.assertEquals("WARN,console",
+ Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
+ Assert.assertEquals("WARN,console",
app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"),
- "HADOOP_CLIENT_OPTS not set for job");
- Assertions.assertEquals("test", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
+ Assert.assertTrue("HADOOP_CLIENT_OPTS not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_CLIENT_OPTS"));
+ Assert.assertEquals("test", app.cmdEnvironment.get("HADOOP_CLIENT_OPTS"));
// Try one more.
app = new MyMRApp(1, 0, true, this.getClass().getName(), true);
@@ -309,9 +304,9 @@ public class TestMapReduceChildJVM {
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
- "HADOOP_ROOT_LOGGER not set for job");
- Assertions.assertEquals("trace",
+ Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
+ Assert.assertEquals("trace",
app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
// Try one using the mapreduce.task.env.var=value syntax
@@ -323,9 +318,9 @@ public class TestMapReduceChildJVM {
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
- Assertions.assertTrue(app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"),
- "HADOOP_ROOT_LOGGER not set for job");
- Assertions.assertEquals("DEBUG,console",
+ Assert.assertTrue("HADOOP_ROOT_LOGGER not set for job",
+ app.cmdEnvironment.containsKey("HADOOP_ROOT_LOGGER"));
+ Assert.assertEquals("DEBUG,console",
app.cmdEnvironment.get("HADOOP_ROOT_LOGGER"));
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java
index 64803a7a111..f44ff81079b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestShuffleProvider.java
@@ -53,8 +53,8 @@ import org.apache.hadoop.yarn.server.api.AuxiliaryService;
import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Test;
+import org.junit.Assert;
public class TestShuffleProvider {
@@ -110,12 +110,9 @@ public class TestShuffleProvider {
credentials);
Map serviceDataMap = launchCtx.getServiceData();
- Assertions.assertNotNull(serviceDataMap.get(TestShuffleHandler1.MAPREDUCE_TEST_SHUFFLE_SERVICEID),
- "TestShuffleHandler1 is missing");
- Assertions.assertNotNull(serviceDataMap.get(TestShuffleHandler2.MAPREDUCE_TEST_SHUFFLE_SERVICEID),
- "TestShuffleHandler2 is missing");
- Assertions.assertTrue(serviceDataMap.size() == 3,
- "mismatch number of services in map"); // 2 that we entered + 1 for the built-in shuffle-provider
+ Assert.assertNotNull("TestShuffleHandler1 is missing", serviceDataMap.get(TestShuffleHandler1.MAPREDUCE_TEST_SHUFFLE_SERVICEID));
+ Assert.assertNotNull("TestShuffleHandler2 is missing", serviceDataMap.get(TestShuffleHandler2.MAPREDUCE_TEST_SHUFFLE_SERVICEID));
+ Assert.assertTrue("mismatch number of services in map", serviceDataMap.size() == 3); // 2 that we entered + 1 for the built-in shuffle-provider
}
static public class StubbedFS extends RawLocalFileSystem {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
index cc9b4206f7c..15682eeefc6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
@@ -20,10 +20,9 @@ package org.apache.hadoop.mapreduce.v2.app.job.impl;
import static org.apache.hadoop.test.GenericTestUtils.waitFor;
import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
@@ -42,10 +41,10 @@ import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent;
import org.apache.hadoop.yarn.util.resource.CustomResourceTypesConfigurationProvider;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.BeforeAll;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -112,7 +111,7 @@ import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.spi.LoggingEvent;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
@@ -152,17 +151,17 @@ public class TestTaskAttempt{
}
}
- @BeforeAll
+ @BeforeClass
public static void setupBeforeClass() {
ResourceUtils.resetResourceTypes(new Configuration());
}
- @BeforeEach
+ @Before
public void before() {
TaskAttemptImpl.RESOURCE_REQUEST_CACHE.clear();
}
- @AfterEach
+ @After
public void tearDown() {
ResourceUtils.resetResourceTypes(new Configuration());
}
@@ -290,7 +289,7 @@ public class TestTaskAttempt{
ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class);
verify(eventHandler, times(2)).handle(arg.capture());
if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) {
- Assertions.fail("Second Event not of type ContainerRequestEvent");
+ Assert.fail("Second Event not of type ContainerRequestEvent");
}
ContainerRequestEvent cre =
(ContainerRequestEvent) arg.getAllValues().get(1);
@@ -324,7 +323,7 @@ public class TestTaskAttempt{
ArgumentCaptor arg = ArgumentCaptor.forClass(Event.class);
verify(eventHandler, times(2)).handle(arg.capture());
if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) {
- Assertions.fail("Second Event not of type ContainerRequestEvent");
+ Assert.fail("Second Event not of type ContainerRequestEvent");
}
Map expected = new HashMap();
expected.put("host1", true);
@@ -362,16 +361,16 @@ public class TestTaskAttempt{
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
Map tasks = job.getTasks();
- Assertions.assertEquals(2, tasks.size(), "Num tasks is not correct");
+ Assert.assertEquals("Num tasks is not correct", 2, tasks.size());
Iterator taskIter = tasks.values().iterator();
Task mTask = taskIter.next();
app.waitForState(mTask, TaskState.RUNNING);
Task rTask = taskIter.next();
app.waitForState(rTask, TaskState.RUNNING);
Map mAttempts = mTask.getAttempts();
- Assertions.assertEquals(1, mAttempts.size(), "Num attempts is not correct");
+ Assert.assertEquals("Num attempts is not correct", 1, mAttempts.size());
Map rAttempts = rTask.getAttempts();
- Assertions.assertEquals(1, rAttempts.size(), "Num attempts is not correct");
+ Assert.assertEquals("Num attempts is not correct", 1, rAttempts.size());
TaskAttempt mta = mAttempts.values().iterator().next();
TaskAttempt rta = rAttempts.values().iterator().next();
app.waitForState(mta, TaskAttemptState.RUNNING);
@@ -393,21 +392,21 @@ public class TestTaskAttempt{
int memoryMb = (int) containerResource.getMemorySize();
int vcores = containerResource.getVirtualCores();
- Assertions.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
+ Assert.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue());
- Assertions.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
+ Assert.assertEquals((int) Math.ceil((float) memoryMb / minContainerSize),
counters.findCounter(JobCounter.SLOTS_MILLIS_REDUCES).getValue());
- Assertions.assertEquals(1,
+ Assert.assertEquals(1,
counters.findCounter(JobCounter.MILLIS_MAPS).getValue());
- Assertions.assertEquals(1,
+ Assert.assertEquals(1,
counters.findCounter(JobCounter.MILLIS_REDUCES).getValue());
- Assertions.assertEquals(memoryMb,
+ Assert.assertEquals(memoryMb,
counters.findCounter(JobCounter.MB_MILLIS_MAPS).getValue());
- Assertions.assertEquals(memoryMb,
+ Assert.assertEquals(memoryMb,
counters.findCounter(JobCounter.MB_MILLIS_REDUCES).getValue());
- Assertions.assertEquals(vcores,
+ Assert.assertEquals(vcores,
counters.findCounter(JobCounter.VCORES_MILLIS_MAPS).getValue());
- Assertions.assertEquals(vcores,
+ Assert.assertEquals(vcores,
counters.findCounter(JobCounter.VCORES_MILLIS_REDUCES).getValue());
}
@@ -453,25 +452,23 @@ public class TestTaskAttempt{
app.waitForState(job, JobState.FAILED);
Map tasks = job.getTasks();
- Assertions.assertEquals(1, tasks.size(),
- "Num tasks is not correct");
+ Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
Task task = tasks.values().iterator().next();
- Assertions.assertEquals(TaskState.FAILED, task.getReport().getTaskState(),
- "Task state not correct");
+ Assert.assertEquals("Task state not correct", TaskState.FAILED, task
+ .getReport().getTaskState());
Map attempts = tasks.values().iterator().next()
.getAttempts();
- Assertions.assertEquals(4, attempts.size(),
- "Num attempts is not correct");
+ Assert.assertEquals("Num attempts is not correct", 4, attempts.size());
Iterator it = attempts.values().iterator();
TaskAttemptReport report = it.next().getReport();
- Assertions.assertEquals(TaskAttemptState.FAILED, report.getTaskAttemptState(),
- "Attempt state not correct");
- Assertions.assertEquals("Test Diagnostic Event", report.getDiagnosticInfo(),
- "Diagnostic Information is not Correct");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
+ report.getTaskAttemptState());
+ Assert.assertEquals("Diagnostic Information is not Correct",
+ "Test Diagnostic Event", report.getDiagnosticInfo());
report = it.next().getReport();
- Assertions.assertEquals(TaskAttemptState.FAILED, report.getTaskAttemptState(),
- "Attempt state not correct ");
+ Assert.assertEquals("Attempt state not correct", TaskAttemptState.FAILED,
+ report.getTaskAttemptState());
}
private void testTaskAttemptAssignedFailHistory
@@ -480,8 +477,8 @@ public class TestTaskAttempt{
Job job = app.submit(conf);
app.waitForState(job, JobState.FAILED);
Map tasks = job.getTasks();
- Assertions.assertTrue(app.getTaStartJHEvent(), "No Ta Started JH Event");
- Assertions.assertTrue(app.getTaFailedJHEvent(), "No Ta Failed JH Event");
+ Assert.assertTrue("No Ta Started JH Event", app.getTaStartJHEvent());
+ Assert.assertTrue("No Ta Failed JH Event", app.getTaFailedJHEvent());
}
private void testTaskAttemptAssignedKilledHistory
@@ -521,8 +518,8 @@ public class TestTaskAttempt{
if (event.getType() == org.apache.hadoop.mapreduce.jobhistory.EventType.MAP_ATTEMPT_FAILED) {
TaskAttemptUnsuccessfulCompletion datum = (TaskAttemptUnsuccessfulCompletion) event
.getHistoryEvent().getDatum();
- Assertions.assertEquals("Test Diagnostic Event", datum.get(8).toString(),
- "Diagnostic Information is not Correct");
+ Assert.assertEquals("Diagnostic Information is not Correct",
+ "Test Diagnostic Event", datum.get(8).toString());
}
}
};
@@ -641,8 +638,8 @@ public class TestTaskAttempt{
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED));
assertFalse(eventHandler.internalError);
- assertEquals(Locality.NODE_LOCAL, taImpl.getLocality(),
- "Task attempt is not assigned on the local node");
+ assertEquals("Task attempt is not assigned on the local node",
+ Locality.NODE_LOCAL, taImpl.getLocality());
}
@Test
@@ -698,10 +695,10 @@ public class TestTaskAttempt{
.isEqualTo(TaskAttemptState.RUNNING);
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_CONTAINER_CLEANED));
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
- assertEquals(Locality.RACK_LOCAL, taImpl.getLocality(),
- "Task attempt is not assigned on the local rack");
+ assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+ eventHandler.internalError);
+ assertEquals("Task attempt is not assigned on the local rack",
+ Locality.RACK_LOCAL, taImpl.getLocality());
}
@Test
@@ -760,10 +757,10 @@ public class TestTaskAttempt{
.isEqualTo(TaskAttemptState.COMMIT_PENDING);
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_CONTAINER_CLEANED));
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
- assertEquals(Locality.OFF_SWITCH,taImpl.getLocality(),
- "Task attempt is assigned locally");
+ assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+ eventHandler.internalError);
+ assertEquals("Task attempt is assigned locally", Locality.OFF_SWITCH,
+ taImpl.getLocality());
}
@Test
@@ -835,8 +832,8 @@ public class TestTaskAttempt{
assertThat(taImpl.getState())
.withFailMessage("Task attempt is not in FAILED state, still")
.isEqualTo(TaskAttemptState.FAILED);
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
+ assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+ eventHandler.internalError);
}
@@ -886,15 +883,16 @@ public class TestTaskAttempt{
TaskAttemptEventType.TA_SCHEDULE));
taImpl.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptId,
"Task got killed"));
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task");
+ assertFalse(
+ "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task",
+ eventHandler.internalError);
try {
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_KILL));
- Assertions.assertTrue(true, "No exception on UNASSIGNED STATE KILL event");
+ Assert.assertTrue("No exception on UNASSIGNED STATE KILL event", true);
} catch (Exception e) {
- Assertions.assertFalse(true,
- "Exception not expected for UNASSIGNED STATE KILL event");
+ Assert.assertFalse(
+ "Exception not expected for UNASSIGNED STATE KILL event", true);
}
}
@@ -964,8 +962,8 @@ public class TestTaskAttempt{
assertThat(taImpl.getState())
.withFailMessage("Task attempt is not in KILLED state, still")
.isEqualTo(TaskAttemptState.KILLED);
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_CONTAINER_CLEANED");
+ assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+ eventHandler.internalError);
}
@Test
@@ -1011,8 +1009,9 @@ public class TestTaskAttempt{
when(container.getNodeHttpAddress()).thenReturn("localhost:0");
taImpl.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptId,
"Task got killed"));
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task");
+ assertFalse(
+ "InternalError occurred trying to handle TA_DIAGNOSTICS_UPDATE on assigned task",
+ eventHandler.internalError);
}
@Test
@@ -1073,8 +1072,8 @@ public class TestTaskAttempt{
.withFailMessage("Task attempt is not in SUCCEEDED state")
.isEqualTo(TaskAttemptState.SUCCEEDED);
- assertTrue(taImpl.getFinishTime() > 0,
- "Task Attempt finish time is not greater than 0");
+ assertTrue("Task Attempt finish time is not greater than 0",
+ taImpl.getFinishTime() > 0);
Long finishTime = taImpl.getFinishTime();
Thread.sleep(5);
@@ -1085,9 +1084,9 @@ public class TestTaskAttempt{
.withFailMessage("Task attempt is not in FAILED state")
.isEqualTo(TaskAttemptState.FAILED);
- assertEquals(finishTime, Long.valueOf(taImpl.getFinishTime()),
- "After TA_TOO_MANY_FETCH_FAILURE,"
- + " Task attempt finish time is not the same ");
+ assertEquals("After TA_TOO_MANY_FETCH_FAILURE,"
+ + " Task attempt finish time is not the same ",
+ finishTime, Long.valueOf(taImpl.getFinishTime()));
}
private void containerKillBeforeAssignment(boolean scheduleAttempt)
@@ -1115,7 +1114,7 @@ public class TestTaskAttempt{
assertThat(taImpl.getInternalState())
.withFailMessage("Task attempt's internal state is not KILLED")
.isEqualTo(TaskAttemptStateInternal.KILLED);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
TaskEvent event = eventHandler.lastTaskEvent;
assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType());
// In NEW state, new map attempt should not be rescheduled.
@@ -1239,8 +1238,8 @@ public class TestTaskAttempt{
.isEqualTo(TaskAttemptState.RUNNING);
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_KILL));
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_KILL");
+ assertFalse("InternalError occurred trying to handle TA_KILL",
+ eventHandler.internalError);
assertThat(taImpl.getInternalState())
.withFailMessage("Task should be in KILL_CONTAINER_CLEANUP state")
.isEqualTo(TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP);
@@ -1302,8 +1301,8 @@ public class TestTaskAttempt{
.isEqualTo(TaskAttemptStateInternal.COMMIT_PENDING);
taImpl.handle(new TaskAttemptEvent(attemptId,
TaskAttemptEventType.TA_KILL));
- assertFalse(eventHandler.internalError,
- "InternalError occurred trying to handle TA_KILL");
+ assertFalse("InternalError occurred trying to handle TA_KILL",
+ eventHandler.internalError);
assertThat(taImpl.getInternalState())
.withFailMessage("Task should be in KILL_CONTAINER_CLEANUP state")
.isEqualTo(TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP);
@@ -1349,7 +1348,7 @@ public class TestTaskAttempt{
.withFailMessage("Task attempt is not in KILLED state")
.isEqualTo(TaskAttemptState.KILLED);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
@Test
@@ -1360,30 +1359,32 @@ public class TestTaskAttempt{
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_DONE));
- assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
- "Task attempt is not in SUCCEEDED state");
- assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_FINISHING_CONTAINER");
+ assertEquals("Task attempt is not in SUCCEEDED state",
+ TaskAttemptState.SUCCEEDED, taImpl.getState());
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_FINISHING_CONTAINER",
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+ taImpl.getInternalState());
// If the map only task is killed when it is in SUCCESS_FINISHING_CONTAINER
// state, the state will move to SUCCESS_CONTAINER_CLEANUP
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_KILL));
- assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
- "Task attempt is not in SUCCEEDED state");
- assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_CONTAINER_CLEANUP");
+ assertEquals("Task attempt is not in SUCCEEDED state",
+ TaskAttemptState.SUCCEEDED, taImpl.getState());
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_CONTAINER_CLEANUP",
+ TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
+ taImpl.getInternalState());
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_CONTAINER_CLEANED));
- assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
- "Task attempt is not in SUCCEEDED state");
- assertEquals(TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState(),
- "Task attempt's internal state is not SUCCEEDED state");
+ assertEquals("Task attempt is not in SUCCEEDED state",
+ TaskAttemptState.SUCCEEDED, taImpl.getState());
+ assertEquals("Task attempt's internal state is not SUCCEEDED state",
+ TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState());
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
@Test
@@ -1413,7 +1414,7 @@ public class TestTaskAttempt{
assertThat(taImpl.getInternalState())
.withFailMessage("Task attempt's internal state is not KILLED")
.isEqualTo(TaskAttemptStateInternal.KILLED);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
TaskEvent event = eventHandler.lastTaskEvent;
assertEquals(TaskEventType.T_ATTEMPT_KILLED, event.getType());
// Send an attempt killed event to TaskImpl forwarding the same reschedule
@@ -1429,21 +1430,22 @@ public class TestTaskAttempt{
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_DONE));
- assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
- "Task attempt is not in SUCCEEDED state");
- assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_FINISHING_CONTAINER");
+ assertEquals("Task attempt is not in SUCCEEDED state",
+ TaskAttemptState.SUCCEEDED, taImpl.getState());
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_FINISHING_CONTAINER",
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+ taImpl.getInternalState());
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_CONTAINER_CLEANED));
// Succeeded
taImpl.handle(new TaskAttemptKillEvent(taImpl.getID(),"", true));
- assertEquals(TaskAttemptState.SUCCEEDED, taImpl.getState(),
- "Task attempt is not in SUCCEEDED state");
- assertEquals(TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState(),
- "Task attempt's internal state is not SUCCEEDED");
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertEquals("Task attempt is not in SUCCEEDED state",
+ TaskAttemptState.SUCCEEDED, taImpl.getState());
+ assertEquals("Task attempt's internal state is not SUCCEEDED",
+ TaskAttemptStateInternal.SUCCEEDED, taImpl.getInternalState());
+ assertFalse("InternalError occurred", eventHandler.internalError);
TaskEvent event = eventHandler.lastTaskEvent;
assertEquals(TaskEventType.T_ATTEMPT_SUCCEEDED, event.getType());
}
@@ -1496,7 +1498,7 @@ public class TestTaskAttempt{
.withFailMessage("Task attempt is not in FAILED state")
.isEqualTo(TaskAttemptState.FAILED);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
@Test
@@ -1529,7 +1531,7 @@ public class TestTaskAttempt{
.withFailMessage("Task attempt is not in FAILED state")
.isEqualTo(TaskAttemptState.FAILED);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
@Test
@@ -1559,7 +1561,7 @@ public class TestTaskAttempt{
"SUCCESS_FINISHING_CONTAINER")
.isEqualTo(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
@Test
@@ -1590,7 +1592,7 @@ public class TestTaskAttempt{
"SUCCESS_CONTAINER_CLEANUP")
.isEqualTo(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
@Test
@@ -1617,7 +1619,7 @@ public class TestTaskAttempt{
"FAIL_CONTAINER_CLEANUP")
.isEqualTo(TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP);
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
@Test
@@ -1634,8 +1636,8 @@ public class TestTaskAttempt{
ResourceInformation resourceInfo =
getResourceInfoFromContainerRequest(taImpl, eventHandler).
getResourceInformation(CUSTOM_RESOURCE_NAME);
- assertEquals("G", resourceInfo.getUnits(),
- "Expecting the default unit (G)");
+ assertEquals("Expecting the default unit (G)",
+ "G", resourceInfo.getUnits());
assertEquals(7L, resourceInfo.getValue());
}
@@ -1652,8 +1654,8 @@ public class TestTaskAttempt{
ResourceInformation resourceInfo =
getResourceInfoFromContainerRequest(taImpl, eventHandler).
getResourceInformation(CUSTOM_RESOURCE_NAME);
- assertEquals("m", resourceInfo.getUnits(),
- "Expecting the specified unit (m)");
+ assertEquals("Expecting the specified unit (m)",
+ "m", resourceInfo.getUnits());
assertEquals(3L, resourceInfo.getValue());
}
@@ -1750,20 +1752,18 @@ public class TestTaskAttempt{
}
}
- @Test
+ @Test(expected=IllegalArgumentException.class)
public void testReducerMemoryRequestMultipleName() {
- assertThrows(IllegalArgumentException.class, () -> {
- EventHandler eventHandler = mock(EventHandler.class);
- Clock clock = SystemClock.getInstance();
- JobConf jobConf = new JobConf();
- for (String memoryName : ImmutableList.of(
- MRJobConfig.RESOURCE_TYPE_NAME_MEMORY,
- MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) {
- jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX + memoryName,
- "3Gi");
- }
- createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
- });
+ EventHandler eventHandler = mock(EventHandler.class);
+ Clock clock = SystemClock.getInstance();
+ JobConf jobConf = new JobConf();
+ for (String memoryName : ImmutableList.of(
+ MRJobConfig.RESOURCE_TYPE_NAME_MEMORY,
+ MRJobConfig.RESOURCE_TYPE_ALTERNATIVE_NAME_MEMORY)) {
+ jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX + memoryName,
+ "3Gi");
+ }
+ createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
}
@Test
@@ -1853,24 +1853,21 @@ public class TestTaskAttempt{
containerRequestEvents.add((ContainerRequestEvent) e);
}
}
- assertEquals(1, containerRequestEvents.size(),
- "Expected one ContainerRequestEvent after scheduling "
- + "task attempt");
+ assertEquals("Expected one ContainerRequestEvent after scheduling "
+ + "task attempt", 1, containerRequestEvents.size());
return containerRequestEvents.get(0).getCapability();
}
- @Test
+ @Test(expected=IllegalArgumentException.class)
public void testReducerCustomResourceTypeWithInvalidUnit() {
- assertThrows(IllegalArgumentException.class, () -> {
- initResourceTypes();
- EventHandler eventHandler = mock(EventHandler.class);
- Clock clock = SystemClock.getInstance();
- JobConf jobConf = new JobConf();
- jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX
- + CUSTOM_RESOURCE_NAME, "3z");
- createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
- });
+ initResourceTypes();
+ EventHandler eventHandler = mock(EventHandler.class);
+ Clock clock = SystemClock.getInstance();
+ JobConf jobConf = new JobConf();
+ jobConf.set(MRJobConfig.REDUCE_RESOURCE_TYPE_PREFIX
+ + CUSTOM_RESOURCE_NAME, "3z");
+ createReduceTaskAttemptImplForTest(eventHandler, clock, jobConf);
}
@Test
@@ -1885,19 +1882,22 @@ public class TestTaskAttempt{
// move in two steps to the desired state (cannot get there directly)
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_DONE));
- assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_FINISHING_CONTAINER");
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_FINISHING_CONTAINER",
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+ taImpl.getInternalState());
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_TIMED_OUT));
- assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_CONTAINER_CLEANUP");
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_CONTAINER_CLEANUP",
+ TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
+ taImpl.getInternalState());
taImpl.handle(new TaskAttemptKillEvent(mapTAId, "", true));
- assertEquals(TaskAttemptState.KILLED,
- taImpl.getState(), "Task attempt is not in KILLED state");
+ assertEquals("Task attempt is not in KILLED state",
+ TaskAttemptState.KILLED,
+ taImpl.getState());
}
@Test
@@ -1912,21 +1912,24 @@ public class TestTaskAttempt{
// move in two steps to the desired state (cannot get there directly)
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_DONE));
- assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_FINISHING_CONTAINER");
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_FINISHING_CONTAINER",
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+ taImpl.getInternalState());
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_TIMED_OUT));
- assertEquals(TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_CONTAINER_CLEANUP");
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_CONTAINER_CLEANUP",
+ TaskAttemptStateInternal.SUCCESS_CONTAINER_CLEANUP,
+ taImpl.getInternalState());
taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(taImpl.getID(),
reduceTAId, "Host"));
- assertEquals(TaskAttemptState.FAILED,
- taImpl.getState(), "Task attempt is not in FAILED state");
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertEquals("Task attempt is not in FAILED state",
+ TaskAttemptState.FAILED,
+ taImpl.getState());
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
private void initResourceTypes() {
@@ -1948,15 +1951,17 @@ public class TestTaskAttempt{
taImpl.handle(new TaskAttemptEvent(taImpl.getID(),
TaskAttemptEventType.TA_DONE));
- assertEquals(TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
- taImpl.getInternalState(), "Task attempt's internal state is not " +
- "SUCCESS_FINISHING_CONTAINER");
+ assertEquals("Task attempt's internal state is not " +
+ "SUCCESS_FINISHING_CONTAINER",
+ TaskAttemptStateInternal.SUCCESS_FINISHING_CONTAINER,
+ taImpl.getInternalState());
taImpl.handle(new TaskAttemptTooManyFetchFailureEvent(taImpl.getID(),
reduceTAId, "Host"));
- assertEquals(TaskAttemptState.FAILED,
- taImpl.getState(), "Task attempt is not in FAILED state");
- assertFalse(eventHandler.internalError, "InternalError occurred");
+ assertEquals("Task attempt is not in FAILED state",
+ TaskAttemptState.FAILED,
+ taImpl.getState());
+ assertFalse("InternalError occurred", eventHandler.internalError);
}
private void setupTaskAttemptFinishingMonitor(
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java
index 3939e2e5153..585b949d7f9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java
@@ -27,8 +27,8 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileStatus;
@@ -58,12 +58,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
@SuppressWarnings({"rawtypes"})
public class TestTaskAttemptContainerRequest {
- @AfterEach
+ @After
public void cleanup() {
UserGroupInformation.reset();
}
@@ -114,8 +114,7 @@ public class TestTaskAttemptContainerRequest {
mock(WrappedJvmID.class), taListener,
credentials);
- Assertions.assertEquals(acls, launchCtx.getApplicationACLs(),
- "ACLs mismatch");
+ Assert.assertEquals("ACLs mismatch", acls, launchCtx.getApplicationACLs());
Credentials launchCredentials = new Credentials();
DataInputByteBuffer dibb = new DataInputByteBuffer();
@@ -126,18 +125,17 @@ public class TestTaskAttemptContainerRequest {
for (Token extends TokenIdentifier> token : credentials.getAllTokens()) {
Token extends TokenIdentifier> launchToken =
launchCredentials.getToken(token.getService());
- Assertions.assertNotNull(launchToken,
- "Token " + token.getService() + " is missing");
- Assertions.assertEquals(token, launchToken,
- "Token " + token.getService() + " mismatch");
+ Assert.assertNotNull("Token " + token.getService() + " is missing",
+ launchToken);
+ Assert.assertEquals("Token " + token.getService() + " mismatch",
+ token, launchToken);
}
// verify the secret key is in the launch context
- Assertions.assertNotNull(launchCredentials.getSecretKey(SECRET_KEY_ALIAS),
- "Secret key missing");
- Assertions.assertTrue(Arrays.equals(SECRET_KEY,
- launchCredentials.getSecretKey(SECRET_KEY_ALIAS)),
- "Secret key mismatch");
+ Assert.assertNotNull("Secret key missing",
+ launchCredentials.getSecretKey(SECRET_KEY_ALIAS));
+ Assert.assertTrue("Secret key mismatch", Arrays.equals(SECRET_KEY,
+ launchCredentials.getSecretKey(SECRET_KEY_ALIAS)));
}
static public class StubbedFS extends RawLocalFileSystem {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
index 8cad334d124..1225c4308cc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
@@ -17,10 +17,10 @@
*/
package org.apache.hadoop.mapreduce.v2.app.job.impl;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -65,9 +65,9 @@ import org.apache.hadoop.yarn.event.InlineDispatcher;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -234,7 +234,7 @@ public class TestTaskImpl {
}
- @BeforeEach
+ @Before
@SuppressWarnings("unchecked")
public void setup() {
dispatcher = new InlineDispatcher();
@@ -273,7 +273,7 @@ public class TestTaskImpl {
startCount, metrics, appContext, taskType);
}
- @AfterEach
+ @After
public void teardown() {
taskAttempts.clear();
}
@@ -510,7 +510,7 @@ public class TestTaskImpl {
assertTaskScheduledState();
}
- @Test
+ @Test
public void testTaskProgress() {
LOG.info("--- START: testTaskProgress ---");
mockTask = createMockTask(TaskType.MAP);
@@ -587,10 +587,10 @@ public class TestTaskImpl {
mockTask.handle(new TaskTAttemptEvent(getLastAttempt().getAttemptId(),
TaskEventType.T_ATTEMPT_SUCCEEDED));
- assertFalse(mockTask.canCommit(taskAttempts.get(0).getAttemptId()),
- "First attempt should not commit");
- assertTrue(mockTask.canCommit(getLastAttempt().getAttemptId()),
- "Second attempt should commit");
+ assertFalse("First attempt should not commit",
+ mockTask.canCommit(taskAttempts.get(0).getAttemptId()));
+ assertTrue("Second attempt should commit",
+ mockTask.canCommit(getLastAttempt().getAttemptId()));
assertTaskSucceededState();
}
@@ -879,8 +879,7 @@ public class TestTaskImpl {
baseAttempt.setProgress(1.0f);
Counters taskCounters = mockTask.getCounters();
- assertEquals(specAttemptCounters, taskCounters,
- "wrong counters for task");
+ assertEquals("wrong counters for task", specAttemptCounters, taskCounters);
}
public static class MockTaskAttemptEventHandler implements EventHandler {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
index 3d8f2b849b8..dda93b682b3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationRequest;
import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RestartContainerResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RollbackResponse;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.Server;
@@ -93,8 +93,7 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
import org.apache.hadoop.yarn.util.Records;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -108,9 +107,9 @@ public class TestContainerLauncher {
static final Logger LOG =
LoggerFactory.getLogger(TestContainerLauncher.class);
- @Test
- @Timeout(10000)
+ @Test (timeout = 10000)
public void testPoolSize() throws InterruptedException {
+
ApplicationId appId = ApplicationId.newInstance(12345, 67);
ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
appId, 3);
@@ -128,10 +127,10 @@ public class TestContainerLauncher {
// No events yet
assertThat(containerLauncher.initialPoolSize).isEqualTo(
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREADPOOL_INITIAL_SIZE);
- Assertions.assertEquals(0, threadPool.getPoolSize());
- Assertions.assertEquals(containerLauncher.initialPoolSize,
+ Assert.assertEquals(0, threadPool.getPoolSize());
+ Assert.assertEquals(containerLauncher.initialPoolSize,
threadPool.getCorePoolSize());
- Assertions.assertNull(containerLauncher.foundErrors);
+ Assert.assertNull(containerLauncher.foundErrors);
containerLauncher.expectedCorePoolSize = containerLauncher.initialPoolSize;
for (int i = 0; i < 10; i++) {
@@ -142,8 +141,8 @@ public class TestContainerLauncher {
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
}
waitForEvents(containerLauncher, 10);
- Assertions.assertEquals(10, threadPool.getPoolSize());
- Assertions.assertNull(containerLauncher.foundErrors);
+ Assert.assertEquals(10, threadPool.getPoolSize());
+ Assert.assertNull(containerLauncher.foundErrors);
// Same set of hosts, so no change
containerLauncher.finishEventHandling = true;
@@ -154,7 +153,7 @@ public class TestContainerLauncher {
+ ". Timeout is " + timeOut);
Thread.sleep(1000);
}
- Assertions.assertEquals(10, containerLauncher.numEventsProcessed.get());
+ Assert.assertEquals(10, containerLauncher.numEventsProcessed.get());
containerLauncher.finishEventHandling = false;
for (int i = 0; i < 10; i++) {
ContainerId containerId = ContainerId.newContainerId(appAttemptId,
@@ -166,8 +165,8 @@ public class TestContainerLauncher {
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
}
waitForEvents(containerLauncher, 20);
- Assertions.assertEquals(10, threadPool.getPoolSize());
- Assertions.assertNull(containerLauncher.foundErrors);
+ Assert.assertEquals(10, threadPool.getPoolSize());
+ Assert.assertNull(containerLauncher.foundErrors);
// Different hosts, there should be an increase in core-thread-pool size to
// 21(11hosts+10buffer)
@@ -180,8 +179,8 @@ public class TestContainerLauncher {
containerId, "host11:1234", null,
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
waitForEvents(containerLauncher, 21);
- Assertions.assertEquals(11, threadPool.getPoolSize());
- Assertions.assertNull(containerLauncher.foundErrors);
+ Assert.assertEquals(11, threadPool.getPoolSize());
+ Assert.assertNull(containerLauncher.foundErrors);
containerLauncher.stop();
@@ -195,8 +194,7 @@ public class TestContainerLauncher {
assertThat(containerLauncher.initialPoolSize).isEqualTo(20);
}
- @Test
- @Timeout(5000)
+ @Test(timeout = 5000)
public void testPoolLimits() throws InterruptedException {
ApplicationId appId = ApplicationId.newInstance(12345, 67);
ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
@@ -224,8 +222,8 @@ public class TestContainerLauncher {
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
}
waitForEvents(containerLauncher, 10);
- Assertions.assertEquals(10, threadPool.getPoolSize());
- Assertions.assertNull(containerLauncher.foundErrors);
+ Assert.assertEquals(10, threadPool.getPoolSize());
+ Assert.assertNull(containerLauncher.foundErrors);
// 4 more different hosts, but thread pool size should be capped at 12
containerLauncher.expectedCorePoolSize = 12 ;
@@ -235,14 +233,14 @@ public class TestContainerLauncher {
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH));
}
waitForEvents(containerLauncher, 12);
- Assertions.assertEquals(12, threadPool.getPoolSize());
- Assertions.assertNull(containerLauncher.foundErrors);
+ Assert.assertEquals(12, threadPool.getPoolSize());
+ Assert.assertNull(containerLauncher.foundErrors);
// Make some threads ideal so that remaining events are also done.
containerLauncher.finishEventHandling = true;
waitForEvents(containerLauncher, 14);
- Assertions.assertEquals(12, threadPool.getPoolSize());
- Assertions.assertNull(containerLauncher.foundErrors);
+ Assert.assertEquals(12, threadPool.getPoolSize());
+ Assert.assertNull(containerLauncher.foundErrors);
containerLauncher.stop();
}
@@ -256,13 +254,13 @@ public class TestContainerLauncher {
+ ". It is now " + containerLauncher.numEventsProcessing.get());
Thread.sleep(1000);
}
- Assertions.assertEquals(expectedNumEvents,
+ Assert.assertEquals(expectedNumEvents,
containerLauncher.numEventsProcessing.get());
}
- @Test
- @Timeout(15000)
+ @Test(timeout = 15000)
public void testSlowNM() throws Exception {
+
conf = new Configuration();
int maxAttempts = 1;
conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, maxAttempts);
@@ -292,16 +290,15 @@ public class TestContainerLauncher {
app.waitForState(job, JobState.RUNNING);
Map tasks = job.getTasks();
- Assertions.assertEquals(1, tasks.size(),
- "Num tasks is not correct");
+ Assert.assertEquals("Num tasks is not correct", 1, tasks.size());
Task task = tasks.values().iterator().next();
app.waitForState(task, TaskState.SCHEDULED);
Map attempts = tasks.values().iterator()
.next().getAttempts();
- Assertions.assertEquals(maxAttempts, attempts.size(),
- "Num attempts is not correct");
+ Assert.assertEquals("Num attempts is not correct", maxAttempts,
+ attempts.size());
TaskAttempt attempt = attempts.values().iterator().next();
app.waitForInternalState((TaskAttemptImpl) attempt,
@@ -312,9 +309,9 @@ public class TestContainerLauncher {
String diagnostics = attempt.getDiagnostics().toString();
LOG.info("attempt.getDiagnostics: " + diagnostics);
- Assertions.assertTrue(diagnostics.contains("Container launch failed for "
+ Assert.assertTrue(diagnostics.contains("Container launch failed for "
+ "container_0_0000_01_000000 : "));
- Assertions
+ Assert
.assertTrue(diagnostics
.contains("java.net.SocketTimeoutException: 3000 millis timeout while waiting for channel"));
@@ -443,7 +440,7 @@ public class TestContainerLauncher {
MRApp.newContainerTokenIdentifier(request.getContainerToken());
// Validate that the container is what RM is giving.
- Assertions.assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT,
+ Assert.assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT,
containerTokenIdentifier.getNmHostAddress());
StartContainersResponse response = recordFactory
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
index 136eda213f4..88ba8943ceb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
@@ -79,9 +79,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Before;
+import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -95,7 +94,7 @@ public class TestContainerLauncherImpl {
private Map serviceResponse =
new HashMap();
- @BeforeEach
+ @Before
public void setup() throws IOException {
serviceResponse.clear();
serviceResponse.put(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID,
@@ -169,8 +168,7 @@ public class TestContainerLauncherImpl {
return MRBuilderUtils.newTaskAttemptId(tID, id);
}
- @Test
- @Timeout(5000)
+ @Test(timeout = 5000)
public void testHandle() throws Exception {
LOG.info("STARTING testHandle");
AppContext mockContext = mock(AppContext.class);
@@ -228,8 +226,7 @@ public class TestContainerLauncherImpl {
}
}
- @Test
- @Timeout(5000)
+ @Test(timeout = 5000)
public void testOutOfOrder() throws Exception {
LOG.info("STARTING testOutOfOrder");
AppContext mockContext = mock(AppContext.class);
@@ -303,8 +300,7 @@ public class TestContainerLauncherImpl {
}
}
- @Test
- @Timeout(5000)
+ @Test(timeout = 5000)
public void testMyShutdown() throws Exception {
LOG.info("in test Shutdown");
@@ -356,8 +352,7 @@ public class TestContainerLauncherImpl {
}
@SuppressWarnings({ "rawtypes", "unchecked" })
- @Test
- @Timeout(5000)
+ @Test(timeout = 5000)
public void testContainerCleaned() throws Exception {
LOG.info("STARTING testContainerCleaned");
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
index b5bf4b6e2ff..de4977205b0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/local/TestLocalContainerAllocator.java
@@ -69,8 +69,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.util.resource.Resources;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Test;
import org.mockito.ArgumentCaptor;
public class TestLocalContainerAllocator {
@@ -90,7 +90,7 @@ public class TestLocalContainerAllocator {
lca.start();
try {
lca.heartbeat();
- Assertions.fail("heartbeat was supposed to throw");
+ Assert.fail("heartbeat was supposed to throw");
} catch (YarnException e) {
// YarnException is expected
} finally {
@@ -104,7 +104,7 @@ public class TestLocalContainerAllocator {
lca.start();
try {
lca.heartbeat();
- Assertions.fail("heartbeat was supposed to throw");
+ Assert.fail("heartbeat was supposed to throw");
} catch (YarnRuntimeException e) {
// YarnRuntimeException is expected
} finally {
@@ -172,13 +172,14 @@ public class TestLocalContainerAllocator {
}
}
- Assertions.assertEquals(1, tokenCount, "too many AMRM tokens");
- Assertions.assertArrayEquals(newToken.getIdentifier(), ugiToken.getIdentifier(),
- "token identifier not updated");
- Assertions.assertArrayEquals(newToken.getPassword(), ugiToken.getPassword(),
- "token password not updated");
- Assertions.assertEquals(new Text(ClientRMProxy.getAMRMTokenService(conf)),
- ugiToken.getService(), "AMRM token service not updated");
+ Assert.assertEquals("too many AMRM tokens", 1, tokenCount);
+ Assert.assertArrayEquals("token identifier not updated",
+ newToken.getIdentifier(), ugiToken.getIdentifier());
+ Assert.assertArrayEquals("token password not updated",
+ newToken.getPassword(), ugiToken.getPassword());
+ Assert.assertEquals("AMRM token service not updated",
+ new Text(ClientRMProxy.getAMRMTokenService(conf)),
+ ugiToken.getService());
}
@Test
@@ -201,7 +202,7 @@ public class TestLocalContainerAllocator {
verify(eventHandler, times(1)).handle(containerAssignedCaptor.capture());
Container container = containerAssignedCaptor.getValue().getContainer();
Resource containerResource = container.getResource();
- Assertions.assertNotNull(containerResource);
+ Assert.assertNotNull(containerResource);
assertThat(containerResource.getMemorySize()).isEqualTo(0);
assertThat(containerResource.getVirtualCores()).isEqualTo(0);
}
@@ -281,8 +282,8 @@ public class TestLocalContainerAllocator {
@Override
public AllocateResponse allocate(AllocateRequest request)
throws YarnException, IOException {
- Assertions.assertEquals(responseId, request.getResponseId(),
- "response ID mismatch");
+ Assert.assertEquals("response ID mismatch",
+ responseId, request.getResponseId());
++responseId;
org.apache.hadoop.yarn.api.records.Token yarnToken = null;
if (amToken != null) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
index eaa06b65810..3fd4cb028a5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/metrics/TestMRAppMetrics.java
@@ -25,20 +25,19 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import static org.apache.hadoop.test.MetricsAsserts.*;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Test;
+import org.junit.After;
+import org.junit.Test;
import static org.mockito.Mockito.*;
public class TestMRAppMetrics {
- @AfterEach
+ @After
public void tearDown() {
DefaultMetricsSystem.shutdown();
}
- @Test
- public void testNames() {
+ @Test public void testNames() {
Job job = mock(Job.class);
Task mapTask = mock(Task.class);
when(mapTask.getType()).thenReturn(TaskType.MAP);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java
index 43154339e37..52db7b5f770 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMCommunicator.java
@@ -23,8 +23,7 @@ import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator.AllocatorRunnable;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.util.Clock;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.Timeout;
+import org.junit.Test;
import org.mockito.stubbing.Answer;
import static org.mockito.Mockito.doThrow;
@@ -46,8 +45,7 @@ public class TestRMCommunicator {
}
}
- @Test
- @Timeout(2000)
+ @Test(timeout = 2000)
public void testRMContainerAllocatorExceptionIsHandled() throws Exception {
ClientService mockClientService = mock(ClientService.class);
AppContext mockContext = mock(AppContext.class);
@@ -68,8 +66,7 @@ public class TestRMCommunicator {
testRunnable.run();
}
- @Test
- @Timeout(2000)
+ @Test(timeout = 2000)
public void testRMContainerAllocatorYarnRuntimeExceptionIsHandled()
throws Exception {
ClientService mockClientService = mock(ClientService.class);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
index fe2f3072141..4c64b11372c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
import static org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestCreator.createRequest;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.Assert.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyFloat;
import static org.mockito.ArgumentMatchers.anyInt;
@@ -150,13 +149,12 @@ import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.Records;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.apache.hadoop.yarn.util.SystemClock;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
import java.util.function.Supplier;
-import org.junit.jupiter.api.Timeout;
import org.mockito.InOrder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -169,7 +167,7 @@ public class TestRMContainerAllocator {
static final RecordFactory recordFactory = RecordFactoryProvider
.getRecordFactory(null);
- @BeforeEach
+ @Before
public void setup() {
MyContainerAllocator.getJobUpdatedNodeEvents().clear();
MyContainerAllocator.getTaskAttemptKillEvents().clear();
@@ -178,7 +176,7 @@ public class TestRMContainerAllocator {
UserGroupInformation.setLoginUser(null);
}
- @AfterEach
+ @After
public void tearDown() {
DefaultMetricsSystem.shutdown();
}
@@ -233,8 +231,8 @@ public class TestRMContainerAllocator {
// as nodes are not added, no allocations
List assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
- Assertions.assertEquals(4, rm.getMyFifoScheduler().lastAsk.size());
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
+ Assert.assertEquals(4, rm.getMyFifoScheduler().lastAsk.size());
// send another request with different resource and priority
ContainerRequestEvent event3 = ContainerRequestCreator.createRequest(jobId,
@@ -245,8 +243,8 @@ public class TestRMContainerAllocator {
// as nodes are not added, no allocations
assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
- Assertions.assertEquals(3, rm.getMyFifoScheduler().lastAsk.size());
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
+ Assert.assertEquals(3, rm.getMyFifoScheduler().lastAsk.size());
// update resources in scheduler
nodeManager1.nodeHeartbeat(true); // Node heartbeat
@@ -256,14 +254,14 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, rm.getMyFifoScheduler().lastAsk.size());
+ Assert.assertEquals(0, rm.getMyFifoScheduler().lastAsk.size());
checkAssignments(new ContainerRequestEvent[] {event1, event2, event3},
assigned, false);
// check that the assigned container requests are cancelled
allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(5, rm.getMyFifoScheduler().lastAsk.size());
+ Assert.assertEquals(5, rm.getMyFifoScheduler().lastAsk.size());
}
@Test
@@ -325,7 +323,7 @@ public class TestRMContainerAllocator {
// as nodes are not added, no allocations
List assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// update resources in scheduler
// Node heartbeat from rack-local first. This makes node h3 the first in the
@@ -344,7 +342,7 @@ public class TestRMContainerAllocator {
for(TaskAttemptContainerAssignedEvent event : assigned) {
if(event.getTaskAttemptID().equals(event3.getAttemptID())) {
assigned.remove(event);
- Assertions.assertEquals("h3", event.getContainer().getNodeId().getHost());
+ Assert.assertEquals("h3", event.getContainer().getNodeId().getHost());
break;
}
}
@@ -404,7 +402,7 @@ public class TestRMContainerAllocator {
// as nodes are not added, no allocations
List assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// update resources in scheduler
nodeManager1.nodeHeartbeat(true); // Node heartbeat
@@ -418,8 +416,7 @@ public class TestRMContainerAllocator {
assigned, false);
}
- @Test
- @Timeout(30000)
+ @Test(timeout = 30000)
public void testReducerRampdownDiagnostics() throws Exception {
LOG.info("Running tesReducerRampdownDiagnostics");
@@ -470,12 +467,11 @@ public class TestRMContainerAllocator {
}
final String killEventMessage = allocator.getTaskAttemptKillEvents().get(0)
.getMessage();
- Assertions.assertTrue(killEventMessage.contains(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC),
- "No reducer rampDown preemption message");
+ Assert.assertTrue("No reducer rampDown preemption message",
+ killEventMessage.contains(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC));
}
- @Test
- @Timeout(30000)
+ @Test(timeout = 30000)
public void testPreemptReducers() throws Exception {
LOG.info("Running testPreemptReducers");
@@ -518,12 +514,11 @@ public class TestRMContainerAllocator {
mock(Container.class));
allocator.preemptReducesIfNeeded();
- Assertions.assertEquals(1, assignedRequests.preemptionWaitingReduces.size(),
- "The reducer is not preempted");
+ Assert.assertEquals("The reducer is not preempted",
+ 1, assignedRequests.preemptionWaitingReduces.size());
}
- @Test
- @Timeout(30000)
+ @Test(timeout = 30000)
public void testNonAggressivelyPreemptReducers() throws Exception {
LOG.info("Running testNonAggressivelyPreemptReducers");
@@ -576,17 +571,16 @@ public class TestRMContainerAllocator {
clock.setTime(clock.getTime() + 1);
allocator.preemptReducesIfNeeded();
- Assertions.assertEquals(0, assignedRequests.preemptionWaitingReduces.size(),
- "The reducer is aggressively preeempted");
+ Assert.assertEquals("The reducer is aggressively preeempted", 0,
+ assignedRequests.preemptionWaitingReduces.size());
clock.setTime(clock.getTime() + (preemptThreshold) * 1000);
allocator.preemptReducesIfNeeded();
- Assertions.assertEquals(1, assignedRequests.preemptionWaitingReduces.size(),
- "The reducer is not preeempted");
+ Assert.assertEquals("The reducer is not preeempted", 1,
+ assignedRequests.preemptionWaitingReduces.size());
}
- @Test
- @Timeout(30000)
+ @Test(timeout = 30000)
public void testUnconditionalPreemptReducers() throws Exception {
LOG.info("Running testForcePreemptReducers");
@@ -641,19 +635,18 @@ public class TestRMContainerAllocator {
clock.setTime(clock.getTime() + 1);
allocator.preemptReducesIfNeeded();
- Assertions.assertEquals(0, assignedRequests.preemptionWaitingReduces.size(),
- "The reducer is preeempted too soon");
+ Assert.assertEquals("The reducer is preeempted too soon", 0,
+ assignedRequests.preemptionWaitingReduces.size());
clock.setTime(clock.getTime() + 1000 * forcePreemptThresholdSecs);
allocator.preemptReducesIfNeeded();
- Assertions.assertEquals(1, assignedRequests.preemptionWaitingReduces.size(),
- "The reducer is not preeempted");
+ Assert.assertEquals("The reducer is not preeempted", 1,
+ assignedRequests.preemptionWaitingReduces.size());
}
- @Test
- @Timeout(30000)
+ @Test(timeout = 30000)
public void testExcessReduceContainerAssign() throws Exception {
- final Configuration conf = new Configuration();
+ final Configuration conf = new Configuration();
conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART, 0.0f);
final MyResourceManager2 rm = new MyResourceManager2(conf);
rm.start();
@@ -750,7 +743,7 @@ public class TestRMContainerAllocator {
allocator.schedule();
// verify all of the host-specific asks were sent plus one for the
// default rack and one for the ANY request
- Assertions.assertEquals(3, mockScheduler.lastAsk.size());
+ Assert.assertEquals(3, mockScheduler.lastAsk.size());
// verify ResourceRequest sent for MAP have appropriate node
// label expression as per the configuration
validateLabelsRequests(mockScheduler.lastAsk.get(0), false);
@@ -761,7 +754,7 @@ public class TestRMContainerAllocator {
ContainerId cid0 = mockScheduler.assignContainer("map", false);
allocator.schedule();
// default rack and one for the ANY request
- Assertions.assertEquals(3, mockScheduler.lastAsk.size());
+ Assert.assertEquals(3, mockScheduler.lastAsk.size());
validateLabelsRequests(mockScheduler.lastAsk.get(0), true);
validateLabelsRequests(mockScheduler.lastAsk.get(1), true);
validateLabelsRequests(mockScheduler.lastAsk.get(2), true);
@@ -776,14 +769,14 @@ public class TestRMContainerAllocator {
case "map":
case "reduce":
case NetworkTopology.DEFAULT_RACK:
- Assertions.assertNull(resourceRequest.getNodeLabelExpression());
+ Assert.assertNull(resourceRequest.getNodeLabelExpression());
break;
case "*":
- Assertions.assertEquals(isReduce ? "ReduceNodes" : "MapNodes",
+ Assert.assertEquals(isReduce ? "ReduceNodes" : "MapNodes",
resourceRequest.getNodeLabelExpression());
break;
default:
- Assertions.fail("Invalid resource location "
+ Assert.fail("Invalid resource location "
+ resourceRequest.getResourceName());
}
}
@@ -937,7 +930,7 @@ public class TestRMContainerAllocator {
// as nodes are not added, no allocations
List assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// update resources in scheduler
nodeManager1.nodeHeartbeat(true); // Node heartbeat
@@ -952,8 +945,8 @@ public class TestRMContainerAllocator {
// validate that no container is assigned to h1 as it doesn't have 2048
for (TaskAttemptContainerAssignedEvent assig : assigned) {
- Assertions.assertFalse("h1".equals(assig.getContainer().getNodeId().getHost()),
- "Assigned count not correct");
+ Assert.assertFalse("Assigned count not correct", "h1".equals(assig
+ .getContainer().getNodeId().getHost()));
}
}
@@ -1044,7 +1037,7 @@ public class TestRMContainerAllocator {
};
};
- Assertions.assertEquals(0.0, rmApp.getProgress(), 0.0);
+ Assert.assertEquals(0.0, rmApp.getProgress(), 0.0);
mrApp.submit(conf);
Job job = mrApp.getContext().getAllJobs().entrySet().iterator().next()
@@ -1083,23 +1076,23 @@ public class TestRMContainerAllocator {
allocator.schedule(); // Send heartbeat
rm.drainEvents();
- Assertions.assertEquals(0.05f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.05f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.05f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.05f, rmApp.getProgress(), 0.001f);
// Finish off 1 map.
Iterator it = job.getTasks().values().iterator();
finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 1);
allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0.095f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.095f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.095f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.095f, rmApp.getProgress(), 0.001f);
// Finish off 7 more so that map-progress is 80%
finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 7);
allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0.41f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.41f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.41f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.41f, rmApp.getProgress(), 0.001f);
// Finish off the 2 remaining maps
finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 2);
@@ -1123,16 +1116,16 @@ public class TestRMContainerAllocator {
allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0.59f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.59f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.59f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.59f, rmApp.getProgress(), 0.001f);
// Finish off the remaining 8 reduces.
finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 8);
allocator.schedule();
rm.drainEvents();
// Remaining is JobCleanup
- Assertions.assertEquals(0.95f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.95f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.95f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.95f, rmApp.getProgress(), 0.001f);
}
private void finishNextNTasks(DrainDispatcher rmDispatcher, MockNM node,
@@ -1196,7 +1189,7 @@ public class TestRMContainerAllocator {
};
};
- Assertions.assertEquals(0.0, rmApp.getProgress(), 0.0);
+ Assert.assertEquals(0.0, rmApp.getProgress(), 0.0);
mrApp.submit(conf);
Job job = mrApp.getContext().getAllJobs().entrySet().iterator().next()
@@ -1231,8 +1224,8 @@ public class TestRMContainerAllocator {
allocator.schedule(); // Send heartbeat
rm.drainEvents();
- Assertions.assertEquals(0.05f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.05f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.05f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.05f, rmApp.getProgress(), 0.001f);
Iterator it = job.getTasks().values().iterator();
@@ -1240,22 +1233,22 @@ public class TestRMContainerAllocator {
finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 1);
allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0.14f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.14f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.14f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.14f, rmApp.getProgress(), 0.001f);
// Finish off 5 more map so that map-progress is 60%
finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 5);
allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0.59f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.59f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.59f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.59f, rmApp.getProgress(), 0.001f);
// Finish off remaining map so that map-progress is 100%
finishNextNTasks(rmDispatcher, amNodeManager, mrApp, it, 4);
allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0.95f, job.getProgress(), 0.001f);
- Assertions.assertEquals(0.95f, rmApp.getProgress(), 0.001f);
+ Assert.assertEquals(0.95f, job.getProgress(), 0.001f);
+ Assert.assertEquals(0.95f, rmApp.getProgress(), 0.001f);
}
@Test
@@ -1306,17 +1299,17 @@ public class TestRMContainerAllocator {
nm1.nodeHeartbeat(true);
rm.drainEvents();
- Assertions.assertEquals(1, allocator.getJobUpdatedNodeEvents().size());
- Assertions.assertEquals(3, allocator.getJobUpdatedNodeEvents().get(0).getUpdatedNodes().size());
+ Assert.assertEquals(1, allocator.getJobUpdatedNodeEvents().size());
+ Assert.assertEquals(3, allocator.getJobUpdatedNodeEvents().get(0).getUpdatedNodes().size());
allocator.getJobUpdatedNodeEvents().clear();
// get the assignment
assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(1, assigned.size());
- Assertions.assertEquals(nm1.getNodeId(), assigned.get(0).getContainer().getNodeId());
+ Assert.assertEquals(1, assigned.size());
+ Assert.assertEquals(nm1.getNodeId(), assigned.get(0).getContainer().getNodeId());
// no updated nodes reported
- Assertions.assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty());
- Assertions.assertTrue(allocator.getTaskAttemptKillEvents().isEmpty());
+ Assert.assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty());
+ Assert.assertTrue(allocator.getTaskAttemptKillEvents().isEmpty());
// mark nodes bad
nm1.nodeHeartbeat(false);
@@ -1326,23 +1319,23 @@ public class TestRMContainerAllocator {
// schedule response returns updated nodes
assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size());
+ Assert.assertEquals(0, assigned.size());
// updated nodes are reported
- Assertions.assertEquals(1, allocator.getJobUpdatedNodeEvents().size());
- Assertions.assertEquals(1, allocator.getTaskAttemptKillEvents().size());
- Assertions.assertEquals(2,
+ Assert.assertEquals(1, allocator.getJobUpdatedNodeEvents().size());
+ Assert.assertEquals(1, allocator.getTaskAttemptKillEvents().size());
+ Assert.assertEquals(2,
allocator.getJobUpdatedNodeEvents().get(0).getUpdatedNodes().size());
- Assertions.assertEquals(attemptId,
+ Assert.assertEquals(attemptId,
allocator.getTaskAttemptKillEvents().get(0).getTaskAttemptID());
allocator.getJobUpdatedNodeEvents().clear();
allocator.getTaskAttemptKillEvents().clear();
assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size());
+ Assert.assertEquals(0, assigned.size());
// no updated nodes reported
- Assertions.assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty());
- Assertions.assertTrue(allocator.getTaskAttemptKillEvents().isEmpty());
+ Assert.assertTrue(allocator.getJobUpdatedNodeEvents().isEmpty());
+ Assert.assertTrue(allocator.getTaskAttemptKillEvents().isEmpty());
}
@Test
@@ -1411,7 +1404,7 @@ public class TestRMContainerAllocator {
// as nodes are not added, no allocations
List assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// Send events to blacklist nodes h1 and h2
ContainerFailedEvent f1 = createFailEvent(jobId, 1, "h1", false);
@@ -1425,9 +1418,9 @@ public class TestRMContainerAllocator {
rm.drainEvents();
assigned = allocator.schedule();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
assertBlacklistAdditionsAndRemovals(2, 0, rm);
// mark h1/h2 as bad nodes
@@ -1438,7 +1431,7 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
rm.drainEvents();
assertBlacklistAdditionsAndRemovals(0, 0, rm);
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
nodeManager3.nodeHeartbeat(true); // Node heartbeat
rm.drainEvents();
@@ -1446,12 +1439,12 @@ public class TestRMContainerAllocator {
rm.drainEvents();
assertBlacklistAdditionsAndRemovals(0, 0, rm);
- Assertions.assertTrue(assigned.size() == 3, "No of assignments must be 3");
+ Assert.assertTrue("No of assignments must be 3", assigned.size() == 3);
// validate that all containers are assigned to h3
for (TaskAttemptContainerAssignedEvent assig : assigned) {
- Assertions.assertTrue("h3".equals(assig.getContainer().getNodeId().getHost()),
- "Assigned container host not correct");
+ Assert.assertTrue("Assigned container host not correct", "h3".equals(assig
+ .getContainer().getNodeId().getHost()));
}
}
@@ -1496,8 +1489,7 @@ public class TestRMContainerAllocator {
assigned =
getContainerOnHost(jobId, 1, 1024, new String[] {"h1"},
nodeManagers[0], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(1, assigned.size(),
- "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
LOG.info("Failing container _1 on H1 (Node should be blacklisted and"
+ " ignore blacklisting enabled");
@@ -1512,51 +1504,47 @@ public class TestRMContainerAllocator {
assigned =
getContainerOnHost(jobId, 2, 1024, new String[] {"h1"},
nodeManagers[0], allocator, 1, 0, 0, 1, rm);
- Assertions.assertEquals(0, assigned.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// Known=1, blacklisted=1, ignore should be true - assign 1
assigned =
getContainerOnHost(jobId, 2, 1024, new String[] {"h1"},
nodeManagers[0], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(1, assigned.size(),
- "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
nodeManagers[nmNum] = registerNodeManager(nmNum++, rm);
// Known=2, blacklisted=1, ignore should be true - assign 1 anyway.
assigned =
getContainerOnHost(jobId, 3, 1024, new String[] {"h2"},
nodeManagers[1], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(1, assigned.size(),
- "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
nodeManagers[nmNum] = registerNodeManager(nmNum++, rm);
// Known=3, blacklisted=1, ignore should be true - assign 1 anyway.
assigned =
getContainerOnHost(jobId, 4, 1024, new String[] {"h3"},
nodeManagers[2], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(1, assigned.size(),
- "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
// Known=3, blacklisted=1, ignore should be true - assign 1
assigned =
getContainerOnHost(jobId, 5, 1024, new String[] {"h1"},
nodeManagers[0], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(1, assigned.size(), "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
nodeManagers[nmNum] = registerNodeManager(nmNum++, rm);
// Known=4, blacklisted=1, ignore should be false - assign 1 anyway
assigned =
getContainerOnHost(jobId, 6, 1024, new String[] {"h4"},
nodeManagers[3], allocator, 0, 0, 1, 0, rm);
- Assertions.assertEquals(1, assigned.size(), "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
// Test blacklisting re-enabled.
// Known=4, blacklisted=1, ignore should be false - no assignment on h1
assigned =
getContainerOnHost(jobId, 7, 1024, new String[] {"h1"},
nodeManagers[0], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// RMContainerRequestor would have created a replacement request.
// Blacklist h2
@@ -1569,20 +1557,20 @@ public class TestRMContainerAllocator {
assigned =
getContainerOnHost(jobId, 8, 1024, new String[] {"h1"},
nodeManagers[0], allocator, 1, 0, 0, 2, rm);
- Assertions.assertEquals(0, assigned.size(), "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// Known=4, blacklisted=2, ignore should be true. Should assign 2
// containers.
assigned =
getContainerOnHost(jobId, 8, 1024, new String[] {"h1"},
nodeManagers[0], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(2, assigned.size(), "No of assignments must be 2");
+ Assert.assertEquals("No of assignments must be 2", 2, assigned.size());
// Known=4, blacklisted=2, ignore should be true.
assigned =
getContainerOnHost(jobId, 9, 1024, new String[] {"h2"},
nodeManagers[1], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(1, assigned.size(), "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
// Test blacklist while ignore blacklisting enabled
ContainerFailedEvent f3 = createFailEvent(jobId, 4, "h3", false);
@@ -1593,7 +1581,7 @@ public class TestRMContainerAllocator {
assigned =
getContainerOnHost(jobId, 10, 1024, new String[] {"h3"},
nodeManagers[2], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(1, assigned.size(), "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
// Assign on 5 more nodes - to re-enable blacklisting
for (int i = 0; i < 5; i++) {
@@ -1602,15 +1590,14 @@ public class TestRMContainerAllocator {
getContainerOnHost(jobId, 11 + i, 1024,
new String[] {String.valueOf(5 + i)}, nodeManagers[4 + i],
allocator, 0, 0, (i == 4 ? 3 : 0), 0, rm);
- Assertions.assertEquals(1, assigned.size(), "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
}
// Test h3 (blacklisted while ignoring blacklisting) is blacklisted.
assigned =
getContainerOnHost(jobId, 20, 1024, new String[] {"h3"},
nodeManagers[2], allocator, 0, 0, 0, 0, rm);
- Assertions.assertEquals(0, assigned.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
}
private MockNM registerNodeManager(int i, MyResourceManager rm)
@@ -1637,8 +1624,7 @@ public class TestRMContainerAllocator {
rm.drainEvents();
assertBlacklistAdditionsAndRemovals(
expectedAdditions1, expectedRemovals1, rm);
- Assertions.assertEquals(0, assigned.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// Heartbeat from the required nodeManager
mockNM.nodeHeartbeat(true);
@@ -1703,8 +1689,7 @@ public class TestRMContainerAllocator {
// as nodes are not added, no allocations
List assigned = allocator.schedule();
rm.drainEvents();
- Assertions.assertEquals(0, assigned.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
LOG.info("h1 Heartbeat (To actually schedule the containers)");
// update resources in scheduler
@@ -1715,8 +1700,7 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
rm.drainEvents();
assertBlacklistAdditionsAndRemovals(0, 0, rm);
- Assertions.assertEquals(1, assigned.size(),
- "No of assignments must be 1");
+ Assert.assertEquals("No of assignments must be 1", 1, assigned.size());
LOG.info("Failing container _1 on H1 (should blacklist the node)");
// Send events to blacklist nodes h1 and h2
@@ -1734,8 +1718,7 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
rm.drainEvents();
assertBlacklistAdditionsAndRemovals(1, 0, rm);
- Assertions.assertEquals(0, assigned.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
// send another request with different resource and priority
ContainerRequestEvent event3 =
@@ -1756,8 +1739,7 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
rm.drainEvents();
assertBlacklistAdditionsAndRemovals(0, 0, rm);
- Assertions.assertEquals(0, assigned.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
//RMContainerAllocator gets assigned a p:5 on a blacklisted node.
@@ -1766,8 +1748,7 @@ public class TestRMContainerAllocator {
assigned = allocator.schedule();
rm.drainEvents();
assertBlacklistAdditionsAndRemovals(0, 0, rm);
- Assertions.assertEquals(0, assigned.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0, assigned.size());
//Hearbeat from H3 to schedule on this host.
LOG.info("h3 Heartbeat (To re-schedule the containers)");
@@ -1786,29 +1767,27 @@ public class TestRMContainerAllocator {
" with priority " + assig.getContainer().getPriority());
}
- Assertions.assertEquals(2, assigned.size(),
- "No of assignments must be 2");
+ Assert.assertEquals("No of assignments must be 2", 2, assigned.size());
// validate that all containers are assigned to h3
for (TaskAttemptContainerAssignedEvent assig : assigned) {
- Assertions.assertEquals("h3", assig.getContainer().getNodeId().getHost(),
- "Assigned container " + assig.getContainer().getId()
- + " host not correct");
+ Assert.assertEquals("Assigned container " + assig.getContainer().getId()
+ + " host not correct", "h3", assig.getContainer().getNodeId().getHost());
}
}
private static void assertBlacklistAdditionsAndRemovals(
int expectedAdditions, int expectedRemovals, MyResourceManager rm) {
- Assertions.assertEquals(expectedAdditions,
+ Assert.assertEquals(expectedAdditions,
rm.getMyFifoScheduler().lastBlacklistAdditions.size());
- Assertions.assertEquals(expectedRemovals,
+ Assert.assertEquals(expectedRemovals,
rm.getMyFifoScheduler().lastBlacklistRemovals.size());
}
private static void assertAsksAndReleases(int expectedAsk,
int expectedRelease, MyResourceManager rm) {
- Assertions.assertEquals(expectedAsk, rm.getMyFifoScheduler().lastAsk.size());
- Assertions.assertEquals(expectedRelease,
+ Assert.assertEquals(expectedAsk, rm.getMyFifoScheduler().lastAsk.size());
+ Assert.assertEquals(expectedRelease,
rm.getMyFifoScheduler().lastRelease.size());
}
@@ -1951,17 +1930,17 @@ public class TestRMContainerAllocator {
private void checkAssignments(ContainerRequestEvent[] requests,
List assignments,
boolean checkHostMatch) {
- Assertions.assertNotNull(assignments, "Container not assigned");
- Assertions.assertEquals(requests.length, assignments.size(),
- "Assigned count not correct");
+ Assert.assertNotNull("Container not assigned", assignments);
+ Assert.assertEquals("Assigned count not correct", requests.length,
+ assignments.size());
// check for uniqueness of containerIDs
Set containerIds = new HashSet();
for (TaskAttemptContainerAssignedEvent assigned : assignments) {
containerIds.add(assigned.getContainer().getId());
}
- Assertions.assertEquals(assignments.size(), containerIds.size(),
- "Assigned containers must be different");
+ Assert.assertEquals("Assigned containers must be different", assignments
+ .size(), containerIds.size());
// check for all assignment
for (ContainerRequestEvent req : requests) {
@@ -1978,14 +1957,14 @@ public class TestRMContainerAllocator {
private void checkAssignment(ContainerRequestEvent request,
TaskAttemptContainerAssignedEvent assigned, boolean checkHostMatch) {
- Assertions.assertNotNull(assigned, "Nothing assigned to attempt "
- + request.getAttemptID());
- Assertions.assertEquals(request.getAttemptID(), assigned.getTaskAttemptID(),
- "assigned to wrong attempt");
+ Assert.assertNotNull("Nothing assigned to attempt "
+ + request.getAttemptID(), assigned);
+ Assert.assertEquals("assigned to wrong attempt", request.getAttemptID(),
+ assigned.getTaskAttemptID());
if (checkHostMatch) {
- Assertions.assertTrue(Arrays.asList(request.getHosts()).contains(
- assigned.getContainer().getNodeId().getHost()),
- "Not assigned to requested host");
+ Assert.assertTrue("Not assigned to requested host", Arrays.asList(
+ request.getHosts()).contains(
+ assigned.getContainer().getNodeId().getHost()));
}
}
@@ -2372,13 +2351,13 @@ public class TestRMContainerAllocator {
allocator.recalculatedReduceSchedule = false;
allocator.schedule();
- Assertions.assertFalse(allocator.recalculatedReduceSchedule,
- "Unexpected recalculate of reduce schedule");
+ Assert.assertFalse("Unexpected recalculate of reduce schedule",
+ allocator.recalculatedReduceSchedule);
doReturn(1).when(job).getCompletedMaps();
allocator.schedule();
- Assertions.assertTrue(allocator.recalculatedReduceSchedule,
- "Expected recalculate of reduce schedule");
+ Assert.assertTrue("Expected recalculate of reduce schedule",
+ allocator.recalculatedReduceSchedule);
}
@Test
@@ -2416,14 +2395,14 @@ public class TestRMContainerAllocator {
Thread.sleep(10);
timeToWaitMs -= 10;
}
- Assertions.assertEquals(5, allocator.getLastHeartbeatTime());
+ Assert.assertEquals(5, allocator.getLastHeartbeatTime());
clock.setTime(7);
timeToWaitMs = 5000;
while (allocator.getLastHeartbeatTime() != 7 && timeToWaitMs > 0) {
Thread.sleep(10);
timeToWaitMs -= 10;
}
- Assertions.assertEquals(7, allocator.getLastHeartbeatTime());
+ Assert.assertEquals(7, allocator.getLastHeartbeatTime());
final AtomicBoolean callbackCalled = new AtomicBoolean(false);
allocator.runOnNextHeartbeat(new Runnable() {
@@ -2438,8 +2417,8 @@ public class TestRMContainerAllocator {
Thread.sleep(10);
timeToWaitMs -= 10;
}
- Assertions.assertEquals(8, allocator.getLastHeartbeatTime());
- Assertions.assertTrue(callbackCalled.get());
+ Assert.assertEquals(8, allocator.getLastHeartbeatTime());
+ Assert.assertTrue(callbackCalled.get());
}
@Test
@@ -2467,12 +2446,12 @@ public class TestRMContainerAllocator {
TaskAttemptEvent event = allocator.createContainerFinishedEvent(status,
attemptId);
- Assertions.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
+ Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
event.getType());
TaskAttemptEvent abortedEvent = allocator.createContainerFinishedEvent(
abortedStatus, attemptId);
- Assertions.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent.getType());
+ Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent.getType());
// PREEMPTED
ContainerId containerId2 =
@@ -2485,12 +2464,12 @@ public class TestRMContainerAllocator {
TaskAttemptEvent event2 = allocator.createContainerFinishedEvent(status2,
attemptId);
- Assertions.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
+ Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
event2.getType());
TaskAttemptEvent abortedEvent2 = allocator.createContainerFinishedEvent(
preemptedStatus, attemptId);
- Assertions.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent2.getType());
+ Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent2.getType());
// KILLED_BY_CONTAINER_SCHEDULER
ContainerId containerId3 =
@@ -2504,12 +2483,12 @@ public class TestRMContainerAllocator {
TaskAttemptEvent event3 = allocator.createContainerFinishedEvent(status3,
attemptId);
- Assertions.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
+ Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
event3.getType());
TaskAttemptEvent abortedEvent3 = allocator.createContainerFinishedEvent(
killedByContainerSchedulerStatus, attemptId);
- Assertions.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent3.getType());
+ Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent3.getType());
}
@Test
@@ -2550,9 +2529,9 @@ public class TestRMContainerAllocator {
MyContainerAllocator allocator =
(MyContainerAllocator) mrApp.getContainerAllocator();
amDispatcher.await();
- Assertions.assertTrue(allocator.isApplicationMasterRegistered());
+ Assert.assertTrue(allocator.isApplicationMasterRegistered());
mrApp.stop();
- Assertions.assertTrue(allocator.isUnregistered());
+ Assert.assertTrue(allocator.isUnregistered());
}
// Step-1 : AM send allocate request for 2 ContainerRequests and 1
@@ -2632,8 +2611,8 @@ public class TestRMContainerAllocator {
List assignedContainers =
allocator.schedule();
rm1.drainEvents();
- Assertions.assertEquals(0, assignedContainers.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0,
+ assignedContainers.size());
// Why ask is 3, not 4? --> ask from blacklisted node h2 is removed
assertAsksAndReleases(3, 0, rm1);
assertBlacklistAdditionsAndRemovals(1, 0, rm1);
@@ -2644,14 +2623,14 @@ public class TestRMContainerAllocator {
// Step-2 : 2 containers are allocated by RM.
assignedContainers = allocator.schedule();
rm1.drainEvents();
- Assertions.assertEquals(2, assignedContainers.size(),
- "No of assignments must be 2");
+ Assert.assertEquals("No of assignments must be 2", 2,
+ assignedContainers.size());
assertAsksAndReleases(0, 0, rm1);
assertBlacklistAdditionsAndRemovals(0, 0, rm1);
assignedContainers = allocator.schedule();
- Assertions.assertEquals(0, assignedContainers.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0,
+ assignedContainers.size());
assertAsksAndReleases(3, 0, rm1);
assertBlacklistAdditionsAndRemovals(0, 0, rm1);
@@ -2670,8 +2649,8 @@ public class TestRMContainerAllocator {
allocator.sendDeallocate(deallocate1);
assignedContainers = allocator.schedule();
- Assertions.assertEquals(0, assignedContainers.size(),
- "No of assignments must be 0");
+ Assert.assertEquals("No of assignments must be 0", 0,
+ assignedContainers.size());
assertAsksAndReleases(3, 1, rm1);
assertBlacklistAdditionsAndRemovals(0, 0, rm1);
@@ -2683,7 +2662,7 @@ public class TestRMContainerAllocator {
// NM should be rebooted on heartbeat, even first heartbeat for nm2
NodeHeartbeatResponse hbResponse = nm1.nodeHeartbeat(true);
- Assertions.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction());
+ Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction());
// new NM to represent NM re-register
nm1 = new MockNM("h1:1234", 10240, rm2.getResourceTrackerService());
@@ -2736,12 +2715,12 @@ public class TestRMContainerAllocator {
assignedContainers = allocator.schedule();
rm2.drainEvents();
- Assertions.assertEquals(3, assignedContainers.size(),
- "Number of container should be 3");
+ Assert.assertEquals("Number of container should be 3", 3,
+ assignedContainers.size());
for (TaskAttemptContainerAssignedEvent assig : assignedContainers) {
- Assertions.assertTrue("h1".equals(assig.getContainer().getNodeId().getHost()),
- "Assigned count not correct");
+ Assert.assertTrue("Assigned count not correct",
+ "h1".equals(assig.getContainer().getNodeId().getHost()));
}
rm1.stop();
@@ -2785,7 +2764,7 @@ public class TestRMContainerAllocator {
allocator.sendRequests(Arrays.asList(mapRequestEvt));
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskMap);
}
@Test
@@ -2828,7 +2807,7 @@ public class TestRMContainerAllocator {
allocator.scheduleAllReduces();
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskReduce);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskReduce);
}
@Test
@@ -2863,20 +2842,19 @@ public class TestRMContainerAllocator {
allocator.jobEvents.clear();
try {
allocator.schedule();
- Assertions.fail("Should Have Exception");
+ Assert.fail("Should Have Exception");
} catch (RMContainerAllocationException e) {
- Assertions.assertTrue(e.getMessage().contains("Could not contact RM after"));
+ Assert.assertTrue(e.getMessage().contains("Could not contact RM after"));
}
rm1.drainEvents();
- Assertions.assertEquals(1, allocator.jobEvents.size(),
- "Should Have 1 Job Event");
+ Assert.assertEquals("Should Have 1 Job Event", 1,
+ allocator.jobEvents.size());
JobEvent event = allocator.jobEvents.get(0);
- Assertions.assertTrue(event.getType().equals(JobEventType.JOB_AM_REBOOT),
- "Should Reboot");
+ Assert.assertTrue("Should Reboot",
+ event.getType().equals(JobEventType.JOB_AM_REBOOT));
}
- @Test
- @Timeout(60000)
+ @Test(timeout=60000)
public void testAMRMTokenUpdate() throws Exception {
LOG.info("Running testAMRMTokenUpdate");
@@ -2914,7 +2892,7 @@ public class TestRMContainerAllocator {
final Token oldToken = rm.getRMContext().getRMApps()
.get(appId).getRMAppAttempt(appAttemptId).getAMRMToken();
- Assertions.assertNotNull(oldToken, "app should have a token");
+ Assert.assertNotNull("app should have a token", oldToken);
UserGroupInformation testUgi = UserGroupInformation.createUserForTesting(
"someuser", new String[0]);
Token newToken = testUgi.doAs(
@@ -2929,7 +2907,7 @@ public class TestRMContainerAllocator {
long startTime = Time.monotonicNow();
while (currentToken == oldToken) {
if (Time.monotonicNow() - startTime > 20000) {
- Assertions.fail("Took to long to see AMRM token change");
+ Assert.fail("Took to long to see AMRM token change");
}
Thread.sleep(100);
allocator.schedule();
@@ -2952,13 +2930,13 @@ public class TestRMContainerAllocator {
}
}
- Assertions.assertEquals(1, tokenCount, "too many AMRM tokens");
- Assertions.assertArrayEquals(newToken.getIdentifier(), ugiToken.getIdentifier(),
- "token identifier not updated");
- Assertions.assertArrayEquals(newToken.getPassword(), ugiToken.getPassword(),
- "token password not updated");
- Assertions.assertEquals(new Text(rmAddr), ugiToken.getService(),
- "AMRM token service not updated");
+ Assert.assertEquals("too many AMRM tokens", 1, tokenCount);
+ Assert.assertArrayEquals("token identifier not updated",
+ newToken.getIdentifier(), ugiToken.getIdentifier());
+ Assert.assertArrayEquals("token password not updated",
+ newToken.getPassword(), ugiToken.getPassword());
+ Assert.assertEquals("AMRM token service not updated",
+ new Text(rmAddr), ugiToken.getService());
}
@Test
@@ -2998,7 +2976,7 @@ public class TestRMContainerAllocator {
@Override
protected void setRequestLimit(Priority priority,
Resource capability, int limit) {
- Assertions.fail("setRequestLimit() should not be invoked");
+ Assert.fail("setRequestLimit() should not be invoked");
}
};
@@ -3080,22 +3058,22 @@ public class TestRMContainerAllocator {
// verify all of the host-specific asks were sent plus one for the
// default rack and one for the ANY request
- Assertions.assertEquals(reqMapEvents.length + 2, mockScheduler.lastAsk.size());
+ Assert.assertEquals(reqMapEvents.length + 2, mockScheduler.lastAsk.size());
// verify AM is only asking for the map limit overall
- Assertions.assertEquals(MAP_LIMIT, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(MAP_LIMIT, mockScheduler.lastAnyAskMap);
// assign a map task and verify we do not ask for any more maps
ContainerId cid0 = mockScheduler.assignContainer("h0", false);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(2, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(2, mockScheduler.lastAnyAskMap);
// complete the map task and verify that we ask for one more
mockScheduler.completeContainer(cid0);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(3, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(3, mockScheduler.lastAnyAskMap);
// assign three more maps and verify we ask for no more maps
ContainerId cid1 = mockScheduler.assignContainer("h1", false);
@@ -3103,7 +3081,7 @@ public class TestRMContainerAllocator {
ContainerId cid3 = mockScheduler.assignContainer("h3", false);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskMap);
// complete two containers and verify we only asked for one more
// since at that point all maps should be scheduled/completed
@@ -3111,7 +3089,7 @@ public class TestRMContainerAllocator {
mockScheduler.completeContainer(cid3);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(1, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(1, mockScheduler.lastAnyAskMap);
// allocate the last container and complete the first one
// and verify there are no more map asks.
@@ -3119,77 +3097,76 @@ public class TestRMContainerAllocator {
ContainerId cid4 = mockScheduler.assignContainer("h4", false);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskMap);
// complete the last map
mockScheduler.completeContainer(cid4);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskMap);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskMap);
// verify only reduce limit being requested
- Assertions.assertEquals(REDUCE_LIMIT, mockScheduler.lastAnyAskReduce);
+ Assert.assertEquals(REDUCE_LIMIT, mockScheduler.lastAnyAskReduce);
// assign a reducer and verify ask goes to zero
cid0 = mockScheduler.assignContainer("h0", true);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskReduce);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskReduce);
// complete the reducer and verify we ask for another
mockScheduler.completeContainer(cid0);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(1, mockScheduler.lastAnyAskReduce);
+ Assert.assertEquals(1, mockScheduler.lastAnyAskReduce);
// assign a reducer and verify ask goes to zero
cid0 = mockScheduler.assignContainer("h0", true);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskReduce);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskReduce);
// complete the reducer and verify no more reducers
mockScheduler.completeContainer(cid0);
allocator.schedule();
allocator.schedule();
- Assertions.assertEquals(0, mockScheduler.lastAnyAskReduce);
+ Assert.assertEquals(0, mockScheduler.lastAnyAskReduce);
allocator.close();
}
- @Test
+ @Test(expected = RMContainerAllocationException.class)
public void testAttemptNotFoundCausesRMCommunicatorException()
throws Exception {
- assertThrows(RMContainerAllocationException.class, () -> {
- Configuration conf = new Configuration();
- MyResourceManager rm = new MyResourceManager(conf);
- rm.start();
- // Submit the application
- RMApp app = MockRMAppSubmitter.submitWithMemory(1024, rm);
- rm.drainEvents();
+ Configuration conf = new Configuration();
+ MyResourceManager rm = new MyResourceManager(conf);
+ rm.start();
- MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
- amNodeManager.nodeHeartbeat(true);
- rm.drainEvents();
+ // Submit the application
+ RMApp app = MockRMAppSubmitter.submitWithMemory(1024, rm);
+ rm.drainEvents();
- ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
- .getAppAttemptId();
- rm.sendAMLaunched(appAttemptId);
- rm.drainEvents();
+ MockNM amNodeManager = rm.registerNode("amNM:1234", 2048);
+ amNodeManager.nodeHeartbeat(true);
+ rm.drainEvents();
- JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0);
- Job mockJob = mock(Job.class);
- when(mockJob.getReport()).thenReturn(
- MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
- 0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
- MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
- appAttemptId, mockJob);
+ ApplicationAttemptId appAttemptId = app.getCurrentAppAttempt()
+ .getAppAttemptId();
+ rm.sendAMLaunched(appAttemptId);
+ rm.drainEvents();
- // Now kill the application
- rm.killApp(app.getApplicationId());
- rm.waitForState(app.getApplicationId(), RMAppState.KILLED);
- allocator.schedule();
- });
+ JobId jobId = MRBuilderUtils.newJobId(appAttemptId.getApplicationId(), 0);
+ Job mockJob = mock(Job.class);
+ when(mockJob.getReport()).thenReturn(
+ MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
+ 0, 0, 0, 0, 0, 0, "jobfile", null, false, ""));
+ MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
+ appAttemptId, mockJob);
+
+ // Now kill the application
+ rm.killApp(app.getApplicationId());
+ rm.waitForState(app.getApplicationId(), RMAppState.KILLED);
+ allocator.schedule();
}
@Test
@@ -3270,29 +3247,29 @@ public class TestRMContainerAllocator {
rm.drainEvents();
// One map is assigned.
- Assertions.assertEquals(1, allocator.getAssignedRequests().maps.size());
+ Assert.assertEquals(1, allocator.getAssignedRequests().maps.size());
// Send deallocate request for map so that no maps are assigned after this.
ContainerAllocatorEvent deallocate = createDeallocateEvent(jobId, 1, false);
allocator.sendDeallocate(deallocate);
// Now one reducer should be scheduled and one should be pending.
- Assertions.assertEquals(1, allocator.getScheduledRequests().reduces.size());
- Assertions.assertEquals(1, allocator.getNumOfPendingReduces());
+ Assert.assertEquals(1, allocator.getScheduledRequests().reduces.size());
+ Assert.assertEquals(1, allocator.getNumOfPendingReduces());
// No map should be assigned and one should be scheduled.
- Assertions.assertEquals(1, allocator.getScheduledRequests().maps.size());
- Assertions.assertEquals(0, allocator.getAssignedRequests().maps.size());
+ Assert.assertEquals(1, allocator.getScheduledRequests().maps.size());
+ Assert.assertEquals(0, allocator.getAssignedRequests().maps.size());
- Assertions.assertEquals(6, allocator.getAsk().size());
+ Assert.assertEquals(6, allocator.getAsk().size());
for (ResourceRequest req : allocator.getAsk()) {
boolean isReduce =
req.getPriority().equals(RMContainerAllocator.PRIORITY_REDUCE);
if (isReduce) {
// 1 reducer each asked on h2, * and default-rack
- Assertions.assertTrue((req.getResourceName().equals("*") ||
+ Assert.assertTrue((req.getResourceName().equals("*") ||
req.getResourceName().equals("/default-rack") ||
req.getResourceName().equals("h2")) && req.getNumContainers() == 1);
} else { //map
// 0 mappers asked on h1 and 1 each on * and default-rack
- Assertions.assertTrue(((req.getResourceName().equals("*") ||
+ Assert.assertTrue(((req.getResourceName().equals("*") ||
req.getResourceName().equals("/default-rack")) &&
req.getNumContainers() == 1) || (req.getResourceName().equals("h1")
&& req.getNumContainers() == 0));
@@ -3305,17 +3282,17 @@ public class TestRMContainerAllocator {
// After allocate response from scheduler, all scheduled reduces are ramped
// down and move to pending. 3 asks are also updated with 0 containers to
// indicate ramping down of reduces to scheduler.
- Assertions.assertEquals(0, allocator.getScheduledRequests().reduces.size());
- Assertions.assertEquals(2, allocator.getNumOfPendingReduces());
- Assertions.assertEquals(3, allocator.getAsk().size());
+ Assert.assertEquals(0, allocator.getScheduledRequests().reduces.size());
+ Assert.assertEquals(2, allocator.getNumOfPendingReduces());
+ Assert.assertEquals(3, allocator.getAsk().size());
for (ResourceRequest req : allocator.getAsk()) {
- Assertions.assertEquals(
+ Assert.assertEquals(
RMContainerAllocator.PRIORITY_REDUCE, req.getPriority());
- Assertions.assertTrue(req.getResourceName().equals("*") ||
+ Assert.assertTrue(req.getResourceName().equals("*") ||
req.getResourceName().equals("/default-rack") ||
req.getResourceName().equals("h2"));
- Assertions.assertEquals(Resource.newInstance(1024, 1), req.getCapability());
- Assertions.assertEquals(0, req.getNumContainers());
+ Assert.assertEquals(Resource.newInstance(1024, 1), req.getCapability());
+ Assert.assertEquals(0, req.getNumContainers());
}
}
@@ -3440,29 +3417,29 @@ public class TestRMContainerAllocator {
rm.drainEvents();
// One map is assigned.
- Assertions.assertEquals(1, allocator.getAssignedRequests().maps.size());
+ Assert.assertEquals(1, allocator.getAssignedRequests().maps.size());
// Send deallocate request for map so that no maps are assigned after this.
ContainerAllocatorEvent deallocate = createDeallocateEvent(jobId, 1, false);
allocator.sendDeallocate(deallocate);
// Now one reducer should be scheduled and one should be pending.
- Assertions.assertEquals(1, allocator.getScheduledRequests().reduces.size());
- Assertions.assertEquals(1, allocator.getNumOfPendingReduces());
+ Assert.assertEquals(1, allocator.getScheduledRequests().reduces.size());
+ Assert.assertEquals(1, allocator.getNumOfPendingReduces());
// No map should be assigned and one should be scheduled.
- Assertions.assertEquals(1, allocator.getScheduledRequests().maps.size());
- Assertions.assertEquals(0, allocator.getAssignedRequests().maps.size());
+ Assert.assertEquals(1, allocator.getScheduledRequests().maps.size());
+ Assert.assertEquals(0, allocator.getAssignedRequests().maps.size());
- Assertions.assertEquals(6, allocator.getAsk().size());
+ Assert.assertEquals(6, allocator.getAsk().size());
for (ResourceRequest req : allocator.getAsk()) {
boolean isReduce =
req.getPriority().equals(RMContainerAllocator.PRIORITY_REDUCE);
if (isReduce) {
// 1 reducer each asked on h2, * and default-rack
- Assertions.assertTrue((req.getResourceName().equals("*") ||
+ Assert.assertTrue((req.getResourceName().equals("*") ||
req.getResourceName().equals("/default-rack") ||
req.getResourceName().equals("h2")) && req.getNumContainers() == 1);
} else { //map
// 0 mappers asked on h1 and 1 each on * and default-rack
- Assertions.assertTrue(((req.getResourceName().equals("*") ||
+ Assert.assertTrue(((req.getResourceName().equals("*") ||
req.getResourceName().equals("/default-rack")) &&
req.getNumContainers() == 1) || (req.getResourceName().equals("h1")
&& req.getNumContainers() == 0));
@@ -3478,17 +3455,17 @@ public class TestRMContainerAllocator {
// After allocate response from scheduler, all scheduled reduces are ramped
// down and move to pending. 3 asks are also updated with 0 containers to
// indicate ramping down of reduces to scheduler.
- Assertions.assertEquals(0, allocator.getScheduledRequests().reduces.size());
- Assertions.assertEquals(2, allocator.getNumOfPendingReduces());
- Assertions.assertEquals(3, allocator.getAsk().size());
+ Assert.assertEquals(0, allocator.getScheduledRequests().reduces.size());
+ Assert.assertEquals(2, allocator.getNumOfPendingReduces());
+ Assert.assertEquals(3, allocator.getAsk().size());
for (ResourceRequest req : allocator.getAsk()) {
- Assertions.assertEquals(
+ Assert.assertEquals(
RMContainerAllocator.PRIORITY_REDUCE, req.getPriority());
- Assertions.assertTrue(req.getResourceName().equals("*") ||
+ Assert.assertTrue(req.getResourceName().equals("*") ||
req.getResourceName().equals("/default-rack") ||
req.getResourceName().equals("h2"));
- Assertions.assertEquals(Resource.newInstance(1024, 1), req.getCapability());
- Assertions.assertEquals(0, req.getNumContainers());
+ Assert.assertEquals(Resource.newInstance(1024, 1), req.getCapability());
+ Assert.assertEquals(0, req.getNumContainers());
}
}
@@ -3576,14 +3553,14 @@ public class TestRMContainerAllocator {
rm.drainEvents();
// Two maps are assigned.
- Assertions.assertEquals(2, allocator.getAssignedRequests().maps.size());
+ Assert.assertEquals(2, allocator.getAssignedRequests().maps.size());
// Send deallocate request for map so that no maps are assigned after this.
ContainerAllocatorEvent deallocate1 = createDeallocateEvent(jobId, 1, false);
allocator.sendDeallocate(deallocate1);
ContainerAllocatorEvent deallocate2 = createDeallocateEvent(jobId, 2, false);
allocator.sendDeallocate(deallocate2);
// No map should be assigned.
- Assertions.assertEquals(0, allocator.getAssignedRequests().maps.size());
+ Assert.assertEquals(0, allocator.getAssignedRequests().maps.size());
nodeManager.nodeHeartbeat(true);
rm.drainEvents();
@@ -3607,18 +3584,18 @@ public class TestRMContainerAllocator {
allocator.schedule();
rm.drainEvents();
// One reducer is assigned and one map is scheduled
- Assertions.assertEquals(1, allocator.getScheduledRequests().maps.size());
- Assertions.assertEquals(1, allocator.getAssignedRequests().reduces.size());
+ Assert.assertEquals(1, allocator.getScheduledRequests().maps.size());
+ Assert.assertEquals(1, allocator.getAssignedRequests().reduces.size());
// Headroom enough to run a mapper if headroom is taken as it is but wont be
// enough if scheduled reducers resources are deducted.
rm.getMyFifoScheduler().forceResourceLimit(Resource.newInstance(1260, 2));
allocator.schedule();
rm.drainEvents();
// After allocate response, the one assigned reducer is preempted and killed
- Assertions.assertEquals(1, MyContainerAllocator.getTaskAttemptKillEvents().size());
- Assertions.assertEquals(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC,
+ Assert.assertEquals(1, MyContainerAllocator.getTaskAttemptKillEvents().size());
+ Assert.assertEquals(RMContainerAllocator.RAMPDOWN_DIAGNOSTIC,
MyContainerAllocator.getTaskAttemptKillEvents().get(0).getMessage());
- Assertions.assertEquals(1, allocator.getNumOfPendingReduces());
+ Assert.assertEquals(1, allocator.getNumOfPendingReduces());
}
private static class MockScheduler implements ApplicationMasterProtocol {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java
index 27cd3678535..cab8f544416 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestResourceCalculatorUtils.java
@@ -19,8 +19,8 @@
package org.apache.hadoop.mapreduce.v2.app.rm;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Test;
import java.util.EnumSet;
@@ -59,17 +59,17 @@ public class TestResourceCalculatorUtils {
Resource nonZeroResource, int expectedNumberOfContainersForMemoryOnly,
int expectedNumberOfContainersOverall) {
- Assertions.assertEquals(expectedNumberOfContainersForMemoryOnly,
+ Assert.assertEquals("Incorrect number of available containers for Memory",
+ expectedNumberOfContainersForMemoryOnly,
ResourceCalculatorUtils.computeAvailableContainers(
clusterAvailableResources, nonZeroResource,
- EnumSet.of(SchedulerResourceTypes.MEMORY)),
- "Incorrect number of available containers for Memory");
+ EnumSet.of(SchedulerResourceTypes.MEMORY)));
- Assertions.assertEquals(expectedNumberOfContainersOverall,
+ Assert.assertEquals("Incorrect number of available containers overall",
+ expectedNumberOfContainersOverall,
ResourceCalculatorUtils.computeAvailableContainers(
clusterAvailableResources, nonZeroResource,
EnumSet.of(SchedulerResourceTypes.CPU,
- SchedulerResourceTypes.MEMORY)),
- "Incorrect number of available containers overall");
+ SchedulerResourceTypes.MEMORY)));
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java
index 3ac360ef53f..d5b817c4828 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/TestDataStatistics.java
@@ -18,8 +18,8 @@
package org.apache.hadoop.mapreduce.v2.app.speculate;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
+import org.junit.Assert;
+import org.junit.Test;
public class TestDataStatistics {
@@ -28,21 +28,21 @@ public class TestDataStatistics {
@Test
public void testEmptyDataStatistics() throws Exception {
DataStatistics statistics = new DataStatistics();
- Assertions.assertEquals(0, statistics.count(), TOL);
- Assertions.assertEquals(0, statistics.mean(), TOL);
- Assertions.assertEquals(0, statistics.var(), TOL);
- Assertions.assertEquals(0, statistics.std(), TOL);
- Assertions.assertEquals(0, statistics.outlier(1.0f), TOL);
+ Assert.assertEquals(0, statistics.count(), TOL);
+ Assert.assertEquals(0, statistics.mean(), TOL);
+ Assert.assertEquals(0, statistics.var(), TOL);
+ Assert.assertEquals(0, statistics.std(), TOL);
+ Assert.assertEquals(0, statistics.outlier(1.0f), TOL);
}
@Test
public void testSingleEntryDataStatistics() throws Exception {
DataStatistics statistics = new DataStatistics(17.29);
- Assertions.assertEquals(1, statistics.count(), TOL);
- Assertions.assertEquals(17.29, statistics.mean(), TOL);
- Assertions.assertEquals(0, statistics.var(), TOL);
- Assertions.assertEquals(0, statistics.std(), TOL);
- Assertions.assertEquals(17.29, statistics.outlier(1.0f), TOL);
+ Assert.assertEquals(1, statistics.count(), TOL);
+ Assert.assertEquals(17.29, statistics.mean(), TOL);
+ Assert.assertEquals(0, statistics.var(), TOL);
+ Assert.assertEquals(0, statistics.std(), TOL);
+ Assert.assertEquals(17.29, statistics.outlier(1.0f), TOL);
}
@Test
@@ -50,24 +50,24 @@ public class TestDataStatistics {
DataStatistics statistics = new DataStatistics();
statistics.add(17);
statistics.add(29);
- Assertions.assertEquals(2, statistics.count(), TOL);
- Assertions.assertEquals(23.0, statistics.mean(), TOL);
- Assertions.assertEquals(36.0, statistics.var(), TOL);
- Assertions.assertEquals(6.0, statistics.std(), TOL);
- Assertions.assertEquals(29.0, statistics.outlier(1.0f), TOL);
+ Assert.assertEquals(2, statistics.count(), TOL);
+ Assert.assertEquals(23.0, statistics.mean(), TOL);
+ Assert.assertEquals(36.0, statistics.var(), TOL);
+ Assert.assertEquals(6.0, statistics.std(), TOL);
+ Assert.assertEquals(29.0, statistics.outlier(1.0f), TOL);
}
@Test
public void testUpdateStatistics() throws Exception {
DataStatistics statistics = new DataStatistics(17);
statistics.add(29);
- Assertions.assertEquals(2, statistics.count(), TOL);
- Assertions.assertEquals(23.0, statistics.mean(), TOL);
- Assertions.assertEquals(36.0, statistics.var(), TOL);
+ Assert.assertEquals(2, statistics.count(), TOL);
+ Assert.assertEquals(23.0, statistics.mean(), TOL);
+ Assert.assertEquals(36.0, statistics.var(), TOL);
statistics.updateStatistics(17, 29);
- Assertions.assertEquals(2, statistics.count(), TOL);
- Assertions.assertEquals(29.0, statistics.mean(), TOL);
- Assertions.assertEquals(0.0, statistics.var(), TOL);
+ Assert.assertEquals(2, statistics.count(), TOL);
+ Assert.assertEquals(29.0, statistics.mean(), TOL);
+ Assert.assertEquals(0.0, statistics.var(), TOL);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java
index 4bd23141304..b799d8c2ccd 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/speculate/forecast/TestSimpleExponentialForecast.java
@@ -18,13 +18,14 @@
package org.apache.hadoop.mapreduce.v2.app.speculate.forecast;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.yarn.util.ControlledClock;
+import org.junit.Assert;
+import org.junit.Test;
+
/**
* Testing the statistical model of simple exponential estimator.
*/
@@ -100,21 +101,21 @@ public class TestSimpleExponentialForecast {
@Test
public void testSimpleExponentialForecastLinearInc() throws Exception {
int res = incTestSimpleExponentialForecast();
- Assertions.assertEquals(res, 0,
- "We got the wrong estimate from simple exponential.");
+ Assert.assertEquals("We got the wrong estimate from simple exponential.",
+ res, 0);
}
@Test
public void testSimpleExponentialForecastLinearDec() throws Exception {
int res = decTestSimpleExponentialForecast();
- Assertions.assertEquals(res, 0,
- "We got the wrong estimate from simple exponential.");
+ Assert.assertEquals("We got the wrong estimate from simple exponential.",
+ res, 0);
}
@Test
public void testSimpleExponentialForecastZeros() throws Exception {
int res = zeroTestSimpleExponentialForecast();
- Assertions.assertEquals(res, 0,
- "We got the wrong estimate from simple exponential.");
+ Assert.assertEquals("We got the wrong estimate from simple exponential.",
+ res, 0);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
index 4b8ed0163d5..adb6a573670 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
@@ -19,7 +19,7 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID;
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.Assert.assertEquals;
import java.io.ByteArrayOutputStream;
import java.io.File;
@@ -39,7 +39,7 @@ import javax.net.ssl.SSLException;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
-import org.junit.jupiter.api.Assertions;
+import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.HttpConfig.Policy;
@@ -65,17 +65,14 @@ import org.apache.hadoop.yarn.webapp.WebApps;
import org.apache.hadoop.yarn.webapp.test.WebAppTests;
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
import org.apache.http.HttpStatus;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Test;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
import org.apache.hadoop.thirdparty.com.google.common.net.HttpHeaders;
import com.google.inject.Injector;
-import org.junit.jupiter.api.extension.ExtendWith;
-import uk.org.webcompere.systemstubs.environment.EnvironmentVariables;
-import uk.org.webcompere.systemstubs.jupiter.SystemStub;
-import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension;
+import org.junit.contrib.java.lang.system.EnvironmentVariables;
-@ExtendWith(SystemStubsExtension.class)
public class TestAMWebApp {
private static final File TEST_DIR = new File(
@@ -83,13 +80,12 @@ public class TestAMWebApp {
System.getProperty("java.io.tmpdir")),
TestAMWebApp.class.getName());
- @AfterEach
+ @After
public void tearDown() {
TEST_DIR.delete();
}
- @Test
- public void testAppControllerIndex() {
+ @Test public void testAppControllerIndex() {
AppContext ctx = new MockAppContext(0, 1, 1, 1);
Injector injector = WebAppTests.createMockInjector(AppContext.class, ctx);
AppController controller = injector.getInstance(AppController.class);
@@ -97,29 +93,25 @@ public class TestAMWebApp {
assertEquals(ctx.getApplicationID().toString(), controller.get(APP_ID,""));
}
- @Test
- public void testAppView() {
+ @Test public void testAppView() {
WebAppTests.testPage(AppView.class, AppContext.class, new MockAppContext(0, 1, 1, 1));
}
- @Test
- public void testJobView() {
+ @Test public void testJobView() {
AppContext appContext = new MockAppContext(0, 1, 1, 1);
Map params = getJobParams(appContext);
WebAppTests.testPage(JobPage.class, AppContext.class, appContext, params);
}
- @Test
- public void testTasksView() {
+ @Test public void testTasksView() {
AppContext appContext = new MockAppContext(0, 1, 1, 1);
Map params = getTaskParams(appContext);
WebAppTests.testPage(TasksPage.class, AppContext.class, appContext, params);
}
- @Test
- public void testTaskView() {
+ @Test public void testTaskView() {
AppContext appContext = new MockAppContext(0, 1, 1, 1);
Map params = getTaskParams(appContext);
App app = new App(appContext);
@@ -146,22 +138,19 @@ public class TestAMWebApp {
return params;
}
- @Test
- public void testConfView() {
+ @Test public void testConfView() {
WebAppTests.testPage(JobConfPage.class, AppContext.class,
new MockAppContext(0, 1, 1, 1));
}
- @Test
- public void testCountersView() {
+ @Test public void testCountersView() {
AppContext appContext = new MockAppContext(0, 1, 1, 1);
Map params = getJobParams(appContext);
WebAppTests.testPage(CountersPage.class, AppContext.class,
appContext, params);
}
- @Test
- public void testSingleCounterView() {
+ @Test public void testSingleCounterView() {
AppContext appContext = new MockAppContext(0, 1, 1, 1);
Job job = appContext.getAllJobs().values().iterator().next();
// add a failed task to the job without any counters
@@ -176,16 +165,14 @@ public class TestAMWebApp {
appContext, params);
}
- @Test
- public void testTaskCountersView() {
+ @Test public void testTaskCountersView() {
AppContext appContext = new MockAppContext(0, 1, 1, 1);
Map params = getTaskParams(appContext);
WebAppTests.testPage(CountersPage.class, AppContext.class,
appContext, params);
}
- @Test
- public void testSingleTaskCounterView() {
+ @Test public void testSingleTaskCounterView() {
AppContext appContext = new MockAppContext(0, 1, 1, 2);
Map params = getTaskParams(appContext);
params.put(AMParams.COUNTER_GROUP,
@@ -226,7 +213,7 @@ public class TestAMWebApp {
InputStream in = conn.getInputStream();
ByteArrayOutputStream out = new ByteArrayOutputStream();
IOUtils.copyBytes(in, out, 1024);
- Assertions.assertTrue(out.toString().contains("MapReduce Application"));
+ Assert.assertTrue(out.toString().contains("MapReduce Application"));
// https:// is not accessible.
URL httpsUrl = new URL("https://" + hostPort);
@@ -234,7 +221,7 @@ public class TestAMWebApp {
HttpURLConnection httpsConn =
(HttpURLConnection) httpsUrl.openConnection();
httpsConn.getInputStream();
- Assertions.fail("https:// is not accessible, expected to fail");
+ Assert.fail("https:// is not accessible, expected to fail");
} catch (SSLException e) {
// expected
}
@@ -243,8 +230,9 @@ public class TestAMWebApp {
app.verifyCompleted();
}
- @SystemStub
- public EnvironmentVariables environmentVariables;
+ @Rule
+ public final EnvironmentVariables environmentVariables
+ = new EnvironmentVariables();
@Test
public void testMRWebAppSSLEnabled() throws Exception {
@@ -282,7 +270,7 @@ public class TestAMWebApp {
InputStream in = httpsConn.getInputStream();
ByteArrayOutputStream out = new ByteArrayOutputStream();
IOUtils.copyBytes(in, out, 1024);
- Assertions.assertTrue(out.toString().contains("MapReduce Application"));
+ Assert.assertTrue(out.toString().contains("MapReduce Application"));
// http:// is not accessible.
URL httpUrl = new URL("http://" + hostPort);
@@ -290,7 +278,7 @@ public class TestAMWebApp {
HttpURLConnection httpConn =
(HttpURLConnection) httpUrl.openConnection();
httpConn.getResponseCode();
- Assertions.fail("http:// is not accessible, expected to fail");
+ Assert.fail("http:// is not accessible, expected to fail");
} catch (SocketException e) {
// expected
}
@@ -349,7 +337,7 @@ public class TestAMWebApp {
InputStream in = httpsConn.getInputStream();
ByteArrayOutputStream out = new ByteArrayOutputStream();
IOUtils.copyBytes(in, out, 1024);
- Assertions.assertTrue(out.toString().contains("MapReduce Application"));
+ Assert.assertTrue(out.toString().contains("MapReduce Application"));
// Try with wrong client cert
KeyPair otherClientKeyPair = KeyStoreTestUtil.generateKeyPair("RSA");
@@ -361,7 +349,7 @@ public class TestAMWebApp {
HttpURLConnection httpConn =
(HttpURLConnection) httpsUrl.openConnection();
httpConn.getResponseCode();
- Assertions.fail("Wrong client certificate, expected to fail");
+ Assert.fail("Wrong client certificate, expected to fail");
} catch (SSLException e) {
// expected
}
@@ -416,9 +404,9 @@ public class TestAMWebApp {
String expectedURL = scheme + conf.get(YarnConfiguration.PROXY_ADDRESS)
+ ProxyUriUtils.getPath(app.getAppID(), "/mapreduce", true);
- Assertions.assertEquals(expectedURL,
+ Assert.assertEquals(expectedURL,
conn.getHeaderField(HttpHeaders.LOCATION));
- Assertions.assertEquals(HttpStatus.SC_MOVED_TEMPORARILY,
+ Assert.assertEquals(HttpStatus.SC_MOVED_TEMPORARILY,
conn.getResponseCode());
app.waitForState(job, JobState.SUCCEEDED);
app.verifyCompleted();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
index 8c9a2d3fa0c..29063668b7d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServices.java
@@ -19,9 +19,9 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.StringReader;
import java.util.Set;
@@ -43,8 +43,8 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.w3c.dom.NodeList;
@@ -93,7 +93,7 @@ public class TestAMWebServices extends JerseyTestBase {
Guice.createInjector(new WebServletModule()));
}
- @BeforeEach
+ @Before
@Override
public void setUp() throws Exception {
super.setUp();
@@ -117,7 +117,7 @@ public class TestAMWebServices extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
verifyAMInfo(json.getJSONObject("info"), appContext);
}
@@ -129,7 +129,7 @@ public class TestAMWebServices extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
verifyAMInfo(json.getJSONObject("info"), appContext);
}
@@ -141,7 +141,7 @@ public class TestAMWebServices extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
verifyAMInfo(json.getJSONObject("info"), appContext);
}
@@ -165,7 +165,7 @@ public class TestAMWebServices extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
verifyAMInfo(json.getJSONObject("info"), appContext);
}
@@ -178,7 +178,7 @@ public class TestAMWebServices extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
verifyAMInfo(json.getJSONObject("info"), appContext);
}
@@ -190,7 +190,7 @@ public class TestAMWebServices extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
verifyAMInfo(json.getJSONObject("info"), appContext);
}
@@ -264,7 +264,7 @@ public class TestAMWebServices extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
verifyBlacklistedNodesInfo(json, appContext);
}
@@ -282,7 +282,7 @@ public class TestAMWebServices extends JerseyTestBase {
public void verifyAMInfo(JSONObject info, AppContext ctx)
throws JSONException {
- assertEquals(5, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 5, info.length());
verifyAMInfoGeneric(ctx, info.getString("appId"), info.getString("user"),
info.getString("name"), info.getLong("startedOn"),
@@ -297,7 +297,7 @@ public class TestAMWebServices extends JerseyTestBase {
is.setCharacterStream(new StringReader(xml));
Document dom = db.parse(is);
NodeList nodes = dom.getElementsByTagName("info");
- assertEquals(1, nodes.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, nodes.getLength());
for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i);
@@ -320,8 +320,8 @@ public class TestAMWebServices extends JerseyTestBase {
WebServicesTestUtils.checkStringMatch("name", ctx.getApplicationName(),
name);
- assertEquals(ctx.getStartTime(), startedOn, "startedOn incorrect");
- assertTrue((elapsedTime > 0), "elapsedTime not greater then 0");
+ assertEquals("startedOn incorrect", ctx.getStartTime(), startedOn);
+ assertTrue("elapsedTime not greater then 0", (elapsedTime > 0));
}
@@ -342,11 +342,11 @@ public class TestAMWebServices extends JerseyTestBase {
is.setCharacterStream(new StringReader(xml));
Document dom = db.parse(is);
NodeList infonodes = dom.getElementsByTagName("blacklistednodesinfo");
- assertEquals(1, infonodes.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, infonodes.getLength());
NodeList nodes = dom.getElementsByTagName("blacklistedNodes");
Set blacklistedNodes = ctx.getBlacklistedNodes();
- assertEquals(blacklistedNodes.size(),
- nodes.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", blacklistedNodes.size(),
+ nodes.getLength());
for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i);
assertTrue(
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
index fb5c8dbeb46..28cfb90a17a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempt.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.Assert.assertEquals;
import java.io.StringReader;
import java.util.Enumeration;
@@ -50,8 +50,8 @@ import org.apache.hadoop.yarn.webapp.GuiceServletConfig;
import org.apache.hadoop.yarn.webapp.JerseyTestBase;
import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
import org.codehaus.jettison.json.JSONObject;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.w3c.dom.NodeList;
@@ -118,7 +118,7 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
Guice.createInjector(new WebServletModule()));
}
- @BeforeEach
+ @Before
@Override
public void setUp() throws Exception {
super.setUp();
@@ -157,7 +157,7 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+ JettyUtils.UTF_8, response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
assertEquals(att.getState().toString(), json.get("state"));
}
}
@@ -226,8 +226,7 @@ public class TestAMWebServicesAttempt extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+ JettyUtils.UTF_8, response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
assertEquals(TaskAttemptState.KILLED.toString(), json.get("state"));
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
index d534759319f..aad41966e8f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesAttempts.java
@@ -19,11 +19,11 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.StringReader;
import java.util.List;
@@ -52,8 +52,8 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.w3c.dom.NodeList;
@@ -101,7 +101,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
Guice.createInjector(new WebServletModule()));
}
- @BeforeEach
+ @Before
@Override
public void setUp() throws Exception {
super.setUp();
@@ -199,7 +199,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
is.setCharacterStream(new StringReader(xml));
Document dom = db.parse(is);
NodeList attempts = dom.getElementsByTagName("taskAttempts");
- assertEquals(1, attempts.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, attempts.getLength());
NodeList nodes = dom.getElementsByTagName("taskAttempt");
verifyAMTaskAttemptsXML(nodes, task);
@@ -229,7 +229,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+ JettyUtils.UTF_8, response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("taskAttempt");
verifyAMTaskAttempt(info, att, task.getType());
}
@@ -259,7 +259,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+ JettyUtils.UTF_8, response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("taskAttempt");
verifyAMTaskAttempt(info, att, task.getType());
}
@@ -288,7 +288,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+ JettyUtils.UTF_8, response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("taskAttempt");
verifyAMTaskAttempt(info, att, task.getType());
}
@@ -391,7 +391,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
+ JettyUtils.UTF_8, response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -434,9 +434,9 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
public void verifyAMTaskAttempt(JSONObject info, TaskAttempt att,
TaskType ttype) throws JSONException {
if (ttype == TaskType.REDUCE) {
- assertEquals(17, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 17, info.length());
} else {
- assertEquals(12, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 12, info.length());
}
verifyTaskAttemptGeneric(att, ttype, info.getString("id"),
@@ -455,9 +455,9 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
public void verifyAMTaskAttempts(JSONObject json, Task task)
throws JSONException {
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject attempts = json.getJSONObject("taskAttempts");
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONArray arr = attempts.getJSONArray("taskAttempt");
for (TaskAttempt att : task.getAttempts().values()) {
TaskAttemptId id = att.getID();
@@ -471,13 +471,13 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
verifyAMTaskAttempt(info, att, task.getType());
}
}
- assertTrue(found, "task attempt with id: " + attid
- + " not in web service output");
+ assertTrue("task attempt with id: " + attid
+ + " not in web service output", found);
}
}
public void verifyAMTaskAttemptsXML(NodeList nodes, Task task) {
- assertEquals(1, nodes.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, nodes.getLength());
for (TaskAttempt att : task.getAttempts().values()) {
TaskAttemptId id = att.getID();
@@ -485,14 +485,15 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
Boolean found = false;
for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i);
- assertFalse(element.hasAttributes(), "task attempt should not contain any attributes, it can lead to incorrect JSON marshaling");
+ assertFalse("task attempt should not contain any attributes, it can lead to incorrect JSON marshaling",
+ element.hasAttributes());
if (attid.matches(WebServicesTestUtils.getXmlString(element, "id"))) {
found = true;
verifyAMTaskAttemptXML(element, att, task.getType());
}
}
- assertTrue(found, "task with id: " + attid + " not in web service output");
+ assertTrue("task with id: " + attid + " not in web service output", found);
}
}
@@ -527,26 +528,26 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
ta.getAssignedContainerID().toString(),
assignedContainerId);
- assertEquals(ta.getLaunchTime(), startTime, "startTime wrong");
- assertEquals(ta.getFinishTime(), finishTime, "finishTime wrong");
- assertEquals(finishTime - startTime, elapsedTime, "elapsedTime wrong");
- assertEquals(ta.getProgress() * 100, progress, 1e-3f, "progress wrong");
+ assertEquals("startTime wrong", ta.getLaunchTime(), startTime);
+ assertEquals("finishTime wrong", ta.getFinishTime(), finishTime);
+ assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
+ assertEquals("progress wrong", ta.getProgress() * 100, progress, 1e-3f);
}
public void verifyReduceTaskAttemptGeneric(TaskAttempt ta,
long shuffleFinishTime, long mergeFinishTime, long elapsedShuffleTime,
long elapsedMergeTime, long elapsedReduceTime) {
- assertEquals(ta.getShuffleFinishTime(),
- shuffleFinishTime, "shuffleFinishTime wrong");
- assertEquals(ta.getSortFinishTime(),
- mergeFinishTime, "mergeFinishTime wrong");
- assertEquals(ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime,
- "elapsedShuffleTime wrong");
- assertEquals(ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime,
- "elapsedMergeTime wrong");
- assertEquals(ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime,
- "elapsedReduceTime wrong");
+ assertEquals("shuffleFinishTime wrong", ta.getShuffleFinishTime(),
+ shuffleFinishTime);
+ assertEquals("mergeFinishTime wrong", ta.getSortFinishTime(),
+ mergeFinishTime);
+ assertEquals("elapsedShuffleTime wrong",
+ ta.getShuffleFinishTime() - ta.getLaunchTime(), elapsedShuffleTime);
+ assertEquals("elapsedMergeTime wrong",
+ ta.getSortFinishTime() - ta.getShuffleFinishTime(), elapsedMergeTime);
+ assertEquals("elapsedReduceTime wrong",
+ ta.getFinishTime() - ta.getSortFinishTime(), elapsedReduceTime);
}
@Test
@@ -571,7 +572,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; "
+ JettyUtils.UTF_8, response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobTaskAttemptCounters");
verifyAMJobTaskAttemptCounters(info, att);
}
@@ -616,7 +617,7 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
public void verifyAMJobTaskAttemptCounters(JSONObject info, TaskAttempt att)
throws JSONException {
- assertEquals(2, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, info.length());
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(att.getID()),
info.getString("id"));
@@ -627,15 +628,15 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
for (int i = 0; i < counterGroups.length(); i++) {
JSONObject counterGroup = counterGroups.getJSONObject(i);
String name = counterGroup.getString("counterGroupName");
- assertTrue((name != null && !name.isEmpty()), "name not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
JSONArray counters = counterGroup.getJSONArray("counter");
for (int j = 0; j < counters.length(); j++) {
JSONObject counter = counters.getJSONObject(j);
String counterName = counter.getString("name");
- assertTrue((counterName != null && !counterName.isEmpty()),
- "name not set");
+ assertTrue("name not set",
+ (counterName != null && !counterName.isEmpty()));
long value = counter.getLong("value");
- assertTrue(value >= 0, "value >= 0");
+ assertTrue("value >= 0", value >= 0);
}
}
}
@@ -653,19 +654,20 @@ public class TestAMWebServicesAttempts extends JerseyTestBase {
for (int j = 0; j < groups.getLength(); j++) {
Element counters = (Element) groups.item(j);
- assertNotNull(counters, "should have counters in the web service info");
+ assertNotNull("should have counters in the web service info", counters);
String name = WebServicesTestUtils.getXmlString(counters,
"counterGroupName");
- assertTrue((name != null && !name.isEmpty()), "name not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
NodeList counterArr = counters.getElementsByTagName("counter");
for (int z = 0; z < counterArr.getLength(); z++) {
Element counter = (Element) counterArr.item(z);
String counterName = WebServicesTestUtils.getXmlString(counter,
"name");
- assertTrue((counterName != null && !counterName.isEmpty()), "counter name not set");
+ assertTrue("counter name not set",
+ (counterName != null && !counterName.isEmpty()));
long value = WebServicesTestUtils.getXmlLong(counter, "value");
- assertTrue(value >= 0, "value not >= 0");
+ assertTrue("value not >= 0", value >= 0);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
index 5d147339de2..6568186648d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobConf.java
@@ -18,10 +18,10 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.File;
import java.io.IOException;
@@ -52,9 +52,9 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeEach;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.w3c.dom.NodeList;
@@ -126,7 +126,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
Guice.createInjector(new WebServletModule()));
}
- @BeforeEach
+ @Before
@Override
public void setUp() throws Exception {
super.setUp();
@@ -135,7 +135,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
Guice.createInjector(new WebServletModule()));
}
- @AfterAll
+ @AfterClass
static public void stop() {
FileUtil.fullyDelete(testConfDir);
}
@@ -161,7 +161,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("conf");
verifyAMJobConf(info, jobsMap.get(id));
}
@@ -180,7 +180,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("conf");
verifyAMJobConf(info, jobsMap.get(id));
}
@@ -198,7 +198,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("conf");
verifyAMJobConf(info, jobsMap.get(id));
}
@@ -229,7 +229,7 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
public void verifyAMJobConf(JSONObject info, Job job) throws JSONException {
- assertEquals(2, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, info.length());
WebServicesTestUtils.checkStringMatch("path", job.getConfFile().toString(),
info.getString("path"));
@@ -240,14 +240,14 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
JSONObject prop = properties.getJSONObject(i);
String name = prop.getString("name");
String value = prop.getString("value");
- assertTrue((name != null && !name.isEmpty()), "name not set");
- assertTrue((value != null && !value.isEmpty()), "value not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
+ assertTrue("value not set", (value != null && !value.isEmpty()));
}
}
public void verifyAMJobConfXML(NodeList nodes, Job job) {
- assertEquals(1, nodes.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, nodes.getLength());
for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i);
@@ -260,11 +260,11 @@ public class TestAMWebServicesJobConf extends JerseyTestBase {
for (int j = 0; j < properties.getLength(); j++) {
Element property = (Element) properties.item(j);
- assertNotNull(property, "should have counters in the web service info");
+ assertNotNull("should have counters in the web service info", property);
String name = WebServicesTestUtils.getXmlString(property, "name");
String value = WebServicesTestUtils.getXmlString(property, "value");
- assertTrue((name != null && !name.isEmpty()), "name not set");
- assertTrue((value != null && !value.isEmpty()), "name not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
+ assertTrue("name not set", (value != null && !value.isEmpty()));
}
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
index 1ff4bc475b4..5e4e9f70b35 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesJobs.java
@@ -20,10 +20,10 @@ package org.apache.hadoop.mapreduce.v2.app.webapp;
import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.StringReader;
import java.util.List;
@@ -54,8 +54,8 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.w3c.dom.NodeList;
@@ -103,7 +103,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
Guice.createInjector(new WebServletModule()));
}
- @BeforeEach
+ @Before
@Override
public void setUp() throws Exception {
super.setUp();
@@ -128,7 +128,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject jobs = json.getJSONObject("jobs");
JSONArray arr = jobs.getJSONArray("job");
JSONObject info = arr.getJSONObject(0);
@@ -146,7 +146,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject jobs = json.getJSONObject("jobs");
JSONArray arr = jobs.getJSONArray("job");
JSONObject info = arr.getJSONObject(0);
@@ -163,7 +163,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject jobs = json.getJSONObject("jobs");
JSONArray arr = jobs.getJSONArray("job");
JSONObject info = arr.getJSONObject(0);
@@ -187,9 +187,9 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
is.setCharacterStream(new StringReader(xml));
Document dom = db.parse(is);
NodeList jobs = dom.getElementsByTagName("jobs");
- assertEquals(1, jobs.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, jobs.getLength());
NodeList job = dom.getElementsByTagName("job");
- assertEquals(1, job.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, job.getLength());
verifyAMJobXML(job, appContext);
}
@@ -207,7 +207,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("job");
verifyAMJob(info, jobsMap.get(id));
}
@@ -227,7 +227,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("job");
verifyAMJob(info, jobsMap.get(id));
}
@@ -245,7 +245,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("job");
verifyAMJob(info, jobsMap.get(id));
}
@@ -267,7 +267,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -295,7 +295,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -319,7 +319,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -383,7 +383,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -425,7 +425,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
public void verifyAMJob(JSONObject info, Job job) throws JSONException {
- assertEquals(31, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 31, info.length());
// everyone access fields
verifyAMJobGeneric(job, info.getString("id"), info.getString("user"),
@@ -476,8 +476,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
} else {
fail("should have acls in the web service info");
}
- assertTrue(found,
- "acl: " + expectName + " not found in webservice output");
+ assertTrue("acl: " + expectName + " not found in webservice output",
+ found);
}
}
@@ -485,14 +485,14 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
public void verifyAMJobXML(NodeList nodes, AppContext appContext) {
- assertEquals(1, nodes.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, nodes.getLength());
for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i);
Job job = appContext.getJob(MRApps.toJobID(WebServicesTestUtils
.getXmlString(element, "id")));
- assertNotNull(job, "Job not found - output incorrect");
+ assertNotNull("Job not found - output incorrect", job);
verifyAMJobGeneric(job, WebServicesTestUtils.getXmlString(element, "id"),
WebServicesTestUtils.getXmlString(element, "user"),
@@ -551,8 +551,8 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
} else {
fail("should have acls in the web service info");
}
- assertTrue(found,
- "acl: " + expectName + " not found in webservice output");
+ assertTrue("acl: " + expectName + " not found in webservice output",
+ found);
}
}
}
@@ -572,21 +572,21 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
WebServicesTestUtils.checkStringMatch("state", job.getState().toString(),
state);
- assertEquals(report.getStartTime(), startTime, "startTime incorrect");
- assertEquals(report.getFinishTime(), finishTime, "finishTime incorrect");
- assertEquals(Times.elapsed(report.getStartTime(), report.getFinishTime()),
- elapsedTime, "elapsedTime incorrect");
- assertEquals(job.getTotalMaps(), mapsTotal, "mapsTotal incorrect");
- assertEquals(job.getCompletedMaps(), mapsCompleted,
- "mapsCompleted incorrect");
- assertEquals(job.getTotalReduces(), reducesTotal,
- "reducesTotal incorrect");
- assertEquals(job.getCompletedReduces(), reducesCompleted,
- "reducesCompleted incorrect");
- assertEquals(report.getMapProgress() * 100, mapProgress, 0,
- "mapProgress incorrect");
- assertEquals(report.getReduceProgress() * 100, reduceProgress, 0,
- "reduceProgress incorrect");
+ assertEquals("startTime incorrect", report.getStartTime(), startTime);
+ assertEquals("finishTime incorrect", report.getFinishTime(), finishTime);
+ assertEquals("elapsedTime incorrect",
+ Times.elapsed(report.getStartTime(), report.getFinishTime()),
+ elapsedTime);
+ assertEquals("mapsTotal incorrect", job.getTotalMaps(), mapsTotal);
+ assertEquals("mapsCompleted incorrect", job.getCompletedMaps(),
+ mapsCompleted);
+ assertEquals("reducesTotal incorrect", job.getTotalReduces(), reducesTotal);
+ assertEquals("reducesCompleted incorrect", job.getCompletedReduces(),
+ reducesCompleted);
+ assertEquals("mapProgress incorrect", report.getMapProgress() * 100,
+ mapProgress, 0);
+ assertEquals("reduceProgress incorrect", report.getReduceProgress() * 100,
+ reduceProgress, 0);
}
public void verifyAMJobGenericSecure(Job job, int mapsPending,
@@ -609,27 +609,28 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
WebServicesTestUtils.checkStringMatch("diagnostics", diagString,
diagnostics);
- assertEquals(job.isUber(), uberized, "isUber incorrect");
+ assertEquals("isUber incorrect", job.isUber(), uberized);
// unfortunately the following fields are all calculated in JobInfo
// so not easily accessible without doing all the calculations again.
// For now just make sure they are present.
- assertTrue(mapsPending >= 0, "mapsPending not >= 0");
- assertTrue(mapsRunning >= 0, "mapsRunning not >= 0");
- assertTrue(reducesPending >= 0, "reducesPending not >= 0");
- assertTrue(reducesRunning >= 0, "reducesRunning not >= 0");
+ assertTrue("mapsPending not >= 0", mapsPending >= 0);
+ assertTrue("mapsRunning not >= 0", mapsRunning >= 0);
+ assertTrue("reducesPending not >= 0", reducesPending >= 0);
+ assertTrue("reducesRunning not >= 0", reducesRunning >= 0);
- assertTrue(newReduceAttempts >= 0, "newReduceAttempts not >= 0");
- assertTrue(runningReduceAttempts >= 0, "runningReduceAttempts not >= 0");
- assertTrue(failedReduceAttempts >= 0, "failedReduceAttempts not >= 0");
- assertTrue(killedReduceAttempts >= 0, "killedReduceAttempts not >= 0");
- assertTrue(successfulReduceAttempts >= 0, "successfulReduceAttempts not >= 0");
+ assertTrue("newReduceAttempts not >= 0", newReduceAttempts >= 0);
+ assertTrue("runningReduceAttempts not >= 0", runningReduceAttempts >= 0);
+ assertTrue("failedReduceAttempts not >= 0", failedReduceAttempts >= 0);
+ assertTrue("killedReduceAttempts not >= 0", killedReduceAttempts >= 0);
+ assertTrue("successfulReduceAttempts not >= 0",
+ successfulReduceAttempts >= 0);
- assertTrue(newMapAttempts >= 0, "newMapAttempts not >= 0");
- assertTrue(runningMapAttempts >= 0, "runningMapAttempts not >= 0");
- assertTrue(failedMapAttempts >= 0, "failedMapAttempts not >= 0");
- assertTrue(killedMapAttempts >= 0, "killedMapAttempts not >= 0");
- assertTrue(successfulMapAttempts >= 0, "successfulMapAttempts not >= 0");
+ assertTrue("newMapAttempts not >= 0", newMapAttempts >= 0);
+ assertTrue("runningMapAttempts not >= 0", runningMapAttempts >= 0);
+ assertTrue("failedMapAttempts not >= 0", failedMapAttempts >= 0);
+ assertTrue("killedMapAttempts not >= 0", killedMapAttempts >= 0);
+ assertTrue("successfulMapAttempts not >= 0", successfulMapAttempts >= 0);
}
@@ -646,8 +647,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobCounters");
verifyAMJobCounters(info, jobsMap.get(id));
}
@@ -666,8 +666,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobCounters");
verifyAMJobCounters(info, jobsMap.get(id));
}
@@ -685,8 +684,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobCounters");
verifyAMJobCounters(info, jobsMap.get(id));
}
@@ -718,8 +716,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
public void verifyAMJobCounters(JSONObject info, Job job)
throws JSONException {
- assertEquals(2, info.length(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, info.length());
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
info.getString("id"));
@@ -729,22 +726,22 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
for (int i = 0; i < counterGroups.length(); i++) {
JSONObject counterGroup = counterGroups.getJSONObject(i);
String name = counterGroup.getString("counterGroupName");
- assertTrue((name != null && !name.isEmpty()), "name not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
JSONArray counters = counterGroup.getJSONArray("counter");
for (int j = 0; j < counters.length(); j++) {
JSONObject counter = counters.getJSONObject(j);
String counterName = counter.getString("name");
- assertTrue((counterName != null && !counterName.isEmpty()),
- "counter name not set");
+ assertTrue("counter name not set",
+ (counterName != null && !counterName.isEmpty()));
long mapValue = counter.getLong("mapCounterValue");
- assertTrue(mapValue >= 0, "mapCounterValue >= 0");
+ assertTrue("mapCounterValue >= 0", mapValue >= 0);
long reduceValue = counter.getLong("reduceCounterValue");
- assertTrue(reduceValue >= 0, "reduceCounterValue >= 0");
+ assertTrue("reduceCounterValue >= 0", reduceValue >= 0);
long totalValue = counter.getLong("totalCounterValue");
- assertTrue(totalValue >= 0, "totalCounterValue >= 0");
+ assertTrue("totalCounterValue >= 0", totalValue >= 0);
}
}
@@ -755,7 +752,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i);
- assertNotNull(job, "Job not found - output incorrect");
+ assertNotNull("Job not found - output incorrect", job);
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(job.getID()),
WebServicesTestUtils.getXmlString(element, "id"));
@@ -765,30 +762,29 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
for (int j = 0; j < groups.getLength(); j++) {
Element counters = (Element) groups.item(j);
- assertNotNull(counters,
- "should have counters in the web service info");
+ assertNotNull("should have counters in the web service info", counters);
String name = WebServicesTestUtils.getXmlString(counters,
"counterGroupName");
- assertTrue((name != null && !name.isEmpty()), "name not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
NodeList counterArr = counters.getElementsByTagName("counter");
for (int z = 0; z < counterArr.getLength(); z++) {
Element counter = (Element) counterArr.item(z);
String counterName = WebServicesTestUtils.getXmlString(counter,
"name");
- assertTrue((counterName != null && !counterName.isEmpty()),
- "counter name not set");
+ assertTrue("counter name not set",
+ (counterName != null && !counterName.isEmpty()));
long mapValue = WebServicesTestUtils.getXmlLong(counter,
"mapCounterValue");
- assertTrue(mapValue >= 0, "mapCounterValue not >= 0");
+ assertTrue("mapCounterValue not >= 0", mapValue >= 0);
long reduceValue = WebServicesTestUtils.getXmlLong(counter,
"reduceCounterValue");
- assertTrue(reduceValue >= 0, "reduceCounterValue >= 0");
+ assertTrue("reduceCounterValue >= 0", reduceValue >= 0);
long totalValue = WebServicesTestUtils.getXmlLong(counter,
"totalCounterValue");
- assertTrue(totalValue >= 0, "totalCounterValue >= 0");
+ assertTrue("totalCounterValue >= 0", totalValue >= 0);
}
}
}
@@ -807,7 +803,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobAttempts");
verifyJobAttempts(info, jobsMap.get(id));
}
@@ -826,7 +822,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobAttempts");
verifyJobAttempts(info, jobsMap.get(id));
}
@@ -845,7 +841,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobAttempts");
verifyJobAttempts(info, jobsMap.get(id));
}
@@ -870,8 +866,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
is.setCharacterStream(new StringReader(xml));
Document dom = db.parse(is);
NodeList attempts = dom.getElementsByTagName("jobAttempts");
- assertEquals(1, attempts.getLength(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, attempts.getLength());
NodeList info = dom.getElementsByTagName("jobAttempt");
verifyJobAttemptsXML(info, jobsMap.get(id));
}
@@ -881,8 +876,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
throws JSONException {
JSONArray attempts = info.getJSONArray("jobAttempt");
- assertEquals(2, attempts.length(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, attempts.length());
for (int i = 0; i < attempts.length(); i++) {
JSONObject attempt = attempts.getJSONObject(i);
verifyJobAttemptsGeneric(job, attempt.getString("nodeHttpAddress"),
@@ -894,8 +888,7 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
public void verifyJobAttemptsXML(NodeList nodes, Job job) {
- assertEquals(2, nodes.getLength(),
- "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, nodes.getLength());
for (int i = 0; i < nodes.getLength(); i++) {
Element element = (Element) nodes.item(i);
verifyJobAttemptsGeneric(job,
@@ -921,17 +914,17 @@ public class TestAMWebServicesJobs extends JerseyTestBase {
+ nmHttpPort, nodeHttpAddress);
WebServicesTestUtils.checkStringMatch("nodeId",
NodeId.newInstance(nmHost, nmPort).toString(), nodeId);
- assertTrue(startTime > 0, "startime not greater than 0");
+ assertTrue("startime not greater than 0", startTime > 0);
WebServicesTestUtils.checkStringMatch("containerId", amInfo
.getContainerId().toString(), containerId);
String localLogsLink =ujoin("node", "containerlogs", containerId,
job.getUserName());
- assertTrue(logsLink.contains(localLogsLink), "logsLink");
+ assertTrue("logsLink", logsLink.contains(localLogsLink));
}
}
- assertTrue(attemptFound, "attempt: " + id + " was not found");
+ assertTrue("attempt: " + id + " was not found", attemptFound);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
index 211d81801d6..49187a016c3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebServicesTasks.java
@@ -19,10 +19,10 @@
package org.apache.hadoop.mapreduce.v2.app.webapp;
import static org.apache.hadoop.yarn.webapp.WebServicesTestUtils.assertResponseStatusCode;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.fail;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
import java.io.StringReader;
import java.util.Map;
@@ -50,8 +50,8 @@ import org.apache.hadoop.yarn.webapp.WebServicesTestUtils;
import org.codehaus.jettison.json.JSONArray;
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
import org.w3c.dom.Document;
import org.w3c.dom.Element;
import org.w3c.dom.NodeList;
@@ -99,7 +99,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
Guice.createInjector(new WebServletModule()));
}
- @BeforeEach
+ @Before
@Override
public void setUp() throws Exception {
super.setUp();
@@ -127,10 +127,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject tasks = json.getJSONObject("tasks");
JSONArray arr = tasks.getJSONArray("task");
- assertEquals(2, arr.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, arr.length());
verifyAMTask(arr, jobsMap.get(id), null);
}
@@ -147,10 +147,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject tasks = json.getJSONObject("tasks");
JSONArray arr = tasks.getJSONArray("task");
- assertEquals(2, arr.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, arr.length());
verifyAMTask(arr, jobsMap.get(id), null);
}
@@ -168,10 +168,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject tasks = json.getJSONObject("tasks");
JSONArray arr = tasks.getJSONArray("task");
- assertEquals(2, arr.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, arr.length());
verifyAMTask(arr, jobsMap.get(id), null);
}
@@ -196,7 +196,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
is.setCharacterStream(new StringReader(xml));
Document dom = db.parse(is);
NodeList tasks = dom.getElementsByTagName("tasks");
- assertEquals(1, tasks.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, tasks.getLength());
NodeList task = dom.getElementsByTagName("task");
verifyAMTaskXML(task, jobsMap.get(id));
}
@@ -215,10 +215,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject tasks = json.getJSONObject("tasks");
JSONArray arr = tasks.getJSONArray("task");
- assertEquals(1, arr.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, arr.length());
verifyAMTask(arr, jobsMap.get(id), type);
}
}
@@ -236,10 +236,10 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject tasks = json.getJSONObject("tasks");
JSONArray arr = tasks.getJSONArray("task");
- assertEquals(1, arr.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, arr.length());
verifyAMTask(arr, jobsMap.get(id), type);
}
}
@@ -265,7 +265,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -294,7 +294,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("task");
verifyAMSingleTask(info, task);
}
@@ -316,7 +316,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("task");
verifyAMSingleTask(info, task);
}
@@ -338,7 +338,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("task");
verifyAMSingleTask(info, task);
}
@@ -363,7 +363,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -398,7 +398,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -431,7 +431,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -466,7 +466,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -501,7 +501,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
response.getType().toString());
JSONObject msg = response.getEntity(JSONObject.class);
JSONObject exception = msg.getJSONObject("RemoteException");
- assertEquals(3, exception.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 3, exception.length());
String message = exception.getString("message");
String type = exception.getString("exception");
String classname = exception.getString("javaClassName");
@@ -550,7 +550,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
public void verifyAMSingleTask(JSONObject info, Task task)
throws JSONException {
- assertEquals(9, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 9, info.length());
verifyTaskGeneric(task, info.getString("id"), info.getString("state"),
info.getString("type"), info.getString("successfulAttempt"),
@@ -574,7 +574,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
verifyAMSingleTask(info, task);
}
}
- assertTrue(found, "task with id: " + tid + " not in web service output");
+ assertTrue("task with id: " + tid + " not in web service output", found);
}
}
}
@@ -593,12 +593,12 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
WebServicesTestUtils.checkStringMatch("state", report.getTaskState()
.toString(), state);
// not easily checked without duplicating logic, just make sure its here
- assertNotNull(successfulAttempt, "successfulAttempt null");
- assertEquals(report.getStartTime(), startTime, "startTime wrong");
- assertEquals(report.getFinishTime(), finishTime, "finishTime wrong");
- assertEquals(finishTime - startTime, elapsedTime, "elapsedTime wrong");
- assertEquals(report.getProgress() * 100, progress, 1e-3f, "progress wrong");
- assertEquals(report.getStatus(), status, "status wrong");
+ assertNotNull("successfulAttempt null", successfulAttempt);
+ assertEquals("startTime wrong", report.getStartTime(), startTime);
+ assertEquals("finishTime wrong", report.getFinishTime(), finishTime);
+ assertEquals("elapsedTime wrong", finishTime - startTime, elapsedTime);
+ assertEquals("progress wrong", report.getProgress() * 100, progress, 1e-3f);
+ assertEquals("status wrong", report.getStatus(), status);
}
public void verifyAMSingleTaskXML(Element element, Task task) {
@@ -615,7 +615,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
public void verifyAMTaskXML(NodeList nodes, Job job) {
- assertEquals(2, nodes.getLength(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, nodes.getLength());
for (Task task : job.getTasks().values()) {
TaskId id = task.getID();
@@ -629,7 +629,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
verifyAMSingleTaskXML(element, task);
}
}
- assertTrue(found, "task with id: " + tid + " not in web service output");
+ assertTrue("task with id: " + tid + " not in web service output", found);
}
}
@@ -648,7 +648,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobTaskCounters");
verifyAMJobTaskCounters(info, task);
}
@@ -670,7 +670,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobTaskCounters");
verifyAMJobTaskCounters(info, task);
}
@@ -692,7 +692,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8,
response.getType().toString());
JSONObject json = response.getEntity(JSONObject.class);
- assertEquals(1, json.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 1, json.length());
JSONObject info = json.getJSONObject("jobTaskCounters");
verifyAMJobTaskCounters(info, task);
}
@@ -728,7 +728,7 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
public void verifyAMJobTaskCounters(JSONObject info, Task task)
throws JSONException {
- assertEquals(2, info.length(), "incorrect number of elements");
+ assertEquals("incorrect number of elements", 2, info.length());
WebServicesTestUtils.checkStringMatch("id", MRApps.toString(task.getID()),
info.getString("id"));
@@ -738,14 +738,15 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
for (int i = 0; i < counterGroups.length(); i++) {
JSONObject counterGroup = counterGroups.getJSONObject(i);
String name = counterGroup.getString("counterGroupName");
- assertTrue((name != null && !name.isEmpty()), "name not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
JSONArray counters = counterGroup.getJSONArray("counter");
for (int j = 0; j < counters.length(); j++) {
JSONObject counter = counters.getJSONObject(j);
String counterName = counter.getString("name");
- assertTrue((counterName != null && !counterName.isEmpty()), "name not set");
+ assertTrue("name not set",
+ (counterName != null && !counterName.isEmpty()));
long value = counter.getLong("value");
- assertTrue(value >= 0, "value >= 0");
+ assertTrue("value >= 0", value >= 0);
}
}
}
@@ -764,20 +765,20 @@ public class TestAMWebServicesTasks extends JerseyTestBase {
for (int j = 0; j < groups.getLength(); j++) {
Element counters = (Element) groups.item(j);
- assertNotNull(counters, "should have counters in the web service info");
+ assertNotNull("should have counters in the web service info", counters);
String name = WebServicesTestUtils.getXmlString(counters,
"counterGroupName");
- assertTrue((name != null && !name.isEmpty()), "name not set");
+ assertTrue("name not set", (name != null && !name.isEmpty()));
NodeList counterArr = counters.getElementsByTagName("counter");
for (int z = 0; z < counterArr.getLength(); z++) {
Element counter = (Element) counterArr.item(z);
String counterName = WebServicesTestUtils.getXmlString(counter,
"name");
- assertTrue((counterName != null && !counterName.isEmpty()),
- "counter name not set");
+ assertTrue("counter name not set",
+ (counterName != null && !counterName.isEmpty()));
long value = WebServicesTestUtils.getXmlLong(counter, "value");
- assertTrue(value >= 0, "value not >= 0");
+ assertTrue("value not >= 0", value >= 0);
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
index d8376e1b51a..ba5c4301214 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
@@ -37,8 +37,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.webapp.Controller.RequestContext;
import org.apache.hadoop.yarn.webapp.MimeType;
import org.apache.hadoop.yarn.webapp.ResponseInfo;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
+import org.junit.Before;
+import org.junit.Test;
import static org.junit.Assert.*;
public class TestAppController {
@@ -48,7 +48,7 @@ public class TestAppController {
private Job job;
private static final String taskId = "task_01_01_m_01";
- @BeforeEach
+ @Before
public void setUp() throws IOException {
AppContext context = mock(AppContext.class);
when(context.getApplicationID()).thenReturn(
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
index 24fb901c958..82b8a37dbea 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestBlocks.java
@@ -26,7 +26,7 @@ import java.util.Map;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
import org.apache.hadoop.yarn.webapp.View;
-import org.junit.jupiter.api.Test;
+import org.junit.Test;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -52,7 +52,7 @@ import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
import org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block;
import static org.mockito.Mockito.*;
-import static org.junit.jupiter.api.Assertions.*;
+import static org.junit.Assert.*;
public class TestBlocks {
private ByteArrayOutputStream data = new ByteArrayOutputStream();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java
index 7fedc7bb2dc..66fa3de94f8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleChannelHandler.java
@@ -225,7 +225,7 @@ public class TestShuffleChannelHandler extends TestShuffleHandlerBase {
final ShuffleTest t = createShuffleTest();
final EmbeddedChannel shuffle = t.createShuffleHandlerChannelFileRegion();
- String dataFile = getDataFile(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_2);
+ String dataFile = getDataFile(TEST_USER, tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_2);
assertTrue("should delete", new File(dataFile).delete());
FullHttpRequest req = t.createRequest(getUri(TEST_JOB_ID, 0,
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
index 37a9210286c..cc46b49b113 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandler.java
@@ -29,6 +29,7 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
@@ -41,6 +42,7 @@ import java.io.DataInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
+import java.io.InputStream;
import java.io.InputStreamReader;
import java.net.HttpURLConnection;
import java.net.MalformedURLException;
@@ -159,7 +161,7 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
shuffleHandler.init(conf);
shuffleHandler.start();
final String port = shuffleHandler.getConfig().get(SHUFFLE_PORT_CONFIG_KEY);
- final SecretKey secretKey = shuffleHandler.addTestApp();
+ final SecretKey secretKey = shuffleHandler.addTestApp(TEST_USER);
// setup connections
HttpURLConnection[] conns = new HttpURLConnection[connAttempts];
@@ -237,7 +239,7 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
shuffleHandler.init(conf);
shuffleHandler.start();
final String port = shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY);
- final SecretKey secretKey = shuffleHandler.addTestApp();
+ final SecretKey secretKey = shuffleHandler.addTestApp(TEST_USER);
HttpURLConnection conn1 = createRequest(
geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_1), true),
@@ -278,18 +280,34 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
UserGroupInformation.setConfiguration(conf);
+ final String randomUser = "randomUser";
+ final String attempt = "attempt_1111111111111_0004_m_000004_0";
+ generateMapOutput(randomUser, tempDir.toAbsolutePath().toString(), attempt,
+ Arrays.asList(TEST_DATA_C, TEST_DATA_B, TEST_DATA_A));
+
ShuffleHandlerMock shuffleHandler = new ShuffleHandlerMock();
shuffleHandler.init(conf);
try {
shuffleHandler.start();
final String port = shuffleHandler.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY);
- final SecretKey secretKey = shuffleHandler.addTestApp();
+ final SecretKey secretKey = shuffleHandler.addTestApp(randomUser);
HttpURLConnection conn = createRequest(
- geURL(port, TEST_JOB_ID, 0, Collections.singletonList(TEST_ATTEMPT_1), false),
+ geURL(port, TEST_JOB_ID, 0, Collections.singletonList(attempt), false),
secretKey);
conn.connect();
- BufferedReader in = new BufferedReader(new InputStreamReader(conn.getInputStream()));
+
+ InputStream is = null;
+ try {
+ is = conn.getInputStream();
+ } catch (IOException ioe) {
+ if (conn.getResponseCode() != HttpURLConnection.HTTP_OK) {
+ is = conn.getErrorStream();
+ }
+ }
+
+ assertNotNull(is);
+ BufferedReader in = new BufferedReader(new InputStreamReader(is));
StringBuilder builder = new StringBuilder();
String inputLine;
while ((inputLine = in.readLine()) != null) {
@@ -299,7 +317,7 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
String receivedString = builder.toString();
//Retrieve file owner name
- String indexFilePath = getIndexFile(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_1);
+ String indexFilePath = getIndexFile(randomUser, tempDir.toAbsolutePath().toString(), attempt);
String owner;
try (FileInputStream fis = new FileInputStream(indexFilePath)) {
owner = NativeIO.POSIX.getFstat(fis.getFD()).getOwner();
@@ -307,11 +325,11 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
String message =
"Owner '" + owner + "' for path " + indexFilePath
- + " did not match expected owner '" + TEST_USER + "'";
+ + " did not match expected owner '" + randomUser + "'";
assertTrue(String.format("Received string '%s' should contain " +
"message '%s'", receivedString, message),
receivedString.contains(message));
- assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+ assertEquals(HttpURLConnection.HTTP_INTERNAL_ERROR, conn.getResponseCode());
LOG.info("received: " + receivedString);
assertNotEquals("", receivedString);
} finally {
@@ -334,7 +352,7 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
shuffle.init(conf);
shuffle.start();
final String port = shuffle.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY);
- final SecretKey secretKey = shuffle.addTestApp();
+ final SecretKey secretKey = shuffle.addTestApp(TEST_USER);
// verify we are authorized to shuffle
int rc = getShuffleResponseCode(port, secretKey);
@@ -387,7 +405,7 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
shuffle.init(conf);
shuffle.start();
final String port = shuffle.getConfig().get(ShuffleHandler.SHUFFLE_PORT_CONFIG_KEY);
- final SecretKey secretKey = shuffle.addTestApp();
+ final SecretKey secretKey = shuffle.addTestApp(TEST_USER);
// verify we are authorized to shuffle
int rc = getShuffleResponseCode(port, secretKey);
@@ -489,14 +507,14 @@ public class TestShuffleHandler extends TestShuffleHandlerBase {
class ShuffleHandlerMock extends ShuffleHandler {
- public SecretKey addTestApp() throws IOException {
+ public SecretKey addTestApp(String user) throws IOException {
DataOutputBuffer outputBuffer = new DataOutputBuffer();
outputBuffer.reset();
Token jt = new Token<>(
- "identifier".getBytes(), "password".getBytes(), new Text(TEST_USER),
+ "identifier".getBytes(), "password".getBytes(), new Text(user),
new Text("shuffleService"));
jt.write(outputBuffer);
- initializeApplication(new ApplicationInitializationContext(TEST_USER, TEST_APP_ID,
+ initializeApplication(new ApplicationInitializationContext(user, TEST_APP_ID,
ByteBuffer.wrap(outputBuffer.getData(), 0,
outputBuffer.getLength())));
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java
index 1bce443381d..406f2866230 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/test/java/org/apache/hadoop/mapred/TestShuffleHandlerBase.java
@@ -55,7 +55,7 @@ public class TestShuffleHandlerBase {
public static final String TEST_ATTEMPT_2 = "attempt_1111111111111_0002_m_000002_0";
public static final String TEST_ATTEMPT_3 = "attempt_1111111111111_0003_m_000003_0";
public static final String TEST_JOB_ID = "job_1111111111111_0001";
- public static final String TEST_USER = "testUser";
+ public static final String TEST_USER = System.getProperty("user.name");
public static final String TEST_DATA_A = "aaaaa";
public static final String TEST_DATA_B = "bbbbb";
public static final String TEST_DATA_C = "ccccc";
@@ -70,11 +70,11 @@ public class TestShuffleHandlerBase {
tempDir = Files.createTempDirectory("test-shuffle-channel-handler");
tempDir.toFile().deleteOnExit();
- generateMapOutput(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_1,
+ generateMapOutput(TEST_USER, tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_1,
Arrays.asList(TEST_DATA_A, TEST_DATA_B, TEST_DATA_C));
- generateMapOutput(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_2,
+ generateMapOutput(TEST_USER, tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_2,
Arrays.asList(TEST_DATA_B, TEST_DATA_A, TEST_DATA_C));
- generateMapOutput(tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_3,
+ generateMapOutput(TEST_USER, tempDir.toAbsolutePath().toString(), TEST_ATTEMPT_3,
Arrays.asList(TEST_DATA_C, TEST_DATA_B, TEST_DATA_A));
outputStreamCaptor.reset();
@@ -101,12 +101,13 @@ public class TestShuffleHandlerBase {
return allMatches;
}
- public static void generateMapOutput(String tempDir, String attempt, List maps)
+ public static void generateMapOutput(String user, String tempDir,
+ String attempt, List maps)
throws IOException {
SpillRecord record = new SpillRecord(maps.size());
- assertTrue(new File(getBasePath(tempDir, attempt)).mkdirs());
- try (PrintWriter writer = new PrintWriter(getDataFile(tempDir, attempt), "UTF-8")) {
+ assertTrue(new File(getBasePath(user, tempDir, attempt)).mkdirs());
+ try (PrintWriter writer = new PrintWriter(getDataFile(user, tempDir, attempt), "UTF-8")) {
long startOffset = 0;
int partition = 0;
for (String map : maps) {
@@ -119,21 +120,21 @@ public class TestShuffleHandlerBase {
partition++;
writer.write(map);
}
- record.writeToFile(new Path(getIndexFile(tempDir, attempt)),
+ record.writeToFile(new Path(getIndexFile(user, tempDir, attempt)),
new JobConf(new Configuration()));
}
}
- public static String getIndexFile(String tempDir, String attempt) {
- return String.format("%s/%s", getBasePath(tempDir, attempt), INDEX_FILE_NAME);
+ public static String getIndexFile(String user, String tempDir, String attempt) {
+ return String.format("%s/%s", getBasePath(user, tempDir, attempt), INDEX_FILE_NAME);
}
- public static String getDataFile(String tempDir, String attempt) {
- return String.format("%s/%s", getBasePath(tempDir, attempt), DATA_FILE_NAME);
+ public static String getDataFile(String user, String tempDir, String attempt) {
+ return String.format("%s/%s", getBasePath(user, tempDir, attempt), DATA_FILE_NAME);
}
- private static String getBasePath(String tempDir, String attempt) {
- return String.format("%s/%s/%s/%s", tempDir, TEST_JOB_ID, TEST_USER, attempt);
+ private static String getBasePath(String user, String tempDir, String attempt) {
+ return String.format("%s/%s/%s/%s", tempDir, TEST_JOB_ID, user, attempt);
}
public static String getUri(String jobId, int reduce, List maps, boolean keepAlive) {
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index 5887d5b7fa9..062abb3f1db 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -129,6 +129,8 @@
1.0-alpha-13.3.14.0.3
+ 10.10.2.0
+ 8.0.296.2.1.jre74.10.03.2.0
@@ -212,7 +214,7 @@
9.8.1v12.22.1v1.22.5
- 1.10.11
+ 1.10.131.20
@@ -1576,6 +1578,36 @@
commons-beanutilscommons-beanutils
+
+ org.apache.commons
+ commons-lang3
+
+
+
+ javax.xml.bind
+ jaxb-api
+
+
+ org.slf4j
+ slf4j-api
+
+
+ com.google.code.gson
+ gson
+
+
+ org.apache.httpcomponents
+ httpcore
+
+
+ commons-logging
+ commons-logging
+
+
+
+ org.codehaus.jettison
+ jettison
+
@@ -1809,6 +1841,16 @@
HikariCP${hikari.version}
+
+ org.apache.derby
+ derby
+ ${derby.version}
+
+
+ mysql
+ mysql-connector-java
+ ${mysql-connector-java.version}
+ com.microsoft.sqlservermssql-jdbc
diff --git a/hadoop-project/src/site/markdown/index.md.vm b/hadoop-project/src/site/markdown/index.md.vm
index e7ed0fe8066..33c86bbc06e 100644
--- a/hadoop-project/src/site/markdown/index.md.vm
+++ b/hadoop-project/src/site/markdown/index.md.vm
@@ -62,7 +62,10 @@ possibly in parallel, with results potentially coming in out-of-order.
Benchmarking of enhanced Apache ORC and Apache Parquet clients through `file://` and `s3a://`
show significant improvements in query performance.
-Further Reading: [FsDataInputStream](./hadoop-project-dist/hadoop-common/filesystem/fsdatainputstream.html).
+Further Reading:
+* [FsDataInputStream](./hadoop-project-dist/hadoop-common/filesystem/fsdatainputstream.html).
+* [Hadoop Vectored IO: Your Data Just Got Faster!](https://apachecon.com/acasia2022/sessions/bigdata-1148.html)
+ Apachecon 2022 talk.
Mapreduce: Manifest Committer for Azure ABFS and google GCS
----------------------------------------------------------
@@ -88,14 +91,6 @@ More details are available in the
documentation.
-HDFS: Router Based Federation
------------------------------
-
-A lot of effort has been invested into stabilizing/improving the HDFS Router Based Federation feature.
-
-1. HDFS-13522, HDFS-16767 & Related Jiras: Allow Observer Reads in HDFS Router Based Federation.
-2. HDFS-13248: RBF supports Client Locality
-
HDFS: Dynamic Datanode Reconfiguration
--------------------------------------
diff --git a/hadoop-tools/hadoop-aliyun/pom.xml b/hadoop-tools/hadoop-aliyun/pom.xml
index 570032f370b..7605b18b538 100644
--- a/hadoop-tools/hadoop-aliyun/pom.xml
+++ b/hadoop-tools/hadoop-aliyun/pom.xml
@@ -125,6 +125,12 @@
+
+
+ org.codehaus.jettison
+ jettison
+
+
org.apache.hadoophadoop-common
diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml
index cec050d2c1b..14ffa3798aa 100644
--- a/hadoop-tools/hadoop-azure-datalake/pom.xml
+++ b/hadoop-tools/hadoop-azure-datalake/pom.xml
@@ -110,6 +110,22 @@
com.microsoft.azureazure-data-lake-store-sdk${azure.data.lake.store.sdk.version}
+
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+
+
+ org.slf4j
+ slf4j-api
+
+
+
+ org.wildfly.openssl
+ wildfly-openssl
+
+
diff --git a/hadoop-tools/hadoop-azure/pom.xml b/hadoop-tools/hadoop-azure/pom.xml
index c313fa28a3a..e8c5fb78efd 100644
--- a/hadoop-tools/hadoop-azure/pom.xml
+++ b/hadoop-tools/hadoop-azure/pom.xml
@@ -164,6 +164,19 @@
org.apache.commonscommons-lang3
+
+
+ com.fasterxml.jackson.core
+ jackson-core
+
+
+ org.slf4j
+ slf4j-api
+
+
+ com.google.guava
+ guava
+
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
index 4ba05794a09..14eb8cfb8a6 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyCommitter.java
@@ -156,10 +156,7 @@ public class CopyCommitter extends FileOutputCommitter {
final boolean directWrite = conf.getBoolean(
DistCpOptionSwitch.DIRECT_WRITE.getConfigLabel(), false);
- final boolean append = conf.getBoolean(
- DistCpOptionSwitch.APPEND.getConfigLabel(), false);
- final boolean useTempTarget = !append && !directWrite;
- if (!useTempTarget) {
+ if (directWrite) {
return;
}
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java
index f2dd246db5a..6a537dc6e7d 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java
@@ -586,13 +586,11 @@ public class TestCopyCommitter {
@Test
public void testCommitWithCleanupTempFiles() throws IOException {
- testCommitWithCleanup(true, false);
- testCommitWithCleanup(false, true);
- testCommitWithCleanup(true, true);
- testCommitWithCleanup(false, false);
+ testCommitWithCleanup(true);
+ testCommitWithCleanup(false);
}
- private void testCommitWithCleanup(boolean append, boolean directWrite)throws IOException {
+ private void testCommitWithCleanup(boolean directWrite) throws IOException {
TaskAttemptContext taskAttemptContext = getTaskAttemptContext(config);
JobID jobID = taskAttemptContext.getTaskAttemptID().getJobID();
JobContext jobContext = new JobContextImpl(
@@ -611,7 +609,7 @@ public class TestCopyCommitter {
DistCpOptions options = new DistCpOptions.Builder(
Collections.singletonList(new Path(sourceBase)),
new Path("/out"))
- .withAppend(append)
+ .withAppend(true)
.withSyncFolder(true)
.withDirectWrite(directWrite)
.build();
@@ -631,7 +629,7 @@ public class TestCopyCommitter {
null, taskAttemptContext);
committer.commitJob(jobContext);
- if (append || directWrite) {
+ if (directWrite) {
ContractTestUtils.assertPathExists(fs, "Temp files should not be cleanup with append or direct option",
tempFilePath);
} else {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshAdminAclsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshAdminAclsRequest.java
index 71c4a2c46d7..5371741331e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshAdminAclsRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshAdminAclsRequest.java
@@ -33,4 +33,27 @@ public abstract class RefreshAdminAclsRequest {
Records.newRecord(RefreshAdminAclsRequest.class);
return request;
}
+
+ @Public
+ @Stable
+ public static RefreshAdminAclsRequest newInstance(String subClusterId) {
+ RefreshAdminAclsRequest request =
+ Records.newRecord(RefreshAdminAclsRequest.class);
+ request.setSubClusterId(subClusterId);
+ return request;
+ }
+
+ /**
+ * Get the subClusterId.
+ *
+ * @return subClusterId.
+ */
+ public abstract String getSubClusterId();
+
+ /**
+ * Set the subClusterId.
+ *
+ * @param subClusterId subCluster Id.
+ */
+ public abstract void setSubClusterId(String subClusterId);
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshServiceAclsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshServiceAclsRequest.java
index 789f54fe29a..e382ebccba1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshServiceAclsRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/RefreshServiceAclsRequest.java
@@ -33,4 +33,27 @@ public abstract class RefreshServiceAclsRequest {
Records.newRecord(RefreshServiceAclsRequest.class);
return request;
}
+
+ @Public
+ @Stable
+ public static RefreshServiceAclsRequest newInstance(String subClusterId) {
+ RefreshServiceAclsRequest request =
+ Records.newRecord(RefreshServiceAclsRequest.class);
+ request.setSubClusterId(subClusterId);
+ return request;
+ }
+
+ /**
+ * Get the subClusterId.
+ *
+ * @return subClusterId.
+ */
+ public abstract String getSubClusterId();
+
+ /**
+ * Set the subClusterId.
+ *
+ * @param subClusterId subCluster Id.
+ */
+ public abstract void setSubClusterId(String subClusterId);
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
index 97e29f954cd..4050a5b356f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
@@ -58,11 +58,13 @@ message RefreshUserToGroupsMappingsResponseProto {
}
message RefreshAdminAclsRequestProto {
+ optional string sub_cluster_id = 1;
}
message RefreshAdminAclsResponseProto {
}
message RefreshServiceAclsRequestProto {
+ optional string sub_cluster_id = 1;
}
message RefreshServiceAclsResponseProto {
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshAdminAclsRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshAdminAclsRequestPBImpl.java
index 47eadc111bb..0738e8a1b0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshAdminAclsRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshAdminAclsRequestPBImpl.java
@@ -18,21 +18,22 @@
package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
+import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProtoOrBuilder;
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
import org.apache.hadoop.thirdparty.protobuf.TextFormat;
@Private
@Unstable
-public class RefreshAdminAclsRequestPBImpl
-extends RefreshAdminAclsRequest {
+public class RefreshAdminAclsRequestPBImpl extends RefreshAdminAclsRequest {
- RefreshAdminAclsRequestProto proto = RefreshAdminAclsRequestProto.getDefaultInstance();
- RefreshAdminAclsRequestProto.Builder builder = null;
- boolean viaProto = false;
+ private RefreshAdminAclsRequestProto proto = RefreshAdminAclsRequestProto.getDefaultInstance();
+ private RefreshAdminAclsRequestProto.Builder builder = null;
+ private boolean viaProto = false;
public RefreshAdminAclsRequestPBImpl() {
builder = RefreshAdminAclsRequestProto.newBuilder();
@@ -48,6 +49,13 @@ extends RefreshAdminAclsRequest {
viaProto = true;
return proto;
}
+
+ private synchronized void maybeInitBuilder() {
+ if (viaProto || builder == null) {
+ builder = RefreshAdminAclsRequestProto.newBuilder(proto);
+ }
+ viaProto = false;
+ }
@Override
public int hashCode() {
@@ -56,16 +64,39 @@ extends RefreshAdminAclsRequest {
@Override
public boolean equals(Object other) {
- if (other == null)
+
+ if (!(other instanceof RefreshAdminAclsRequest)) {
return false;
- if (other.getClass().isAssignableFrom(this.getClass())) {
- return this.getProto().equals(this.getClass().cast(other).getProto());
}
- return false;
+
+ RefreshAdminAclsRequestPBImpl otherImpl = this.getClass().cast(other);
+ return new EqualsBuilder()
+ .append(this.getProto(), otherImpl.getProto())
+ .isEquals();
}
@Override
public String toString() {
return TextFormat.shortDebugString(getProto());
}
+
+ @Override
+ public String getSubClusterId() {
+ RefreshAdminAclsRequestProtoOrBuilder p = viaProto ? proto : builder;
+ boolean hasSubClusterId = p.hasSubClusterId();
+ if (hasSubClusterId) {
+ return p.getSubClusterId();
+ }
+ return null;
+ }
+
+ @Override
+ public void setSubClusterId(String subClusterId) {
+ maybeInitBuilder();
+ if (subClusterId == null) {
+ builder.clearSubClusterId();
+ return;
+ }
+ builder.setSubClusterId(subClusterId);
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java
index d4529f43e65..4c30d0f2a54 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/RefreshServiceAclsRequestPBImpl.java
@@ -18,9 +18,11 @@
package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb;
+import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProtoOrBuilder;
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
import org.apache.hadoop.thirdparty.protobuf.TextFormat;
@@ -29,10 +31,10 @@ import org.apache.hadoop.thirdparty.protobuf.TextFormat;
@Unstable
public class RefreshServiceAclsRequestPBImpl extends RefreshServiceAclsRequest {
- RefreshServiceAclsRequestProto proto =
+ private RefreshServiceAclsRequestProto proto =
RefreshServiceAclsRequestProto.getDefaultInstance();
- RefreshServiceAclsRequestProto.Builder builder = null;
- boolean viaProto = false;
+ private RefreshServiceAclsRequestProto.Builder builder = null;
+ private boolean viaProto = false;
public RefreshServiceAclsRequestPBImpl() {
builder = RefreshServiceAclsRequestProto.newBuilder();
@@ -50,6 +52,13 @@ public class RefreshServiceAclsRequestPBImpl extends RefreshServiceAclsRequest {
return proto;
}
+ private synchronized void maybeInitBuilder() {
+ if (viaProto || builder == null) {
+ builder = RefreshServiceAclsRequestProto.newBuilder(proto);
+ }
+ viaProto = false;
+ }
+
@Override
public int hashCode() {
return getProto().hashCode();
@@ -57,16 +66,39 @@ public class RefreshServiceAclsRequestPBImpl extends RefreshServiceAclsRequest {
@Override
public boolean equals(Object other) {
- if (other == null)
+
+ if (!(other instanceof RefreshServiceAclsRequest)) {
return false;
- if (other.getClass().isAssignableFrom(this.getClass())) {
- return this.getProto().equals(this.getClass().cast(other).getProto());
}
- return false;
+
+ RefreshServiceAclsRequestPBImpl otherImpl = this.getClass().cast(other);
+ return new EqualsBuilder()
+ .append(this.getProto(), otherImpl.getProto())
+ .isEquals();
}
@Override
public String toString() {
return TextFormat.shortDebugString(getProto());
}
+
+ @Override
+ public String getSubClusterId() {
+ RefreshServiceAclsRequestProtoOrBuilder p = viaProto ? proto : builder;
+ boolean hasSubClusterId = p.hasSubClusterId();
+ if (hasSubClusterId) {
+ return p.getSubClusterId();
+ }
+ return null;
+ }
+
+ @Override
+ public void setSubClusterId(String subClusterId) {
+ maybeInitBuilder();
+ if (subClusterId == null) {
+ builder.clearSubClusterId();
+ return;
+ }
+ builder.setSubClusterId(subClusterId);
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java
index 3ca8ccc2bfb..e65bdf42e2e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/FederationStateStore.java
@@ -60,7 +60,21 @@ public interface FederationStateStore extends
* Load the version information from the federation state store.
*
* @return the {@link Version} of the federation state store
+ * @throws Exception an exception occurred in load version.
*/
- Version loadVersion();
+ Version loadVersion() throws Exception;
+ /**
+ * Store the Version information in federation state store.
+ *
+ * @throws Exception an exception occurred in store version.
+ */
+ void storeVersion() throws Exception;
+
+ /**
+ * Check the version of federation stateStore.
+ *
+ * @throws Exception an exception occurred in check version.
+ */
+ void checkVersion() throws Exception;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
index b91de3ae808..273e736e887 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/MemoryFederationStateStore.java
@@ -31,6 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.Comparator;
+import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.api.records.ReservationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.federation.store.FederationStateStore;
+import org.apache.hadoop.yarn.server.federation.store.metrics.FederationStateStoreClientMetrics;
import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterRequest;
import org.apache.hadoop.yarn.server.federation.store.records.AddApplicationHomeSubClusterResponse;
import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
@@ -107,7 +109,7 @@ import static org.apache.hadoop.yarn.server.federation.store.utils.FederationSta
public class MemoryFederationStateStore implements FederationStateStore {
private Map membership;
- private Map applications;
+ private Map applications;
private Map reservations;
private Map policies;
private RouterRMDTSecretManagerState routerRMSecretManagerState;
@@ -122,10 +124,10 @@ public class MemoryFederationStateStore implements FederationStateStore {
@Override
public void init(Configuration conf) {
- membership = new ConcurrentHashMap();
- applications = new ConcurrentHashMap();
- reservations = new ConcurrentHashMap();
- policies = new ConcurrentHashMap();
+ membership = new ConcurrentHashMap<>();
+ applications = new ConcurrentHashMap<>();
+ reservations = new ConcurrentHashMap<>();
+ policies = new ConcurrentHashMap<>();
routerRMSecretManagerState = new RouterRMDTSecretManagerState();
maxAppsInStateStore = conf.getInt(
YarnConfiguration.FEDERATION_STATESTORE_MAX_APPLICATIONS,
@@ -143,14 +145,15 @@ public class MemoryFederationStateStore implements FederationStateStore {
}
@Override
- public SubClusterRegisterResponse registerSubCluster(
- SubClusterRegisterRequest request) throws YarnException {
+ public SubClusterRegisterResponse registerSubCluster(SubClusterRegisterRequest request)
+ throws YarnException {
+ long startTime = clock.getTime();
+
FederationMembershipStateStoreInputValidator.validate(request);
SubClusterInfo subClusterInfo = request.getSubClusterInfo();
long currentTime =
Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTimeInMillis();
-
SubClusterInfo subClusterInfoToSave =
SubClusterInfo.newInstance(subClusterInfo.getSubClusterId(),
subClusterInfo.getAMRMServiceAddress(),
@@ -161,18 +164,21 @@ public class MemoryFederationStateStore implements FederationStateStore {
subClusterInfo.getCapability());
membership.put(subClusterInfo.getSubClusterId(), subClusterInfoToSave);
+ long stopTime = clock.getTime();
+
+ FederationStateStoreClientMetrics.succeededStateStoreCall(stopTime - startTime);
return SubClusterRegisterResponse.newInstance();
}
@Override
- public SubClusterDeregisterResponse deregisterSubCluster(
- SubClusterDeregisterRequest request) throws YarnException {
+ public SubClusterDeregisterResponse deregisterSubCluster(SubClusterDeregisterRequest request)
+ throws YarnException {
+
FederationMembershipStateStoreInputValidator.validate(request);
SubClusterInfo subClusterInfo = membership.get(request.getSubClusterId());
if (subClusterInfo == null) {
- String errMsg =
- "SubCluster " + request.getSubClusterId().toString() + " not found";
- FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+ FederationStateStoreUtils.logAndThrowStoreException(
+ LOG, "SubCluster %s not found", request.getSubClusterId());
} else {
subClusterInfo.setState(request.getState());
}
@@ -181,17 +187,16 @@ public class MemoryFederationStateStore implements FederationStateStore {
}
@Override
- public SubClusterHeartbeatResponse subClusterHeartbeat(
- SubClusterHeartbeatRequest request) throws YarnException {
+ public SubClusterHeartbeatResponse subClusterHeartbeat(SubClusterHeartbeatRequest request)
+ throws YarnException {
FederationMembershipStateStoreInputValidator.validate(request);
SubClusterId subClusterId = request.getSubClusterId();
SubClusterInfo subClusterInfo = membership.get(subClusterId);
if (subClusterInfo == null) {
- String errMsg = "SubCluster " + subClusterId.toString()
- + " does not exist; cannot heartbeat";
- FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+ FederationStateStoreUtils.logAndThrowStoreException(
+ LOG, "SubCluster %s does not exist; cannot heartbeat.", request.getSubClusterId());
}
long currentTime =
@@ -205,11 +210,12 @@ public class MemoryFederationStateStore implements FederationStateStore {
}
@Override
- public GetSubClusterInfoResponse getSubCluster(
- GetSubClusterInfoRequest request) throws YarnException {
+ public GetSubClusterInfoResponse getSubCluster(GetSubClusterInfoRequest request)
+ throws YarnException {
FederationMembershipStateStoreInputValidator.validate(request);
SubClusterId subClusterId = request.getSubClusterId();
+
if (!membership.containsKey(subClusterId)) {
LOG.warn("The queried SubCluster: {} does not exist.", subClusterId);
return null;
@@ -219,16 +225,17 @@ public class MemoryFederationStateStore implements FederationStateStore {
}
@Override
- public GetSubClustersInfoResponse getSubClusters(
- GetSubClustersInfoRequest request) throws YarnException {
- List result = new ArrayList();
+ public GetSubClustersInfoResponse getSubClusters(GetSubClustersInfoRequest request)
+ throws YarnException {
+
+ List result = new ArrayList<>();
for (SubClusterInfo info : membership.values()) {
- if (!request.getFilterInactiveSubClusters()
- || info.getState().isActive()) {
+ if (!request.getFilterInactiveSubClusters() || info.getState().isActive()) {
result.add(info);
}
}
+
return GetSubClustersInfoResponse.newInstance(result);
}
@@ -239,16 +246,16 @@ public class MemoryFederationStateStore implements FederationStateStore {
AddApplicationHomeSubClusterRequest request) throws YarnException {
FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
- ApplicationId appId =
- request.getApplicationHomeSubCluster().getApplicationId();
+ ApplicationHomeSubCluster homeSubCluster = request.getApplicationHomeSubCluster();
+
+ ApplicationId appId = homeSubCluster.getApplicationId();
if (!applications.containsKey(appId)) {
- applications.put(appId,
- request.getApplicationHomeSubCluster().getHomeSubCluster());
+ applications.put(appId, homeSubCluster);
}
- return AddApplicationHomeSubClusterResponse
- .newInstance(applications.get(appId));
+ ApplicationHomeSubCluster respHomeSubCluster = applications.get(appId);
+ return AddApplicationHomeSubClusterResponse.newInstance(respHomeSubCluster.getHomeSubCluster());
}
@Override
@@ -256,15 +263,16 @@ public class MemoryFederationStateStore implements FederationStateStore {
UpdateApplicationHomeSubClusterRequest request) throws YarnException {
FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
+
ApplicationId appId =
request.getApplicationHomeSubCluster().getApplicationId();
+
if (!applications.containsKey(appId)) {
- String errMsg = "Application " + appId + " does not exist";
- FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+ FederationStateStoreUtils.logAndThrowStoreException(LOG,
+ "Application %s does not exist.", appId);
}
- applications.put(appId,
- request.getApplicationHomeSubCluster().getHomeSubCluster());
+ applications.put(appId, request.getApplicationHomeSubCluster());
return UpdateApplicationHomeSubClusterResponse.newInstance();
}
@@ -275,11 +283,12 @@ public class MemoryFederationStateStore implements FederationStateStore {
FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
ApplicationId appId = request.getApplicationId();
if (!applications.containsKey(appId)) {
- String errMsg = "Application " + appId + " does not exist";
- FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+ FederationStateStoreUtils.logAndThrowStoreException(LOG,
+ "Application %s does not exist.", appId);
}
- return GetApplicationHomeSubClusterResponse.newInstance(appId, applications.get(appId));
+ return GetApplicationHomeSubClusterResponse.newInstance(appId,
+ applications.get(appId).getHomeSubCluster());
}
@Override
@@ -303,7 +312,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
}
private ApplicationHomeSubCluster generateAppHomeSC(ApplicationId applicationId) {
- SubClusterId subClusterId = applications.get(applicationId);
+ SubClusterId subClusterId = applications.get(applicationId).getHomeSubCluster();
return ApplicationHomeSubCluster.newInstance(applicationId, subClusterId);
}
@@ -314,8 +323,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
FederationApplicationHomeSubClusterStoreInputValidator.validate(request);
ApplicationId appId = request.getApplicationId();
if (!applications.containsKey(appId)) {
- String errMsg = "Application " + appId + " does not exist";
- FederationStateStoreUtils.logAndThrowStoreException(LOG, errMsg);
+ FederationStateStoreUtils.logAndThrowStoreException(LOG,
+ "Application %s does not exist.", appId);
}
applications.remove(appId);
@@ -329,12 +338,11 @@ public class MemoryFederationStateStore implements FederationStateStore {
FederationPolicyStoreInputValidator.validate(request);
String queue = request.getQueue();
if (!policies.containsKey(queue)) {
- LOG.warn("Policy for queue: {} does not exist.", queue);
+ LOG.warn("Policy for queue : {} does not exist.", queue);
return null;
}
- return GetSubClusterPolicyConfigurationResponse
- .newInstance(policies.get(queue));
+ return GetSubClusterPolicyConfigurationResponse.newInstance(policies.get(queue));
}
@Override
@@ -350,8 +358,7 @@ public class MemoryFederationStateStore implements FederationStateStore {
@Override
public GetSubClusterPoliciesConfigurationsResponse getPoliciesConfigurations(
GetSubClusterPoliciesConfigurationsRequest request) throws YarnException {
- ArrayList result =
- new ArrayList();
+ ArrayList result = new ArrayList<>();
for (SubClusterPolicyConfiguration policy : policies.values()) {
result.add(policy);
}
@@ -360,12 +367,22 @@ public class MemoryFederationStateStore implements FederationStateStore {
@Override
public Version getCurrentVersion() {
- return null;
+ throw new NotImplementedException("Code is not implemented");
}
@Override
- public Version loadVersion() {
- return null;
+ public Version loadVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
+ }
+
+ @Override
+ public void storeVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
+ }
+
+ @Override
+ public void checkVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
}
@Override
@@ -386,7 +403,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
FederationReservationHomeSubClusterStoreInputValidator.validate(request);
ReservationId reservationId = request.getReservationId();
if (!reservations.containsKey(reservationId)) {
- throw new YarnException("Reservation " + reservationId + " does not exist");
+ FederationStateStoreUtils.logAndThrowStoreException(LOG,
+ "Reservation %s does not exist.", reservationId);
}
SubClusterId subClusterId = reservations.get(reservationId);
ReservationHomeSubCluster homeSubCluster =
@@ -417,7 +435,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
ReservationId reservationId = request.getReservationHomeSubCluster().getReservationId();
if (!reservations.containsKey(reservationId)) {
- throw new YarnException("Reservation " + reservationId + " does not exist.");
+ FederationStateStoreUtils.logAndThrowStoreException(LOG,
+ "Reservation %s does not exist.", reservationId);
}
SubClusterId subClusterId = request.getReservationHomeSubCluster().getHomeSubCluster();
@@ -431,7 +450,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
FederationReservationHomeSubClusterStoreInputValidator.validate(request);
ReservationId reservationId = request.getReservationId();
if (!reservations.containsKey(reservationId)) {
- throw new YarnException("Reservation " + reservationId + " does not exist");
+ FederationStateStoreUtils.logAndThrowStoreException(LOG,
+ "Reservation %s does not exist.", reservationId);
}
reservations.remove(reservationId);
return DeleteReservationHomeSubClusterResponse.newInstance();
@@ -446,9 +466,8 @@ public class MemoryFederationStateStore implements FederationStateStore {
Set rmDTMasterKeyState = routerRMSecretManagerState.getMasterKeyState();
if (rmDTMasterKeyState.contains(delegationKey)) {
- LOG.info("Error storing info for RMDTMasterKey with keyID: {}.", delegationKey.getKeyId());
- throw new IOException("RMDTMasterKey with keyID: " + delegationKey.getKeyId() +
- " is already stored");
+ FederationStateStoreUtils.logAndThrowStoreException(LOG,
+ "Error storing info for RMDTMasterKey with keyID: %s.", delegationKey.getKeyId());
}
routerRMSecretManagerState.getMasterKeyState().add(delegationKey);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
index f16fe673ce3..e1fc3f2a47e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/SQLFederationStateStore.java
@@ -997,7 +997,17 @@ public class SQLFederationStateStore implements FederationStateStore {
}
@Override
- public Version loadVersion() {
+ public Version loadVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
+ }
+
+ @Override
+ public void storeVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
+ }
+
+ @Override
+ public void checkVersion() throws Exception {
throw new NotImplementedException("Code is not implemented");
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java
index 95903b81d18..536faa31dca 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/federation/store/impl/ZookeeperFederationStateStore.java
@@ -30,6 +30,7 @@ import java.util.TimeZone;
import java.util.Comparator;
import java.util.stream.Collectors;
+import org.apache.commons.lang3.NotImplementedException;
import org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.curator.framework.recipes.shared.VersionedValue;
import org.apache.hadoop.classification.VisibleForTesting;
@@ -642,12 +643,22 @@ public class ZookeeperFederationStateStore implements FederationStateStore {
@Override
public Version getCurrentVersion() {
- return null;
+ throw new NotImplementedException("Code is not implemented");
}
@Override
- public Version loadVersion() {
- return null;
+ public Version loadVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
+ }
+
+ @Override
+ public void storeVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
+ }
+
+ @Override
+ public void checkVersion() throws Exception {
+ throw new NotImplementedException("Code is not implemented");
}
/**
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
index c93115ccfd3..b93763583d7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/federation/store/impl/FederationStateStoreBaseTest.java
@@ -26,6 +26,7 @@ import java.util.Set;
import java.util.HashSet;
import java.util.TimeZone;
+import org.apache.commons.lang3.NotImplementedException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.delegation.DelegationKey;
@@ -164,12 +165,9 @@ public abstract class FederationStateStoreBaseTest {
SubClusterDeregisterRequest deregisterRequest = SubClusterDeregisterRequest
.newInstance(subClusterId, SubClusterState.SC_UNREGISTERED);
- try {
- stateStore.deregisterSubCluster(deregisterRequest);
- Assert.fail();
- } catch (FederationStateStoreException e) {
- Assert.assertTrue(e.getMessage().startsWith("SubCluster SC not found"));
- }
+
+ LambdaTestUtils.intercept(YarnException.class,
+ "SubCluster SC not found", () -> stateStore.deregisterSubCluster(deregisterRequest));
}
@Test
@@ -266,13 +264,9 @@ public abstract class FederationStateStoreBaseTest {
SubClusterHeartbeatRequest heartbeatRequest = SubClusterHeartbeatRequest
.newInstance(subClusterId, SubClusterState.SC_RUNNING, "capability");
- try {
- stateStore.subClusterHeartbeat(heartbeatRequest);
- Assert.fail();
- } catch (FederationStateStoreException e) {
- Assert.assertTrue(e.getMessage()
- .startsWith("SubCluster SC does not exist; cannot heartbeat"));
- }
+ LambdaTestUtils.intercept(YarnException.class,
+ "SubCluster SC does not exist; cannot heartbeat",
+ () -> stateStore.subClusterHeartbeat(heartbeatRequest));
}
// Test FederationApplicationHomeSubClusterStore
@@ -1050,4 +1044,24 @@ public abstract class FederationStateStoreBaseTest {
checkRouterStoreToken(identifier, getStoreTokenResp);
}
+
+ @Test(expected = NotImplementedException.class)
+ public void testGetCurrentVersion() {
+ stateStore.getCurrentVersion();
+ }
+
+ @Test(expected = NotImplementedException.class)
+ public void testStoreVersion() throws Exception {
+ stateStore.storeVersion();
+ }
+
+ @Test(expected = NotImplementedException.class)
+ public void testLoadVersion() throws Exception {
+ stateStore.loadVersion();
+ }
+
+ @Test(expected = NotImplementedException.class)
+ public void testCheckVersion() throws Exception {
+ stateStore.checkVersion();
+ }
}
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 24be1ebd2a9..a66b6664020 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
@@ -30,7 +30,6 @@
${project.parent.parent.basedir}../etc/hadoop
- ../lib/native
@@ -224,7 +223,6 @@
${container-executor.conf.dir}
- ${extra.libhadoop.rpath}${sun.arch.data.model}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
index e2cfbc52df2..677429bb99c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/CMakeLists.txt
@@ -25,8 +25,6 @@ set(GTEST_SRC_DIR ${CMAKE_SOURCE_DIR}/../../../../../hadoop-common-project/hadoo
set(HADOOP_COMMON_SEC_PATH ${HADOOP_COMMON_PATH}/src/main/native/src/org/apache/hadoop/security)
-set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
-
# determine if container-executor.conf.dir is an absolute
# path in case the OS we're compiling on doesn't have
# a hook in get_executable. We'll use this define
@@ -159,18 +157,6 @@ add_executable(container-executor
main/native/container-executor/impl/main.c
)
-# By embedding '$ORIGIN' into the RPATH of container-executor, dlopen will look in
-# the directory containing container-executor. However, $ORIGIN is not supported by
-# all operating systems.
-if(${CMAKE_SYSTEM_NAME} MATCHES "Linux|SunOS")
- set(RPATH "\$ORIGIN/")
- if(EXTRA_LIBHADOOP_RPATH)
- set(RPATH "${RPATH}:${EXTRA_LIBHADOOP_RPATH}/")
- endif()
- message("RPATH SET AS ${RPATH}.")
- set_target_properties(container-executor PROPERTIES INSTALL_RPATH "${RPATH}")
-endif()
-
target_link_libraries(container-executor
container
crypto
@@ -182,19 +168,6 @@ output_directory(container-executor target/usr/local/bin)
add_executable(test-container-executor
main/native/container-executor/test/test-container-executor.c
)
-
-# By embedding '$ORIGIN' into the RPATH of test-container-executor, dlopen will look in
-# the directory containing test-container-executor. However, $ORIGIN is not supported by
-# all operating systems.
-if(${CMAKE_SYSTEM_NAME} MATCHES "Linux|SunOS")
- set(RPATH "\$ORIGIN/")
- if(EXTRA_LIBHADOOP_RPATH)
- set(RPATH "${RPATH}:${EXTRA_LIBHADOOP_RPATH}/")
- endif()
- message("RPATH SET AS ${RPATH}.")
- set_target_properties(test-container-executor PROPERTIES INSTALL_RPATH "${RPATH}")
-endif()
-
target_link_libraries(test-container-executor
container
${EXTRA_LIBS}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
index 14950cac1cf..afa85483d11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
@@ -618,12 +618,12 @@ public class NodesListManager extends CompositeService implements
}
/**
- * Refresh the nodes gracefully
+ * Refresh the nodes gracefully.
*
- * @param yarnConf
+ * @param yarnConf yarn configuration.
* @param timeout decommission timeout, null means default timeout.
- * @throws IOException
- * @throws YarnException
+ * @throws IOException io error occur.
+ * @throws YarnException exceptions from yarn servers.
*/
public void refreshNodesGracefully(Configuration yarnConf, Integer timeout)
throws IOException, YarnException {
@@ -685,6 +685,9 @@ public class NodesListManager extends CompositeService implements
/**
* A NodeId instance needed upon startup for populating inactive nodes Map.
* It only knows the hostname/ip and marks the port to -1 or invalid.
+ *
+ * @param host host name.
+ * @return node id.
*/
public static NodeId createUnknownNodeId(String host) {
return NodeId.newInstance(host, -1);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
index cc54d0b5861..1b3a49433fd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAuditLogger.java
@@ -489,6 +489,7 @@ public class RMAuditLogger {
* @param description Some additional information as to why the operation
* failed.
* @param appId ApplicationId in which operation was performed.
+ * @param attemptId Application Attempt Id in which operation was performed.
*
*
* Note that the {@link RMAuditLogger} uses tabs ('\t') as a key-val delimiter
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.java
index 32369ba53ed..1a0047b8215 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.java
@@ -44,7 +44,8 @@ public class RMSecretManagerService extends AbstractService {
/**
* Construct the service.
- *
+ * @param conf Configuration.
+ * @param rmContext RMContext.
*/
public RMSecretManagerService(Configuration conf, RMContextImpl rmContext) {
super(RMSecretManagerService.class.getName());
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
index de7a7626600..39a27e73feb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMServerUtils.java
@@ -241,7 +241,15 @@ public class RMServerUtils {
/**
* Utility method to validate a list resource requests, by ensuring that the
- * requested memory/vcore is non-negative and not greater than max
+ * requested memory/vcore is non-negative and not greater than max.
+ *
+ * @param ask resource request.
+ * @param maximumAllocation Maximum Allocation.
+ * @param queueName queue name.
+ * @param scheduler YarnScheduler.
+ * @param rmContext RMContext.
+ * @param nodeLabelsEnabled the node labels feature enabled.
+ * @throws InvalidResourceRequestException when there is invalid request.
*/
public static void normalizeAndValidateRequests(List ask,
Resource maximumAllocation, String queueName, YarnScheduler scheduler,
@@ -264,9 +272,13 @@ public class RMServerUtils {
/**
* Validate increase/decrease request.
+ *
*
* - Throw exception when any other error happens
*
+ * @param request SchedContainerChangeRequest.
+ * @param increase true, add container; false, decrease container.
+ * @throws InvalidResourceRequestException when there is invalid request.
*/
public static void checkSchedContainerChangeRequest(
SchedContainerChangeRequest request, boolean increase)
@@ -362,6 +374,7 @@ public class RMServerUtils {
* application master.
* @param appAttemptId Application attempt Id
* @throws InvalidContainerReleaseException
+ * an Application Master tries to release containers not belonging to it using.
*/
public static void
validateContainerReleaseRequest(List containerReleaseList,
@@ -394,7 +407,7 @@ public class RMServerUtils {
* @param module like AdminService or NodeLabelManager
* @param LOG the logger to use
* @return {@link UserGroupInformation} of the current user
- * @throws IOException
+ * @throws IOException an I/O exception has occurred.
*/
public static UserGroupInformation verifyAdminAccess(
YarnAuthorizationProvider authorizer, String method, String module,
@@ -509,7 +522,9 @@ public class RMServerUtils {
/**
* Find all configs whose name starts with
* YarnConfiguration.RM_PROXY_USER_PREFIX, and add a record for each one by
- * replacing the prefix with ProxyUsers.CONF_HADOOP_PROXYUSER
+ * replacing the prefix with ProxyUsers.CONF_HADOOP_PROXYUSER.
+ *
+ * @param conf Configuration.
*/
public static void processRMProxyUsersConf(Configuration conf) {
Map rmProxyUsers = new HashMap();
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 1bcfdbbafa8..da500add4cb 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
@@ -1657,6 +1657,7 @@ public class ResourceManager extends CompositeService
/**
* Create RMDelegatedNodeLabelsUpdater based on configuration.
+ * @return RMDelegatedNodeLabelsUpdater.
*/
protected RMDelegatedNodeLabelsUpdater createRMDelegatedNodeLabelsUpdater() {
if (conf.getBoolean(YarnConfiguration.NODE_LABELS_ENABLED,
@@ -1813,9 +1814,9 @@ public class ResourceManager extends CompositeService
}
/**
- * Retrieve RM bind address from configuration
+ * Retrieve RM bind address from configuration.
*
- * @param conf
+ * @param conf Configuration.
* @return InetSocketAddress
*/
public static InetSocketAddress getBindAddress(Configuration conf) {
@@ -1826,8 +1827,8 @@ public class ResourceManager extends CompositeService
/**
* Deletes the RMStateStore
*
- * @param conf
- * @throws Exception
+ * @param conf Configuration.
+ * @throws Exception error occur.
*/
@VisibleForTesting
static void deleteRMStateStore(Configuration conf) throws Exception {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index b0b148c117f..cf2aed41e00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -194,8 +194,8 @@ public class ResourceTrackerService extends AbstractService implements
/**
* Load DynamicResourceConfiguration from dynamic-resources.xml.
- * @param conf
- * @throws IOException
+ * @param conf Configuration.
+ * @throws IOException an I/O exception has occurred.
*/
public void loadDynamicResourceConfiguration(Configuration conf)
throws IOException {
@@ -219,7 +219,7 @@ public class ResourceTrackerService extends AbstractService implements
/**
* Update DynamicResourceConfiguration with new configuration.
- * @param conf
+ * @param conf DynamicResourceConfiguration.
*/
public void updateDynamicResourceConfiguration(
DynamicResourceConfiguration conf) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
index 92376872919..90dcadb721e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/federation/FederationStateStoreService.java
@@ -266,10 +266,20 @@ public class FederationStateStoreService extends AbstractService
}
@Override
- public Version loadVersion() {
+ public Version loadVersion() throws Exception {
return stateStoreClient.getCurrentVersion();
}
+ @Override
+ public void storeVersion() throws Exception {
+ stateStoreClient.storeVersion();
+ }
+
+ @Override
+ public void checkVersion() throws Exception {
+ stateStoreClient.checkVersion();
+ }
+
@Override
public GetSubClusterPolicyConfigurationResponse getPolicyConfiguration(
GetSubClusterPolicyConfigurationRequest request) throws YarnException {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
index 3382ead7412..99c691c7733 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/CapacitySchedulerPreemptionUtils.java
@@ -144,6 +144,7 @@ public class CapacitySchedulerPreemptionUtils {
* stop preempt container when: all major resource type
* {@literal <=} 0 for to-preempt.
* This is default preemption behavior of inter-queue preemption
+ * @param curCandidates RMContainer Set.
* @return should we preempt rmContainer. If we should, deduct from
* resourceToObtainByPartition
*/
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.java
index 7563c36b924..2c4c7164fed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/PreemptableResourceCalculator.java
@@ -42,13 +42,22 @@ public class PreemptableResourceCalculator
LoggerFactory.getLogger(PreemptableResourceCalculator.class);
/**
- * PreemptableResourceCalculator constructor
+ * PreemptableResourceCalculator constructor.
*
- * @param preemptionContext
+ * @param preemptionContext context.
* @param isReservedPreemptionCandidatesSelector this will be set by
* different implementation of candidate selectors, please refer to
* TempQueuePerPartition#offer for details.
* @param allowQueuesBalanceAfterAllQueuesSatisfied
+ * Should resources be preempted from an over-served queue when the
+ * requesting queues are all at or over their guarantees?
+ * An example is, there're 10 queues under root, guaranteed resource
+ * of them are all 10%.
+ * Assume there're two queues are using resources, queueA uses 10%
+ * queueB uses 90%. For all queues are guaranteed, but it's not fair
+ * for queueA.
+ * We wanna make this behavior can be configured. By default it is
+ * not allowed.
*/
public PreemptableResourceCalculator(
CapacitySchedulerPreemptionContext preemptionContext,
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/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
index b0b98ad9b48..161e104e75b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java
@@ -271,9 +271,11 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
}
/**
- * @param nodeAttributeMapping
- * @param newAttributesToBeAdded
- * @param isRemoveOperation : to indicate whether its a remove operation.
+ * Validate for attributes.
+ *
+ * @param nodeAttributeMapping NodeAttribute Mapping
+ * @param newAttributesToBeAdded new Attributes ToBeAdded
+ * @param isRemoveOperation : to indicate whether it's a remove operation.
* @return Map of String to Map of NodeAttribute to AttributeValue
* @throws IOException : on invalid mapping in the current request or against
* already existing NodeAttributes.
@@ -328,12 +330,13 @@ public class NodeAttributesManagerImpl extends NodeAttributesManager {
}
/**
+ * Validate For AttributeType Mismatch.
*
- * @param isRemoveOperation
- * @param attribute
- * @param newAttributes
- * @return Whether its a new Attribute added
- * @throws IOException
+ * @param isRemoveOperation to indicate whether it's a remove operation.
+ * @param attribute NodeAttribute.
+ * @param newAttributes new Attributes.
+ * @return Whether it's a new Attribute added
+ * @throws IOException an I/O exception of some sort has occurred.
*/
private boolean validateForAttributeTypeMismatch(boolean isRemoveOperation,
NodeAttribute attribute,
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 ba0c430d77f..a4f74a91f7e 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
@@ -102,7 +102,8 @@ public class RMDelegatedNodeLabelsUpdater extends CompositeService {
/**
* Terminate the timer.
- * @throws Exception
+ *
+ * @throws Exception exception occurs.
*/
@Override
protected void serviceStop() throws Exception {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementFactory.java
index 301b3a314dc..bab17775b43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementFactory.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementFactory.java
@@ -46,6 +46,8 @@ public final class PlacementFactory {
* @param ruleStr The name of the class to instantiate
* @param conf The configuration object to set for the rule
* @return Created class instance
+ * @throws ClassNotFoundException
+ * no definition for the class with the specified name could be found.
*/
public static PlacementRule getPlacementRule(String ruleStr,
Configuration conf)
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementRule.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementRule.java
index 50d686ad370..ca4c9e1d7a2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementRule.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/PlacementRule.java
@@ -36,6 +36,7 @@ public abstract class PlacementRule {
/**
* Set the config based on the passed in argument. This construct is used to
* not pollute this abstract class with implementation specific references.
+ * @param initArg initialization arguments.
*/
public void setConfig(Object initArg) {
// Default is a noop
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/VariableContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/VariableContext.java
index e8e419c64eb..f2c5f5422d8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/VariableContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/placement/VariableContext.java
@@ -179,6 +179,7 @@ public class VariableContext {
/**
* Returns the dataset referenced by the name.
* @param name Name of the set to be returned.
+ * @return the dataset referenced by the name.
*/
public Set getExtraDataset(String name) {
return extraDataset.get(name);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
index a93ed1bc6b4..49e25eb8f13 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
@@ -791,7 +791,9 @@ public abstract class RMStateStore extends AbstractService {
/**
* Dispatcher used to send state operation completion events to
- * ResourceManager services
+ * ResourceManager services.
+ *
+ * @param dispatcher Dispatcher.
*/
public void setRMDispatcher(Dispatcher dispatcher) {
this.rmDispatcher = dispatcher;
@@ -827,13 +829,18 @@ public abstract class RMStateStore extends AbstractService {
/**
* Derived classes initialize themselves using this method.
+ *
+ * @param conf Configuration.
+ * @throws Exception error occur.
*/
protected abstract void initInternal(Configuration conf) throws Exception;
/**
* Derived classes start themselves using this method.
* The base class is started and the event dispatcher is ready to use at
- * this point
+ * this point.
+ *
+ * @throws Exception error occur.
*/
protected abstract void startInternal() throws Exception;
@@ -846,7 +853,9 @@ public abstract class RMStateStore extends AbstractService {
/**
* Derived classes close themselves using this method.
* The base class will be closed and the event dispatcher will be shutdown
- * after this
+ * after this.
+ *
+ * @throws Exception error occur.
*/
protected abstract void closeInternal() throws Exception;
@@ -860,6 +869,8 @@ public abstract class RMStateStore extends AbstractService {
* 5) Within a major upgrade, say 1.2 to 2.0:
* throw exception and indicate user to use a separate upgrade tool to
* upgrade RM state.
+ *
+ * @throws Exception error occur.
*/
public void checkVersion() throws Exception {
Version loadedVersion = loadVersion();
@@ -884,22 +895,28 @@ public abstract class RMStateStore extends AbstractService {
/**
* Derived class use this method to load the version information from state
* store.
+ * @throws Exception error occur.
+ * @return current version.
*/
protected abstract Version loadVersion() throws Exception;
/**
* Derived class use this method to store the version information.
+ * @throws Exception error occur.
*/
protected abstract void storeVersion() throws Exception;
/**
* Get the current version of the underlying state store.
+ * @return current version.
*/
protected abstract Version getCurrentVersion();
/**
* Get the current epoch of RM and increment the value.
+ * @throws Exception error occur.
+ * @return current epoch.
*/
public abstract long getAndIncrementEpoch() throws Exception;
@@ -907,6 +924,9 @@ public abstract class RMStateStore extends AbstractService {
* Compute the next epoch value by incrementing by one.
* Wraps around if the epoch range is exceeded so that
* when federation is enabled epoch collisions can be avoided.
+ *
+ * @param epoch epoch value.
+ * @return next epoch value.
*/
protected long nextEpoch(long epoch){
long epochVal = epoch - baseEpoch + 1;
@@ -920,7 +940,9 @@ public abstract class RMStateStore extends AbstractService {
* Blocking API
* The derived class must recover state from the store and return a new
* RMState object populated with that state
- * This must not be called on the dispatcher thread
+ * This must not be called on the dispatcher thread.
+ * @throws Exception error occur.
+ * @return RMState.
*/
public abstract RMState loadState() throws Exception;
@@ -928,7 +950,9 @@ public abstract class RMStateStore extends AbstractService {
* Non-Blocking API
* ResourceManager services use this to store the application's state
* This does not block the dispatcher threads
- * RMAppStoredEvent will be sent on completion to notify the RMApp
+ * RMAppStoredEvent will be sent on completion to notify the RMApp.
+ *
+ * @param app rmApp.
*/
@SuppressWarnings("unchecked")
public void storeNewApplication(RMApp app) {
@@ -969,6 +993,10 @@ public abstract class RMStateStore extends AbstractService {
* Blocking API
* Derived classes must implement this method to store the state of an
* application.
+ *
+ * @param appId application Id.
+ * @param appStateData application StateData.
+ * @throws Exception error occur.
*/
protected abstract void storeApplicationStateInternal(ApplicationId appId,
ApplicationStateData appStateData) throws Exception;
@@ -981,7 +1009,9 @@ public abstract class RMStateStore extends AbstractService {
* Non-blocking API
* ResourceManager services call this to store state on an application attempt
* This does not block the dispatcher threads
- * RMAppAttemptStoredEvent will be sent on completion to notify the RMAppAttempt
+ * RMAppAttemptStoredEvent will be sent on completion to notify the RMAppAttempt.
+ *
+ * @param appAttempt RM AppAttempt.
*/
public void storeNewApplicationAttempt(RMAppAttempt appAttempt) {
Credentials credentials = getCredentialsFromAppAttempt(appAttempt);
@@ -1011,7 +1041,11 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
* Derived classes must implement this method to store the state of an
- * application attempt
+ * application attempt.
+ *
+ * @param attemptId Application AttemptId.
+ * @param attemptStateData Application AttemptStateData.
+ * @throws Exception error occur.
*/
protected abstract void storeApplicationAttemptStateInternal(
ApplicationAttemptId attemptId,
@@ -1023,7 +1057,10 @@ public abstract class RMStateStore extends AbstractService {
/**
* RMDTSecretManager call this to store the state of a delegation token
- * and sequence number
+ * and sequence number.
+ *
+ * @param rmDTIdentifier RMDelegationTokenIdentifier.
+ * @param renewDate token renew date.
*/
public void storeRMDelegationToken(
RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate) {
@@ -1034,14 +1071,20 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
* Derived classes must implement this method to store the state of
- * RMDelegationToken and sequence number
+ * RMDelegationToken and sequence number.
+ *
+ * @param rmDTIdentifier RMDelegationTokenIdentifier.
+ * @param renewDate token renew date.
+ * @throws Exception error occur.
*/
protected abstract void storeRMDelegationTokenState(
RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
throws Exception;
/**
- * RMDTSecretManager call this to remove the state of a delegation token
+ * RMDTSecretManager call this to remove the state of a delegation token.
+ *
+ * @param rmDTIdentifier RMDelegationTokenIdentifier.
*/
public void removeRMDelegationToken(
RMDelegationTokenIdentifier rmDTIdentifier) {
@@ -1051,14 +1094,20 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
- * Derived classes must implement this method to remove the state of RMDelegationToken
+ * Derived classes must implement this method to remove the state of RMDelegationToken.
+ *
+ * @param rmDTIdentifier RMDelegationTokenIdentifier.
+ * @throws Exception error occurs.
*/
protected abstract void removeRMDelegationTokenState(
RMDelegationTokenIdentifier rmDTIdentifier) throws Exception;
/**
* RMDTSecretManager call this to update the state of a delegation token
- * and sequence number
+ * and sequence number.
+ *
+ * @param rmDTIdentifier RMDelegationTokenIdentifier.
+ * @param renewDate token renew date.
*/
public void updateRMDelegationToken(
RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate) {
@@ -1069,14 +1118,20 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
* Derived classes must implement this method to update the state of
- * RMDelegationToken and sequence number
+ * RMDelegationToken and sequence number.
+ *
+ * @param rmDTIdentifier RMDelegationTokenIdentifier.
+ * @param renewDate token renew date.
+ * @throws Exception error occurs.
*/
protected abstract void updateRMDelegationTokenState(
RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate)
throws Exception;
/**
- * RMDTSecretManager call this to store the state of a master key
+ * RMDTSecretManager call this to store the state of a master key.
+ *
+ * @param delegationKey DelegationToken Master Key.
*/
public void storeRMDTMasterKey(DelegationKey delegationKey) {
handleStoreEvent(new RMStateStoreRMDTMasterKeyEvent(delegationKey,
@@ -1086,13 +1141,18 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
* Derived classes must implement this method to store the state of
- * DelegationToken Master Key
+ * DelegationToken Master Key.
+ *
+ * @param delegationKey DelegationToken Master Key.
+ * @throws Exception error occur.
*/
protected abstract void storeRMDTMasterKeyState(DelegationKey delegationKey)
throws Exception;
/**
- * RMDTSecretManager call this to remove the state of a master key
+ * RMDTSecretManager call this to remove the state of a master key.
+ *
+ * @param delegationKey DelegationToken Master Key.
*/
public void removeRMDTMasterKey(DelegationKey delegationKey) {
handleStoreEvent(new RMStateStoreRMDTMasterKeyEvent(delegationKey,
@@ -1101,6 +1161,10 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking Apis to maintain reservation state.
+ *
+ * @param reservationAllocation reservation Allocation.
+ * @param planName plan Name.
+ * @param reservationIdName reservationId Name.
*/
public void storeNewReservation(
ReservationAllocationStateProto reservationAllocation, String planName,
@@ -1120,6 +1184,11 @@ public abstract class RMStateStore extends AbstractService {
* Blocking API
* Derived classes must implement this method to store the state of
* a reservation allocation.
+ *
+ * @param reservationAllocation reservation Allocation.
+ * @param planName plan Name.
+ * @param reservationIdName reservationId Name.
+ * @throws Exception error occurs.
*/
protected abstract void storeReservationState(
ReservationAllocationStateProto reservationAllocation, String planName,
@@ -1129,6 +1198,10 @@ public abstract class RMStateStore extends AbstractService {
* Blocking API
* Derived classes must implement this method to remove the state of
* a reservation allocation.
+ *
+ * @param planName plan Name.
+ * @param reservationIdName reservationId Name.
+ * @throws Exception exception occurs.
*/
protected abstract void removeReservationState(String planName,
String reservationIdName) throws Exception;
@@ -1136,21 +1209,31 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
* Derived classes must implement this method to remove the state of
- * DelegationToken Master Key
+ * DelegationToken Master Key.
+ *
+ * @param delegationKey DelegationKey.
+ * @throws Exception exception occurs.
*/
protected abstract void removeRMDTMasterKeyState(DelegationKey delegationKey)
throws Exception;
/**
* Blocking API Derived classes must implement this method to store or update
- * the state of AMRMToken Master Key
+ * the state of AMRMToken Master Key.
+ *
+ * @param amrmTokenSecretManagerState amrmTokenSecretManagerState.
+ * @param isUpdate true, update; otherwise not update.
+ * @throws Exception exception occurs.
*/
protected abstract void storeOrUpdateAMRMTokenSecretManagerState(
AMRMTokenSecretManagerState amrmTokenSecretManagerState, boolean isUpdate)
throws Exception;
/**
- * Store or Update state of AMRMToken Master Key
+ * Store or Update state of AMRMToken Master Key.
+ *
+ * @param amrmTokenSecretManagerState amrmTokenSecretManagerState.
+ * @param isUpdate true, update; otherwise not update.
*/
public void storeOrUpdateAMRMTokenSecretManager(
AMRMTokenSecretManagerState amrmTokenSecretManagerState, boolean isUpdate) {
@@ -1165,6 +1248,8 @@ public abstract class RMStateStore extends AbstractService {
* store
* This does not block the dispatcher threads
* There is no notification of completion for this operation.
+ *
+ * @param app RMApp.
*/
@SuppressWarnings("unchecked")
public void removeApplication(RMApp app) {
@@ -1183,7 +1268,10 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
* Derived classes must implement this method to remove the state of an
- * application and its attempts
+ * application and its attempts.
+ *
+ * @param appState ApplicationStateData.
+ * @throws Exception error occurs.
*/
protected abstract void removeApplicationStateInternal(
ApplicationStateData appState) throws Exception;
@@ -1194,6 +1282,8 @@ public abstract class RMStateStore extends AbstractService {
* store
* This does not block the dispatcher threads
* There is no notification of completion for this operation.
+ *
+ * @param applicationAttemptId applicationAttemptId.
*/
@SuppressWarnings("unchecked")
public synchronized void removeApplicationAttempt(
@@ -1206,6 +1296,8 @@ public abstract class RMStateStore extends AbstractService {
* Blocking API
* Derived classes must implement this method to remove the state of specified
* attempt.
+ * @param attemptId application attempt id.
+ * @throws Exception exception occurs.
*/
protected abstract void removeApplicationAttemptInternal(
ApplicationAttemptId attemptId) throws Exception;
@@ -1316,7 +1408,7 @@ public abstract class RMStateStore extends AbstractService {
/**
* EventHandler implementation which forward events to the FSRMStateStore
- * This hides the EventHandle methods of the store from its public interface
+ * This hides the EventHandle methods of the store from its public interface
*/
private final class ForwardingEventHandler
implements EventHandler {
@@ -1328,16 +1420,18 @@ public abstract class RMStateStore extends AbstractService {
}
/**
- * Derived classes must implement this method to delete the state store
- * @throws Exception
+ * Derived classes must implement this method to delete the state store.
+ *
+ * @throws Exception exception occurs.
*/
public abstract void deleteStore() throws Exception;
/**
* Derived classes must implement this method to remove application from the
- * state store
- *
- * @throws Exception
+ * state store.
+ *
+ * @param removeAppId application Id.
+ * @throws Exception exception occurs.
*/
public abstract void removeApplication(ApplicationId removeAppId)
throws Exception;
@@ -1362,6 +1456,8 @@ public abstract class RMStateStore extends AbstractService {
/**
* ProxyCAManager calls this to store the CA Certificate and Private Key.
+ * @param caCert X509Certificate.
+ * @param caPrivateKey PrivateKey.
*/
public void storeProxyCACert(X509Certificate caCert,
PrivateKey caPrivateKey) {
@@ -1372,7 +1468,11 @@ public abstract class RMStateStore extends AbstractService {
/**
* Blocking API
* Derived classes must implement this method to store the CA Certificate
- * and Private Key
+ * and Private Key.
+ *
+ * @param caCert X509Certificate.
+ * @param caPrivateKey PrivateKey.
+ * @throws Exception error occurs.
*/
protected abstract void storeProxyCACertState(
X509Certificate caCert, PrivateKey caPrivateKey) throws Exception;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreUtils.java
index 870c0ff30a9..13a4d849556 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreUtils.java
@@ -48,6 +48,7 @@ public class RMStateStoreUtils {
* @param fsIn The {@link DataInputStream} containing RM Delegation Token data
* @return An {@link RMDelegationTokenIdentifierData} containing the read in
* RM Delegation Token
+ * @throws IOException an I/O exception has occurred.
*/
public static RMDelegationTokenIdentifierData
readRMDelegationTokenIdentifierData(DataInputStream fsIn)
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/package-info.java
index cbbeb3d01d0..da5455f7b6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/package-info.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/package-info.java
@@ -16,6 +16,9 @@
* limitations under the License.
*/
+/**
+ * This package contains classes to recovery the resourcemanager.
+ */
@InterfaceAudience.Private
@InterfaceStability.Unstable
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/AMRMTokenSecretManagerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/AMRMTokenSecretManagerState.java
index 89b4ff0fc7a..fc03d7a948a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/AMRMTokenSecretManagerState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/AMRMTokenSecretManagerState.java
@@ -51,7 +51,8 @@ public abstract class AMRMTokenSecretManagerState {
}
/**
- * {@link AMRMTokenSecretManager} current Master key
+ * {@link AMRMTokenSecretManager} current Master key.
+ * @return MasterKey.
*/
@Public
@Unstable
@@ -62,7 +63,8 @@ public abstract class AMRMTokenSecretManagerState {
public abstract void setCurrentMasterKey(MasterKey currentMasterKey);
/**
- * {@link AMRMTokenSecretManager} next Master key
+ * {@link AMRMTokenSecretManager} next Master key.
+ * @return MasterKey.
*/
@Public
@Unstable
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java
index 27e80cd30e3..31972d47129 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationAttemptStateData.java
@@ -143,7 +143,7 @@ public abstract class ApplicationAttemptStateData {
/**
* Set the final tracking Url of the AM.
- * @param url
+ * @param url tracking url.
*/
public abstract void setFinalTrackingUrl(String url);
/**
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java
index 8b908e56d0a..c94b27770f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/records/ApplicationStateData.java
@@ -192,7 +192,8 @@ public abstract class ApplicationStateData {
public abstract void setLaunchTime(long launchTime);
/**
- * The application submitter
+ * The application submitter.
+ * @param user submitter user name.
*/
@Public
@Unstable
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
index 3f350961d59..687890e286e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/AbstractSchedulerPlanFollower.java
@@ -380,6 +380,10 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
/**
* Add a new reservation queue for reservation currResId for this planQueue.
+ *
+ * @param planQueueName name of the reservable queue.
+ * @param queue the queue for the current {@link Plan}.
+ * @param currResId curr reservationId.
*/
protected abstract void addReservationQueue(String planQueueName, Queue queue,
String currResId);
@@ -399,6 +403,7 @@ public abstract class AbstractSchedulerPlanFollower implements PlanFollower {
* Get plan resources for this planQueue.
*
* @param plan the current {@link Plan} being considered
+ * @param queue the queue for the current {@link Plan}
* @param clusterResources the resources available in the cluster
*
* @return the resources allocated to the specified {@link Plan}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
index 39b69249f94..894abd0f073 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/PeriodicRLESparseResourceAllocation.java
@@ -178,7 +178,6 @@ public class PeriodicRLESparseResourceAllocation
* maximum capacity.
* @param period periodic offset at which capacities are evaluated.
* @return the maximum {@link Resource} across the specified time instants.
- * @return true if removal is successful, false otherwise
*/
public Resource getMaximumPeriodicCapacity(long tick, long period) {
Resource maxResource;
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index 949b34e8bb5..fbf54a18131 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -290,8 +290,10 @@ public interface RMApp extends EventHandler {
Map getLogAggregationReportsForApp();
LogAggregationStatus getLogAggregationStatusForAppReport();
+
/**
* Return the node label expression of the AM container.
+ * @return the node label expression.
*/
String getAmNodeLabelExpression();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index ca88b8be328..e73b66a259d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -1900,10 +1900,11 @@ public class RMAppImpl implements RMApp, Recoverable {
}
/**
- * catch the InvalidStateTransition.
- * @param state
- * @param rmAppEventType
- */
+ * catch the InvalidStateTransition.
+ *
+ * @param state RMAppState.
+ * @param rmAppEventType RMAppEventType.
+ */
protected void onInvalidStateTransition(RMAppEventType rmAppEventType,
RMAppState state){
/* TODO fail the application on the failed transition */
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
index 5d78c8b354d..5044169607b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
@@ -241,6 +241,8 @@ public interface RMAppAttempt extends EventHandler {
*
*
killed by RM because of RM restart or failover.
*
+ *
+ * @return attempt retry count.
*/
boolean shouldCountTowardsMaxAttemptRetry();
@@ -258,7 +260,7 @@ public interface RMAppAttempt extends EventHandler {
/**
* To capture Launch diagnostics of the app.
- * @param amLaunchDiagnostics
+ * @param amLaunchDiagnostics amLaunchDiagnostics.
*/
void updateAMLaunchDiagnostics(String amLaunchDiagnostics);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
index 7e417cc9dc8..22d4eaafd75 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainerImpl.java
@@ -922,8 +922,8 @@ public class RMContainerImpl implements RMContainer {
/**
* catch the InvalidStateTransition.
- * @param state
- * @param rmContainerEventType
+ * @param state RMContainerState.
+ * @param rmContainerEventType RMContainerEventType.
*/
@VisibleForTesting
protected void onInvalidStateTransition(
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
index 5d60b4fbe06..daf44891b6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
@@ -95,7 +95,8 @@ public interface RMNode {
/**
* the node manager version of the node received as part of the
- * registration with the resource manager
+ * registration with the resource manager.
+ * @return node manager version.
*/
public String getNodeManagerVersion();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
index b8aaea5de33..81a9ed8efc1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
@@ -1203,8 +1203,8 @@ public class RMNodeImpl implements RMNode, EventHandler {
/**
* Put a node in deactivated (decommissioned or shutdown) status.
- * @param rmNode
- * @param finalState
+ * @param rmNode RMNode.
+ * @param finalState NodeState.
*/
public static void deactivateNode(RMNodeImpl rmNode, NodeState finalState) {
@@ -1226,8 +1226,8 @@ public class RMNodeImpl implements RMNode, EventHandler {
/**
* Report node is RUNNING.
- * @param rmNode
- * @param containers
+ * @param rmNode RMNode.
+ * @param containers NMContainerStatus List.
*/
public static void reportNodeRunning(RMNodeImpl rmNode,
List containers) {
@@ -1242,8 +1242,8 @@ public class RMNodeImpl implements RMNode, EventHandler {
/**
* Report node is UNUSABLE and update metrics.
- * @param rmNode
- * @param finalState
+ * @param rmNode RMNode.
+ * @param finalState NodeState.
*/
public static void reportNodeUnusable(RMNodeImpl rmNode,
NodeState finalState) {
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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 8772ddb44e6..4ccc30ce39f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -314,6 +314,7 @@ public abstract class AbstractYarnScheduler
* Add blacklisted NodeIds to the list that is passed.
*
* @param app application attempt.
+ * @return blacklisted NodeIds.
*/
public List getBlacklistedNodes(final SchedulerApplicationAttempt app) {
@@ -850,6 +851,9 @@ public abstract class AbstractYarnScheduler
/**
* Process resource update on a node.
+ *
+ * @param nm RMNode.
+ * @param resourceOption resourceOption.
*/
public void updateNodeResource(RMNode nm,
ResourceOption resourceOption) {
@@ -1329,6 +1333,7 @@ public abstract class AbstractYarnScheduler
* Normalize a list of resource requests
* using queue maximum resource allocations.
* @param asks resource requests
+ * @param queueName queue Name.
*/
protected void normalizeResourceRequests(List asks,
String 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/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 71d657debee..33504a5ceae 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
@@ -39,6 +39,9 @@ public class ApplicationPlacementAllocatorFactory {
*
* @param appPlacementAllocatorName
* allocator class name.
+ * @param appSchedulingInfo app SchedulingInfo.
+ * @param schedulerRequestKey scheduler RequestKey.
+ * @param rmContext RMContext.
* @return Specific AppPlacementAllocator instance based on type
*/
public static AppPlacementAllocator getAppPlacementAllocator(
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/MutableConfigurationProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
index b44e2bf650a..a9059872af5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/MutableConfigurationProvider.java
@@ -82,6 +82,7 @@ public interface MutableConfigurationProvider {
/**
* Get the last updated scheduler config version.
* @return Last updated scheduler config version.
+ * @throws Exception exception occurs.
*/
long getConfigVersion() throws Exception;
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/NodeFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/NodeFilter.java
index 7b3e7a2ce75..ce1f375d249 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/NodeFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/NodeFilter.java
@@ -28,6 +28,9 @@ public interface NodeFilter {
/**
* Criteria to accept node in the filtered list.
+ *
+ * @param node SchedulerNode.
+ * @return true, node accept; false, node not accept.
*/
boolean accept(SchedulerNode node);
}
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/PreemptableResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
index b73c5386690..249637c3a6d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PreemptableResourceScheduler.java
@@ -42,8 +42,8 @@ public interface PreemptableResourceScheduler extends ResourceScheduler {
void markContainerForPreemption(ApplicationAttemptId aid, RMContainer container);
/**
- * Ask the scheduler to forcibly interrupt the container given as input
- * @param container
+ * Ask the scheduler to forcibly interrupt the container given as input.
+ * @param container RMContainer.
*/
void markContainerForKillable(RMContainer 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/QueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
index 0bfee4d3350..82bff453aa0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
@@ -314,7 +314,7 @@ public class QueueMetrics implements MetricsSource {
* QueueMetrics (B)
* metrics
*
- * @param partition
+ * @param partition Node Partition
* @return QueueMetrics
*/
public synchronized QueueMetrics getPartitionQueueMetrics(String partition) {
@@ -603,7 +603,7 @@ public class QueueMetrics implements MetricsSource {
/**
* Set Available resources with support for resource vectors.
*
- * @param limit
+ * @param limit Resource.
*/
public void setAvailableResources(Resource limit) {
availableMB.set(limit.getMemorySize());
@@ -631,7 +631,7 @@ public class QueueMetrics implements MetricsSource {
* resources become available.
*
* @param partition Node Partition
- * @param user
+ * @param user Name of the user.
* @param limit resource limit
*/
public void setAvailableResourcesToUser(String partition, String user,
@@ -657,8 +657,8 @@ public class QueueMetrics implements MetricsSource {
* Increment pending resource metrics
*
* @param partition Node Partition
- * @param user
- * @param containers
+ * @param user Name of the user.
+ * @param containers containers count.
* @param res the TOTAL delta of resources note this is different from the
* other APIs which use per container resource
*/
@@ -852,8 +852,8 @@ public class QueueMetrics implements MetricsSource {
/**
* Allocate Resource for container size change.
* @param partition Node Partition
- * @param user
- * @param res
+ * @param user Name of the user
+ * @param res Resource.
*/
public void allocateResources(String partition, String user, Resource res) {
allocatedMB.incr(res.getMemorySize());
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/ResourceScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java
index dcb6edd3ba3..84f3467703b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceScheduler.java
@@ -49,7 +49,8 @@ public interface ResourceScheduler extends YarnScheduler, Recoverable {
/**
* Re-initialize the ResourceScheduler.
* @param conf configuration
- * @throws IOException
+ * @param rmContext RMContext.
+ * @throws IOException an I/O exception has occurred.
*/
void reinitialize(Configuration conf, RMContext rmContext) throws IOException;
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/SchedContainerChangeRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
index 94b006cdb22..fb3181e0b2c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedContainerChangeRequest.java
@@ -65,7 +65,8 @@ public class SchedContainerChangeRequest implements
}
/**
* Delta capacity = target - before, so if it is a decrease request, delta
- * capacity will be negative
+ * capacity will be negative.
+ * @return delta Capacity.
*/
public synchronized Resource getDeltaCapacity() {
// Only calculate deltaCapacity once
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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 2c84d399242..5121453e395 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -801,6 +801,7 @@ public class SchedulerApplicationAttempt implements SchedulableEntity {
* Called when AM heartbeats. These containers were recovered by the RM after
* the AM had registered. They are reported to the AM in the
* AllocateResponse#containersFromPreviousAttempts.
+ * @return Container List.
*/
public List pullPreviousAttemptContainers() {
writeLock.lock();
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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 2d1ddd70cf7..1ffd6a758f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -246,6 +246,8 @@ public abstract class SchedulerNode {
/**
* Check if a container is launched by this node.
+ *
+ * @param containerId containerId.
* @return If the container is launched by the node.
*/
public synchronized boolean isValidContainer(ContainerId containerId) {
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/SchedulerUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
index 05370c6c525..1c7446b4936 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerUtils.java
@@ -199,6 +199,12 @@ public class SchedulerUtils {
/**
* Utility method to normalize a resource request, by ensuring that the
* requested memory is a multiple of minMemory and is not zero.
+ *
+ * @param ask resource request.
+ * @param resourceCalculator {@link ResourceCalculator} the resource
+ * calculator to use.
+ * @param minimumResource minimum Resource.
+ * @param maximumResource maximum Resource.
*/
@VisibleForTesting
public static void normalizeRequest(
@@ -215,6 +221,12 @@ public class SchedulerUtils {
* Utility method to normalize a resource request, by ensuring that the
* requested memory is a multiple of increment resource and is not zero.
*
+ * @param ask resource request.
+ * @param resourceCalculator {@link ResourceCalculator} the resource
+ * calculator to use.
+ * @param minimumResource minimum Resource.
+ * @param maximumResource maximum Resource.
+ * @param incrementResource increment Resource.
* @return normalized resource
*/
public static Resource getNormalizedResource(
@@ -531,7 +543,14 @@ public class SchedulerUtils {
/**
* Check queue label expression, check if node label in queue's
- * node-label-expression existed in clusterNodeLabels if rmContext != null
+ * node-label-expression existed in clusterNodeLabels if rmContext != null.
+ *
+ * @param queueLabels queue Labels.
+ * @param labelExpression label expression.
+ * @param rmContext rmContext.
+ * @return true, if node label in queue's node-label-expression existed in clusterNodeLabels;
+ * otherwise false.
+ *
*/
public static boolean checkQueueLabelExpression(Set queueLabels,
String labelExpression, RMContext rmContext) {
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/YarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
index a975ad90bd6..e06517bf484 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/YarnScheduler.java
@@ -61,12 +61,13 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.SettableFu
public interface YarnScheduler extends EventHandler {
/**
- * Get queue information
+ * Get queue information.
+ *
* @param queueName queue name
* @param includeChildQueues include child queues?
* @param recursive get children queues?
* @return queue information
- * @throws IOException
+ * @throws IOException an I/O exception has occurred.
*/
@Public
@Stable
@@ -153,7 +154,8 @@ public interface YarnScheduler extends EventHandler {
/**
* Get node resource usage report.
- * @param nodeId
+ *
+ * @param nodeId nodeId.
* @return the {@link SchedulerNodeReport} for the node or null
* if nodeId does not point to a defined node.
*/
@@ -191,10 +193,11 @@ public interface YarnScheduler extends EventHandler {
/**
* Check if the user has permission to perform the operation.
* If the user has {@link QueueACL#ADMINISTER_QUEUE} permission,
- * this user can view/modify the applications in this queue
- * @param callerUGI
- * @param acl
- * @param queueName
+ * this user can view/modify the applications in this queue.
+ *
+ * @param callerUGI caller UserGroupInformation.
+ * @param acl queue ACL.
+ * @param queueName queue Name.
* @return true if the user has the permission,
* false otherwise
*/
@@ -212,7 +215,8 @@ public interface YarnScheduler extends EventHandler {
/**
* Get the container for the given containerId.
- * @param containerId
+ *
+ * @param containerId the given containerId.
* @return the container for the given containerId.
*/
@LimitedPrivate("yarn")
@@ -220,9 +224,9 @@ public interface YarnScheduler extends EventHandler {
public RMContainer getRMContainer(ContainerId containerId);
/**
- * Moves the given application to the given queue
- * @param appId
- * @param newQueue
+ * Moves the given application to the given queue.
+ * @param appId application Id
+ * @param newQueue the given queue.
* @return the name of the queue the application was placed into
* @throws YarnException if the move cannot be carried out
*/
@@ -246,9 +250,9 @@ public interface YarnScheduler extends EventHandler {
* Completely drain sourceQueue of applications, by moving all of them to
* destQueue.
*
- * @param sourceQueue
- * @param destQueue
- * @throws YarnException
+ * @param sourceQueue sourceQueue.
+ * @param destQueue destQueue.
+ * @throws YarnException when yarn exception occur.
*/
void moveAllApps(String sourceQueue, String destQueue) throws YarnException;
@@ -256,7 +260,7 @@ public interface YarnScheduler extends EventHandler {
* Terminate all applications in the specified queue.
*
* @param queueName the name of queue to be drained
- * @throws YarnException
+ * @throws YarnException when yarn exception occur.
*/
void killAllAppsInQueue(String queueName) throws YarnException;
@@ -266,7 +270,7 @@ public interface YarnScheduler extends EventHandler {
* must be a leaf, etc..).
*
* @param queueName name of the queue to remove
- * @throws YarnException
+ * @throws YarnException when yarn exception occur.
*/
void removeQueue(String queueName) throws YarnException;
@@ -276,7 +280,8 @@ public interface YarnScheduler extends EventHandler {
* attached to existing parent, must have zero entitlement).
*
* @param newQueue the queue being added.
- * @throws YarnException
+ * @throws YarnException when yarn exception occur.
+ * @throws IOException when io exception occur.
*/
void addQueue(Queue newQueue) throws YarnException, IOException;
@@ -289,14 +294,15 @@ public interface YarnScheduler extends EventHandler {
* @param queue the queue for which we change entitlement
* @param entitlement the new entitlement for the queue (capacity,
* maxCapacity, etc..)
- * @throws YarnException
+ * @throws YarnException when yarn exception occur.
*/
void setEntitlement(String queue, QueueEntitlement entitlement)
throws YarnException;
/**
- * Gets the list of names for queues managed by the Reservation System
+ * Gets the list of names for queues managed by the Reservation System.
* @return the list of queues which support reservations
+ * @throws YarnException when yarn exception occur.
*/
public Set getPlanQueues() throws YarnException;
@@ -322,6 +328,7 @@ public interface YarnScheduler extends EventHandler {
* @param applicationId
* Application ID
* @return Updated Priority from scheduler
+ * @throws YarnException when yarn exception occur.
*/
public Priority checkAndGetApplicationPriority(Priority priorityRequestedByApp,
UserGroupInformation user, String queuePath, ApplicationId applicationId)
@@ -339,6 +346,7 @@ public interface YarnScheduler extends EventHandler {
* @param user who submitted the application
*
* @return updated priority
+ * @throws YarnException when yarn exception occur.
*/
public Priority updateApplicationPriority(Priority newPriority,
ApplicationId applicationId, SettableFuture future,
@@ -355,15 +363,18 @@ public interface YarnScheduler extends EventHandler {
List getTransferredContainers(ApplicationAttemptId appAttemptId);
/**
- * Set the cluster max priority
+ * Set the cluster max priority.
*
- * @param conf
- * @throws YarnException
+ * @param conf Configuration.
+ * @throws YarnException when yarn exception occur.
*/
void setClusterMaxPriority(Configuration conf) throws YarnException;
/**
- * @param attemptId
+ * Get pending resource request for specified application attempt.
+ *
+ * @param attemptId the id of the application attempt
+ * @return pending resource requests.
*/
List getPendingResourceRequestsForAttempt(
ApplicationAttemptId attemptId);
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/AbstractAutoCreatedLeafQueue.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/AbstractAutoCreatedLeafQueue.java
index 15960c8de84..ca8a7f9e44e 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/AbstractAutoCreatedLeafQueue.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/AbstractAutoCreatedLeafQueue.java
@@ -53,7 +53,7 @@ public class AbstractAutoCreatedLeafQueue extends AbstractLeafQueue {
*
* @param entitlement the new entitlement for the queue (capacity,
* maxCapacity, etc..)
- * @throws SchedulerDynamicEditException
+ * @throws SchedulerDynamicEditException when setEntitlement fails.
*/
public void setEntitlement(QueueEntitlement entitlement)
throws SchedulerDynamicEditException {
@@ -86,11 +86,12 @@ public class AbstractAutoCreatedLeafQueue extends AbstractLeafQueue {
/**
* This methods to change capacity for a queue and adjusts its
- * absoluteCapacity
+ * absoluteCapacity.
*
+ * @param nodeLabel nodeLabel.
* @param entitlement the new entitlement for the queue (capacity,
* maxCapacity, etc..)
- * @throws SchedulerDynamicEditException
+ * @throws SchedulerDynamicEditException when setEntitlement fails.
*/
public void setEntitlement(String nodeLabel, QueueEntitlement entitlement)
throws SchedulerDynamicEditException {
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 f9304cc9604..e8bed4604f6 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
@@ -659,6 +659,10 @@ public abstract class AbstractCSQueue implements CSQueue {
/**
* Returns whether we should continue to look at all heart beating nodes even
* after the reservation limit was hit.
+ *
+ * @return true,
+ * continue to look at all heart beating nodes even after the reservation limit was hit.
+ * otherwise false.
*/
@Private
public boolean isReservationsContinueLooking() {
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 72ea63a2fc5..b451ee7c3a4 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
@@ -323,6 +323,7 @@ public class AbstractLeafQueue extends AbstractCSQueue {
/**
* Used only by tests.
+ * @return minimumAllocationFactor.
*/
@Private
public float getMinimumAllocationFactor() {
@@ -331,6 +332,7 @@ public class AbstractLeafQueue extends AbstractCSQueue {
/**
* Used only by tests.
+ * @return maxAMResourcePerQueuePercent.
*/
@Private
public float getMaxAMResourcePerQueuePercent() {
@@ -2102,6 +2104,7 @@ public class AbstractLeafQueue extends AbstractCSQueue {
/**
* Obtain (read-only) collection of pending applications.
+ * @return collection of pending applications.
*/
public Collection getPendingApplications() {
return Collections.unmodifiableCollection(pendingOrderingPolicy
@@ -2110,6 +2113,8 @@ public class AbstractLeafQueue extends AbstractCSQueue {
/**
* Obtain (read-only) collection of active applications.
+ *
+ * @return collection of active applications.
*/
public Collection getApplications() {
return Collections.unmodifiableCollection(orderingPolicy
@@ -2118,6 +2123,8 @@ public class AbstractLeafQueue extends AbstractCSQueue {
/**
* Obtain (read-only) collection of all applications.
+ *
+ * @return collection of all applications.
*/
public Collection getAllApplications() {
Collection apps = new HashSet(
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/AbstractManagedParentQueue.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/AbstractManagedParentQueue.java
index 1c25ce2928b..e6d61e33075 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/AbstractManagedParentQueue.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/AbstractManagedParentQueue.java
@@ -64,7 +64,8 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
/**
* Add the specified child queue.
* @param childQueue reference to the child queue to be added
- * @throws SchedulerDynamicEditException
+ * @throws SchedulerDynamicEditException when addChildQueue fails.
+ * @throws IOException an I/O exception has occurred.
*/
public void addChildQueue(CSQueue childQueue)
throws SchedulerDynamicEditException, IOException {
@@ -87,7 +88,7 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
/**
* Remove the specified child queue.
* @param childQueue reference to the child queue to be removed
- * @throws SchedulerDynamicEditException
+ * @throws SchedulerDynamicEditException when removeChildQueue fails.
*/
public void removeChildQueue(CSQueue childQueue)
throws SchedulerDynamicEditException {
@@ -113,7 +114,8 @@ public abstract class AbstractManagedParentQueue extends ParentQueue {
/**
* Remove the specified child queue.
* @param childQueueName name of the child queue to be removed
- * @throws SchedulerDynamicEditException
+ * @return child queue.
+ * @throws SchedulerDynamicEditException when removeChildQueue fails.
*/
public CSQueue removeChildQueue(String childQueueName)
throws SchedulerDynamicEditException {
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/AutoCreatedQueueManagementPolicy.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/AutoCreatedQueueManagementPolicy.java
index bf99c3f30c3..cea3df4e03b 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/AutoCreatedQueueManagementPolicy.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/AutoCreatedQueueManagementPolicy.java
@@ -26,22 +26,25 @@ import java.util.List;
public interface AutoCreatedQueueManagementPolicy {
/**
- * Initialize policy
+ * Initialize policy.
* @param parentQueue parent queue
+ * @throws IOException an I/O exception has occurred.
*/
void init(ParentQueue parentQueue) throws IOException;
/**
- * Reinitialize policy state ( if required )
+ * Reinitialize policy state ( if required ).
* @param parentQueue parent queue
+ * @throws IOException an I/O exception has occurred.
*/
void reinitialize(ParentQueue parentQueue) throws IOException;
/**
- * Get initial template for the specified leaf queue
+ * Get initial template for the specified leaf queue.
* @param leafQueue the leaf queue
* @return initial leaf queue template configurations and capacities for
* auto created queue
+ * @throws SchedulerDynamicEditException when get initialLeafQueue Configuration fails.
*/
AutoCreatedLeafQueueConfig getInitialLeafQueueConfiguration(
AbstractAutoCreatedLeafQueue leafQueue)
@@ -56,13 +59,17 @@ public interface AutoCreatedQueueManagementPolicy {
* are done in commitQueueManagementChanges.
*
* @return returns a list of suggested QueueEntitlementChange(s) which may
- * or may not be be enforced by the scheduler
+ * or may not be enforced by the scheduler
+ * @throws SchedulerDynamicEditException when compute QueueManagementChanges fails.
*/
List computeQueueManagementChanges()
throws SchedulerDynamicEditException;
/**
* Commit/Update state for the specified queue management changes.
+ *
+ * @param queueManagementChanges QueueManagementChange List.
+ * @throws SchedulerDynamicEditException when commit QueueManagementChanges fails.
*/
void commitQueueManagementChanges(
List queueManagementChanges)
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/CSMaxRunningAppsEnforcer.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/CSMaxRunningAppsEnforcer.java
index fedde057a90..83068e20786 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/CSMaxRunningAppsEnforcer.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/CSMaxRunningAppsEnforcer.java
@@ -172,6 +172,8 @@ public class CSMaxRunningAppsEnforcer {
*
* Runs in O(n log(n)) where n is the number of queues that are under the
* highest queue that went from having no slack to having slack.
+ *
+ * @param app FiCaSchedulerApp.
*/
public void updateRunnabilityOnAppRemoval(FiCaSchedulerApp app) {
// childqueueX might have no pending apps itself, but if a queue higher up
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 91dab98ce76..72f3cd16fe8 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
@@ -190,12 +190,16 @@ public interface CSQueue extends SchedulerQueue {
* @param applicationId the applicationId of the application being submitted
* @param user user who submitted the application
* @param queue queue to which the application is submitted
+ * @throws AccessControlException if any acl violation is there.
*/
public void submitApplication(ApplicationId applicationId, String user,
String queue) throws AccessControlException;
/**
* Submit an application attempt to the queue.
+ *
+ * @param application application whose attempt is being submitted.
+ * @param userName userName who submitted the application.
*/
public void submitApplicationAttempt(FiCaSchedulerApp application,
String userName);
@@ -211,13 +215,16 @@ public interface CSQueue extends SchedulerQueue {
/**
* An application submitted to this queue has finished.
- * @param applicationId
+ * @param applicationId applicationId.
* @param user user who submitted the application
*/
public void finishApplication(ApplicationId applicationId, String user);
/**
* An application attempt submitted to this queue has finished.
+ *
+ * @param application application attempt.
+ * @param queue queue.
*/
public void finishApplicationAttempt(FiCaSchedulerApp application,
String queue);
@@ -266,6 +273,7 @@ public interface CSQueue extends SchedulerQueue {
* Reinitialize the queue.
* @param newlyParsedQueue new queue to re-initalize from
* @param clusterResource resources in the cluster
+ * @throws IOException an I/O exception has occurred.
*/
public void reinitialize(CSQueue newlyParsedQueue, Resource clusterResource)
throws IOException;
@@ -346,6 +354,10 @@ public interface CSQueue extends SchedulerQueue {
/**
* When partition of node updated, we will update queue's resource usage if it
* has container(s) running on that.
+ *
+ * @param nodePartition node label.
+ * @param resourceToInc resource.
+ * @param application application.
*/
public void incUsedResource(String nodePartition, Resource resourceToInc,
SchedulerApplicationAttempt application);
@@ -353,6 +365,10 @@ public interface CSQueue extends SchedulerQueue {
/**
* When partition of node updated, we will update queue's resource usage if it
* has container(s) running on that.
+ *
+ * @param nodePartition node label.
+ * @param resourceToDec resource.
+ * @param application application.
*/
public void decUsedResource(String nodePartition, Resource resourceToDec,
SchedulerApplicationAttempt application);
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 ecb285748b5..aa90f14e3d4 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
@@ -97,6 +97,11 @@ public class CSQueueUtils {
/**
* Update partitioned resource usage, if nodePartition == null, will update
* used resource for all partitions of this queue.
+ *
+ * @param rc resource calculator.
+ * @param totalPartitionResource total Partition Resource.
+ * @param nodePartition node label.
+ * @param childQueue child queue.
*/
public static void updateUsedCapacity(final ResourceCalculator rc,
final Resource totalPartitionResource, String nodePartition,
@@ -212,6 +217,12 @@ public class CSQueueUtils {
* When nodePartition is null, all partition of
* used-capacity/absolute-used-capacity will be updated.
*
+ *
+ * @param rc resource calculator.
+ * @param cluster cluster resource.
+ * @param childQueue child queue.
+ * @param nlm RMNodeLabelsManager.
+ * @param nodePartition node label.
*/
@Lock(CSQueue.class)
public static void updateQueueStatistics(
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/CapacitySchedulerConfigValidator.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/CapacitySchedulerConfigValidator.java
index 5ddc709b57c..a0467e21c81 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/CapacitySchedulerConfigValidator.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/CapacitySchedulerConfigValidator.java
@@ -119,6 +119,8 @@ public final class CapacitySchedulerConfigValidator {
*
* @param queues existing queues
* @param newQueues new queues
+ * @param newConf Capacity Scheduler Configuration.
+ * @throws IOException an I/O exception has occurred.
*/
public static void validateQueueHierarchy(
CSQueueStore 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/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 757120e1621..85731310e42 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
@@ -1709,7 +1709,10 @@ public class CapacitySchedulerConfiguration extends ReservationSchedulerConfigur
}
/**
- * Get configured node labels in a given queuePath
+ * Get configured node labels in a given queuePath.
+ *
+ * @param queuePath queue path.
+ * @return configured node labels.
*/
public Set getConfiguredNodeLabels(String queuePath) {
Set configuredNodeLabels = new HashSet();
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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
index 5795a7e30fa..e60a92ffd33 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java
@@ -58,6 +58,7 @@ public interface CapacitySchedulerContext {
/**
* Get the yarn configuration.
+ * @return yarn configuration.
*/
Configuration getConf();
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/ManagedParentQueue.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/ManagedParentQueue.java
index 6867c6baf9d..4f7ed405184 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/ManagedParentQueue.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/ManagedParentQueue.java
@@ -389,9 +389,11 @@ public class ManagedParentQueue extends AbstractManagedParentQueue {
}
/**
- * Asynchronously called from scheduler to apply queue management changes
+ * Asynchronously called from scheduler to apply queue management changes.
*
- * @param queueManagementChanges
+ * @param queueManagementChanges QueueManagementChange List.
+ * @throws IOException an I/O exception has occurred.
+ * @throws SchedulerDynamicEditException when validate and apply QueueManagementChanges fails.
*/
public void validateAndApplyQueueManagementChanges(
List queueManagementChanges)
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 3bbbffe5881..5051c739d5c 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
@@ -188,7 +188,8 @@ public class PlanQueue extends AbstractManagedParentQueue {
}
/**
- * Determine whether to hide/show the ReservationQueues
+ * Determine whether to hide/show the ReservationQueues.
+ * @return true, show ReservationQueues; false, hide ReservationQueues.
*/
public boolean showReservationsAsQueues() {
return showReservationsAsQueues;
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/conf/InMemoryConfigurationStore.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/conf/InMemoryConfigurationStore.java
index 0fc0ecc41fa..20e33f70236 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/conf/InMemoryConfigurationStore.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/conf/InMemoryConfigurationStore.java
@@ -112,7 +112,7 @@ public class InMemoryConfigurationStore extends YarnConfigurationStore {
* As there is no configuration store for versioning purposes,
* a conf store version is not applicable.
* @return null Conf store version not applicable for this store.
- * @throws Exception
+ * @throws Exception if any exception occurs during getConfStoreVersion.
*/
@Override
public Version getConfStoreVersion() throws Exception {
@@ -123,7 +123,7 @@ public class InMemoryConfigurationStore extends YarnConfigurationStore {
/**
* Configuration mutations not logged (i.e. not persisted). As such, they are
* not persisted and not versioned. Hence, no version information to store.
- * @throws Exception
+ * @throws Exception if any exception occurs during store Version.
*/
@Override
public void storeVersion() throws Exception {
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/conf/YarnConfigurationStore.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/conf/YarnConfigurationStore.java
index 425d63f6a66..0f44f439056 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/conf/YarnConfigurationStore.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/conf/YarnConfigurationStore.java
@@ -126,6 +126,7 @@ public abstract class YarnConfigurationStore {
/**
* Retrieve the persisted configuration.
* @return configuration as key-value
+ * @throws IOException an I/O exception has occurred.
*/
public abstract Configuration retrieve() throws IOException;
@@ -139,6 +140,7 @@ public abstract class YarnConfigurationStore {
/**
* Get the last updated config version.
* @return Last updated config version.
+ * @throws Exception On version fetch failure.
*/
public abstract long getConfigVersion() throws Exception;
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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java
index 46bb40aee47..7461ef76aa2 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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.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/queuemanagement/GuaranteedOrZeroCapacityOverTimePolicy.java
@@ -304,7 +304,7 @@ public class GuaranteedOrZeroCapacityOverTimePolicy
*
* @return List of Queue Management change suggestions which could potentially
* be committed/rejected by the scheduler due to validation failures
- * @throws SchedulerDynamicEditException
+ * @throws SchedulerDynamicEditException when compute queueManagement changes fails.
*/
@Override
public List computeQueueManagementChanges()
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/constraint/AllocationTagsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java
index 850da23effb..dc69eba2bbb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/AllocationTagsManager.java
@@ -405,9 +405,10 @@ public class AllocationTagsManager {
/**
* Helper method to just remove the tags associated with a container.
- * @param nodeId
- * @param applicationId
- * @param allocationTags
+ *
+ * @param nodeId nodeId.
+ * @param applicationId application Id
+ * @param allocationTags application Tags.
*/
public void removeTags(NodeId nodeId, ApplicationId applicationId,
Set allocationTags) {
@@ -644,7 +645,7 @@ public class AllocationTagsManager {
* Returns a map whose key is the allocation tag and value is the
* count of allocations with this tag.
*
- * @param nodeId
+ * @param nodeId nodeId.
* @return allocation tag to count mapping
*/
public Map getAllocationTagsWithCount(NodeId nodeId) {
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/constraint/Evaluable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/Evaluable.java
index 6a7e54e4708..981e342234e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/Evaluable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/Evaluable.java
@@ -32,7 +32,7 @@ public interface Evaluable {
* of current class.
*
* @param target a generic type target that impacts this evaluation.
- * @throws YarnException
+ * @throws YarnException if evaluate error.
*/
void evaluate(T target) throws YarnException;
}
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/constraint/PlacementConstraintsUtil.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
index 23a3c566959..3f6dbf203c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/PlacementConstraintsUtil.java
@@ -347,7 +347,7 @@ public final class PlacementConstraintsUtil {
* @param atm allocation tags manager
* @param dcOpt optional diagnostics collector
* @return true if the given node satisfies the constraint of the request
- * @throws InvalidAllocationTagsQueryException
+ * @throws InvalidAllocationTagsQueryException if given string is not in valid format.
*/
public static boolean canSatisfyConstraints(ApplicationId applicationId,
SchedulingRequest request, SchedulerNode schedulerNode,
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/constraint/TargetApplicationsNamespace.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TargetApplicationsNamespace.java
index 97fa3a33c41..d413ae76716 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TargetApplicationsNamespace.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/constraint/TargetApplicationsNamespace.java
@@ -92,7 +92,7 @@ public abstract class TargetApplicationsNamespace implements
* in this method.
*
* @param target a generic type target that impacts this evaluation.
- * @throws InvalidAllocationTagsQueryException
+ * @throws InvalidAllocationTagsQueryException if given string is not in valid format.
*/
@Override
public void evaluate(TargetApplications target)
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/distributed/CentralizedOpportunisticContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/CentralizedOpportunisticContainerAllocator.java
index 482612de250..e3ca8b8065b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/CentralizedOpportunisticContainerAllocator.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/distributed/CentralizedOpportunisticContainerAllocator.java
@@ -77,6 +77,7 @@ public class CentralizedOpportunisticContainerAllocator extends
* @param tokenSecretManager TokenSecretManager
* @param maxAllocationsPerAMHeartbeat max number of containers to be
* allocated in one AM heartbeat
+ * @param nodeQueueLoadMonitor NodeQueueLoadMonitor.
*/
public CentralizedOpportunisticContainerAllocator(
BaseContainerTokenSecretManager tokenSecretManager,
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/fair/AllocationConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
index ee80b2265f7..b3705c60057 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationConfiguration.java
@@ -108,7 +108,7 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
* configuration.
* @param allocationFileParser The allocation file parser
* @param globalReservationQueueConfig The reservation queue config
- * @throws AllocationConfigurationException
+ * @throws AllocationConfigurationException for any errors.
*/
public AllocationConfiguration(QueueProperties queueProperties,
AllocationFileParser allocationFileParser,
@@ -202,6 +202,9 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
/**
* Get a queue's min share preemption timeout configured in the allocation
* file, in milliseconds. Return -1 if not set.
+ *
+ * @param queueName queue name.
+ * @return min share preemption timeout, return -1f if not set.
*/
public long getMinSharePreemptionTimeout(String queueName) {
Long minSharePreemptionTimeout = minSharePreemptionTimeouts.get(queueName);
@@ -211,6 +214,9 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
/**
* Get a queue's fair share preemption timeout configured in the allocation
* file, in milliseconds. Return -1 if not set.
+ *
+ * @param queueName queue Name.
+ * @return fair share preemption timeout, return -1f if not set.
*/
public long getFairSharePreemptionTimeout(String queueName) {
Long fairSharePreemptionTimeout = fairSharePreemptionTimeouts.get(queueName);
@@ -221,6 +227,9 @@ public class AllocationConfiguration extends ReservationSchedulerConfiguration {
/**
* Get a queue's fair share preemption threshold in the allocation file.
* Return -1f if not set.
+ *
+ * @param queueName queue Name.
+ * @return preemption threshold, return -1f if not set.
*/
public float getFairSharePreemptionThreshold(String queueName) {
Float fairSharePreemptionThreshold =
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/fair/AllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
index 7a5bad5d580..7fab417d893 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
@@ -189,6 +189,11 @@ public class AllocationFileLoaderService extends AbstractService {
* Path to XML file containing allocations. If the
* path is relative, it is searched for in the
* classpath, but loaded like a regular File.
+ *
+ * @param conf configuration.
+ * @return Allocation File Path.
+ * @throws UnsupportedFileSystemException
+ * File system for a given file system name/scheme is not supported.
*/
@VisibleForTesting
public Path getAllocationFile(Configuration conf)
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/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index 3b7055c757c..0218eb3e8c5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -131,6 +131,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
/**
* Get metrics reference from containing queue.
+ * @return metrics reference from containing queue.
*/
public QueueMetrics getMetrics() {
return queue.getMetrics();
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/fair/FSLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
index 0479dd7cc48..7a643ef9295 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSLeafQueue.java
@@ -483,6 +483,7 @@ public class FSLeafQueue extends FSQueue {
/**
* TODO: Based on how frequently this is called, we might want to club
* counting pending and active apps in the same method.
+ * @return active apps.
*/
public int getNumActiveApps() {
int numActiveApps = 0;
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/fair/FSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
index fea2dc0fe8c..3e99ebae9f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
@@ -304,7 +304,10 @@ public abstract class FSQueue implements Queue, Schedulable {
LOG.debug("The updated fairShare for {} is {}", getName(), fairShare);
}
- /** Get the steady fair share assigned to this Schedulable. */
+ /**
+ * Get the steady fair share assigned to this Schedulable.
+ * @return the steady fair share assigned to this Schedulable.
+ */
public Resource getSteadyFairShare() {
return steadyFairShare;
}
@@ -364,7 +367,7 @@ public abstract class FSQueue implements Queue, Schedulable {
*
* To be called holding the scheduler writelock.
*
- * @param fairShare
+ * @param fairShare queue's fairshare.
*/
public void update(Resource fairShare) {
setFairShare(fairShare);
@@ -407,6 +410,8 @@ public abstract class FSQueue implements Queue, Schedulable {
/**
* Gets the children of this queue, if any.
+ *
+ * @return the children of this queue.
*/
public abstract List getChildQueues();
@@ -420,6 +425,8 @@ public abstract class FSQueue implements Queue, Schedulable {
/**
* Return the number of apps for which containers can be allocated.
* Includes apps in subqueues.
+ *
+ * @return the number of apps.
*/
public abstract int getNumRunnableApps();
@@ -447,6 +454,8 @@ public abstract class FSQueue implements Queue, Schedulable {
/**
* Returns true if queue has at least one app running.
+ *
+ * @return true, if queue has at least one app running; otherwise, false;
*/
public boolean isActive() {
return getNumRunnableApps() > 0;
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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index a865d7543dd..32a3b7be4be 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -465,6 +465,12 @@ public class FairScheduler extends
* Add a new application to the scheduler, with a given id, queue name, and
* user. This will accept a new app even if the user or queue is above
* configured limits, but the app will not be marked as runnable.
+ *
+ * @param applicationId applicationId.
+ * @param queueName queue name.
+ * @param user submit application user.
+ * @param isAppRecovering true, app recover; false, app not recover.
+ * @param placementContext application placement context.
*/
protected void addApplication(ApplicationId applicationId,
String queueName, String user, boolean isAppRecovering,
@@ -588,6 +594,10 @@ public class FairScheduler extends
/**
* Add a new application attempt to the scheduler.
+ *
+ * @param applicationAttemptId application AttemptId.
+ * @param transferStateFromPreviousAttempt transferStateFromPreviousAttempt.
+ * @param isAttemptRecovering true, attempt recovering;false, attempt not recovering.
*/
protected void addApplicationAttempt(
ApplicationAttemptId applicationAttemptId,
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/fair/MaxRunningAppsEnforcer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/MaxRunningAppsEnforcer.java
index fb3a2c37f5c..29501540c16 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/MaxRunningAppsEnforcer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/MaxRunningAppsEnforcer.java
@@ -119,6 +119,8 @@ public class MaxRunningAppsEnforcer {
/**
* Tracks the given new runnable app for purposes of maintaining max running
* app limits.
+ *
+ * @param app FSAppAttempt, the given application.
*/
public void trackRunnableApp(FSAppAttempt app) {
String user = app.getUser();
@@ -138,6 +140,8 @@ public class MaxRunningAppsEnforcer {
/**
* Tracks the given new non runnable app so that it can be made runnable when
* it would not violate max running app limits.
+ *
+ * @param app FSAppAttempt, the given application.
*/
public void trackNonRunnableApp(FSAppAttempt app) {
String user = app.getUser();
@@ -170,6 +174,9 @@ public class MaxRunningAppsEnforcer {
*
* Runs in O(n log(n)) where n is the number of queues that are under the
* highest queue that went from having no slack to having slack.
+ *
+ * @param queue FSLeafQueue, the given queue.
+ * @param app FSAppAttempt, the given application.
*/
public void updateRunnabilityOnAppRemoval(FSAppAttempt app, FSLeafQueue queue) {
AllocationConfiguration allocConf = scheduler.getAllocationConfiguration();
@@ -271,6 +278,8 @@ public class MaxRunningAppsEnforcer {
/**
* Updates the relevant tracking variables after a runnable app with the given
* queue and user has been removed.
+ *
+ * @param app FSAppAttempt.
*/
public void untrackRunnableApp(FSAppAttempt app) {
// Update usersRunnableApps
@@ -292,7 +301,9 @@ public class MaxRunningAppsEnforcer {
}
/**
- * Stops tracking the given non-runnable app
+ * Stops tracking the given non-runnable app.
+ *
+ * @param app FSAppAttempt.
*/
public void untrackNonRunnableApp(FSAppAttempt app) {
usersNonRunnableApps.remove(app.getUser(), app);
@@ -301,6 +312,9 @@ public class MaxRunningAppsEnforcer {
/**
* Traverses the queue hierarchy under the given queue to gather all lists
* of non-runnable applications.
+ *
+ * @param queue FSQueue
+ * @param appLists FSAppAttempt List.
*/
private void gatherPossiblyRunnableAppLists(FSQueue queue,
List> appLists) {
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/fair/QueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
index 07cbb1d7082..1b12e563ab4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
@@ -543,6 +543,8 @@ public class QueueManager {
/**
* Gets a queue by name.
+ * @param name queue name.
+ * @return queue objects, FSQueue.
*/
public FSQueue getQueue(String name) {
name = ensureRootPrefix(name);
@@ -553,6 +555,10 @@ public class QueueManager {
/**
* Return whether a queue exists already.
+ *
+ * @param name queue name.
+ * @return Returns true if the queue exists,
+ * otherwise returns false.
*/
public boolean exists(String name) {
name = ensureRootPrefix(name);
@@ -563,6 +569,7 @@ public class QueueManager {
/**
* Get a collection of all leaf queues.
+ * @return a collection of all leaf queues.
*/
public Collection getLeafQueues() {
synchronized (queues) {
@@ -572,6 +579,7 @@ public class QueueManager {
/**
* Get a collection of all queues.
+ * @return a collection of all queues.
*/
public Collection getQueues() {
synchronized (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/fair/Schedulable.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
index bd1ff7ada25..ee393dffe64 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/Schedulable.java
@@ -48,6 +48,7 @@ public interface Schedulable {
/**
* Name of job/queue, used for debugging as well as for breaking ties in
* scheduling order deterministically.
+ * @return Name of job/queue.
*/
String getName();
@@ -55,16 +56,26 @@ public interface Schedulable {
* Maximum number of resources required by this Schedulable. This is defined as
* number of currently utilized resources + number of unlaunched resources (that
* are either not yet launched or need to be speculated).
+ * @return resources required by this Schedulable.
*/
Resource getDemand();
- /** Get the aggregate amount of resources consumed by the schedulable. */
+ /**
+ * Get the aggregate amount of resources consumed by the schedulable.
+ * @return aggregate amount of resources.
+ */
Resource getResourceUsage();
- /** Minimum Resource share assigned to the schedulable. */
+ /**
+ * Minimum Resource share assigned to the schedulable.
+ * @return Minimum Resource share.
+ */
Resource getMinShare();
- /** Maximum Resource share assigned to the schedulable. */
+ /**
+ * Maximum Resource share assigned to the schedulable.
+ * @return Maximum Resource share.
+ */
Resource getMaxShare();
/**
@@ -77,10 +88,16 @@ public interface Schedulable {
*/
float getWeight();
- /** Start time for jobs in FIFO queues; meaningless for QueueSchedulables.*/
+ /**
+ * Start time for jobs in FIFO queues; meaningless for QueueSchedulables.
+ * @return Start time for jobs.
+ */
long getStartTime();
- /** Job priority for jobs in FIFO queues; meaningless for QueueSchedulables. */
+ /**
+ * Job priority for jobs in FIFO queues; meaningless for QueueSchedulables.
+ * @return Job priority.
+ */
Priority getPriority();
/** Refresh the Schedulable's demand and those of its children if any. */
@@ -89,13 +106,22 @@ public interface Schedulable {
/**
* Assign a container on this node if possible, and return the amount of
* resources assigned.
+ *
+ * @param node FSSchedulerNode.
+ * @return the amount of resources assigned.
*/
Resource assignContainer(FSSchedulerNode node);
- /** Get the fair share assigned to this Schedulable. */
+ /**
+ * Get the fair share assigned to this Schedulable.
+ * @return the fair share assigned to this Schedulable.
+ */
Resource getFairShare();
- /** Assign a fair share to this Schedulable. */
+ /**
+ * Assign a fair share to this Schedulable.
+ * @param fairShare a fair share to this Schedulable.
+ */
void setFairShare(Resource fairShare);
/**
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/fair/SchedulingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
index 1fed9b08c55..8ec455a6bcb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/SchedulingPolicy.java
@@ -84,7 +84,7 @@ public abstract class SchedulingPolicy {
*
* @param policy canonical class name or "drf" or "fair" or "fifo"
* @return a {@link SchedulingPolicy} instance parsed from given policy
- * @throws AllocationConfigurationException
+ * @throws AllocationConfigurationException for any errors.
*
*/
@SuppressWarnings("unchecked")
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/fair/converter/FSQueueConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSQueueConverter.java
index c29d020b10c..34400d229d2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSQueueConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/FSQueueConverter.java
@@ -16,6 +16,8 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.converter;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.AUTO_QUEUE_CREATION_V2_ENABLED;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DEFAULT_AUTO_QUEUE_CREATION_ENABLED;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.PREFIX;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.DOT;
import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration.USER_LIMIT_FACTOR;
@@ -82,13 +84,13 @@ public class FSQueueConverter {
emitMaxParallelApps(queueName, queue);
emitMaxAllocations(queueName, queue);
emitPreemptionDisabled(queueName, queue);
- emitDefaultUserLimitFactor(queueName, children);
emitChildCapacity(queue);
emitMaximumCapacity(queueName, queue);
emitSizeBasedWeight(queueName);
emitOrderingPolicy(queueName, queue);
checkMaxChildCapacitySetting(queue);
+ emitDefaultUserLimitFactor(queueName, children);
for (FSQueue childQueue : children) {
convertQueueHierarchy(childQueue);
@@ -220,7 +222,7 @@ public class FSQueueConverter {
}
public void emitDefaultUserLimitFactor(String queueName, List children) {
- if (children.isEmpty()) {
+ if (children.isEmpty() && checkAutoQueueCreationV2Disabled(queueName)) {
capacitySchedulerConfig.setFloat(
CapacitySchedulerConfiguration.
PREFIX + queueName + DOT + USER_LIMIT_FACTOR,
@@ -309,6 +311,12 @@ public class FSQueueConverter {
}
}
+ private boolean checkAutoQueueCreationV2Disabled(String queueName) {
+ return !capacitySchedulerConfig.getBoolean(
+ PREFIX + queueName + DOT + AUTO_QUEUE_CREATION_V2_ENABLED,
+ DEFAULT_AUTO_QUEUE_CREATION_ENABLED);
+ }
+
private String getQueueShortName(String queueName) {
int lastDot = queueName.lastIndexOf(".");
return queueName.substring(lastDot + 1);
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/fair/policies/ComputeFairShares.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
index 97bb4c56175..6d9cad25ab9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/policies/ComputeFairShares.java
@@ -45,9 +45,9 @@ public final class ComputeFairShares {
* shares considering only active schedulables ie schedulables which have
* running apps.
*
- * @param schedulables
- * @param totalResources
- * @param type
+ * @param schedulables given schedulables.
+ * @param totalResources totalResources.
+ * @param type type of the resource.
*/
public static void computeShares(
Collection extends Schedulable> schedulables, Resource totalResources,
@@ -60,9 +60,9 @@ public final class ComputeFairShares {
* share is an allocation of shares considering all queues, i.e.,
* active and inactive.
*
- * @param queues
- * @param totalResources
- * @param type
+ * @param queues {@link FSQueue}s whose shares are to be updated.
+ * @param totalResources totalResources.
+ * @param type type of the resource.
*/
public static void computeSteadyShares(
Collection extends FSQueue> queues, Resource totalResources,
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/placement/MultiNodeLookupPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
index 662e34d1dc6..509187823b0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/MultiNodeLookupPolicy.java
@@ -51,6 +51,8 @@ public interface MultiNodeLookupPolicy {
*
* @param nodes
* a collection working nm's.
+ * @param partition
+ * node label
*/
void addAndRefreshNodesSet(Collection nodes, String partition);
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/SchedulableEntity.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/SchedulableEntity.java
index e2a690947ae..3c5ef26c0c0 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/SchedulableEntity.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/SchedulableEntity.java
@@ -29,35 +29,43 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
public interface SchedulableEntity {
/**
- * Id - each entity must have a unique id
+ * Id - each entity must have a unique id.
+ * @return id.
*/
public String getId();
/**
* Compare the passed SchedulableEntity to this one for input order.
* Input order is implementation defined and should reflect the
- * correct ordering for first-in first-out processing
+ * correct ordering for first-in first-out processing.
+ *
+ * @param other SchedulableEntity.
+ * @return correct ordering.
*/
public int compareInputOrderTo(SchedulableEntity other);
/**
- * View of Resources wanted and consumed by the entity
+ * View of Resources wanted and consumed by the entity.
+ * @return ResourceUsage.
*/
public ResourceUsage getSchedulingResourceUsage();
/**
- * Get the priority of the application
+ * Get the priority of the application.
+ * @return priority of the application.
*/
public Priority getPriority();
/**
* Whether application was running before RM restart.
+ * @return true, application was running before RM restart;
+ * otherwise false.
*/
public boolean isRecovering();
/**
* Get partition corresponding to this entity.
- * @return partition
+ * @return partition node label.
*/
String getPartition();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/AMRMTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/AMRMTokenSecretManager.java
index b398721c813..59fdc57234c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/AMRMTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/AMRMTokenSecretManager.java
@@ -76,7 +76,9 @@ public class AMRMTokenSecretManager extends
new HashSet();
/**
- * Create an {@link AMRMTokenSecretManager}
+ * Create an {@link AMRMTokenSecretManager}.
+ * @param conf configuration.
+ * @param rmContext rm context.
*/
public AMRMTokenSecretManager(Configuration conf, RMContext rmContext) {
this.rmContext = rmContext;
@@ -219,6 +221,8 @@ public class AMRMTokenSecretManager extends
/**
* Populate persisted password of AMRMToken back to AMRMTokenSecretManager.
+ * @param token AMRMTokenIdentifier.
+ * @throws IOException an I/O exception has occurred.
*/
public void addPersistedPassword(Token token)
throws IOException {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index d0e0bf61748..8040ce9771a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -632,7 +632,9 @@ public class DelegationTokenRenewer extends AbstractService {
}
/**
- * set task to renew the token
+ * set task to renew the token.
+ * @param token DelegationTokenToRenew.
+ * @throws IOException if an IO error occurred.
*/
@VisibleForTesting
protected void setTimerForTokenRenewal(DelegationTokenToRenew token)
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/NMTokenSecretManagerInRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/NMTokenSecretManagerInRM.java
index e3596fa34c5..1f686c96f93 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/NMTokenSecretManagerInRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/NMTokenSecretManagerInRM.java
@@ -262,7 +262,7 @@ public class NMTokenSecretManagerInRM extends BaseNMTokenSecretManager {
/**
* This is to be called when NodeManager reconnects or goes down. This will
* remove if NMTokens if present for any running application from cache.
- * @param nodeId
+ * @param nodeId Node Id.
*/
public void removeNodeKey(NodeId nodeId) {
this.writeLock.lock();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/QueueACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/QueueACLsManager.java
index 96e14c86742..af176a0454d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/QueueACLsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/QueueACLsManager.java
@@ -52,7 +52,7 @@ public abstract class QueueACLsManager {
/**
* Get queue acl manager corresponding to the scheduler.
* @param scheduler the scheduler for which the queue acl manager is required
- * @param conf
+ * @param conf Configuration.
* @return {@link QueueACLsManager}
*/
public static QueueACLsManager getQueueACLsManager(
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
index 54cd67cf312..0cb2eee55ed 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/RMContainerTokenSecretManager.java
@@ -193,6 +193,7 @@ public class RMContainerTokenSecretManager extends
* @param containerType Container Type
* @param execType Execution Type
* @param allocationRequestId allocationRequestId
+ * @param allocationTags allocation Tags
* @return the container-token
*/
public Token createContainerToken(ContainerId containerId,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/VolumeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/VolumeManager.java
index 32563cb3673..b67d143d9f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/VolumeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/volume/csi/VolumeManager.java
@@ -43,30 +43,32 @@ public interface VolumeManager {
/**
* Start to supervise on a volume.
- * @param volume
+ * @param volume volume.
* @return the volume being managed by the manager.
*/
Volume addOrGetVolume(Volume volume);
/**
* Execute volume provisioning tasks as backend threads.
- * @param volumeProvisioningTask
- * @param delaySecond
+ * @param volumeProvisioningTask A provisioning task encapsulates
+ * all the logic required by a storage system to provision a volume.
+ * @param delaySecond delay Second.
+ * @return ScheduledFuture.
*/
ScheduledFuture schedule(
VolumeProvisioningTask volumeProvisioningTask, int delaySecond);
/**
* Register a csi-driver-adaptor to the volume manager.
- * @param driverName
- * @param client
+ * @param driverName driver name.
+ * @param client csi adaptor protocol client.
*/
void registerCsiDriverAdaptor(String driverName, CsiAdaptorProtocol client);
/**
* Returns the csi-driver-adaptor client from cache by the given driver name.
* If the client is not found, null is returned.
- * @param driverName
+ * @param driverName driver name.
* @return a csi-driver-adaptor client working for given driver or null
* if the adaptor could not be found.
*/
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/RMWebServiceProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java
index 4034ed7afc6..47de8ee591a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebServiceProtocol.java
@@ -98,6 +98,7 @@ public interface RMWebServiceProtocol {
* This method retrieves the cluster user information, and it is reachable by using
* {@link RMWSConsts#CLUSTER_USER_INFO}.
*
+ * @param hsr the servlet request
* @return the cluster user information
*/
ClusterUserInfo getClusterUserInfo(HttpServletRequest hsr);
@@ -161,6 +162,7 @@ public interface RMWebServiceProtocol {
* It is a PathParam.
* @param resourceOption The resource change.
* @throws AuthorizationException If the user is not authorized.
+ * @return the resources of a specific node.
*/
ResourceInfo updateNodeResource(HttpServletRequest hsr, String nodeId,
ResourceOptionInfo resourceOption) throws AuthorizationException;
@@ -223,6 +225,7 @@ public interface RMWebServiceProtocol {
* aggregated. It is a QueryParam.
* @param activitiesCount number of activities
* @return last n activities
+ * @throws InterruptedException if interrupted.
*/
BulkActivitiesInfo getBulkActivities(HttpServletRequest hsr,
String groupBy, int activitiesCount) throws InterruptedException;
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/FairSchedulerInfo.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/FairSchedulerInfo.java
index 442a54ceeef..245582fd571 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/FairSchedulerInfo.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/FairSchedulerInfo.java
@@ -50,7 +50,7 @@ public class FairSchedulerInfo extends SchedulerInfo {
/**
* Get the fair share assigned to the appAttemptId.
- * @param appAttemptId
+ * @param appAttemptId the application attempt id
* @return The fair share assigned to the appAttemptId,
* FairSchedulerInfo#INVALID_FAIR_SHARE if the scheduler does
* not know about this application attempt.
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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
index 9804a36099c..446b7928bf9 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/FairSchedulerQueueInfo.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/FairSchedulerQueueInfo.java
@@ -154,6 +154,7 @@ public class FairSchedulerQueueInfo {
/**
* Returns the steady fair share as a fraction of the entire cluster capacity.
+ * @return steady fairshare memoryfraction.
*/
public float getSteadyFairShareMemoryFraction() {
return fractionMemSteadyFairShare;
@@ -161,6 +162,7 @@ public class FairSchedulerQueueInfo {
/**
* Returns the fair share as a fraction of the entire cluster capacity.
+ * @return fair share memory fraction.
*/
public float getFairShareMemoryFraction() {
return fractionMemFairShare;
@@ -168,13 +170,15 @@ public class FairSchedulerQueueInfo {
/**
* Returns the steady fair share of this queue in megabytes.
+ * @return steady fair share.
*/
public ResourceInfo getSteadyFairShare() {
return steadyFairResources;
}
/**
- * Returns the fair share of this queue in megabytes
+ * Returns the fair share of this queue in megabytes.
+ * @return fair share.
*/
public ResourceInfo getFairShare() {
return fairResources;
@@ -232,6 +236,7 @@ public class FairSchedulerQueueInfo {
/**
* Returns the memory used by this queue as a fraction of the entire
* cluster capacity.
+ * @return used memory fraction.
*/
public float getUsedMemoryFraction() {
return fractionMemUsed;
@@ -240,6 +245,7 @@ public class FairSchedulerQueueInfo {
/**
* Returns the capacity of this queue as a fraction of the entire cluster
* capacity.
+ * @return max resources fraction.
*/
public float getMaxResourcesFraction() {
return fractionMemMaxShare;
@@ -247,6 +253,7 @@ public class FairSchedulerQueueInfo {
/**
* Returns the name of the scheduling policy used by this queue.
+ * @return SchedulingPolicy.
*/
public String getSchedulingPolicy() {
return schedulingPolicy;
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/NodeToLabelsEntry.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/NodeToLabelsEntry.java
index 702d6f0d1a4..905ceb64b79 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/NodeToLabelsEntry.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/NodeToLabelsEntry.java
@@ -44,6 +44,11 @@ public class NodeToLabelsEntry {
this.labels = labels;
}
+ public NodeToLabelsEntry(String nodeId, Collection pLabels) {
+ this.nodeId = nodeId;
+ this.labels.addAll(pLabels);
+ }
+
public String getNodeId() {
return nodeId;
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 316f8e06cb5..faa5ddb7186 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.MockNM.createMockNod
import org.apache.hadoop.classification.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils;
@@ -55,8 +56,13 @@ import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
@@ -789,6 +795,7 @@ public class MockRM extends ResourceManager {
@Override
protected AdminService createAdminService() {
+ RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
return new AdminService(this) {
@Override
protected void startServer() {
@@ -799,6 +806,19 @@ public class MockRM extends ResourceManager {
protected void stopServer() {
// don't do anything
}
+
+ @Override
+ public RefreshServiceAclsResponse refreshServiceAcls(RefreshServiceAclsRequest request)
+ throws YarnException, IOException {
+ Configuration config = this.getConfig();
+ boolean authorization =
+ config.getBoolean(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, false);
+ if (!authorization) {
+ throw RPCUtil.getRemoteException(new IOException("Service Authorization (" +
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION + ") not enabled."));
+ }
+ return recordFactory.newRecordInstance(RefreshServiceAclsResponse.class);
+ }
};
}
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/fair/converter/TestFSConfigToCSConfigConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java
index 86bf113d64e..beb8f2f1de9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/converter/TestFSConfigToCSConfigConverter.java
@@ -194,6 +194,8 @@ public class TestFSConfigToCSConfigConverter {
assertNull("root.users user-limit-factor should be null",
conf.get(PREFIX + "root.users." + USER_LIMIT_FACTOR));
+ assertEquals("root.users auto-queue-creation-v2.enabled", "true",
+ conf.get(PREFIX + "root.users.auto-queue-creation-v2.enabled"));
assertEquals("root.default user-limit-factor", "-1.0",
conf.get(PREFIX + "root.default.user-limit-factor"));
@@ -203,6 +205,8 @@ public class TestFSConfigToCSConfigConverter {
assertEquals("root.admins.bob user-limit-factor", "-1.0",
conf.get(PREFIX + "root.admins.bob.user-limit-factor"));
+ assertNull("root.admin.bob auto-queue-creation-v2.enabled should be null",
+ conf.get(PREFIX + "root.admin.bob.auto-queue-creation-v2.enabled"));
}
@Test
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java
index 47396371ff4..fdcd890ea6e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java
@@ -147,6 +147,18 @@ public final class RouterMetrics {
private MutableGaugeInt numRefreshSuperUserGroupsConfigurationFailedRetrieved;
@Metric("# of refreshUserToGroupsMappings failed to be retrieved")
private MutableGaugeInt numRefreshUserToGroupsMappingsFailedRetrieved;
+ @Metric("# of refreshAdminAcls failed to be retrieved")
+ private MutableGaugeInt numRefreshAdminAclsFailedRetrieved;
+ @Metric("# of refreshServiceAcls failed to be retrieved")
+ private MutableGaugeInt numRefreshServiceAclsFailedRetrieved;
+ @Metric("# of replaceLabelsOnNodes failed to be retrieved")
+ private MutableGaugeInt numReplaceLabelsOnNodesFailedRetrieved;
+ @Metric("# of replaceLabelsOnNode failed to be retrieved")
+ private MutableGaugeInt numReplaceLabelsOnNodeFailedRetrieved;
+ @Metric("# of addToClusterNodeLabels failed to be retrieved")
+ private MutableGaugeInt numAddToClusterNodeLabelsFailedRetrieved;
+ @Metric("# of removeFromClusterNodeLabels failed to be retrieved")
+ private MutableGaugeInt numRemoveFromClusterNodeLabelsFailedRetrieved;
// Aggregate metrics are shared, and don't have to be looked up per call
@Metric("Total number of successful Submitted apps and latency(ms)")
@@ -253,9 +265,20 @@ public final class RouterMetrics {
private MutableRate totalSucceededRefreshSuperUserGroupsConfigurationRetrieved;
@Metric("Total number of successful Retrieved RefreshUserToGroupsMappings and latency(ms)")
private MutableRate totalSucceededRefreshUserToGroupsMappingsRetrieved;
-
+ @Metric("Total number of successful Retrieved ReplaceLabelsOnNodes and latency(ms)")
+ private MutableRate totalSucceededReplaceLabelsOnNodesRetrieved;
+ @Metric("Total number of successful Retrieved ReplaceLabelsOnNode and latency(ms)")
+ private MutableRate totalSucceededReplaceLabelsOnNodeRetrieved;
@Metric("Total number of successful Retrieved GetSchedulerInfo and latency(ms)")
private MutableRate totalSucceededGetSchedulerInfoRetrieved;
+ @Metric("Total number of successful Retrieved RefreshAdminAcls and latency(ms)")
+ private MutableRate totalSucceededRefreshAdminAclsRetrieved;
+ @Metric("Total number of successful Retrieved RefreshServiceAcls and latency(ms)")
+ private MutableRate totalSucceededRefreshServiceAclsRetrieved;
+ @Metric("Total number of successful Retrieved AddToClusterNodeLabels and latency(ms)")
+ private MutableRate totalSucceededAddToClusterNodeLabelsRetrieved;
+ @Metric("Total number of successful Retrieved RemoveFromClusterNodeLabels and latency(ms)")
+ private MutableRate totalSucceededRemoveFromClusterNodeLabelsRetrieved;
/**
* Provide quantile counters for all latencies.
@@ -313,6 +336,12 @@ public final class RouterMetrics {
private MutableQuantiles getSchedulerInfoRetrievedLatency;
private MutableQuantiles refreshSuperUserGroupsConfLatency;
private MutableQuantiles refreshUserToGroupsMappingsLatency;
+ private MutableQuantiles refreshAdminAclsLatency;
+ private MutableQuantiles refreshServiceAclsLatency;
+ private MutableQuantiles replaceLabelsOnNodesLatency;
+ private MutableQuantiles replaceLabelsOnNodeLatency;
+ private MutableQuantiles addToClusterNodeLabelsLatency;
+ private MutableQuantiles removeFromClusterNodeLabelsLatency;
private static volatile RouterMetrics instance = null;
private static MetricsRegistry registry;
@@ -504,6 +533,24 @@ public final class RouterMetrics {
refreshUserToGroupsMappingsLatency = registry.newQuantiles("refreshUserToGroupsMappingsLatency",
"latency of refresh user to groups mappings timeouts", "ops", "latency", 10);
+
+ refreshAdminAclsLatency = registry.newQuantiles("refreshAdminAclsLatency",
+ "latency of refresh admin acls timeouts", "ops", "latency", 10);
+
+ refreshServiceAclsLatency = registry.newQuantiles("refreshServiceAclsLatency",
+ "latency of refresh service acls timeouts", "ops", "latency", 10);
+
+ replaceLabelsOnNodesLatency = registry.newQuantiles("replaceLabelsOnNodesLatency",
+ "latency of replace labels on nodes timeouts", "ops", "latency", 10);
+
+ replaceLabelsOnNodeLatency = registry.newQuantiles("replaceLabelsOnNodeLatency",
+ "latency of replace labels on node timeouts", "ops", "latency", 10);
+
+ addToClusterNodeLabelsLatency = registry.newQuantiles("addToClusterNodeLabelsLatency",
+ "latency of add cluster nodelabels timeouts", "ops", "latency", 10);
+
+ removeFromClusterNodeLabelsLatency = registry.newQuantiles("removeFromClusterNodeLabelsLatency",
+ "latency of remove cluster nodelabels timeouts", "ops", "latency", 10);
}
public static RouterMetrics getMetrics() {
@@ -780,11 +827,41 @@ public final class RouterMetrics {
return totalSucceededGetSchedulerInfoRetrieved.lastStat().numSamples();
}
+ @VisibleForTesting
+ public long getNumSucceededRefreshAdminAclsRetrieved() {
+ return totalSucceededRefreshAdminAclsRetrieved.lastStat().numSamples();
+ }
+
+ @VisibleForTesting
+ public long getNumSucceededRefreshServiceAclsRetrieved() {
+ return totalSucceededRefreshServiceAclsRetrieved.lastStat().numSamples();
+ }
+
+ @VisibleForTesting
+ public long getNumSucceededAddToClusterNodeLabelsRetrieved() {
+ return totalSucceededAddToClusterNodeLabelsRetrieved.lastStat().numSamples();
+ }
+
+ @VisibleForTesting
+ public long getNumSucceededRemoveFromClusterNodeLabelsRetrieved() {
+ return totalSucceededRemoveFromClusterNodeLabelsRetrieved.lastStat().numSamples();
+ }
+
@VisibleForTesting
public long getNumSucceededRefreshSuperUserGroupsConfigurationRetrieved() {
return totalSucceededRefreshSuperUserGroupsConfigurationRetrieved.lastStat().numSamples();
}
+ @VisibleForTesting
+ public long getNumSucceededReplaceLabelsOnNodesRetrieved() {
+ return totalSucceededReplaceLabelsOnNodesRetrieved.lastStat().numSamples();
+ }
+
+ @VisibleForTesting
+ public long getNumSucceededReplaceLabelsOnNodeRetrieved() {
+ return totalSucceededReplaceLabelsOnNodeRetrieved.lastStat().numSamples();
+ }
+
@VisibleForTesting
public double getLatencySucceededAppsCreated() {
return totalSucceededAppsCreated.lastStat().mean();
@@ -1040,11 +1117,41 @@ public final class RouterMetrics {
return totalSucceededGetSchedulerInfoRetrieved.lastStat().mean();
}
+ @VisibleForTesting
+ public double getLatencySucceededRefreshAdminAclsRetrieved() {
+ return totalSucceededRefreshAdminAclsRetrieved.lastStat().mean();
+ }
+
+ @VisibleForTesting
+ public double getLatencySucceededRefreshServiceAclsRetrieved() {
+ return totalSucceededRefreshServiceAclsRetrieved.lastStat().mean();
+ }
+
+ @VisibleForTesting
+ public double getLatencySucceededAddToClusterNodeLabelsRetrieved() {
+ return totalSucceededAddToClusterNodeLabelsRetrieved.lastStat().mean();
+ }
+
+ @VisibleForTesting
+ public double getLatencySucceededRemoveFromClusterNodeLabelsRetrieved() {
+ return totalSucceededRemoveFromClusterNodeLabelsRetrieved.lastStat().mean();
+ }
+
@VisibleForTesting
public double getLatencySucceededRefreshSuperUserGroupsConfigurationRetrieved() {
return totalSucceededRefreshSuperUserGroupsConfigurationRetrieved.lastStat().mean();
}
+ @VisibleForTesting
+ public double getLatencySucceededReplaceLabelsOnNodesRetrieved() {
+ return totalSucceededReplaceLabelsOnNodesRetrieved.lastStat().mean();
+ }
+
+ @VisibleForTesting
+ public double getLatencySucceededReplaceLabelsOnNodeRetrieved() {
+ return totalSucceededReplaceLabelsOnNodeRetrieved.lastStat().mean();
+ }
+
@VisibleForTesting
public int getAppsFailedCreated() {
return numAppsFailedCreated.value();
@@ -1251,6 +1358,30 @@ public final class RouterMetrics {
return numRefreshUserToGroupsMappingsFailedRetrieved.value();
}
+ public int getNumRefreshAdminAclsFailedRetrieved() {
+ return numRefreshAdminAclsFailedRetrieved.value();
+ }
+
+ public int getNumRefreshServiceAclsFailedRetrieved() {
+ return numRefreshServiceAclsFailedRetrieved.value();
+ }
+
+ public int getNumReplaceLabelsOnNodesFailedRetrieved() {
+ return numReplaceLabelsOnNodesFailedRetrieved.value();
+ }
+
+ public int getNumReplaceLabelsOnNodeFailedRetrieved() {
+ return numReplaceLabelsOnNodeFailedRetrieved.value();
+ }
+
+ public int getNumAddToClusterNodeLabelsFailedRetrieved() {
+ return numAddToClusterNodeLabelsFailedRetrieved.value();
+ }
+
+ public int getNumRemoveFromClusterNodeLabelsFailedRetrieved() {
+ return numRemoveFromClusterNodeLabelsFailedRetrieved.value();
+ }
+
public int getDelegationTokenFailedRetrieved() {
return numGetDelegationTokenFailedRetrieved.value();
}
@@ -1534,6 +1665,26 @@ public final class RouterMetrics {
getSchedulerInfoRetrievedLatency.add(duration);
}
+ public void succeededRefreshAdminAclsRetrieved(long duration) {
+ totalSucceededRefreshAdminAclsRetrieved.add(duration);
+ refreshAdminAclsLatency.add(duration);
+ }
+
+ public void succeededRefreshServiceAclsRetrieved(long duration) {
+ totalSucceededRefreshServiceAclsRetrieved.add(duration);
+ refreshServiceAclsLatency.add(duration);
+ }
+
+ public void succeededAddToClusterNodeLabelsRetrieved(long duration) {
+ totalSucceededAddToClusterNodeLabelsRetrieved.add(duration);
+ addToClusterNodeLabelsLatency.add(duration);
+ }
+
+ public void succeededRemoveFromClusterNodeLabelsRetrieved(long duration) {
+ totalSucceededRemoveFromClusterNodeLabelsRetrieved.add(duration);
+ removeFromClusterNodeLabelsLatency.add(duration);
+ }
+
public void succeededRefreshSuperUserGroupsConfRetrieved(long duration) {
totalSucceededRefreshSuperUserGroupsConfigurationRetrieved.add(duration);
refreshSuperUserGroupsConfLatency.add(duration);
@@ -1544,6 +1695,16 @@ public final class RouterMetrics {
refreshUserToGroupsMappingsLatency.add(duration);
}
+ public void succeededReplaceLabelsOnNodesRetrieved(long duration) {
+ totalSucceededReplaceLabelsOnNodesRetrieved.add(duration);
+ replaceLabelsOnNodesLatency.add(duration);
+ }
+
+ public void succeededReplaceLabelsOnNodeRetrieved(long duration) {
+ totalSucceededReplaceLabelsOnNodeRetrieved.add(duration);
+ replaceLabelsOnNodeLatency.add(duration);
+ }
+
public void incrAppsFailedCreated() {
numAppsFailedCreated.incr();
}
@@ -1728,6 +1889,22 @@ public final class RouterMetrics {
numRefreshUserToGroupsMappingsFailedRetrieved.incr();
}
+ public void incrRefreshAdminAclsFailedRetrieved() {
+ numRefreshAdminAclsFailedRetrieved.incr();
+ }
+
+ public void incrRefreshServiceAclsFailedRetrieved() {
+ numRefreshServiceAclsFailedRetrieved.incr();
+ }
+
+ public void incrAddToClusterNodeLabelsFailedRetrieved() {
+ numAddToClusterNodeLabelsFailedRetrieved.incr();
+ }
+
+ public void incrRemoveFromClusterNodeLabelsFailedRetrieved() {
+ numRemoveFromClusterNodeLabelsFailedRetrieved.incr();
+ }
+
public void incrGetDelegationTokenFailedRetrieved() {
numGetDelegationTokenFailedRetrieved.incr();
}
@@ -1740,6 +1917,14 @@ public final class RouterMetrics {
numCancelDelegationTokenFailedRetrieved.incr();
}
+ public void incrReplaceLabelsOnNodesFailedRetrieved() {
+ numReplaceLabelsOnNodesFailedRetrieved.incr();
+ }
+
+ public void incrReplaceLabelsOnNodeFailedRetrieved() {
+ numReplaceLabelsOnNodeFailedRetrieved.incr();
+ }
+
public void incrDumpSchedulerLogsFailedRetrieved() {
numDumpSchedulerLogsFailedRetrieved.incr();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java
index 41d87c3f588..93e864bb980 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/rmadmin/FederationRMAdminInterceptor.java
@@ -372,13 +372,67 @@ public class FederationRMAdminInterceptor extends AbstractRMAdminRequestIntercep
@Override
public RefreshAdminAclsResponse refreshAdminAcls(RefreshAdminAclsRequest request)
throws YarnException, IOException {
- throw new NotImplementedException();
+
+ // parameter verification.
+ if (request == null) {
+ routerMetrics.incrRefreshAdminAclsFailedRetrieved();
+ RouterServerUtil.logAndThrowException("Missing RefreshAdminAcls request.", null);
+ }
+
+ // call refreshAdminAcls of activeSubClusters.
+ try {
+ long startTime = clock.getTime();
+ RMAdminProtocolMethod remoteMethod = new RMAdminProtocolMethod(
+ new Class[] {RefreshAdminAclsRequest.class}, new Object[] {request});
+ String subClusterId = request.getSubClusterId();
+ Collection refreshAdminAclsResps =
+ remoteMethod.invokeConcurrent(this, RefreshAdminAclsResponse.class, subClusterId);
+ if (CollectionUtils.isNotEmpty(refreshAdminAclsResps)) {
+ long stopTime = clock.getTime();
+ routerMetrics.succeededRefreshAdminAclsRetrieved(stopTime - startTime);
+ return RefreshAdminAclsResponse.newInstance();
+ }
+ } catch (YarnException e) {
+ routerMetrics.incrRefreshAdminAclsFailedRetrieved();
+ RouterServerUtil.logAndThrowException(e,
+ "Unable to refreshAdminAcls due to exception. " + e.getMessage());
+ }
+
+ routerMetrics.incrRefreshAdminAclsFailedRetrieved();
+ throw new YarnException("Unable to refreshAdminAcls.");
}
@Override
public RefreshServiceAclsResponse refreshServiceAcls(RefreshServiceAclsRequest request)
throws YarnException, IOException {
- throw new NotImplementedException();
+
+ // parameter verification.
+ if (request == null) {
+ routerMetrics.incrRefreshServiceAclsFailedRetrieved();
+ RouterServerUtil.logAndThrowException("Missing RefreshServiceAcls request.", null);
+ }
+
+ // call refreshAdminAcls of activeSubClusters.
+ try {
+ long startTime = clock.getTime();
+ RMAdminProtocolMethod remoteMethod = new RMAdminProtocolMethod(
+ new Class[]{RefreshServiceAclsRequest.class}, new Object[]{request});
+ String subClusterId = request.getSubClusterId();
+ Collection refreshServiceAclsResps =
+ remoteMethod.invokeConcurrent(this, RefreshServiceAclsResponse.class, subClusterId);
+ if (CollectionUtils.isNotEmpty(refreshServiceAclsResps)) {
+ long stopTime = clock.getTime();
+ routerMetrics.succeededRefreshServiceAclsRetrieved(stopTime - startTime);
+ return RefreshServiceAclsResponse.newInstance();
+ }
+ } catch (YarnException e) {
+ routerMetrics.incrRefreshServiceAclsFailedRetrieved();
+ RouterServerUtil.logAndThrowException(e,
+ "Unable to refreshAdminAcls due to exception. " + e.getMessage());
+ }
+
+ routerMetrics.incrRefreshServiceAclsFailedRetrieved();
+ throw new YarnException("Unable to refreshServiceAcls.");
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java
index 7cc403a492a..94b4b1ca251 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/webapp/FederationInterceptorREST.java
@@ -43,6 +43,7 @@ import javax.ws.rs.core.HttpHeaders;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
+import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.lang3.NotImplementedException;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
@@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
+import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
import org.apache.hadoop.yarn.server.federation.policies.RouterPolicyFacade;
import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyException;
@@ -116,6 +118,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.BulkActivitiesIn
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.SchedulerTypeInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeLabelInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDefinitionInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntry;
import org.apache.hadoop.yarn.server.router.RouterMetrics;
import org.apache.hadoop.yarn.server.router.RouterServerUtil;
import org.apache.hadoop.yarn.server.router.clientrm.ClientMethod;
@@ -1537,16 +1540,130 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
"getLabelsToNodes by labels = %s Failed.", StringUtils.join(labels, ","));
}
+ /**
+ * This method replaces all the node labels for specific nodes, and it is
+ * reachable by using {@link RMWSConsts#REPLACE_NODE_TO_LABELS}.
+ *
+ * @see ResourceManagerAdministrationProtocol#replaceLabelsOnNode
+ * @param newNodeToLabels the list of new labels. It is a content param.
+ * @param hsr the servlet request
+ * @return Response containing the status code
+ * @throws IOException if an exception happened
+ */
@Override
public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
HttpServletRequest hsr) throws IOException {
- throw new NotImplementedException("Code is not implemented");
+
+ // Step1. Check the parameters to ensure that the parameters are not empty.
+ if (newNodeToLabels == null) {
+ routerMetrics.incrReplaceLabelsOnNodesFailedRetrieved();
+ throw new IllegalArgumentException("Parameter error, newNodeToLabels must not be empty.");
+ }
+ List nodeToLabelsEntries = newNodeToLabels.getNodeToLabels();
+ if (CollectionUtils.isEmpty(nodeToLabelsEntries)) {
+ routerMetrics.incrReplaceLabelsOnNodesFailedRetrieved();
+ throw new IllegalArgumentException("Parameter error, " +
+ "nodeToLabelsEntries must not be empty.");
+ }
+
+ try {
+
+ // Step2. We map the NodeId and NodeToLabelsEntry in the request.
+ Map nodeIdToLabels = new HashMap<>();
+ newNodeToLabels.getNodeToLabels().stream().forEach(nodeIdToLabel -> {
+ String nodeId = nodeIdToLabel.getNodeId();
+ nodeIdToLabels.put(nodeId, nodeIdToLabel);
+ });
+
+ // Step3. We map SubCluster with NodeToLabelsEntryList
+ Map subClusterToNodeToLabelsEntryList =
+ new HashMap<>();
+ nodeIdToLabels.forEach((nodeId, nodeToLabelsEntry) -> {
+ SubClusterInfo subClusterInfo = getNodeSubcluster(nodeId);
+ NodeToLabelsEntryList nodeToLabelsEntryList = subClusterToNodeToLabelsEntryList.
+ getOrDefault(subClusterInfo, new NodeToLabelsEntryList());
+ nodeToLabelsEntryList.getNodeToLabels().add(nodeToLabelsEntry);
+ subClusterToNodeToLabelsEntryList.put(subClusterInfo, nodeToLabelsEntryList);
+ });
+
+ // Step4. Traverse the subCluster and call the replaceLabelsOnNodes interface.
+ long startTime = clock.getTime();
+ final HttpServletRequest hsrCopy = clone(hsr);
+ StringBuilder builder = new StringBuilder();
+ subClusterToNodeToLabelsEntryList.forEach((subCluster, nodeToLabelsEntryList) -> {
+ SubClusterId subClusterId = subCluster.getSubClusterId();
+ try {
+ DefaultRequestInterceptorREST interceptor = getOrCreateInterceptorForSubCluster(
+ subCluster.getSubClusterId(), subCluster.getRMWebServiceAddress());
+ interceptor.replaceLabelsOnNodes(nodeToLabelsEntryList, hsrCopy);
+ builder.append("subCluster-").append(subClusterId.getId()).append(":Success,");
+ } catch (Exception e) {
+ LOG.error("replaceLabelsOnNodes Failed. subClusterId = {}.", subClusterId, e);
+ builder.append("subCluster-").append(subClusterId.getId()).append(":Failed,");
+ }
+ });
+ long stopTime = clock.getTime();
+ routerMetrics.succeededReplaceLabelsOnNodesRetrieved(stopTime - startTime);
+
+ // Step5. return call result.
+ return Response.status(Status.OK).entity(builder.toString()).build();
+ } catch (NotFoundException e) {
+ routerMetrics.incrReplaceLabelsOnNodesFailedRetrieved();
+ throw e;
+ } catch (Exception e) {
+ routerMetrics.incrReplaceLabelsOnNodesFailedRetrieved();
+ throw e;
+ }
}
+ /**
+ * This method replaces all the node labels for specific node, and it is
+ * reachable by using {@link RMWSConsts#NODES_NODEID_REPLACE_LABELS}.
+ *
+ * @see ResourceManagerAdministrationProtocol#replaceLabelsOnNode
+ * @param newNodeLabelsName the list of new labels. It is a QueryParam.
+ * @param hsr the servlet request
+ * @param nodeId the node we want to replace the node labels. It is a
+ * PathParam.
+ * @return Response containing the status code
+ * @throws Exception if an exception happened
+ */
@Override
public Response replaceLabelsOnNode(Set newNodeLabelsName,
HttpServletRequest hsr, String nodeId) throws Exception {
- throw new NotImplementedException("Code is not implemented");
+
+ // Step1. Check the parameters to ensure that the parameters are not empty.
+ if (StringUtils.isBlank(nodeId)) {
+ routerMetrics.incrReplaceLabelsOnNodeFailedRetrieved();
+ throw new IllegalArgumentException("Parameter error, nodeId must not be null or empty.");
+ }
+ if (CollectionUtils.isEmpty(newNodeLabelsName)) {
+ routerMetrics.incrReplaceLabelsOnNodeFailedRetrieved();
+ throw new IllegalArgumentException("Parameter error, newNodeLabelsName must not be empty.");
+ }
+
+ try {
+ // Step2. We find the subCluster according to the nodeId,
+ // and then call the replaceLabelsOnNode of the subCluster.
+ long startTime = clock.getTime();
+ SubClusterInfo subClusterInfo = getNodeSubcluster(nodeId);
+ DefaultRequestInterceptorREST interceptor = getOrCreateInterceptorForSubCluster(
+ subClusterInfo.getSubClusterId(), subClusterInfo.getRMWebServiceAddress());
+ final HttpServletRequest hsrCopy = clone(hsr);
+ interceptor.replaceLabelsOnNode(newNodeLabelsName, hsrCopy, nodeId);
+
+ // Step3. Return the response result.
+ long stopTime = clock.getTime();
+ routerMetrics.succeededReplaceLabelsOnNodeRetrieved(stopTime - startTime);
+ String msg = "subCluster#" + subClusterInfo.getSubClusterId().getId() + ":Success;";
+ return Response.status(Status.OK).entity(msg).build();
+ } catch (NotFoundException e) {
+ routerMetrics.incrReplaceLabelsOnNodeFailedRetrieved();
+ throw e;
+ } catch (Exception e){
+ routerMetrics.incrReplaceLabelsOnNodeFailedRetrieved();
+ throw e;
+ }
}
@Override
@@ -1580,16 +1697,126 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
throw new RuntimeException("getClusterNodeLabels Failed.");
}
+ /**
+ * This method adds specific node labels for specific nodes, and it is
+ * reachable by using {@link RMWSConsts#ADD_NODE_LABELS}.
+ *
+ * @see ResourceManagerAdministrationProtocol#addToClusterNodeLabels
+ * @param newNodeLabels the node labels to add. It is a content param.
+ * @param hsr the servlet request
+ * @return Response containing the status code
+ * @throws Exception in case of bad request
+ */
@Override
public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels,
HttpServletRequest hsr) throws Exception {
- throw new NotImplementedException("Code is not implemented");
+
+ if (newNodeLabels == null) {
+ routerMetrics.incrAddToClusterNodeLabelsFailedRetrieved();
+ throw new IllegalArgumentException("Parameter error, the newNodeLabels is null.");
+ }
+
+ List nodeLabelInfos = newNodeLabels.getNodeLabelsInfo();
+ if (CollectionUtils.isEmpty(nodeLabelInfos)) {
+ routerMetrics.incrAddToClusterNodeLabelsFailedRetrieved();
+ throw new IllegalArgumentException("Parameter error, the nodeLabelsInfo is null or empty.");
+ }
+
+ try {
+ long startTime = clock.getTime();
+ Map subClustersActive = getActiveSubclusters();
+ final HttpServletRequest hsrCopy = clone(hsr);
+ Class[] argsClasses = new Class[]{NodeLabelsInfo.class, HttpServletRequest.class};
+ Object[] args = new Object[]{newNodeLabels, hsrCopy};
+ ClientMethod remoteMethod = new ClientMethod("addToClusterNodeLabels", argsClasses, args);
+ Map responseInfoMap =
+ invokeConcurrent(subClustersActive.values(), remoteMethod, Response.class);
+ StringBuffer buffer = new StringBuffer();
+ // SubCluster-0:SUCCESS,SubCluster-1:SUCCESS
+ responseInfoMap.forEach((subClusterInfo, response) -> {
+ buildAppendMsg(subClusterInfo, buffer, response);
+ });
+ long stopTime = clock.getTime();
+ routerMetrics.succeededAddToClusterNodeLabelsRetrieved((stopTime - startTime));
+ return Response.status(Status.OK).entity(buffer.toString()).build();
+ } catch (NotFoundException e) {
+ routerMetrics.incrAddToClusterNodeLabelsFailedRetrieved();
+ RouterServerUtil.logAndThrowIOException("get all active sub cluster(s) error.", e);
+ } catch (YarnException e) {
+ routerMetrics.incrAddToClusterNodeLabelsFailedRetrieved();
+ RouterServerUtil.logAndThrowIOException("addToClusterNodeLabels with yarn error.", e);
+ }
+
+ routerMetrics.incrAddToClusterNodeLabelsFailedRetrieved();
+ throw new RuntimeException("addToClusterNodeLabels Failed.");
}
+ /**
+ * This method removes all the node labels for specific nodes, and it is
+ * reachable by using {@link RMWSConsts#REMOVE_NODE_LABELS}.
+ *
+ * @see ResourceManagerAdministrationProtocol#removeFromClusterNodeLabels
+ * @param oldNodeLabels the node labels to remove. It is a QueryParam.
+ * @param hsr the servlet request
+ * @return Response containing the status code
+ * @throws Exception in case of bad request
+ */
@Override
public Response removeFromClusterNodeLabels(Set oldNodeLabels,
HttpServletRequest hsr) throws Exception {
- throw new NotImplementedException("Code is not implemented");
+
+ if (CollectionUtils.isEmpty(oldNodeLabels)) {
+ routerMetrics.incrRemoveFromClusterNodeLabelsFailedRetrieved();
+ throw new IllegalArgumentException("Parameter error, the oldNodeLabels is null or empty.");
+ }
+
+ try {
+ long startTime = clock.getTime();
+ Map subClustersActive = getActiveSubclusters();
+ final HttpServletRequest hsrCopy = clone(hsr);
+ Class[] argsClasses = new Class[]{Set.class, HttpServletRequest.class};
+ Object[] args = new Object[]{oldNodeLabels, hsrCopy};
+ ClientMethod remoteMethod =
+ new ClientMethod("removeFromClusterNodeLabels", argsClasses, args);
+ Map responseInfoMap =
+ invokeConcurrent(subClustersActive.values(), remoteMethod, Response.class);
+ StringBuffer buffer = new StringBuffer();
+ // SubCluster-0:SUCCESS,SubCluster-1:SUCCESS
+ responseInfoMap.forEach((subClusterInfo, response) -> {
+ buildAppendMsg(subClusterInfo, buffer, response);
+ });
+ long stopTime = clock.getTime();
+ routerMetrics.succeededRemoveFromClusterNodeLabelsRetrieved(stopTime - startTime);
+ return Response.status(Status.OK).entity(buffer.toString()).build();
+ } catch (NotFoundException e) {
+ routerMetrics.incrRemoveFromClusterNodeLabelsFailedRetrieved();
+ RouterServerUtil.logAndThrowIOException("get all active sub cluster(s) error.", e);
+ } catch (YarnException e) {
+ routerMetrics.incrRemoveFromClusterNodeLabelsFailedRetrieved();
+ RouterServerUtil.logAndThrowIOException("removeFromClusterNodeLabels with yarn error.", e);
+ }
+
+ routerMetrics.incrRemoveFromClusterNodeLabelsFailedRetrieved();
+ throw new RuntimeException("removeFromClusterNodeLabels Failed.");
+ }
+
+ /**
+ * Bbulid Append information.
+ *
+ * @param subClusterInfo subCluster information.
+ * @param buffer StringBuffer.
+ * @param response response message.
+ */
+ private void buildAppendMsg(SubClusterInfo subClusterInfo, StringBuffer buffer,
+ Response response) {
+ SubClusterId subClusterId = subClusterInfo.getSubClusterId();
+ String state = response != null &&
+ (response.getStatus() == Status.OK.getStatusCode()) ? "SUCCESS" : "FAILED";
+ buffer.append("SubCluster-")
+ .append(subClusterId.getId())
+ .append(":")
+ .append(state)
+ .append(",");
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java
index b86d85a94fd..a3756174573 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java
@@ -534,6 +534,26 @@ public class TestRouterMetrics {
metrics.incrRenewDelegationTokenFailedRetrieved();
}
+ public void getRefreshAdminAclsFailedRetrieved() {
+ LOG.info("Mocked: failed refreshAdminAcls call");
+ metrics.incrRefreshAdminAclsFailedRetrieved();
+ }
+
+ public void getRefreshServiceAclsFailedRetrieved() {
+ LOG.info("Mocked: failed refreshServiceAcls call");
+ metrics.incrRefreshServiceAclsFailedRetrieved();
+ }
+
+ public void getReplaceLabelsOnNodesFailed() {
+ LOG.info("Mocked: failed replaceLabelsOnNodes call");
+ metrics.incrReplaceLabelsOnNodesFailedRetrieved();
+ }
+
+ public void getReplaceLabelsOnNodeFailed() {
+ LOG.info("Mocked: failed ReplaceLabelOnNode call");
+ metrics.incrReplaceLabelsOnNodeFailedRetrieved();
+ }
+
public void getDumpSchedulerLogsFailed() {
LOG.info("Mocked: failed DumpSchedulerLogs call");
metrics.incrDumpSchedulerLogsFailedRetrieved();
@@ -779,6 +799,26 @@ public class TestRouterMetrics {
metrics.succeededRenewDelegationTokenRetrieved(duration);
}
+ public void getRefreshAdminAclsRetrieved(long duration) {
+ LOG.info("Mocked: successful RefreshAdminAcls call with duration {}", duration);
+ metrics.succeededRefreshAdminAclsRetrieved(duration);
+ }
+
+ public void getRefreshServiceAclsRetrieved(long duration) {
+ LOG.info("Mocked: successful RefreshServiceAcls call with duration {}", duration);
+ metrics.succeededRefreshServiceAclsRetrieved(duration);
+ }
+
+ public void getNumSucceededReplaceLabelsOnNodesRetrieved(long duration) {
+ LOG.info("Mocked: successful ReplaceLabelsOnNodes call with duration {}", duration);
+ metrics.succeededReplaceLabelsOnNodesRetrieved(duration);
+ }
+
+ public void getNumSucceededReplaceLabelsOnNodeRetrieved(long duration) {
+ LOG.info("Mocked: successful ReplaceLabelOnNode call with duration {}", duration);
+ metrics.succeededReplaceLabelsOnNodeRetrieved(duration);
+ }
+
public void getDumpSchedulerLogsRetrieved(long duration) {
LOG.info("Mocked: successful DumpSchedulerLogs call with duration {}", duration);
metrics.succeededDumpSchedulerLogsRetrieved(duration);
@@ -793,6 +833,11 @@ public class TestRouterMetrics {
LOG.info("Mocked: successful GetBulkActivities call with duration {}", duration);
metrics.succeededGetBulkActivitiesRetrieved(duration);
}
+
+ public void addToClusterNodeLabelsRetrieved(long duration) {
+ LOG.info("Mocked: successful AddToClusterNodeLabels call with duration {}", duration);
+ metrics.succeededAddToClusterNodeLabelsRetrieved(duration);
+ }
}
@Test
@@ -1628,6 +1673,98 @@ public class TestRouterMetrics {
metrics.getRenewDelegationTokenFailedRetrieved());
}
+ @Test
+ public void testRefreshAdminAclsRetrieved() {
+ long totalGoodBefore = metrics.getNumSucceededRefreshAdminAclsRetrieved();
+ goodSubCluster.getRefreshAdminAclsRetrieved(150);
+ Assert.assertEquals(totalGoodBefore + 1,
+ metrics.getNumSucceededRefreshAdminAclsRetrieved());
+ Assert.assertEquals(150,
+ metrics.getLatencySucceededRefreshAdminAclsRetrieved(), ASSERT_DOUBLE_DELTA);
+ goodSubCluster.getRefreshAdminAclsRetrieved(300);
+ Assert.assertEquals(totalGoodBefore + 2,
+ metrics.getNumSucceededRefreshAdminAclsRetrieved());
+ Assert.assertEquals(225,
+ metrics.getLatencySucceededRefreshAdminAclsRetrieved(), ASSERT_DOUBLE_DELTA);
+ }
+
+ @Test
+ public void testRefreshAdminAclsRetrievedFailed() {
+ long totalBadBefore = metrics.getNumRefreshAdminAclsFailedRetrieved();
+ badSubCluster.getRefreshAdminAclsFailedRetrieved();
+ Assert.assertEquals(totalBadBefore + 1,
+ metrics.getNumRefreshAdminAclsFailedRetrieved());
+ }
+
+ @Test
+ public void testRefreshServiceAclsRetrieved() {
+ long totalGoodBefore = metrics.getNumSucceededRefreshServiceAclsRetrieved();
+ goodSubCluster.getRefreshServiceAclsRetrieved(150);
+ Assert.assertEquals(totalGoodBefore + 1,
+ metrics.getNumSucceededRefreshServiceAclsRetrieved());
+ Assert.assertEquals(150,
+ metrics.getLatencySucceededRefreshServiceAclsRetrieved(), ASSERT_DOUBLE_DELTA);
+ goodSubCluster.getRefreshServiceAclsRetrieved(300);
+ Assert.assertEquals(totalGoodBefore + 2,
+ metrics.getNumSucceededRefreshServiceAclsRetrieved());
+ Assert.assertEquals(225,
+ metrics.getLatencySucceededRefreshServiceAclsRetrieved(), ASSERT_DOUBLE_DELTA);
+ }
+
+ @Test
+ public void testRefreshServiceAclsRetrievedFailed() {
+ long totalBadBefore = metrics.getNumRefreshServiceAclsFailedRetrieved();
+ badSubCluster.getRefreshServiceAclsFailedRetrieved();
+ Assert.assertEquals(totalBadBefore + 1,
+ metrics.getNumRefreshServiceAclsFailedRetrieved());
+ }
+
+ @Test
+ public void testReplaceLabelsOnNodesRetrieved() {
+ long totalGoodBefore = metrics.getNumSucceededReplaceLabelsOnNodesRetrieved();
+ goodSubCluster.getNumSucceededReplaceLabelsOnNodesRetrieved(150);
+ Assert.assertEquals(totalGoodBefore + 1,
+ metrics.getNumSucceededReplaceLabelsOnNodesRetrieved());
+ Assert.assertEquals(150,
+ metrics.getLatencySucceededReplaceLabelsOnNodesRetrieved(), ASSERT_DOUBLE_DELTA);
+ goodSubCluster.getNumSucceededReplaceLabelsOnNodesRetrieved(300);
+ Assert.assertEquals(totalGoodBefore + 2,
+ metrics.getNumSucceededReplaceLabelsOnNodesRetrieved());
+ Assert.assertEquals(225,
+ metrics.getLatencySucceededReplaceLabelsOnNodesRetrieved(), ASSERT_DOUBLE_DELTA);
+ }
+
+ @Test
+ public void testReplaceLabelsOnNodesRetrievedFailed() {
+ long totalBadBefore = metrics.getNumReplaceLabelsOnNodesFailedRetrieved();
+ badSubCluster.getReplaceLabelsOnNodesFailed();
+ Assert.assertEquals(totalBadBefore + 1,
+ metrics.getNumReplaceLabelsOnNodesFailedRetrieved());
+ }
+
+ @Test
+ public void testReplaceLabelsOnNodeRetrieved() {
+ long totalGoodBefore = metrics.getNumSucceededReplaceLabelsOnNodeRetrieved();
+ goodSubCluster.getNumSucceededReplaceLabelsOnNodeRetrieved(150);
+ Assert.assertEquals(totalGoodBefore + 1,
+ metrics.getNumSucceededReplaceLabelsOnNodeRetrieved());
+ Assert.assertEquals(150,
+ metrics.getLatencySucceededReplaceLabelsOnNodeRetrieved(), ASSERT_DOUBLE_DELTA);
+ goodSubCluster.getNumSucceededReplaceLabelsOnNodeRetrieved(300);
+ Assert.assertEquals(totalGoodBefore + 2,
+ metrics.getNumSucceededReplaceLabelsOnNodeRetrieved());
+ Assert.assertEquals(225,
+ metrics.getLatencySucceededReplaceLabelsOnNodeRetrieved(), ASSERT_DOUBLE_DELTA);
+ }
+
+ @Test
+ public void testReplaceLabelOnNodeRetrievedFailed() {
+ long totalBadBefore = metrics.getNumReplaceLabelsOnNodeFailedRetrieved();
+ badSubCluster.getReplaceLabelsOnNodeFailed();
+ Assert.assertEquals(totalBadBefore + 1,
+ metrics.getNumReplaceLabelsOnNodeFailedRetrieved());
+ }
+
@Test
public void testDumpSchedulerLogsRetrieved() {
long totalGoodBefore = metrics.getNumSucceededDumpSchedulerLogsRetrieved();
@@ -1696,4 +1833,19 @@ public class TestRouterMetrics {
Assert.assertEquals(totalBadBefore + 1,
metrics.getBulkActivitiesFailedRetrieved());
}
+
+ @Test
+ public void testAddToClusterNodeLabelsRetrieved() {
+ long totalGoodBefore = metrics.getNumSucceededAddToClusterNodeLabelsRetrieved();
+ goodSubCluster.addToClusterNodeLabelsRetrieved(150);
+ Assert.assertEquals(totalGoodBefore + 1,
+ metrics.getNumSucceededAddToClusterNodeLabelsRetrieved());
+ Assert.assertEquals(150,
+ metrics.getLatencySucceededAddToClusterNodeLabelsRetrieved(), ASSERT_DOUBLE_DELTA);
+ goodSubCluster.addToClusterNodeLabelsRetrieved(300);
+ Assert.assertEquals(totalGoodBefore + 2,
+ metrics.getNumSucceededAddToClusterNodeLabelsRetrieved());
+ Assert.assertEquals(225,
+ metrics.getLatencySucceededAddToClusterNodeLabelsRetrieved(), ASSERT_DOUBLE_DELTA);
+ }
}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java
index 977f82dd3cd..60a782bd8a9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestFederationRMAdminInterceptor.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.server.router.rmadmin;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.yarn.api.records.DecommissionType;
@@ -30,6 +31,10 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsC
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsResponse;
import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
@@ -113,6 +118,8 @@ public class TestFederationRMAdminInterceptor extends BaseRouterRMAdminTest {
config.set(YarnConfiguration.ROUTER_RMADMIN_INTERCEPTOR_CLASS_PIPELINE,
mockPassThroughInterceptorClass + "," + mockPassThroughInterceptorClass + "," +
TestFederationRMAdminInterceptor.class.getName());
+ config.setBoolean(
+ CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, true);
return config;
}
@@ -259,4 +266,58 @@ public class TestFederationRMAdminInterceptor extends BaseRouterRMAdminTest {
"subClusterId = SC-NON is not an active subCluster.",
() -> interceptor.refreshUserToGroupsMappings(request1));
}
+
+ @Test
+ public void testRefreshAdminAcls() throws Exception {
+ // null request.
+ LambdaTestUtils.intercept(YarnException.class, "Missing RefreshAdminAcls request.",
+ () -> interceptor.refreshAdminAcls(null));
+
+ // normal request.
+ RefreshAdminAclsRequest request = RefreshAdminAclsRequest.newInstance();
+ RefreshAdminAclsResponse response = interceptor.refreshAdminAcls(request);
+ assertNotNull(response);
+ }
+
+ @Test
+ public void testSC1RefreshAdminAcls() throws Exception {
+ // case 1, test the existing subCluster (SC-1).
+ String existSubCluster = "SC-1";
+ RefreshAdminAclsRequest request = RefreshAdminAclsRequest.newInstance(existSubCluster);
+ RefreshAdminAclsResponse response = interceptor.refreshAdminAcls(request);
+ assertNotNull(response);
+
+ // case 2, test the non-exist subCluster.
+ String notExistsSubCluster = "SC-NON";
+ RefreshAdminAclsRequest request1 = RefreshAdminAclsRequest.newInstance(notExistsSubCluster);
+ LambdaTestUtils.intercept(Exception.class, "subClusterId = SC-NON is not an active subCluster.",
+ () -> interceptor.refreshAdminAcls(request1));
+ }
+
+ @Test
+ public void testRefreshServiceAcls() throws Exception {
+ // null request.
+ LambdaTestUtils.intercept(YarnException.class, "Missing RefreshServiceAcls request.",
+ () -> interceptor.refreshServiceAcls(null));
+
+ // normal request.
+ RefreshServiceAclsRequest request = RefreshServiceAclsRequest.newInstance();
+ RefreshServiceAclsResponse response = interceptor.refreshServiceAcls(request);
+ assertNotNull(response);
+ }
+
+ @Test
+ public void testSC1RefreshServiceAcls() throws Exception {
+ // case 1, test the existing subCluster (SC-1).
+ String existSubCluster = "SC-1";
+ RefreshServiceAclsRequest request = RefreshServiceAclsRequest.newInstance(existSubCluster);
+ RefreshServiceAclsResponse response = interceptor.refreshServiceAcls(request);
+ assertNotNull(response);
+
+ // case 2, test the non-exist subCluster.
+ String notExistsSubCluster = "SC-NON";
+ RefreshServiceAclsRequest request1 = RefreshServiceAclsRequest.newInstance(notExistsSubCluster);
+ LambdaTestUtils.intercept(Exception.class, "subClusterId = SC-NON is not an active subCluster.",
+ () -> interceptor.refreshServiceAcls(request1));
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestableFederationRMAdminInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestableFederationRMAdminInterceptor.java
index 26f50f88b89..b95bcd4a62b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestableFederationRMAdminInterceptor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/rmadmin/TestableFederationRMAdminInterceptor.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.router.rmadmin;
import org.apache.commons.collections.MapUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
@@ -35,6 +36,8 @@ import java.util.HashSet;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_CLUSTER_ID;
+
public class TestableFederationRMAdminInterceptor extends FederationRMAdminInterceptor {
// Record log information
@@ -55,11 +58,13 @@ public class TestableFederationRMAdminInterceptor extends FederationRMAdminInter
if (mockRMs.containsKey(subClusterId)) {
mockRM = mockRMs.get(subClusterId);
} else {
- mockRM = new MockRM();
+ YarnConfiguration config = new YarnConfiguration(super.getConf());
+ config.set(RM_CLUSTER_ID, "subcluster." + subClusterId);
+ mockRM = new MockRM(config);
if (badSubCluster.contains(subClusterId)) {
return new MockRMAdminBadService(mockRM);
}
- mockRM.init(super.getConf());
+ mockRM.init(config);
mockRM.start();
mockRMs.put(subClusterId, mockRM);
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java
index 91f3a7d4cea..653224a7d37 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/MockDefaultRequestInterceptorREST.java
@@ -137,6 +137,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationReque
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationRequestsInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationUpdateResponseInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationDeleteResponseInfo;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ActivitiesInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.BulkActivitiesInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWSConsts;
@@ -305,9 +306,14 @@ public class MockDefaultRequestInterceptorREST
if (!isRunning) {
throw new RuntimeException("RM is stopped");
}
- NodeInfo node = new NodeInfo();
- node.setId(nodeId);
- node.setLastHealthUpdate(Integer.valueOf(getSubClusterId().getId()));
+ NodeInfo node = null;
+ SubClusterId subCluster = getSubClusterId();
+ String subClusterId = subCluster.getId();
+ if (nodeId.contains(subClusterId) || nodeId.contains("test")) {
+ node = new NodeInfo();
+ node.setId(nodeId);
+ node.setLastHealthUpdate(Integer.valueOf(getSubClusterId().getId()));
+ }
return node;
}
@@ -1236,7 +1242,17 @@ public class MockDefaultRequestInterceptorREST
return webSvc.dumpSchedulerLogs(time, hsr);
}
+ public Response replaceLabelsOnNodes(NodeToLabelsEntryList newNodeToLabels,
+ HttpServletRequest hsr) throws IOException {
+ return super.replaceLabelsOnNodes(newNodeToLabels, hsr);
+ }
+
@Override
+ public Response replaceLabelsOnNode(Set newNodeLabelsName,
+ HttpServletRequest hsr, String nodeId) throws Exception {
+ return super.replaceLabelsOnNode(newNodeLabelsName, hsr, nodeId);
+ }
+
public ActivitiesInfo getActivities(HttpServletRequest hsr, String nodeId, String groupBy) {
if (!EnumUtils.isValidEnum(RMWSConsts.ActivitiesGroupBy.class, groupBy.toUpperCase())) {
String errMessage = "Got invalid groupBy: " + groupBy + ", valid groupBy types: "
@@ -1307,4 +1323,44 @@ public class MockDefaultRequestInterceptorREST
throw new RuntimeException(e);
}
}
+
+ @Override
+ public Response addToClusterNodeLabels(NodeLabelsInfo newNodeLabels, HttpServletRequest hsr)
+ throws Exception {
+ List nodeLabelInfoList = newNodeLabels.getNodeLabelsInfo();
+ NodeLabelInfo nodeLabelInfo = nodeLabelInfoList.get(0);
+ String nodeLabelName = nodeLabelInfo.getName();
+
+ // If nodeLabelName is ALL, we let all subclusters pass
+ if (StringUtils.equals("ALL", nodeLabelName)) {
+ return Response.status(Status.OK).build();
+ } else if (StringUtils.equals("A0", nodeLabelName)) {
+ SubClusterId subClusterId = getSubClusterId();
+ String id = subClusterId.getId();
+ if (StringUtils.contains("A0", id)) {
+ return Response.status(Status.OK).build();
+ } else {
+ return Response.status(Status.BAD_REQUEST).entity(null).build();
+ }
+ }
+ throw new YarnException("addToClusterNodeLabels Error");
+ }
+
+ @Override
+ public Response removeFromClusterNodeLabels(Set oldNodeLabels, HttpServletRequest hsr)
+ throws Exception {
+ // If oldNodeLabels contains ALL, we let all subclusters pass
+ if (oldNodeLabels.contains("ALL")) {
+ return Response.status(Status.OK).build();
+ } else if (oldNodeLabels.contains("A0")) {
+ SubClusterId subClusterId = getSubClusterId();
+ String id = subClusterId.getId();
+ if (StringUtils.contains("A0", id)) {
+ return Response.status(Status.OK).build();
+ } else {
+ return Response.status(Status.BAD_REQUEST).entity(null).build();
+ }
+ }
+ throw new YarnException("removeFromClusterNodeLabels Error");
+ }
}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java
index b7ddda7d30b..a2831657dc8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestFederationInterceptorREST.java
@@ -25,6 +25,7 @@ import java.util.List;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
+import java.util.Arrays;
import java.util.HashSet;
import java.util.Collections;
import java.util.stream.Collectors;
@@ -40,6 +41,7 @@ import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.util.Lists;
+import org.apache.hadoop.util.Sets;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -97,6 +99,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ReservationSubmissionRequestInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.RMQueueAclInfo;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.DelegationToken;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntry;
+import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.NodeToLabelsEntryList;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.NodeIDsInfo;
import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService;
import org.apache.hadoop.yarn.server.router.clientrm.RouterClientRMService.RequestInterceptorChainWrapper;
@@ -1784,6 +1788,102 @@ public class TestFederationInterceptorREST extends BaseRouterWebServicesTest {
Assert.assertEquals(response.getStatus(), Status.OK.getStatusCode());
}
+ @Test
+ public void testReplaceLabelsOnNodes() throws Exception {
+ // subCluster0 -> node0:0 -> label:NodeLabel0
+ // subCluster1 -> node1:1 -> label:NodeLabel1
+ // subCluster2 -> node2:2 -> label:NodeLabel2
+ // subCluster3 -> node3:3 -> label:NodeLabel3
+ NodeToLabelsEntryList nodeToLabelsEntryList = new NodeToLabelsEntryList();
+ for (int i = 0; i < NUM_SUBCLUSTER; i++) {
+ // labels
+ List labels = new ArrayList<>();
+ labels.add("NodeLabel" + i);
+ // nodes
+ String nodeId = "node" + i + ":" + i;
+ NodeToLabelsEntry nodeToLabelsEntry = new NodeToLabelsEntry(nodeId, labels);
+ List nodeToLabelsEntries = nodeToLabelsEntryList.getNodeToLabels();
+ nodeToLabelsEntries.add(nodeToLabelsEntry);
+ }
+
+ // one of the results:
+ // subCluster#0:Success;subCluster#1:Success;subCluster#3:Success;subCluster#2:Success;
+ // We can't confirm the complete return order.
+ Response response = interceptor.replaceLabelsOnNodes(nodeToLabelsEntryList, null);
+ Assert.assertNotNull(response);
+ Assert.assertEquals(200, response.getStatus());
+
+ Object entityObject = response.getEntity();
+ Assert.assertNotNull(entityObject);
+
+ String entityValue = String.valueOf(entityObject);
+ String[] entities = entityValue.split(",");
+ Assert.assertNotNull(entities);
+ Assert.assertEquals(4, entities.length);
+ String expectValue =
+ "subCluster-0:Success,subCluster-1:Success,subCluster-2:Success,subCluster-3:Success,";
+ for (String entity : entities) {
+ Assert.assertTrue(expectValue.contains(entity));
+ }
+ }
+
+ @Test
+ public void testReplaceLabelsOnNodesError() throws Exception {
+ // newNodeToLabels is null
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, newNodeToLabels must not be empty.",
+ () -> interceptor.replaceLabelsOnNodes(null, null));
+
+ // nodeToLabelsEntryList is Empty
+ NodeToLabelsEntryList nodeToLabelsEntryList = new NodeToLabelsEntryList();
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, nodeToLabelsEntries must not be empty.",
+ () -> interceptor.replaceLabelsOnNodes(nodeToLabelsEntryList, null));
+ }
+
+ @Test
+ public void testReplaceLabelsOnNode() throws Exception {
+ // subCluster3 -> node3:3 -> label:NodeLabel3
+ String nodeId = "node3:3";
+ Set labels = Collections.singleton("NodeLabel3");
+
+ // We expect the following result: subCluster#3:Success;
+ String expectValue = "subCluster#3:Success;";
+ Response response = interceptor.replaceLabelsOnNode(labels, null, nodeId);
+ Assert.assertNotNull(response);
+ Assert.assertEquals(200, response.getStatus());
+
+ Object entityObject = response.getEntity();
+ Assert.assertNotNull(entityObject);
+
+ String entityValue = String.valueOf(entityObject);
+ Assert.assertNotNull(entityValue);
+ Assert.assertEquals(expectValue, entityValue);
+ }
+
+ @Test
+ public void testReplaceLabelsOnNodeError() throws Exception {
+ // newNodeToLabels is null
+ String nodeId = "node3:3";
+ Set labels = Collections.singleton("NodeLabel3");
+ Set labelsEmpty = new HashSet<>();
+
+ // nodeId is null
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, nodeId must not be null or empty.",
+ () -> interceptor.replaceLabelsOnNode(labels, null, null));
+
+ // labels is null
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, newNodeLabelsName must not be empty.",
+ () -> interceptor.replaceLabelsOnNode(null, null, nodeId));
+
+ // labels is empty
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, newNodeLabelsName must not be empty.",
+ () -> interceptor.replaceLabelsOnNode(labelsEmpty, null, nodeId));
+ }
+
@Test
public void testDumpSchedulerLogs() throws Exception {
HttpServletRequest mockHsr = mockHttpServletRequestByUserName("admin");
@@ -1899,4 +1999,132 @@ public class TestFederationInterceptorREST extends BaseRouterWebServicesTest {
"'groupBy' must not be empty.",
() -> interceptor.getBulkActivities(null, "", 1));
}
+
+ @Test
+ public void testAddToClusterNodeLabels1() throws Exception {
+ // In this test, we try to add ALL label, all subClusters will return success.
+ NodeLabelsInfo nodeLabelsInfo = new NodeLabelsInfo();
+ NodeLabelInfo nodeLabelInfo = new NodeLabelInfo("ALL", true);
+ nodeLabelsInfo.getNodeLabelsInfo().add(nodeLabelInfo);
+
+ Response response = interceptor.addToClusterNodeLabels(nodeLabelsInfo, null);
+ Assert.assertNotNull(response);
+
+ Object entityObj = response.getEntity();
+ Assert.assertNotNull(entityObj);
+
+ String entity = String.valueOf(entityObj);
+ String[] entities = StringUtils.split(entity, ",");
+ Assert.assertNotNull(entities);
+ Assert.assertEquals(4, entities.length);
+
+ // The order in which the cluster returns messages is uncertain,
+ // we confirm the result by contains
+ String expectedMsg =
+ "SubCluster-0:SUCCESS,SubCluster-1:SUCCESS,SubCluster-2:SUCCESS,SubCluster-3:SUCCESS";
+ Arrays.stream(entities).forEach(item -> {
+ Assert.assertTrue(expectedMsg.contains(item));
+ });
+ }
+
+ @Test
+ public void testAddToClusterNodeLabels2() throws Exception {
+ // In this test, we try to add A0 label,
+ // subCluster0 will return success, and other subClusters will return null
+ NodeLabelsInfo nodeLabelsInfo = new NodeLabelsInfo();
+ NodeLabelInfo nodeLabelInfo = new NodeLabelInfo("A0", true);
+ nodeLabelsInfo.getNodeLabelsInfo().add(nodeLabelInfo);
+
+ Response response = interceptor.addToClusterNodeLabels(nodeLabelsInfo, null);
+ Assert.assertNotNull(response);
+
+ Object entityObj = response.getEntity();
+ Assert.assertNotNull(entityObj);
+
+ String expectedValue = "SubCluster-0:SUCCESS,";
+ String entity = String.valueOf(entityObj);
+ Assert.assertTrue(entity.contains(expectedValue));
+ }
+
+ @Test
+ public void testAddToClusterNodeLabelsError() throws Exception {
+ // the newNodeLabels is null
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, the newNodeLabels is null.",
+ () -> interceptor.addToClusterNodeLabels(null, null));
+
+ // the nodeLabelsInfo is null
+ NodeLabelsInfo nodeLabelsInfo = new NodeLabelsInfo();
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, the nodeLabelsInfo is null or empty.",
+ () -> interceptor.addToClusterNodeLabels(nodeLabelsInfo, null));
+
+ // error nodeLabelsInfo
+ NodeLabelsInfo nodeLabelsInfo1 = new NodeLabelsInfo();
+ NodeLabelInfo nodeLabelInfo1 = new NodeLabelInfo("A", true);
+ nodeLabelsInfo1.getNodeLabelsInfo().add(nodeLabelInfo1);
+ LambdaTestUtils.intercept(YarnRuntimeException.class, "addToClusterNodeLabels Error",
+ () -> interceptor.addToClusterNodeLabels(nodeLabelsInfo1, null));
+ }
+
+ @Test
+ public void testRemoveFromClusterNodeLabels1() throws Exception {
+ Set oldNodeLabels = Sets.newHashSet();
+ oldNodeLabels.add("ALL");
+
+ Response response = interceptor.removeFromClusterNodeLabels(oldNodeLabels, null);
+ Assert.assertNotNull(response);
+
+ Object entityObj = response.getEntity();
+ Assert.assertNotNull(entityObj);
+
+ String entity = String.valueOf(entityObj);
+ String[] entities = StringUtils.split(entity, ",");
+ Assert.assertNotNull(entities);
+ Assert.assertEquals(4, entities.length);
+
+ // The order in which the cluster returns messages is uncertain,
+ // we confirm the result by contains
+ String expectedMsg =
+ "SubCluster-0:SUCCESS,SubCluster-1:SUCCESS,SubCluster-2:SUCCESS,SubCluster-3:SUCCESS";
+ Arrays.stream(entities).forEach(item -> {
+ Assert.assertTrue(expectedMsg.contains(item));
+ });
+ }
+
+ @Test
+ public void testRemoveFromClusterNodeLabels2() throws Exception {
+ Set oldNodeLabels = Sets.newHashSet();
+ oldNodeLabels.add("A0");
+
+ Response response = interceptor.removeFromClusterNodeLabels(oldNodeLabels, null);
+ Assert.assertNotNull(response);
+
+ Object entityObj = response.getEntity();
+ Assert.assertNotNull(entityObj);
+
+ String expectedValue = "SubCluster-0:SUCCESS,";
+ String entity = String.valueOf(entityObj);
+ Assert.assertTrue(entity.contains(expectedValue));
+ }
+
+ @Test
+ public void testRemoveFromClusterNodeLabelsError() throws Exception {
+ // the oldNodeLabels is null
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, the oldNodeLabels is null or empty.",
+ () -> interceptor.removeFromClusterNodeLabels(null, null));
+
+ // the oldNodeLabels is empty
+ Set oldNodeLabels = Sets.newHashSet();
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Parameter error, the oldNodeLabels is null or empty.",
+ () -> interceptor.removeFromClusterNodeLabels(oldNodeLabels, null));
+
+ // error oldNodeLabels
+ Set oldNodeLabels1 = Sets.newHashSet();
+ oldNodeLabels1.add("A1");
+ LambdaTestUtils.intercept(YarnRuntimeException.class, "removeFromClusterNodeLabels Error",
+ () -> interceptor.removeFromClusterNodeLabels(oldNodeLabels1, null));
+ }
}
\ No newline at end of file
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/README.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/README.md
index 6e6ca594536..0ee58985ffc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/README.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/README.md
@@ -24,7 +24,7 @@ The YARN UI is an Ember based web-app that provides visualization of the applica
You can point the UI to custom locations by setting the environment variables in `src/main/webapp/config/configs.env`.
In order to make the UI running on Ember server (started by `yarn start`)
-work with independently running ResouceManager,
+work with independently running ResourceManager,
you need to enable CORS by setting `hadoop.http.cross-origin.enabled` to true
and adding `org.apache.hadoop.security.HttpCrossOriginFilterInitializer`
to `hadoop.http.filter.initializers` in core-site.xml of the ResourceManager.
diff --git a/pom.xml b/pom.xml
index 51a80c76341..fa768296e37 100644
--- a/pom.xml
+++ b/pom.xml
@@ -118,7 +118,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x
4.2.01.1.13.10.1
- 2.7.3bash
@@ -500,19 +499,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x
maven-compiler-plugin${maven-compiler-plugin.version}
-
- org.cyclonedx
- cyclonedx-maven-plugin
- ${cyclonedx.version}
-
-
- package
-
- makeBom
-
-
-
-
@@ -621,10 +607,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x
com.github.spotbugsspotbugs-maven-plugin
-
- org.cyclonedx
- cyclonedx-maven-plugin
-