Merge branch 'trunk' into HDFS-16864

This commit is contained in:
Dave Marion 2023-03-03 12:53:25 +00:00
commit 6e63d38340
227 changed files with 6090 additions and 2596 deletions

View File

@ -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

View File

@ -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];
}

View File

@ -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.
*

View File

@ -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

View File

@ -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 */

View File

@ -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<TokenIdent
extends AbstractDelegationTokenIdentifier>
extends AbstractDelegationTokenSecretManager<TokenIdent> {
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<TokenIdent> 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;
}

View File

@ -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 |

View File

@ -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.

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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);

View File

@ -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

View File

@ -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<DatanodeInfo> 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<DatanodeInfo> 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<DatanodeInfo> 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.

View File

@ -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());
}
}

View File

@ -117,6 +117,15 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>com.zaxxer</groupId>
<artifactId>HikariCP</artifactId>
</dependency>
<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
@ -153,6 +162,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<artifactId>curator-test</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
@ -170,6 +184,14 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemProperties>
<property>
<name>derby.stream.error.file</name>
<value>${project.build.directory}/derby.log</value>
</property>
</systemProperties>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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.

View File

@ -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<StateStoreSerializerPBImpl>

View File

@ -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<String> 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<String, LongAccumulator> 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));
}
}

View File

@ -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;
}
}
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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<AbstractDelegationTokenIdentifier> {
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;
}
}

View File

@ -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> T execute(SQLCommand<T> command) throws SQLException;
@FunctionalInterface
interface SQLCommandVoid {
void doCall() throws SQLException;
}
@FunctionalInterface
interface SQLCommand<T> {
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<Class<? extends Exception>, 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> T execute(SQLCommand<T> 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);
}
}
}

View File

@ -884,4 +884,14 @@
of namespaces in use and the latency of the msync requests.
</description>
</property>
<property>
<name>dfs.federation.router.observer.state.id.refresh.period</name>
<value>15s</value>
<description>
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.
</description>
</property>
</configuration>

View File

@ -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);
}
}

View File

@ -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<FileSystem>) () -> 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 =

View File

@ -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

View File

@ -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<Integer> sequenceNums1 = new HashSet<>();
Set<Integer> sequenceNums2 = new HashSet<>();
Set<Integer> sequenceNums3 = new HashSet<>();
Set<Integer> 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<Integer> 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<Integer> 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);
}
}
}

View File

@ -310,4 +310,14 @@
<Method name="reconcile" />
<Bug pattern="SWL_SLEEP_WITH_LOCK_HELD" />
</Match>
<!--
conversionPattern is only set once and used to initiate PatternLayout object
only once. It is set by log4j framework if set as part of log4j properties and accessed
only during first append operation.
-->
<Match>
<Class name="org.apache.hadoop.hdfs.util.AsyncRFAAppender"/>
<Field name="conversionPattern"/>
<Bug pattern="IS2_INCONSISTENT_SYNC"/>
</Match>
</FindBugsFilter>

View File

@ -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.
* <p>
* 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.
* <p>
* 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";

View File

@ -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);
}
}
}

View File

@ -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<Appender> 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);
}
}
}

View File

@ -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);

View File

@ -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) {

View File

@ -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);
}

View File

@ -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=&lt;permissions (optional)&gt;
* </code>
*/
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<AuditLogger> 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<Appender> 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.
*/

View File

@ -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(

View File

@ -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;
}
}

View File

@ -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<DatanodeInfo> 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;

View File

@ -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());
}
}

View File

@ -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);

View File

@ -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;
}

View File

@ -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<Boolean>() {
@ -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;
}
}
}

View File

@ -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());
}
}

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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<Object[]> data() {
Collection<Object[]> params = new ArrayList<Object[]>();
params.add(new Object[]{Boolean.FALSE, Boolean.FALSE});
params.add(new Object[]{Boolean.TRUE, Boolean.FALSE});
params.add(new Object[]{Boolean.FALSE, Boolean.TRUE});
params.add(new Object[]{Boolean.TRUE, Boolean.TRUE});
Collection<Object[]> params = new ArrayList<>();
params.add(new Object[]{Boolean.FALSE});
params.add(new Object[]{Boolean.TRUE});
return params;
}
public TestAuditLogs(boolean useAsyncLog, boolean useAsyncEdits) {
this.useAsyncLog = useAsyncLog;
public TestAuditLogs(boolean useAsyncEdits) {
this.useAsyncEdits = useAsyncEdits;
}
// Pattern for:
// allowed=(true|false) ugi=name ip=/address cmd={cmd} src={path} dst=null perm=null
static final Pattern auditPattern = Pattern.compile(
private static final Pattern AUDIT_PATTERN = Pattern.compile(
"allowed=.*?\\s" +
"ugi=.*?\\s" +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\s" +
"cmd=.*?\\ssrc=.*?\\sdst=null\\s" +
"perm=.*?");
static final Pattern successPattern = Pattern.compile(
private static final Pattern SUCCESS_PATTERN = Pattern.compile(
".*allowed=true.*");
static final Pattern webOpenPattern = Pattern.compile(
private static final Pattern FAILURE_PATTERN = Pattern.compile(
".*allowed=false.*");
private static final Pattern WEB_OPEN_PATTERN = Pattern.compile(
".*cmd=open.*proto=webhdfs.*");
static final String username = "bob";
@ -113,14 +111,15 @@ public class TestAuditLogs {
@Before
public void setupCluster() throws Exception {
try (PrintWriter writer = new PrintWriter(AUDIT_LOG_FILE)) {
writer.print("");
}
// must configure prior to instantiating the namesystem because it
// will reconfigure the logger if async is enabled
configureAuditLogs();
conf = new HdfsConfiguration();
final long precision = 1L;
conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY, useAsyncLog);
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_EDITS_ASYNC_LOGGING, useAsyncEdits);
util = new DFSTestUtil.Builder().setName("TestAuditAllowed").
setNumFiles(20).build();
@ -129,19 +128,25 @@ public class TestAuditLogs {
util.createFiles(fs, fileName);
// make sure the appender is what it's supposed to be
Logger logger = FSNamesystem.AUDIT_LOG;
Logger logger = org.apache.log4j.Logger.getLogger(
"org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit");
@SuppressWarnings("unchecked")
List<Appender> appenders = Collections.list(logger.getAllAppenders());
assertEquals(1, appenders.size());
assertEquals(useAsyncLog, appenders.get(0) instanceof AsyncAppender);
assertTrue(appenders.get(0) instanceof AsyncAppender);
fnames = util.getFileNames(fileName);
util.waitReplication(fs, fileName, (short)3);
userGroupInfo = UserGroupInformation.createUserForTesting(username, groups);
LOG.info("Audit log file: {}, exists: {}, length: {}", AUDIT_LOG_FILE, AUDIT_LOG_FILE.exists(),
AUDIT_LOG_FILE.length());
}
@After
public void teardownCluster() throws Exception {
try (PrintWriter writer = new PrintWriter(AUDIT_LOG_FILE)) {
writer.print("");
}
util.cleanup(fs, "/srcdat");
if (fs != null) {
fs.close();
@ -159,11 +164,10 @@ public class TestAuditLogs {
final Path file = new Path(fnames[0]);
FileSystem userfs = DFSTestUtil.getFileSystemAs(userGroupInfo, conf);
setupAuditLogs();
InputStream istream = userfs.open(file);
int val = istream.read();
istream.close();
verifyAuditLogs(true);
verifySuccessCommandsAuditLogs(2, fnames[0], "cmd=open");
assertTrue("failed to read from file", val >= 0);
}
@ -173,9 +177,8 @@ public class TestAuditLogs {
final Path file = new Path(fnames[0]);
FileSystem userfs = DFSTestUtil.getFileSystemAs(userGroupInfo, conf);
setupAuditLogs();
FileStatus st = userfs.getFileStatus(file);
verifyAuditLogs(true);
verifySuccessCommandsAuditLogs(2, fnames[0], "cmd=getfileinfo");
assertTrue("failed to stat file", st != null && st.isFile());
}
@ -188,15 +191,13 @@ public class TestAuditLogs {
fs.setPermission(file, new FsPermission((short)0600));
fs.setOwner(file, "root", null);
setupAuditLogs();
try {
userfs.open(file);
fail("open must not succeed");
} catch(AccessControlException e) {
System.out.println("got access denied, as expected.");
}
verifyAuditLogs(false);
verifyFailedCommandsAuditLogs(1, fnames[0], "cmd=open");
}
/** test that access via webhdfs puts proper entry in audit log */
@ -207,14 +208,12 @@ public class TestAuditLogs {
fs.setPermission(file, new FsPermission((short)0644));
fs.setOwner(file, "root", null);
setupAuditLogs();
WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(userGroupInfo, conf, WebHdfsConstants.WEBHDFS_SCHEME);
InputStream istream = webfs.open(file);
int val = istream.read();
istream.close();
verifyAuditLogsRepeat(true, 3);
verifySuccessCommandsAuditLogs(3, fnames[0], "cmd=open");
assertTrue("failed to read from file", val >= 0);
}
@ -226,12 +225,10 @@ public class TestAuditLogs {
fs.setPermission(file, new FsPermission((short)0644));
fs.setOwner(file, "root", null);
setupAuditLogs();
WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(userGroupInfo, conf, WebHdfsConstants.WEBHDFS_SCHEME);
FileStatus st = webfs.getFileStatus(file);
verifyAuditLogs(true);
verifySuccessCommandsAuditLogs(2, fnames[0], "cmd=getfileinfo");
assertTrue("failed to stat file", st != null && st.isFile());
}
@ -243,7 +240,6 @@ public class TestAuditLogs {
fs.setPermission(file, new FsPermission((short)0600));
fs.setOwner(file, "root", null);
setupAuditLogs();
try {
WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(userGroupInfo, conf, WebHdfsConstants.WEBHDFS_SCHEME);
InputStream istream = webfs.open(file);
@ -252,7 +248,7 @@ public class TestAuditLogs {
} catch(AccessControlException E) {
System.out.println("got access denied, as expected.");
}
verifyAuditLogsRepeat(false, 2);
verifyFailedCommandsAuditLogs(1, fnames[0], "cmd=open");
}
/** test that open via webhdfs puts proper entry in audit log */
@ -263,124 +259,68 @@ public class TestAuditLogs {
fs.setPermission(file, new FsPermission((short)0644));
fs.setOwner(file, "root", null);
setupAuditLogs();
WebHdfsFileSystem webfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(userGroupInfo, conf, WebHdfsConstants.WEBHDFS_SCHEME);
webfs.open(file).read();
verifyAuditLogsCheckPattern(true, 3, webOpenPattern);
verifySuccessCommandsAuditLogs(3, fnames[0], "cmd=open");
}
/** make sure that "\r\n" isn't made into a newline in audit log */
@Test
public void testAuditCharacterEscape() throws Exception {
final Path file = new Path("foo" + "\r\n" + "bar");
setupAuditLogs();
fs.create(file);
verifyAuditLogsRepeat(true, 1);
verifySuccessCommandsAuditLogs(1, "foo", "cmd=create");
}
/** Sets up log4j logger for auditlogs */
private void setupAuditLogs() throws IOException {
Logger logger = FSNamesystem.AUDIT_LOG;
// enable logging now that the test is ready to run
logger.setLevel(Level.INFO);
}
private void configureAuditLogs() throws IOException {
// Shutdown the LogManager to release all logger open file handles.
// Unfortunately, Apache commons logging library does not provide
// means to release underlying loggers. For additional info look up
// commons library FAQ.
LogManager.shutdown();
File file = new File(auditLogFile);
if (file.exists()) {
assertTrue(file.delete());
}
// disable logging while the cluster startup preps files
disableAuditLog();
PatternLayout layout = new PatternLayout("%m%n");
RollingFileAppender appender = new RollingFileAppender(layout, auditLogFile);
Logger logger = FSNamesystem.AUDIT_LOG;
logger.addAppender(appender);
}
// Ensure audit log has only one entry
private void verifyAuditLogs(boolean expectSuccess) throws IOException {
verifyAuditLogsRepeat(expectSuccess, 1);
}
// Ensure audit log has exactly N entries
private void verifyAuditLogsRepeat(boolean expectSuccess, int ndupe)
private void verifySuccessCommandsAuditLogs(int leastExpected, String file, String cmd)
throws IOException {
// Turn off the logs
disableAuditLog();
// Close the appenders and force all logs to be flushed
Logger logger = FSNamesystem.AUDIT_LOG;
Enumeration<?> appenders = logger.getAllAppenders();
while (appenders.hasMoreElements()) {
Appender appender = (Appender)appenders.nextElement();
appender.close();
}
BufferedReader reader = new BufferedReader(new FileReader(auditLogFile));
String line = null;
boolean ret = true;
try {
for (int i = 0; i < ndupe; i++) {
line = reader.readLine();
try (BufferedReader reader = new BufferedReader(new FileReader(AUDIT_LOG_FILE))) {
String line;
int success = 0;
while ((line = reader.readLine()) != null) {
assertNotNull(line);
assertTrue("Expected audit event not found in audit log",
auditPattern.matcher(line).matches());
ret &= successPattern.matcher(line).matches();
}
assertNull("Unexpected event in audit log", reader.readLine());
assertTrue("Expected success=" + expectSuccess, ret == expectSuccess);
} finally {
reader.close();
}
}
// Ensure audit log has exactly N entries
private void verifyAuditLogsCheckPattern(boolean expectSuccess, int ndupe, Pattern pattern)
throws IOException {
// Turn off the logs
disableAuditLog();
// Close the appenders and force all logs to be flushed
Logger logger = FSNamesystem.AUDIT_LOG;
Enumeration<?> appenders = logger.getAllAppenders();
while (appenders.hasMoreElements()) {
Appender appender = (Appender)appenders.nextElement();
appender.close();
}
BufferedReader reader = new BufferedReader(new FileReader(auditLogFile));
String line = null;
boolean ret = true;
boolean patternMatches = false;
try {
for (int i = 0; i < ndupe; i++) {
line = reader.readLine();
assertNotNull(line);
patternMatches |= pattern.matcher(line).matches();
ret &= successPattern.matcher(line).matches();
LOG.info("Line: {}", line);
if (SUCCESS_PATTERN.matcher(line).matches() && line.contains(file) && line.contains(
cmd)) {
assertTrue("Expected audit event not found in audit log",
AUDIT_PATTERN.matcher(line).matches());
LOG.info("Successful verification. Log line: {}", line);
success++;
}
assertNull("Unexpected event in audit log", reader.readLine());
assertTrue("Expected audit event not found in audit log", patternMatches);
assertTrue("Expected success=" + expectSuccess, ret == expectSuccess);
} finally {
reader.close();
}
if (success < leastExpected) {
throw new AssertionError(
"Least expected: " + leastExpected + ". Actual success: " + success);
}
}
}
private void disableAuditLog() {
GenericTestUtils.disableLog(LoggerFactory.getLogger(
FSNamesystem.class.getName() + ".audit"));
private void verifyFailedCommandsAuditLogs(int leastExpected, String file, String cmd)
throws IOException {
try (BufferedReader reader = new BufferedReader(new FileReader(AUDIT_LOG_FILE))) {
String line;
int success = 0;
while ((line = reader.readLine()) != null) {
assertNotNull(line);
LOG.info("Line: {}", line);
if (FAILURE_PATTERN.matcher(line).matches() && line.contains(file) && line.contains(
cmd)) {
assertTrue("Expected audit event not found in audit log",
AUDIT_PATTERN.matcher(line).matches());
LOG.info("Failure verification. Log line: {}", line);
success++;
}
}
assertEquals("Expected: " + leastExpected + ". Actual failure: " + success, leastExpected,
success);
if (success < leastExpected) {
throw new AssertionError(
"Least expected: " + leastExpected + ". Actual success: " + success);
}
}
}
}

View File

@ -103,7 +103,7 @@ public class TestFSNamesystemLockReport {
FSDataOutputStream os = testLockReport(() ->
userfs.create(new Path("/file")),
".* by create \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"perm=bob:hadoop:rw-r--r--\\) .*");
os.close();
@ -111,7 +111,7 @@ public class TestFSNamesystemLockReport {
// ip=/127.0.0.1,src=/file,dst=null,perm=null)"
FSDataInputStream is = testLockReport(() -> userfs.open(new Path("/file")),
".* by open \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"perm=null\\) .*");
is.close();
@ -120,49 +120,49 @@ public class TestFSNamesystemLockReport {
testLockReport(() ->
userfs.setPermission(new Path("/file"), new FsPermission(644)),
".* by setPermission \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"perm=bob:hadoop:-w----r-T\\) .*");
// The log output should contain "by setOwner (ugi=bob (auth:SIMPLE),
// ip=/127.0.0.1,src=/file,dst=null,perm=alice:group1:-w----r-T)"
testLockReport(() -> userfs.setOwner(new Path("/file"), "alice", "group1"),
".* by setOwner \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"perm=alice:group1:-w----r-T\\) .*");
// The log output should contain "by listStatus (ugi=bob (auth:SIMPLE),
// ip=/127.0.0.1,src=/,dst=null,perm=null)"
testLockReport(() -> userfs.listStatus(new Path("/")),
".* by listStatus \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/,dst=null," +
"perm=null\\) .*");
// The log output should contain "by getfileinfo (ugi=bob (auth:SIMPLE),
// ip=/127.0.0.1,src=/file,dst=null,perm=null)"
testLockReport(() -> userfs.getFileStatus(new Path("/file")),
".* by getfileinfo \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=null," +
"perm=null\\) .*");
// The log output should contain "by mkdirs (ugi=bob (auth:SIMPLE),
// ip=/127.0.0.1,src=/dir,dst=null,perm=bob:hadoop:rwxr-xr-x)"
testLockReport(() -> userfs.mkdirs(new Path("/dir")),
".* by mkdirs \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/dir,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/dir,dst=null," +
"perm=bob:hadoop:rwxr-xr-x\\) .*");
// The log output should contain "by delete (ugi=bob (auth:SIMPLE),
// ip=/127.0.0.1,src=/file2,dst=null,perm=null)"
testLockReport(() -> userfs.rename(new Path("/file"), new Path("/file2")),
".* by rename \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=/file2," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file,dst=/file2," +
"perm=alice:group1:-w----r-T\\) .*");
// The log output should contain "by rename (ugi=bob (auth:SIMPLE),
// ip=/127.0.0.1,src=/file,dst=/file2,perm=alice:group1:-w----r-T)"
testLockReport(() -> userfs.delete(new Path("/file2"), false),
".* by delete \\(ugi=bob \\(auth:SIMPLE\\)," +
"ip=/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file2,dst=null," +
"ip=[a-zA-Z0-9.]+/\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3},src=/file2,dst=null," +
"perm=null\\) .*");
}

View File

@ -23,7 +23,6 @@ import static org.apache.hadoop.hdfs.MiniDFSCluster.HDFS_MINIDFS_BASEDIR;
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.ArgumentMatchers.any;
@ -119,11 +118,8 @@ import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.ToolRunner;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.log4j.PatternLayout;
import org.apache.log4j.RollingFileAppender;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -136,9 +132,9 @@ public class TestFsck {
private static final org.slf4j.Logger LOG =
LoggerFactory.getLogger(TestFsck.class.getName());
static final String AUDITLOG_FILE =
GenericTestUtils.getTempPath("TestFsck-audit.log");
private static final File AUDIT_LOG_FILE =
new File(System.getProperty("hadoop.log.dir"), "hdfs-audit.log");
// Pattern for:
// allowed=true ugi=name ip=/address cmd=FSCK src=/ dst=null perm=null
static final Pattern FSCK_PATTERN = Pattern.compile(
@ -195,6 +191,11 @@ public class TestFsck {
shutdownCluster();
}
@AfterClass
public static void afterClass() throws Exception {
assertTrue(AUDIT_LOG_FILE.delete());
}
private void shutdownCluster() throws Exception {
if (cluster != null) {
cluster.shutdown();
@ -221,7 +222,6 @@ public class TestFsck {
final Path file = new Path(fileName);
long aTime = fs.getFileStatus(file).getAccessTime();
Thread.sleep(precision);
setupAuditLogs();
String outStr = runFsck(conf, 0, true, "/");
verifyAuditLogs();
assertEquals(aTime, fs.getFileStatus(file).getAccessTime());
@ -245,54 +245,27 @@ public class TestFsck {
util.cleanup(fs, "/srcdat");
}
/** Sets up log4j logger for auditlogs. */
private void setupAuditLogs() throws IOException {
File file = new File(AUDITLOG_FILE);
if (file.exists()) {
file.delete();
}
Logger logger = FSNamesystem.AUDIT_LOG;
logger.removeAllAppenders();
logger.setLevel(Level.INFO);
PatternLayout layout = new PatternLayout("%m%n");
RollingFileAppender appender =
new RollingFileAppender(layout, AUDITLOG_FILE);
logger.addAppender(appender);
}
private void verifyAuditLogs() throws IOException {
// Turn off the logs
GenericTestUtils.disableLog(LoggerFactory.getLogger(
FSNamesystem.class.getName() + ".audit"));
BufferedReader reader = null;
try {
try (BufferedReader reader = new BufferedReader(new FileReader(AUDIT_LOG_FILE))) {
// Audit log should contain one getfileinfo and one fsck
reader = new BufferedReader(new FileReader(AUDITLOG_FILE));
String line;
// one extra getfileinfo stems from resolving the path
//
for (int i = 0; i < 2; i++) {
line = reader.readLine();
assertNotNull(line);
assertTrue("Expected getfileinfo event not found in audit log",
GET_FILE_INFO_PATTERN.matcher(line).matches());
int getFileStatusSuccess = 0;
int fsckCount = 0;
while ((line = reader.readLine()) != null) {
LOG.info("Line: {}", line);
if (line.contains("cmd=getfileinfo") && GET_FILE_INFO_PATTERN.matcher(line).matches()) {
getFileStatusSuccess++;
} else if (FSCK_PATTERN.matcher(line).matches()) {
fsckCount++;
}
}
line = reader.readLine();
assertNotNull(line);
assertTrue("Expected fsck event not found in audit log", FSCK_PATTERN
.matcher(line).matches());
assertNull("Unexpected event in audit log", reader.readLine());
} finally {
// Close the reader and remove the appender to release the audit log file
// handle after verifying the content of the file.
if (reader != null) {
reader.close();
if (getFileStatusSuccess < 2) {
throw new AssertionError(
"getfileinfo cmd should occur at least 2 times. Actual count: " + getFileStatusSuccess);
}
Logger logger = FSNamesystem.AUDIT_LOG;
if (logger != null) {
logger.removeAllAppenders();
if (fsckCount < 1) {
throw new AssertionError(
"fsck should be present at least once. Actual count: " + fsckCount);
}
}
}
@ -1411,7 +1384,6 @@ public class TestFsck {
util.waitReplication(fs, fileName, (short)3);
long aTime = fc.getFileStatus(symlink).getAccessTime();
Thread.sleep(precision);
setupAuditLogs();
String outStr = runFsck(conf, 0, true, "/");
verifyAuditLogs();
assertEquals(aTime, fc.getFileStatus(symlink).getAccessTime());
@ -2055,7 +2027,6 @@ public class TestFsck {
long replTime = fs.getFileStatus(replFilePath).getAccessTime();
long ecTime = fs.getFileStatus(largeFilePath).getAccessTime();
Thread.sleep(precision);
setupAuditLogs();
String outStr = runFsck(conf, 0, true, "/");
verifyAuditLogs();
assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime());

View File

@ -26,9 +26,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
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.Rule;
import org.junit.Test;
import org.junit.rules.Timeout;
@ -37,7 +36,6 @@ import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeoutException;
import java.util.regex.Pattern;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.junit.Assert.*;
@ -86,8 +84,8 @@ public class TestNameNodeMetricsLogger {
"DummyMetrics", metricsProvider);
makeNameNode(true); // Log metrics early and often.
final PatternMatchingAppender appender =
new PatternMatchingAppender("^.*FakeMetric42.*$");
addAppender(org.apache.log4j.Logger.getLogger(NameNode.METRICS_LOG_NAME), appender);
(PatternMatchingAppender) org.apache.log4j.Logger.getLogger(NameNode.METRICS_LOG_NAME)
.getAppender("PATTERNMATCHERAPPENDER");
// Ensure that the supplied pattern was matched.
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@ -115,12 +113,6 @@ public class TestNameNodeMetricsLogger {
return new TestNameNode(conf);
}
private void addAppender(org.apache.log4j.Logger logger, Appender appender) {
@SuppressWarnings("unchecked")
List<Appender> appenders = Collections.list(logger.getAllAppenders());
((AsyncAppender) appenders.get(0)).addAppender(appender);
}
/**
* A NameNode that stubs out the NameSystem for testing.
*/
@ -149,37 +141,4 @@ public class TestNameNodeMetricsLogger {
}
}
/**
* 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;
}
}
}

View File

@ -45,7 +45,7 @@ import java.util.function.Supplier;
*/
public class TestDNFencingWithReplication {
static {
GenericTestUtils.setLogLevel(FSNamesystem.AUDIT_LOG, org.apache.log4j.Level.WARN);
GenericTestUtils.setLogLevel(FSNamesystem.AUDIT_LOG, Level.WARN);
GenericTestUtils.setLogLevel(Server.LOG, Level.ERROR);
GenericTestUtils.setLogLevel(RetryInvocationHandler.LOG, Level.ERROR);
}

View File

@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.tools.GetGroups;
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
import org.apache.hadoop.ipc.metrics.RpcMetrics;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.junit.After;
@ -652,6 +653,17 @@ public class TestObserverNode {
}
}
@Test
public void testGetListingForDeletedDir() throws Exception {
Path path = new Path("/dir1/dir2/testFile");
dfs.create(path).close();
assertTrue(dfs.delete(new Path("/dir1/dir2"), true));
LambdaTestUtils.intercept(FileNotFoundException.class,
() -> dfs.listLocatedStatus(new Path("/dir1/dir2")));
}
@Test
public void testSimpleReadEmptyDirOrFile() throws IOException {
// read empty dir

View File

@ -22,31 +22,60 @@ log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
# Only to be used for testing
log4j.appender.PATTERNMATCHERAPPENDER=org.apache.hadoop.hdfs.server.namenode.PatternMatchingAppender
#
# NameNode metrics logging.
# The default is to retain two namenode-metrics.log files up to 64MB each.
#
log4j.logger.NameNodeMetricsLog=INFO,NNMETRICSRFA
# TODO : While migrating to log4j2, replace AsyncRFAAppender with AsyncAppender as
# log4j2 properties support wrapping of other appenders to AsyncAppender using appender ref
namenode.metrics.logger=INFO,ASYNCNNMETRICSRFA,PATTERNMATCHERAPPENDER
log4j.logger.NameNodeMetricsLog=${namenode.metrics.logger}
log4j.additivity.NameNodeMetricsLog=false
log4j.appender.NNMETRICSRFA=org.apache.log4j.RollingFileAppender
log4j.appender.NNMETRICSRFA.File=${hadoop.log.dir}/namenode-metrics.log
log4j.appender.NNMETRICSRFA.layout=org.apache.log4j.PatternLayout
log4j.appender.NNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
log4j.appender.NNMETRICSRFA.MaxBackupIndex=1
log4j.appender.NNMETRICSRFA.MaxFileSize=64MB
log4j.appender.ASYNCNNMETRICSRFA=org.apache.hadoop.hdfs.util.AsyncRFAAppender
log4j.appender.ASYNCNNMETRICSRFA.conversionPattern=%d{ISO8601} %m%n
log4j.appender.ASYNCNNMETRICSRFA.maxFileSize=64MB
log4j.appender.ASYNCNNMETRICSRFA.fileName=${hadoop.log.dir}/namenode-metrics.log
log4j.appender.ASYNCNNMETRICSRFA.maxBackupIndex=1
#
# DataNode metrics logging.
# The default is to retain two datanode-metrics.log files up to 64MB each.
#
log4j.logger.DataNodeMetricsLog=INFO,DNMETRICSRFA
# TODO : While migrating to log4j2, replace AsyncRFAAppender with AsyncAppender as
# log4j2 properties support wrapping of other appenders to AsyncAppender using appender ref
datanode.metrics.logger=INFO,ASYNCDNMETRICSRFA,PATTERNMATCHERAPPENDER
log4j.logger.DataNodeMetricsLog=${datanode.metrics.logger}
log4j.additivity.DataNodeMetricsLog=false
log4j.appender.DNMETRICSRFA=org.apache.log4j.RollingFileAppender
log4j.appender.DNMETRICSRFA.File=${hadoop.log.dir}/datanode-metrics.log
log4j.appender.DNMETRICSRFA.layout=org.apache.log4j.PatternLayout
log4j.appender.DNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
log4j.appender.DNMETRICSRFA.MaxBackupIndex=1
log4j.appender.DNMETRICSRFA.MaxFileSize=64MB
log4j.appender.ASYNCDNMETRICSRFA=org.apache.hadoop.hdfs.util.AsyncRFAAppender
log4j.appender.ASYNCDNMETRICSRFA.conversionPattern=%d{ISO8601} %m%n
log4j.appender.ASYNCDNMETRICSRFA.maxFileSize=64MB
log4j.appender.ASYNCDNMETRICSRFA.fileName=${hadoop.log.dir}/datanode-metrics.log
log4j.appender.ASYNCDNMETRICSRFA.maxBackupIndex=1
# Supress KMS error log
log4j.logger.com.sun.jersey.server.wadl.generators.WadlGeneratorJAXBGrammarGenerator=OFF
log4j.logger.com.sun.jersey.server.wadl.generators.WadlGeneratorJAXBGrammarGenerator=OFF
#
# hdfs audit logging
#
# TODO : While migrating to log4j2, replace AsyncRFAAppender with AsyncAppender as
# log4j2 properties support wrapping of other appenders to AsyncAppender using appender ref
hdfs.audit.logger=INFO,ASYNCAUDITAPPENDER
hdfs.audit.log.maxfilesize=256MB
hdfs.audit.log.maxbackupindex=20
log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=${hdfs.audit.logger}
log4j.additivity.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=false
log4j.appender.ASYNCAUDITAPPENDER=org.apache.hadoop.hdfs.util.AsyncRFAAppender
log4j.appender.ASYNCAUDITAPPENDER.blocking=false
log4j.appender.ASYNCAUDITAPPENDER.bufferSize=256
log4j.appender.ASYNCAUDITAPPENDER.conversionPattern=%m%n
log4j.appender.ASYNCAUDITAPPENDER.maxFileSize=${hdfs.audit.log.maxfilesize}
log4j.appender.ASYNCAUDITAPPENDER.fileName=${hadoop.log.dir}/hdfs-audit.log
log4j.appender.ASYNCAUDITAPPENDER.maxBackupIndex=${hdfs.audit.log.maxbackupindex}

View File

@ -100,39 +100,6 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-api</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-engine</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter-params</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-junit-jupiter</artifactId>
<version>4.11.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>uk.org.webcompere</groupId>
<artifactId>system-stubs-core</artifactId>
<version>1.1.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>uk.org.webcompere</groupId>
<artifactId>system-stubs-jupiter</artifactId>
<version>1.1.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>

View File

@ -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);

View File

@ -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);
}

View File

@ -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();

View File

@ -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);

View File

@ -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();

View File

@ -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();

View File

@ -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"));
}
}

View File

@ -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 {

View File

@ -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 {

View File

@ -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());
}
}
}

View File

@ -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;

View File

@ -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<Task> 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<AMInfo> 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();
}
}

View File

@ -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);

View File

@ -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<TaskId,Task> 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<TaskAttemptId, TaskAttempt> 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<TaskAttempt> 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<TaskId,Task> 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<TaskAttemptId, TaskAttempt> 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<TaskId, Task> 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<TaskAttemptId, TaskAttempt> 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);

View File

@ -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<Task> 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<TaskAttempt> 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<Task> 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<Task> 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<TaskAttempt> 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) {

View File

@ -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());
}

View File

@ -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<TaskId,Task> 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<TaskAttemptId, TaskAttempt> 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<TaskAttempt> 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<TaskId,Task> 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<Task> 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<TaskAttemptId, TaskAttempt> 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<TaskAttempt> 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<Task> 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<Task> 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<Task> 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<TaskId,Task> 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<Task> 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<TaskAttemptId, TaskAttempt> 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<TaskAttempt> 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 {

View File

@ -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

View File

@ -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<Task> 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<Task> 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<Task> 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<NodeReport> updatedNodes = new ArrayList<NodeReport>();
@ -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<Task> 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<Task> 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<Task> 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);
}

View File

@ -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 {

View File

@ -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,

View File

@ -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<Task> 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<String> 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<Task> 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<MRClientProtocol>() {
@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<AMInfo> 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());
}

View File

@ -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<Task> 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<TaskAttempt> 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<Task> 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<Task> 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<Task> 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<Task> 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<Task> 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<Task> 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<Task> 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<Task> 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<Task> 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<Event> arg, List<EventType> expectedJobHistoryEvents,
long expectedMapLaunches, long expectedFailedMaps) {
assertEquals(finalState, checkTask.getState(), "Final State of Task");
assertEquals("Final State of Task", finalState, checkTask.getState());
Map<TaskAttemptId, TaskAttempt> 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<Event> 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) {

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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<Event> {
@ -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();

View File

@ -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<NodeReport, TaskId>

View File

@ -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><LOG_DIR>/stdout" +
" 2><LOG_DIR>/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><LOG_DIR>/stdout" +
" 2><LOG_DIR>/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"));
}
}

View File

@ -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<String, ByteBuffer> 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 {

View File

@ -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<Event> 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<Event> 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<String, Boolean> expected = new HashMap<String, Boolean>();
expected.put("host1", true);
@ -362,16 +361,16 @@ public class TestTaskAttempt{
Job job = app.submit(conf);
app.waitForState(job, JobState.RUNNING);
Map<TaskId, Task> tasks = job.getTasks();
Assertions.assertEquals(2, tasks.size(), "Num tasks is not correct");
Assert.assertEquals("Num tasks is not correct", 2, tasks.size());
Iterator<Task> taskIter = tasks.values().iterator();
Task mTask = taskIter.next();
app.waitForState(mTask, TaskState.RUNNING);
Task rTask = taskIter.next();
app.waitForState(rTask, TaskState.RUNNING);
Map<TaskAttemptId, TaskAttempt> mAttempts = mTask.getAttempts();
Assertions.assertEquals(1, mAttempts.size(), "Num attempts is not correct");
Assert.assertEquals("Num attempts is not correct", 1, mAttempts.size());
Map<TaskAttemptId, TaskAttempt> 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<TaskId, Task> 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<TaskAttemptId, TaskAttempt> 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<TaskAttempt> 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<TaskId, Task> 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(

View File

@ -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 {

View File

@ -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 {

View File

@ -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<TaskId, Task> 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<TaskAttemptId, TaskAttempt> 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

View File

@ -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<String, ByteBuffer> serviceResponse =
new HashMap<String, ByteBuffer>();
@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");

View File

@ -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) {

View File

@ -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);

View File

@ -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);

View File

@ -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)));
}
}

View File

@ -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);
}
}

View File

@ -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);
}
}

View File

@ -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<String, String> 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<String, String> 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<String, String> 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<String, String> 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<String, String> 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<String, String> 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();

Some files were not shown because too many files have changed in this diff Show More