Merging r1541618 through r1542122 from trunk to branch HDFS-2832
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832@1542125 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
132a8ff7c7
@ -185,7 +185,7 @@ public void authenticate(URL url, AuthenticatedURL.Token token)
|
|||||||
conn.setRequestMethod(AUTH_HTTP_METHOD);
|
conn.setRequestMethod(AUTH_HTTP_METHOD);
|
||||||
conn.connect();
|
conn.connect();
|
||||||
|
|
||||||
if (conn.getRequestProperty(AUTHORIZATION) != null && conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
|
if (conn.getResponseCode() == HttpURLConnection.HTTP_OK) {
|
||||||
LOG.debug("JDK performed authentication on our behalf.");
|
LOG.debug("JDK performed authentication on our behalf.");
|
||||||
// If the JDK already did the SPNEGO back-and-forth for
|
// If the JDK already did the SPNEGO back-and-forth for
|
||||||
// us, just pull out the token.
|
// us, just pull out the token.
|
||||||
|
@ -136,7 +136,6 @@ protected void _testAuthentication(Authenticator authenticator, boolean doPost)
|
|||||||
TestConnectionConfigurator connConf = new TestConnectionConfigurator();
|
TestConnectionConfigurator connConf = new TestConnectionConfigurator();
|
||||||
AuthenticatedURL aUrl = new AuthenticatedURL(authenticator, connConf);
|
AuthenticatedURL aUrl = new AuthenticatedURL(authenticator, connConf);
|
||||||
HttpURLConnection conn = aUrl.openConnection(url, token);
|
HttpURLConnection conn = aUrl.openConnection(url, token);
|
||||||
Assert.assertTrue(token.isSet());
|
|
||||||
Assert.assertTrue(connConf.invoked);
|
Assert.assertTrue(connConf.invoked);
|
||||||
String tokenStr = token.toString();
|
String tokenStr = token.toString();
|
||||||
if (doPost) {
|
if (doPost) {
|
||||||
|
@ -284,7 +284,7 @@ Trunk (Unreleased)
|
|||||||
|
|
||||||
HADOOP-7761. Improve the performance of raw comparisons. (todd)
|
HADOOP-7761. Improve the performance of raw comparisons. (todd)
|
||||||
|
|
||||||
HADOOP-8589 ViewFs tests fail when tests and home dirs are nested (sanjay Radia)
|
HADOOP-8589. ViewFs tests fail when tests and home dirs are nested (sanjay Radia)
|
||||||
|
|
||||||
Release 2.3.0 - UNRELEASED
|
Release 2.3.0 - UNRELEASED
|
||||||
|
|
||||||
@ -375,6 +375,14 @@ Release 2.3.0 - UNRELEASED
|
|||||||
HADOOP-10064. Upgrade to maven antrun plugin version 1.7 (Arpit Agarwal via
|
HADOOP-10064. Upgrade to maven antrun plugin version 1.7 (Arpit Agarwal via
|
||||||
jeagles)
|
jeagles)
|
||||||
|
|
||||||
|
HADOOP-9594. Update apache commons math dependency (Timothy St. Clair via
|
||||||
|
stevel)
|
||||||
|
|
||||||
|
HADOOP-10095. In CodecPool, synchronize pool and codecList separately in
|
||||||
|
order to reduce lock contention. (Nicolas Liochon via szetszwo)
|
||||||
|
|
||||||
|
HADOOP-10067. Missing POM dependency on jsr305 (Robert Rati via stevel)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
|
HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
|
||||||
@ -426,6 +434,9 @@ Release 2.3.0 - UNRELEASED
|
|||||||
HADOOP-10093. hadoop-env.cmd sets HADOOP_CLIENT_OPTS with a max heap size
|
HADOOP-10093. hadoop-env.cmd sets HADOOP_CLIENT_OPTS with a max heap size
|
||||||
that is too small. (Shanyu Zhao via cnauroth)
|
that is too small. (Shanyu Zhao via cnauroth)
|
||||||
|
|
||||||
|
HADOOP-10094. NPE in GenericOptionsParser#preProcessForWindows().
|
||||||
|
(Enis Soztutar via cnauroth)
|
||||||
|
|
||||||
Release 2.2.1 - UNRELEASED
|
Release 2.2.1 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
@ -476,6 +487,8 @@ Release 2.2.1 - UNRELEASED
|
|||||||
as [-Dkey, value] which breaks GenericsOptionParser.
|
as [-Dkey, value] which breaks GenericsOptionParser.
|
||||||
(Enis Soztutar via cnauroth)
|
(Enis Soztutar via cnauroth)
|
||||||
|
|
||||||
|
HADOOP-10078. KerberosAuthenticator always does SPNEGO. (rkanter via tucu)
|
||||||
|
|
||||||
Release 2.2.0 - 2013-10-13
|
Release 2.2.0 - 2013-10-13
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
@ -2298,6 +2311,10 @@ Release 0.23.10 - UNRELEASED
|
|||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
|
HADOOP-9956. RPC listener inefficiently assigns connections to readers (daryn)
|
||||||
|
|
||||||
|
HADOOP-9955. RPC idle connection closing is extremely inefficient (daryn)
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
||||||
Release 0.23.9 - 2013-07-08
|
Release 0.23.9 - 2013-07-08
|
||||||
|
@ -55,7 +55,7 @@
|
|||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.commons</groupId>
|
<groupId>org.apache.commons</groupId>
|
||||||
<artifactId>commons-math</artifactId>
|
<artifactId>commons-math3</artifactId>
|
||||||
<scope>compile</scope>
|
<scope>compile</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
@ -83,6 +83,11 @@
|
|||||||
<artifactId>commons-net</artifactId>
|
<artifactId>commons-net</artifactId>
|
||||||
<scope>compile</scope>
|
<scope>compile</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>commons-collections</groupId>
|
||||||
|
<artifactId>commons-collections</artifactId>
|
||||||
|
<scope>compile</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>javax.servlet</groupId>
|
<groupId>javax.servlet</groupId>
|
||||||
<artifactId>servlet-api</artifactId>
|
<artifactId>servlet-api</artifactId>
|
||||||
@ -213,6 +218,11 @@
|
|||||||
<groupId>com.jcraft</groupId>
|
<groupId>com.jcraft</groupId>
|
||||||
<artifactId>jsch</artifactId>
|
<artifactId>jsch</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.google.code.findbugs</groupId>
|
||||||
|
<artifactId>jsr305</artifactId>
|
||||||
|
<scope>compile</scope>
|
||||||
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.zookeeper</groupId>
|
<groupId>org.apache.zookeeper</groupId>
|
||||||
|
@ -65,6 +65,13 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
|
|||||||
/** Default value for IPC_SERVER_RPC_READ_THREADS_KEY */
|
/** Default value for IPC_SERVER_RPC_READ_THREADS_KEY */
|
||||||
public static final int IPC_SERVER_RPC_READ_THREADS_DEFAULT = 1;
|
public static final int IPC_SERVER_RPC_READ_THREADS_DEFAULT = 1;
|
||||||
|
|
||||||
|
/** Number of pending connections that may be queued per socket reader */
|
||||||
|
public static final String IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY =
|
||||||
|
"ipc.server.read.connection-queue.size";
|
||||||
|
/** Default value for IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE */
|
||||||
|
public static final int IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_DEFAULT =
|
||||||
|
100;
|
||||||
|
|
||||||
public static final String IPC_MAXIMUM_DATA_LENGTH =
|
public static final String IPC_MAXIMUM_DATA_LENGTH =
|
||||||
"ipc.maximum.data.length";
|
"ipc.maximum.data.length";
|
||||||
|
|
||||||
@ -219,4 +226,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
|
|||||||
public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "ipc.client.fallback-to-simple-auth-allowed";
|
public static final String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_KEY = "ipc.client.fallback-to-simple-auth-allowed";
|
||||||
public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
|
public static final boolean IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH_ALLOWED_DEFAULT = false;
|
||||||
|
|
||||||
|
/** How often the server scans for idle connections */
|
||||||
|
public static final String IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_KEY =
|
||||||
|
"ipc.client.connection.idle-scan-interval.ms";
|
||||||
|
/** Default value for IPC_SERVER_CONNECTION_IDLE_SCAN_INTERVAL_KEY */
|
||||||
|
public static final int IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_DEFAULT =
|
||||||
|
10000;
|
||||||
}
|
}
|
||||||
|
@ -85,16 +85,15 @@ private static <T> T borrow(Map<Class<T>, List<T>> pool,
|
|||||||
T codec = null;
|
T codec = null;
|
||||||
|
|
||||||
// Check if an appropriate codec is available
|
// Check if an appropriate codec is available
|
||||||
|
List<T> codecList;
|
||||||
synchronized (pool) {
|
synchronized (pool) {
|
||||||
if (pool.containsKey(codecClass)) {
|
codecList = pool.get(codecClass);
|
||||||
List<T> codecList = pool.get(codecClass);
|
}
|
||||||
|
|
||||||
if (codecList != null) {
|
if (codecList != null) {
|
||||||
synchronized (codecList) {
|
synchronized (codecList) {
|
||||||
if (!codecList.isEmpty()) {
|
if (!codecList.isEmpty()) {
|
||||||
codec = codecList.remove(codecList.size()-1);
|
codec = codecList.remove(codecList.size() - 1);
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -105,18 +104,20 @@ private static <T> T borrow(Map<Class<T>, List<T>> pool,
|
|||||||
private static <T> void payback(Map<Class<T>, List<T>> pool, T codec) {
|
private static <T> void payback(Map<Class<T>, List<T>> pool, T codec) {
|
||||||
if (codec != null) {
|
if (codec != null) {
|
||||||
Class<T> codecClass = ReflectionUtils.getClass(codec);
|
Class<T> codecClass = ReflectionUtils.getClass(codec);
|
||||||
|
List<T> codecList;
|
||||||
synchronized (pool) {
|
synchronized (pool) {
|
||||||
if (!pool.containsKey(codecClass)) {
|
codecList = pool.get(codecClass);
|
||||||
pool.put(codecClass, new ArrayList<T>());
|
if (codecList == null) {
|
||||||
|
codecList = new ArrayList<T>();
|
||||||
|
pool.put(codecClass, codecList);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
List<T> codecList = pool.get(codecClass);
|
|
||||||
synchronized (codecList) {
|
synchronized (codecList) {
|
||||||
codecList.add(codec);
|
codecList.add(codec);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
private static <T> int getLeaseCount(
|
private static <T> int getLeaseCount(
|
||||||
|
@ -115,6 +115,16 @@ public static interface CacheTracker {
|
|||||||
public void fadvise(String identifier, long offset, long len, int flags);
|
public void fadvise(String identifier, long offset, long len, int flags);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static CacheManipulator cacheManipulator = new CacheManipulator();
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public static class CacheManipulator {
|
||||||
|
public void mlock(String identifier, ByteBuffer buffer,
|
||||||
|
long len) throws IOException {
|
||||||
|
POSIX.mlock(buffer, len);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
static {
|
static {
|
||||||
if (NativeCodeLoader.isNativeCodeLoaded()) {
|
if (NativeCodeLoader.isNativeCodeLoaded()) {
|
||||||
try {
|
try {
|
||||||
@ -249,7 +259,7 @@ static native void munlock_native(
|
|||||||
*
|
*
|
||||||
* @throws NativeIOException
|
* @throws NativeIOException
|
||||||
*/
|
*/
|
||||||
public static void mlock(ByteBuffer buffer, long len)
|
static void mlock(ByteBuffer buffer, long len)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
assertCodeLoaded();
|
assertCodeLoaded();
|
||||||
if (!buffer.isDirect()) {
|
if (!buffer.isDirect()) {
|
||||||
|
@ -51,11 +51,13 @@
|
|||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Random;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.Timer;
|
||||||
|
import java.util.TimerTask;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import javax.security.sasl.Sasl;
|
import javax.security.sasl.Sasl;
|
||||||
import javax.security.sasl.SaslException;
|
import javax.security.sasl.SaslException;
|
||||||
@ -345,17 +347,8 @@ public static boolean isRpcInvocation() {
|
|||||||
private int port; // port we listen on
|
private int port; // port we listen on
|
||||||
private int handlerCount; // number of handler threads
|
private int handlerCount; // number of handler threads
|
||||||
private int readThreads; // number of read threads
|
private int readThreads; // number of read threads
|
||||||
|
private int readerPendingConnectionQueue; // number of connections to queue per read thread
|
||||||
private Class<? extends Writable> rpcRequestClass; // class used for deserializing the rpc request
|
private Class<? extends Writable> rpcRequestClass; // class used for deserializing the rpc request
|
||||||
private int maxIdleTime; // the maximum idle time after
|
|
||||||
// which a client may be disconnected
|
|
||||||
private int thresholdIdleConnections; // the number of idle connections
|
|
||||||
// after which we will start
|
|
||||||
// cleaning up idle
|
|
||||||
// connections
|
|
||||||
int maxConnectionsToNuke; // the max number of
|
|
||||||
// connections to nuke
|
|
||||||
//during a cleanup
|
|
||||||
|
|
||||||
protected RpcMetrics rpcMetrics;
|
protected RpcMetrics rpcMetrics;
|
||||||
protected RpcDetailedMetrics rpcDetailedMetrics;
|
protected RpcDetailedMetrics rpcDetailedMetrics;
|
||||||
|
|
||||||
@ -373,13 +366,10 @@ public static boolean isRpcInvocation() {
|
|||||||
volatile private boolean running = true; // true while server runs
|
volatile private boolean running = true; // true while server runs
|
||||||
private BlockingQueue<Call> callQueue; // queued calls
|
private BlockingQueue<Call> callQueue; // queued calls
|
||||||
|
|
||||||
private List<Connection> connectionList =
|
// maintains the set of client connections and handles idle timeouts
|
||||||
Collections.synchronizedList(new LinkedList<Connection>());
|
private ConnectionManager connectionManager;
|
||||||
//maintain a list
|
|
||||||
//of client connections
|
|
||||||
private Listener listener = null;
|
private Listener listener = null;
|
||||||
private Responder responder = null;
|
private Responder responder = null;
|
||||||
private int numConnections = 0;
|
|
||||||
private Handler[] handlers = null;
|
private Handler[] handlers = null;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -449,8 +439,8 @@ Iterable<? extends Thread> getHandlers() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
List<Connection> getConnections() {
|
Connection[] getConnections() {
|
||||||
return connectionList;
|
return connectionManager.toArray();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -518,11 +508,6 @@ private class Listener extends Thread {
|
|||||||
private Reader[] readers = null;
|
private Reader[] readers = null;
|
||||||
private int currentReader = 0;
|
private int currentReader = 0;
|
||||||
private InetSocketAddress address; //the address we bind at
|
private InetSocketAddress address; //the address we bind at
|
||||||
private Random rand = new Random();
|
|
||||||
private long lastCleanupRunTime = 0; //the last time when a cleanup connec-
|
|
||||||
//-tion (for idle connections) ran
|
|
||||||
private long cleanupInterval = 10000; //the minimum interval between
|
|
||||||
//two cleanup runs
|
|
||||||
private int backlogLength = conf.getInt(
|
private int backlogLength = conf.getInt(
|
||||||
CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_KEY,
|
CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_KEY,
|
||||||
CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_DEFAULT);
|
CommonConfigurationKeysPublic.IPC_SERVER_LISTEN_QUEUE_SIZE_DEFAULT);
|
||||||
@ -553,12 +538,14 @@ public Listener() throws IOException {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private class Reader extends Thread {
|
private class Reader extends Thread {
|
||||||
private volatile boolean adding = false;
|
final private BlockingQueue<Connection> pendingConnections;
|
||||||
private final Selector readSelector;
|
private final Selector readSelector;
|
||||||
|
|
||||||
Reader(String name) throws IOException {
|
Reader(String name) throws IOException {
|
||||||
super(name);
|
super(name);
|
||||||
|
|
||||||
|
this.pendingConnections =
|
||||||
|
new LinkedBlockingQueue<Connection>(readerPendingConnectionQueue);
|
||||||
this.readSelector = Selector.open();
|
this.readSelector = Selector.open();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -580,10 +567,14 @@ private synchronized void doRunLoop() {
|
|||||||
while (running) {
|
while (running) {
|
||||||
SelectionKey key = null;
|
SelectionKey key = null;
|
||||||
try {
|
try {
|
||||||
readSelector.select();
|
// consume as many connections as currently queued to avoid
|
||||||
while (adding) {
|
// unbridled acceptance of connections that starves the select
|
||||||
this.wait(1000);
|
int size = pendingConnections.size();
|
||||||
|
for (int i=size; i>0; i--) {
|
||||||
|
Connection conn = pendingConnections.take();
|
||||||
|
conn.channel.register(readSelector, SelectionKey.OP_READ, conn);
|
||||||
}
|
}
|
||||||
|
readSelector.select();
|
||||||
|
|
||||||
Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
|
Iterator<SelectionKey> iter = readSelector.selectedKeys().iterator();
|
||||||
while (iter.hasNext()) {
|
while (iter.hasNext()) {
|
||||||
@ -607,27 +598,15 @@ private synchronized void doRunLoop() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This gets reader into the state that waits for the new channel
|
* Updating the readSelector while it's being used is not thread-safe,
|
||||||
* to be registered with readSelector. If it was waiting in select()
|
* so the connection must be queued. The reader will drain the queue
|
||||||
* the thread will be woken up, otherwise whenever select() is called
|
* and update its readSelector before performing the next select
|
||||||
* it will return even if there is nothing to read and wait
|
|
||||||
* in while(adding) for finishAdd call
|
|
||||||
*/
|
*/
|
||||||
public void startAdd() {
|
public void addConnection(Connection conn) throws InterruptedException {
|
||||||
adding = true;
|
pendingConnections.put(conn);
|
||||||
readSelector.wakeup();
|
readSelector.wakeup();
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized SelectionKey registerChannel(SocketChannel channel)
|
|
||||||
throws IOException {
|
|
||||||
return channel.register(readSelector, SelectionKey.OP_READ);
|
|
||||||
}
|
|
||||||
|
|
||||||
public synchronized void finishAdd() {
|
|
||||||
adding = false;
|
|
||||||
this.notify();
|
|
||||||
}
|
|
||||||
|
|
||||||
void shutdown() {
|
void shutdown() {
|
||||||
assert !running;
|
assert !running;
|
||||||
readSelector.wakeup();
|
readSelector.wakeup();
|
||||||
@ -638,58 +617,12 @@ void shutdown() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
/** cleanup connections from connectionList. Choose a random range
|
|
||||||
* to scan and also have a limit on the number of the connections
|
|
||||||
* that will be cleanedup per run. The criteria for cleanup is the time
|
|
||||||
* for which the connection was idle. If 'force' is true then all
|
|
||||||
* connections will be looked at for the cleanup.
|
|
||||||
*/
|
|
||||||
private void cleanupConnections(boolean force) {
|
|
||||||
if (force || numConnections > thresholdIdleConnections) {
|
|
||||||
long currentTime = Time.now();
|
|
||||||
if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
int start = 0;
|
|
||||||
int end = numConnections - 1;
|
|
||||||
if (!force) {
|
|
||||||
start = rand.nextInt() % numConnections;
|
|
||||||
end = rand.nextInt() % numConnections;
|
|
||||||
int temp;
|
|
||||||
if (end < start) {
|
|
||||||
temp = start;
|
|
||||||
start = end;
|
|
||||||
end = temp;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
int i = start;
|
|
||||||
int numNuked = 0;
|
|
||||||
while (i <= end) {
|
|
||||||
Connection c;
|
|
||||||
synchronized (connectionList) {
|
|
||||||
try {
|
|
||||||
c = connectionList.get(i);
|
|
||||||
} catch (Exception e) {return;}
|
|
||||||
}
|
|
||||||
if (c.timedOut(currentTime)) {
|
|
||||||
if (LOG.isDebugEnabled())
|
|
||||||
LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
|
|
||||||
closeConnection(c);
|
|
||||||
numNuked++;
|
|
||||||
end--;
|
|
||||||
c = null;
|
|
||||||
if (!force && numNuked == maxConnectionsToNuke) break;
|
|
||||||
}
|
|
||||||
else i++;
|
|
||||||
}
|
|
||||||
lastCleanupRunTime = Time.now();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
LOG.info(getName() + ": starting");
|
LOG.info(getName() + ": starting");
|
||||||
SERVER.set(Server.this);
|
SERVER.set(Server.this);
|
||||||
|
connectionManager.startIdleScan();
|
||||||
while (running) {
|
while (running) {
|
||||||
SelectionKey key = null;
|
SelectionKey key = null;
|
||||||
try {
|
try {
|
||||||
@ -713,12 +646,11 @@ public void run() {
|
|||||||
// some thread(s) a chance to finish
|
// some thread(s) a chance to finish
|
||||||
LOG.warn("Out of Memory in server select", e);
|
LOG.warn("Out of Memory in server select", e);
|
||||||
closeCurrentConnection(key, e);
|
closeCurrentConnection(key, e);
|
||||||
cleanupConnections(true);
|
connectionManager.closeIdle(true);
|
||||||
try { Thread.sleep(60000); } catch (Exception ie) {}
|
try { Thread.sleep(60000); } catch (Exception ie) {}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
closeCurrentConnection(key, e);
|
closeCurrentConnection(key, e);
|
||||||
}
|
}
|
||||||
cleanupConnections(false);
|
|
||||||
}
|
}
|
||||||
LOG.info("Stopping " + this.getName());
|
LOG.info("Stopping " + this.getName());
|
||||||
|
|
||||||
@ -731,10 +663,9 @@ public void run() {
|
|||||||
selector= null;
|
selector= null;
|
||||||
acceptChannel= null;
|
acceptChannel= null;
|
||||||
|
|
||||||
// clean up all connections
|
// close all connections
|
||||||
while (!connectionList.isEmpty()) {
|
connectionManager.stopIdleScan();
|
||||||
closeConnection(connectionList.remove(0));
|
connectionManager.closeAll();
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -742,8 +673,6 @@ private void closeCurrentConnection(SelectionKey key, Throwable e) {
|
|||||||
if (key != null) {
|
if (key != null) {
|
||||||
Connection c = (Connection)key.attachment();
|
Connection c = (Connection)key.attachment();
|
||||||
if (c != null) {
|
if (c != null) {
|
||||||
if (LOG.isDebugEnabled())
|
|
||||||
LOG.debug(getName() + ": disconnecting client " + c.getHostAddress());
|
|
||||||
closeConnection(c);
|
closeConnection(c);
|
||||||
c = null;
|
c = null;
|
||||||
}
|
}
|
||||||
@ -754,8 +683,7 @@ InetSocketAddress getAddress() {
|
|||||||
return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
|
return (InetSocketAddress)acceptChannel.socket().getLocalSocketAddress();
|
||||||
}
|
}
|
||||||
|
|
||||||
void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
|
void doAccept(SelectionKey key) throws InterruptedException, IOException, OutOfMemoryError {
|
||||||
Connection c = null;
|
|
||||||
ServerSocketChannel server = (ServerSocketChannel) key.channel();
|
ServerSocketChannel server = (ServerSocketChannel) key.channel();
|
||||||
SocketChannel channel;
|
SocketChannel channel;
|
||||||
while ((channel = server.accept()) != null) {
|
while ((channel = server.accept()) != null) {
|
||||||
@ -765,22 +693,9 @@ void doAccept(SelectionKey key) throws IOException, OutOfMemoryError {
|
|||||||
channel.socket().setKeepAlive(true);
|
channel.socket().setKeepAlive(true);
|
||||||
|
|
||||||
Reader reader = getReader();
|
Reader reader = getReader();
|
||||||
try {
|
Connection c = connectionManager.register(channel);
|
||||||
reader.startAdd();
|
key.attach(c); // so closeCurrentConnection can get the object
|
||||||
SelectionKey readKey = reader.registerChannel(channel);
|
reader.addConnection(c);
|
||||||
c = new Connection(readKey, channel, Time.now());
|
|
||||||
readKey.attach(c);
|
|
||||||
synchronized (connectionList) {
|
|
||||||
connectionList.add(numConnections, c);
|
|
||||||
numConnections++;
|
|
||||||
}
|
|
||||||
if (LOG.isDebugEnabled())
|
|
||||||
LOG.debug("Server connection from " + c.toString() +
|
|
||||||
"; # active connections: " + numConnections +
|
|
||||||
"; # queued calls: " + callQueue.size());
|
|
||||||
} finally {
|
|
||||||
reader.finishAdd();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -808,10 +723,6 @@ void doRead(SelectionKey key) throws InterruptedException {
|
|||||||
count = -1; //so that the (count < 0) block is executed
|
count = -1; //so that the (count < 0) block is executed
|
||||||
}
|
}
|
||||||
if (count < 0) {
|
if (count < 0) {
|
||||||
if (LOG.isDebugEnabled())
|
|
||||||
LOG.debug(getName() + ": disconnecting client " +
|
|
||||||
c + ". Number of active connections: "+
|
|
||||||
numConnections);
|
|
||||||
closeConnection(c);
|
closeConnection(c);
|
||||||
c = null;
|
c = null;
|
||||||
}
|
}
|
||||||
@ -1190,8 +1101,7 @@ public class Connection {
|
|||||||
private boolean sentNegotiate = false;
|
private boolean sentNegotiate = false;
|
||||||
private boolean useWrap = false;
|
private boolean useWrap = false;
|
||||||
|
|
||||||
public Connection(SelectionKey key, SocketChannel channel,
|
public Connection(SocketChannel channel, long lastContact) {
|
||||||
long lastContact) {
|
|
||||||
this.channel = channel;
|
this.channel = channel;
|
||||||
this.lastContact = lastContact;
|
this.lastContact = lastContact;
|
||||||
this.data = null;
|
this.data = null;
|
||||||
@ -1253,12 +1163,6 @@ private void incRpcCount() {
|
|||||||
rpcCount++;
|
rpcCount++;
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean timedOut(long currentTime) {
|
|
||||||
if (isIdle() && currentTime - lastContact > maxIdleTime)
|
|
||||||
return true;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
private UserGroupInformation getAuthorizedUgi(String authorizedId)
|
private UserGroupInformation getAuthorizedUgi(String authorizedId)
|
||||||
throws InvalidToken, AccessControlException {
|
throws InvalidToken, AccessControlException {
|
||||||
if (authMethod == AuthMethod.TOKEN) {
|
if (authMethod == AuthMethod.TOKEN) {
|
||||||
@ -2189,16 +2093,10 @@ protected Server(String bindAddress, int port,
|
|||||||
CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY,
|
CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY,
|
||||||
CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT);
|
CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_DEFAULT);
|
||||||
}
|
}
|
||||||
|
this.readerPendingConnectionQueue = conf.getInt(
|
||||||
|
CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY,
|
||||||
|
CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_DEFAULT);
|
||||||
this.callQueue = new LinkedBlockingQueue<Call>(maxQueueSize);
|
this.callQueue = new LinkedBlockingQueue<Call>(maxQueueSize);
|
||||||
this.maxIdleTime = 2 * conf.getInt(
|
|
||||||
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
|
|
||||||
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT);
|
|
||||||
this.maxConnectionsToNuke = conf.getInt(
|
|
||||||
CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_KEY,
|
|
||||||
CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_DEFAULT);
|
|
||||||
this.thresholdIdleConnections = conf.getInt(
|
|
||||||
CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_KEY,
|
|
||||||
CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_DEFAULT);
|
|
||||||
this.secretManager = (SecretManager<TokenIdentifier>) secretManager;
|
this.secretManager = (SecretManager<TokenIdentifier>) secretManager;
|
||||||
this.authorize =
|
this.authorize =
|
||||||
conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
|
conf.getBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
|
||||||
@ -2219,6 +2117,7 @@ protected Server(String bindAddress, int port,
|
|||||||
|
|
||||||
// Create the responder here
|
// Create the responder here
|
||||||
responder = new Responder();
|
responder = new Responder();
|
||||||
|
connectionManager = new ConnectionManager();
|
||||||
|
|
||||||
if (secretManager != null) {
|
if (secretManager != null) {
|
||||||
SaslRpcServer.init(conf);
|
SaslRpcServer.init(conf);
|
||||||
@ -2277,11 +2176,7 @@ private List<AuthMethod> getAuthMethods(SecretManager<?> secretManager,
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void closeConnection(Connection connection) {
|
private void closeConnection(Connection connection) {
|
||||||
synchronized (connectionList) {
|
connectionManager.close(connection);
|
||||||
if (connectionList.remove(connection))
|
|
||||||
numConnections--;
|
|
||||||
}
|
|
||||||
connection.close();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -2536,7 +2431,7 @@ public int getPort() {
|
|||||||
* @return the number of open rpc connections
|
* @return the number of open rpc connections
|
||||||
*/
|
*/
|
||||||
public int getNumOpenConnections() {
|
public int getNumOpenConnections() {
|
||||||
return numConnections;
|
return connectionManager.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -2646,4 +2541,151 @@ private static int channelIO(ReadableByteChannel readCh,
|
|||||||
int nBytes = initialRemaining - buf.remaining();
|
int nBytes = initialRemaining - buf.remaining();
|
||||||
return (nBytes > 0) ? nBytes : ret;
|
return (nBytes > 0) ? nBytes : ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private class ConnectionManager {
|
||||||
|
final private AtomicInteger count = new AtomicInteger();
|
||||||
|
final private Set<Connection> connections;
|
||||||
|
|
||||||
|
final private Timer idleScanTimer;
|
||||||
|
final private int idleScanThreshold;
|
||||||
|
final private int idleScanInterval;
|
||||||
|
final private int maxIdleTime;
|
||||||
|
final private int maxIdleToClose;
|
||||||
|
|
||||||
|
ConnectionManager() {
|
||||||
|
this.idleScanTimer = new Timer(
|
||||||
|
"IPC Server idle connection scanner for port " + getPort(), true);
|
||||||
|
this.idleScanThreshold = conf.getInt(
|
||||||
|
CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_KEY,
|
||||||
|
CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_DEFAULT);
|
||||||
|
this.idleScanInterval = conf.getInt(
|
||||||
|
CommonConfigurationKeys.IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_KEY,
|
||||||
|
CommonConfigurationKeys.IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_DEFAULT);
|
||||||
|
this.maxIdleTime = 2 * conf.getInt(
|
||||||
|
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
|
||||||
|
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_DEFAULT);
|
||||||
|
this.maxIdleToClose = conf.getInt(
|
||||||
|
CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_KEY,
|
||||||
|
CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_DEFAULT);
|
||||||
|
// create a set with concurrency -and- a thread-safe iterator, add 2
|
||||||
|
// for listener and idle closer threads
|
||||||
|
this.connections = Collections.newSetFromMap(
|
||||||
|
new ConcurrentHashMap<Connection,Boolean>(
|
||||||
|
maxQueueSize, 0.75f, readThreads+2));
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean add(Connection connection) {
|
||||||
|
boolean added = connections.add(connection);
|
||||||
|
if (added) {
|
||||||
|
count.getAndIncrement();
|
||||||
|
}
|
||||||
|
return added;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean remove(Connection connection) {
|
||||||
|
boolean removed = connections.remove(connection);
|
||||||
|
if (removed) {
|
||||||
|
count.getAndDecrement();
|
||||||
|
}
|
||||||
|
return removed;
|
||||||
|
}
|
||||||
|
|
||||||
|
int size() {
|
||||||
|
return count.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
Connection[] toArray() {
|
||||||
|
return connections.toArray(new Connection[0]);
|
||||||
|
}
|
||||||
|
|
||||||
|
Connection register(SocketChannel channel) {
|
||||||
|
Connection connection = new Connection(channel, Time.now());
|
||||||
|
add(connection);
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Server connection from " + connection +
|
||||||
|
"; # active connections: " + size() +
|
||||||
|
"; # queued calls: " + callQueue.size());
|
||||||
|
}
|
||||||
|
return connection;
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean close(Connection connection) {
|
||||||
|
boolean exists = remove(connection);
|
||||||
|
if (exists) {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug(Thread.currentThread().getName() +
|
||||||
|
": disconnecting client " + connection +
|
||||||
|
". Number of active connections: "+ size());
|
||||||
|
}
|
||||||
|
// only close if actually removed to avoid double-closing due
|
||||||
|
// to possible races
|
||||||
|
connection.close();
|
||||||
|
}
|
||||||
|
return exists;
|
||||||
|
}
|
||||||
|
|
||||||
|
// synch'ed to avoid explicit invocation upon OOM from colliding with
|
||||||
|
// timer task firing
|
||||||
|
synchronized void closeIdle(boolean scanAll) {
|
||||||
|
long minLastContact = Time.now() - maxIdleTime;
|
||||||
|
// concurrent iterator might miss new connections added
|
||||||
|
// during the iteration, but that's ok because they won't
|
||||||
|
// be idle yet anyway and will be caught on next scan
|
||||||
|
int closed = 0;
|
||||||
|
for (Connection connection : connections) {
|
||||||
|
// stop if connections dropped below threshold unless scanning all
|
||||||
|
if (!scanAll && size() < idleScanThreshold) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
// stop if not scanning all and max connections are closed
|
||||||
|
if (connection.isIdle() &&
|
||||||
|
connection.getLastContact() < minLastContact &&
|
||||||
|
close(connection) &&
|
||||||
|
!scanAll && (++closed == maxIdleToClose)) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void closeAll() {
|
||||||
|
// use a copy of the connections to be absolutely sure the concurrent
|
||||||
|
// iterator doesn't miss a connection
|
||||||
|
for (Connection connection : toArray()) {
|
||||||
|
close(connection);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void startIdleScan() {
|
||||||
|
scheduleIdleScanTask();
|
||||||
|
}
|
||||||
|
|
||||||
|
void stopIdleScan() {
|
||||||
|
idleScanTimer.cancel();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void scheduleIdleScanTask() {
|
||||||
|
if (!running) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
TimerTask idleScanTask = new TimerTask(){
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
if (!running) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug(Thread.currentThread().getName()+": task running");
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
closeIdle(false);
|
||||||
|
} finally {
|
||||||
|
// explicitly reschedule so next execution occurs relative
|
||||||
|
// to the end of this scan, not the beginning
|
||||||
|
scheduleIdleScanTask();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
idleScanTimer.schedule(idleScanTask, idleScanInterval);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -38,7 +38,7 @@
|
|||||||
import org.apache.commons.configuration.PropertiesConfiguration;
|
import org.apache.commons.configuration.PropertiesConfiguration;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.commons.math.util.MathUtils;
|
import org.apache.commons.math3.util.ArithmeticUtils;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.metrics2.MetricsInfo;
|
import org.apache.hadoop.metrics2.MetricsInfo;
|
||||||
import org.apache.hadoop.metrics2.MetricsCollector;
|
import org.apache.hadoop.metrics2.MetricsCollector;
|
||||||
@ -457,7 +457,7 @@ private synchronized void configureSinks() {
|
|||||||
MetricsConfig conf = entry.getValue();
|
MetricsConfig conf = entry.getValue();
|
||||||
int sinkPeriod = conf.getInt(PERIOD_KEY, PERIOD_DEFAULT);
|
int sinkPeriod = conf.getInt(PERIOD_KEY, PERIOD_DEFAULT);
|
||||||
confPeriod = confPeriod == 0 ? sinkPeriod
|
confPeriod = confPeriod == 0 ? sinkPeriod
|
||||||
: MathUtils.gcd(confPeriod, sinkPeriod);
|
: ArithmeticUtils.gcd(confPeriod, sinkPeriod);
|
||||||
String clsName = conf.getClassName("");
|
String clsName = conf.getClassName("");
|
||||||
if (clsName == null) continue; // sink can be registered later on
|
if (clsName == null) continue; // sink can be registered later on
|
||||||
String sinkName = entry.getKey();
|
String sinkName = entry.getKey();
|
||||||
|
@ -431,6 +431,9 @@ private String[] preProcessForWindows(String[] args) {
|
|||||||
if (!Shell.WINDOWS) {
|
if (!Shell.WINDOWS) {
|
||||||
return args;
|
return args;
|
||||||
}
|
}
|
||||||
|
if (args == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
List<String> newArgs = new ArrayList<String>(args.length);
|
List<String> newArgs = new ArrayList<String>(args.length);
|
||||||
for (int i=0; i < args.length; i++) {
|
for (int i=0; i < args.length; i++) {
|
||||||
String prop = null;
|
String prop = null;
|
||||||
|
@ -71,6 +71,20 @@ private static int getPermFromString(String permString) {
|
|||||||
return perm;
|
return perm;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper method to remove a subset of permissions (remove) from a
|
||||||
|
* given set (perms).
|
||||||
|
* @param perms The permissions flag to remove from. Should be an OR of a
|
||||||
|
* some combination of {@link ZooDefs.Perms}
|
||||||
|
* @param remove The permissions to be removed. Should be an OR of a
|
||||||
|
* some combination of {@link ZooDefs.Perms}
|
||||||
|
* @return A permissions flag that is an OR of {@link ZooDefs.Perms}
|
||||||
|
* present in perms and not present in remove
|
||||||
|
*/
|
||||||
|
public static int removeSpecificPerms(int perms, int remove) {
|
||||||
|
return perms ^ remove;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Parse comma separated list of ACL entries to secure generated nodes, e.g.
|
* Parse comma separated list of ACL entries to secure generated nodes, e.g.
|
||||||
* <code>sasl:hdfs/host1@MY.DOMAIN:cdrwa,sasl:hdfs/host2@MY.DOMAIN:cdrwa</code>
|
* <code>sasl:hdfs/host1@MY.DOMAIN:cdrwa,sasl:hdfs/host2@MY.DOMAIN:cdrwa</code>
|
||||||
|
@ -44,12 +44,19 @@
|
|||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
import java.util.concurrent.BrokenBarrierException;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
import java.util.concurrent.CyclicBarrier;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import javax.net.SocketFactory;
|
import javax.net.SocketFactory;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.commons.logging.impl.Log4JLogger;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.io.IntWritable;
|
import org.apache.hadoop.io.IntWritable;
|
||||||
@ -64,8 +71,10 @@
|
|||||||
import org.apache.hadoop.net.ConnectTimeoutException;
|
import org.apache.hadoop.net.ConnectTimeoutException;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.util.StringUtils;
|
import org.apache.hadoop.util.StringUtils;
|
||||||
|
import org.apache.log4j.Level;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Assume;
|
import org.junit.Assume;
|
||||||
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
import org.mockito.invocation.InvocationOnMock;
|
import org.mockito.invocation.InvocationOnMock;
|
||||||
@ -79,7 +88,7 @@ public class TestIPC {
|
|||||||
public static final Log LOG =
|
public static final Log LOG =
|
||||||
LogFactory.getLog(TestIPC.class);
|
LogFactory.getLog(TestIPC.class);
|
||||||
|
|
||||||
final private static Configuration conf = new Configuration();
|
private static Configuration conf;
|
||||||
final static private int PING_INTERVAL = 1000;
|
final static private int PING_INTERVAL = 1000;
|
||||||
final static private int MIN_SLEEP_TIME = 1000;
|
final static private int MIN_SLEEP_TIME = 1000;
|
||||||
/**
|
/**
|
||||||
@ -89,7 +98,9 @@ public class TestIPC {
|
|||||||
static boolean WRITABLE_FAULTS_ENABLED = true;
|
static boolean WRITABLE_FAULTS_ENABLED = true;
|
||||||
static int WRITABLE_FAULTS_SLEEP = 0;
|
static int WRITABLE_FAULTS_SLEEP = 0;
|
||||||
|
|
||||||
static {
|
@Before
|
||||||
|
public void setupConf() {
|
||||||
|
conf = new Configuration();
|
||||||
Client.setPingInterval(conf, PING_INTERVAL);
|
Client.setPingInterval(conf, PING_INTERVAL);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -613,6 +624,255 @@ public void testIpcWithServiceClass() throws IOException {
|
|||||||
server.stop();
|
server.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class TestServerQueue extends Server {
|
||||||
|
final CountDownLatch firstCallLatch = new CountDownLatch(1);
|
||||||
|
final CountDownLatch callBlockLatch = new CountDownLatch(1);
|
||||||
|
|
||||||
|
TestServerQueue(int expectedCalls, int readers, int callQ, int handlers,
|
||||||
|
Configuration conf) throws IOException {
|
||||||
|
super(ADDRESS, 0, LongWritable.class, handlers, readers, callQ, conf, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Writable call(RPC.RpcKind rpcKind, String protocol, Writable param,
|
||||||
|
long receiveTime) throws IOException {
|
||||||
|
firstCallLatch.countDown();
|
||||||
|
try {
|
||||||
|
callBlockLatch.await();
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
return param;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check that reader queueing works
|
||||||
|
* @throws BrokenBarrierException
|
||||||
|
* @throws InterruptedException
|
||||||
|
*/
|
||||||
|
@Test(timeout=60000)
|
||||||
|
public void testIpcWithReaderQueuing() throws Exception {
|
||||||
|
// 1 reader, 1 connectionQ slot, 1 callq
|
||||||
|
for (int i=0; i < 10; i++) {
|
||||||
|
checkBlocking(1, 1, 1);
|
||||||
|
}
|
||||||
|
// 4 readers, 5 connectionQ slots, 2 callq
|
||||||
|
for (int i=0; i < 10; i++) {
|
||||||
|
checkBlocking(4, 5, 2);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// goal is to jam a handler with a connection, fill the callq with
|
||||||
|
// connections, in turn jamming the readers - then flood the server and
|
||||||
|
// ensure that the listener blocks when the reader connection queues fill
|
||||||
|
private void checkBlocking(int readers, int readerQ, int callQ) throws Exception {
|
||||||
|
int handlers = 1; // makes it easier
|
||||||
|
|
||||||
|
final Configuration conf = new Configuration();
|
||||||
|
conf.setInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_CONNECTION_QUEUE_SIZE_KEY, readerQ);
|
||||||
|
|
||||||
|
// send in enough clients to block up the handlers, callq, and readers
|
||||||
|
int initialClients = readers + callQ + handlers;
|
||||||
|
// max connections we should ever end up accepting at once
|
||||||
|
int maxAccept = initialClients + readers*readerQ + 1; // 1 = listener
|
||||||
|
// stress it with 2X the max
|
||||||
|
int clients = maxAccept*2;
|
||||||
|
|
||||||
|
final AtomicInteger failures = new AtomicInteger(0);
|
||||||
|
final CountDownLatch callFinishedLatch = new CountDownLatch(clients);
|
||||||
|
|
||||||
|
// start server
|
||||||
|
final TestServerQueue server =
|
||||||
|
new TestServerQueue(clients, readers, callQ, handlers, conf);
|
||||||
|
final InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
||||||
|
server.start();
|
||||||
|
|
||||||
|
Client.setConnectTimeout(conf, 10000);
|
||||||
|
|
||||||
|
// instantiate the threads, will start in batches
|
||||||
|
Thread[] threads = new Thread[clients];
|
||||||
|
for (int i=0; i<clients; i++) {
|
||||||
|
threads[i] = new Thread(new Runnable() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
Client client = new Client(LongWritable.class, conf);
|
||||||
|
try {
|
||||||
|
client.call(new LongWritable(Thread.currentThread().getId()),
|
||||||
|
addr, null, null, 60000, conf);
|
||||||
|
} catch (Throwable e) {
|
||||||
|
LOG.error(e);
|
||||||
|
failures.incrementAndGet();
|
||||||
|
return;
|
||||||
|
} finally {
|
||||||
|
callFinishedLatch.countDown();
|
||||||
|
client.stop();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
// start enough clients to block up the handler, callq, and each reader;
|
||||||
|
// let the calls sequentially slot in to avoid some readers blocking
|
||||||
|
// and others not blocking in the race to fill the callq
|
||||||
|
for (int i=0; i < initialClients; i++) {
|
||||||
|
threads[i].start();
|
||||||
|
if (i==0) {
|
||||||
|
// let first reader block in a call
|
||||||
|
server.firstCallLatch.await();
|
||||||
|
} else if (i <= callQ) {
|
||||||
|
// let subsequent readers jam the callq, will happen immediately
|
||||||
|
while (server.getCallQueueLen() != i) {
|
||||||
|
Thread.sleep(1);
|
||||||
|
}
|
||||||
|
} // additional threads block the readers trying to add to the callq
|
||||||
|
}
|
||||||
|
|
||||||
|
// wait till everything is slotted, should happen immediately
|
||||||
|
Thread.sleep(10);
|
||||||
|
if (server.getNumOpenConnections() < initialClients) {
|
||||||
|
LOG.info("(initial clients) need:"+initialClients+" connections have:"+server.getNumOpenConnections());
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
LOG.info("ipc layer should be blocked");
|
||||||
|
assertEquals(callQ, server.getCallQueueLen());
|
||||||
|
assertEquals(initialClients, server.getNumOpenConnections());
|
||||||
|
|
||||||
|
// now flood the server with the rest of the connections, the reader's
|
||||||
|
// connection queues should fill and then the listener should block
|
||||||
|
for (int i=initialClients; i<clients; i++) {
|
||||||
|
threads[i].start();
|
||||||
|
}
|
||||||
|
Thread.sleep(10);
|
||||||
|
if (server.getNumOpenConnections() < maxAccept) {
|
||||||
|
LOG.info("(max clients) need:"+maxAccept+" connections have:"+server.getNumOpenConnections());
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
// check a few times to make sure we didn't go over
|
||||||
|
for (int i=0; i<4; i++) {
|
||||||
|
assertEquals(maxAccept, server.getNumOpenConnections());
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
|
// sanity check that no calls have finished
|
||||||
|
assertEquals(clients, callFinishedLatch.getCount());
|
||||||
|
LOG.info("releasing the calls");
|
||||||
|
server.callBlockLatch.countDown();
|
||||||
|
callFinishedLatch.await();
|
||||||
|
for (Thread t : threads) {
|
||||||
|
t.join();
|
||||||
|
}
|
||||||
|
assertEquals(0, failures.get());
|
||||||
|
server.stop();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(timeout=30000)
|
||||||
|
public void testConnectionIdleTimeouts() throws Exception {
|
||||||
|
((Log4JLogger)Server.LOG).getLogger().setLevel(Level.DEBUG);
|
||||||
|
final int maxIdle = 1000;
|
||||||
|
final int cleanupInterval = maxIdle*3/4; // stagger cleanups
|
||||||
|
final int killMax = 3;
|
||||||
|
final int clients = 1 + killMax*2; // 1 to block, 2 batches to kill
|
||||||
|
|
||||||
|
conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, maxIdle);
|
||||||
|
conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_IDLETHRESHOLD_KEY, 0);
|
||||||
|
conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_KILL_MAX_KEY, killMax);
|
||||||
|
conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECTION_IDLESCANINTERVAL_KEY, cleanupInterval);
|
||||||
|
|
||||||
|
final CyclicBarrier firstCallBarrier = new CyclicBarrier(2);
|
||||||
|
final CyclicBarrier callBarrier = new CyclicBarrier(clients);
|
||||||
|
final CountDownLatch allCallLatch = new CountDownLatch(clients);
|
||||||
|
final AtomicBoolean error = new AtomicBoolean();
|
||||||
|
|
||||||
|
final TestServer server = new TestServer(clients, false);
|
||||||
|
Thread[] threads = new Thread[clients];
|
||||||
|
try {
|
||||||
|
server.callListener = new Runnable(){
|
||||||
|
AtomicBoolean first = new AtomicBoolean(true);
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
allCallLatch.countDown();
|
||||||
|
// block first call
|
||||||
|
if (first.compareAndSet(true, false)) {
|
||||||
|
firstCallBarrier.await();
|
||||||
|
} else {
|
||||||
|
callBarrier.await();
|
||||||
|
}
|
||||||
|
} catch (Throwable t) {
|
||||||
|
LOG.error(t);
|
||||||
|
error.set(true);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
server.start();
|
||||||
|
|
||||||
|
// start client
|
||||||
|
final CountDownLatch callReturned = new CountDownLatch(clients-1);
|
||||||
|
final InetSocketAddress addr = NetUtils.getConnectAddress(server);
|
||||||
|
final Configuration clientConf = new Configuration();
|
||||||
|
clientConf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 10000);
|
||||||
|
for (int i=0; i < clients; i++) {
|
||||||
|
threads[i] = new Thread(new Runnable(){
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
Client client = new Client(LongWritable.class, clientConf);
|
||||||
|
try {
|
||||||
|
client.call(new LongWritable(Thread.currentThread().getId()),
|
||||||
|
addr, null, null, 0, clientConf);
|
||||||
|
callReturned.countDown();
|
||||||
|
Thread.sleep(10000);
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error(e);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
threads[i].start();
|
||||||
|
}
|
||||||
|
|
||||||
|
// all calls blocked in handler so all connections made
|
||||||
|
allCallLatch.await();
|
||||||
|
assertFalse(error.get());
|
||||||
|
assertEquals(clients, server.getNumOpenConnections());
|
||||||
|
|
||||||
|
// wake up blocked calls and wait for client call to return, no
|
||||||
|
// connections should have closed
|
||||||
|
callBarrier.await();
|
||||||
|
callReturned.await();
|
||||||
|
assertEquals(clients, server.getNumOpenConnections());
|
||||||
|
|
||||||
|
// server won't close till maxIdle*2, so give scanning thread time to
|
||||||
|
// be almost ready to close idle connection. after which it should
|
||||||
|
// close max connections on every cleanupInterval
|
||||||
|
Thread.sleep(maxIdle*2-cleanupInterval);
|
||||||
|
for (int i=clients; i > 1; i -= killMax) {
|
||||||
|
Thread.sleep(cleanupInterval);
|
||||||
|
assertFalse(error.get());
|
||||||
|
assertEquals(i, server.getNumOpenConnections());
|
||||||
|
}
|
||||||
|
|
||||||
|
// connection for the first blocked call should still be open
|
||||||
|
Thread.sleep(cleanupInterval);
|
||||||
|
assertFalse(error.get());
|
||||||
|
assertEquals(1, server.getNumOpenConnections());
|
||||||
|
|
||||||
|
// wake up call and ensure connection times out
|
||||||
|
firstCallBarrier.await();
|
||||||
|
Thread.sleep(maxIdle*2);
|
||||||
|
assertFalse(error.get());
|
||||||
|
assertEquals(0, server.getNumOpenConnections());
|
||||||
|
} finally {
|
||||||
|
for (Thread t : threads) {
|
||||||
|
if (t != null) {
|
||||||
|
t.interrupt();
|
||||||
|
t.join();
|
||||||
|
}
|
||||||
|
server.stop();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Make a call from a client and verify if header info is changed in server side
|
* Make a call from a client and verify if header info is changed in server side
|
||||||
*/
|
*/
|
||||||
@ -622,7 +882,7 @@ private void callAndVerify(Server server, InetSocketAddress addr,
|
|||||||
|
|
||||||
client.call(new LongWritable(RANDOM.nextLong()),
|
client.call(new LongWritable(RANDOM.nextLong()),
|
||||||
addr, null, null, MIN_SLEEP_TIME, serviceClass, conf);
|
addr, null, null, MIN_SLEEP_TIME, serviceClass, conf);
|
||||||
Connection connection = server.getConnections().get(0);
|
Connection connection = server.getConnections()[0];
|
||||||
int serviceClass2 = connection.getServiceClass();
|
int serviceClass2 = connection.getServiceClass();
|
||||||
assertFalse(noChanged ^ serviceClass == serviceClass2);
|
assertFalse(noChanged ^ serviceClass == serviceClass2);
|
||||||
client.stop();
|
client.stop();
|
||||||
|
@ -282,4 +282,12 @@ private void assertDOptionParsing(String[] args,
|
|||||||
Arrays.toString(remainingArgs) + Arrays.toString(expectedRemainingArgs),
|
Arrays.toString(remainingArgs) + Arrays.toString(expectedRemainingArgs),
|
||||||
expectedRemainingArgs, remainingArgs);
|
expectedRemainingArgs, remainingArgs);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Test passing null as args. Some classes still call
|
||||||
|
* Tool interface from java passing null.
|
||||||
|
*/
|
||||||
|
public void testNullArgs() throws IOException {
|
||||||
|
GenericOptionsParser parser = new GenericOptionsParser(conf, null);
|
||||||
|
parser.getRemainingArgs();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -24,7 +24,6 @@
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.util.ZKUtil;
|
|
||||||
import org.apache.hadoop.util.ZKUtil.BadAclFormatException;
|
import org.apache.hadoop.util.ZKUtil.BadAclFormatException;
|
||||||
import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
|
import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
|
||||||
import org.apache.zookeeper.ZooDefs.Perms;
|
import org.apache.zookeeper.ZooDefs.Perms;
|
||||||
@ -75,6 +74,14 @@ private static void badAcl(String acls, String expectedErr) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRemoveSpecificPerms() {
|
||||||
|
int perms = Perms.ALL;
|
||||||
|
int remove = Perms.CREATE;
|
||||||
|
int newPerms = ZKUtil.removeSpecificPerms(perms, remove);
|
||||||
|
assertEquals("Removal failed", 0, newPerms & Perms.CREATE);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGoodACLs() {
|
public void testGoodACLs() {
|
||||||
List<ACL> result = ZKUtil.parseACLs(
|
List<ACL> result = ZKUtil.parseACLs(
|
||||||
|
@ -55,7 +55,7 @@ print "Setting HTTPFS_HOME: ${HTTPFS_HOME}"
|
|||||||
#
|
#
|
||||||
if [ -e "${HTTPFS_HOME}/bin/httpfs-env.sh" ]; then
|
if [ -e "${HTTPFS_HOME}/bin/httpfs-env.sh" ]; then
|
||||||
print "Sourcing: ${HTTPFS_HOME}/bin/httpfs-env.sh"
|
print "Sourcing: ${HTTPFS_HOME}/bin/httpfs-env.sh"
|
||||||
source ${HTTPFS_HOME}/bin/HTTPFS-env.sh
|
source ${HTTPFS_HOME}/bin/httpfs-env.sh
|
||||||
grep "^ *export " ${HTTPFS_HOME}/bin/httpfs-env.sh | sed 's/ *export/ setting/'
|
grep "^ *export " ${HTTPFS_HOME}/bin/httpfs-env.sh | sed 's/ *export/ setting/'
|
||||||
fi
|
fi
|
||||||
|
|
||||||
|
@ -13,6 +13,9 @@ Trunk (Unreleased)
|
|||||||
|
|
||||||
HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
|
HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
|
||||||
|
|
||||||
|
HDFS-5444. Choose default web UI based on browser capabilities. (Haohui Mai
|
||||||
|
via jing9)
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
|
|
||||||
HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
|
HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
|
||||||
@ -194,6 +197,8 @@ Trunk (Unreleased)
|
|||||||
|
|
||||||
HDFS-5485. Add command-line support for modifyDirective. (cmccabe)
|
HDFS-5485. Add command-line support for modifyDirective. (cmccabe)
|
||||||
|
|
||||||
|
HDFS-5366. recaching improvements (cmccabe)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
|
HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
|
||||||
|
|
||||||
@ -475,6 +480,20 @@ Release 2.3.0 - UNRELEASED
|
|||||||
HDFS-5495. Remove further JUnit3 usages from HDFS.
|
HDFS-5495. Remove further JUnit3 usages from HDFS.
|
||||||
(Jarek Jarcec Cecho via wang)
|
(Jarek Jarcec Cecho via wang)
|
||||||
|
|
||||||
|
HDFS-5325. Remove WebHdfsFileSystem#ConnRunner. (Haohui Mai via jing9)
|
||||||
|
|
||||||
|
HDFS-5488. Clean up TestHftpURLTimeout. (Haohui Mai via jing9)
|
||||||
|
|
||||||
|
HDFS-5440. Extract the logic of handling delegation tokens in HftpFileSystem
|
||||||
|
to the TokenAspect class. (Haohui Mai via jing9)
|
||||||
|
|
||||||
|
HDFS-5487. Introduce unit test for TokenAspect. (Haohui Mai via jing9)
|
||||||
|
|
||||||
|
HDFS-4995. Make getContentSummary less expensive. (kihwal)
|
||||||
|
|
||||||
|
HDFS-5506. Use URLConnectionFactory in DelegationTokenFetcher. (Haohui Mai
|
||||||
|
via jing9)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
HDFS-5239. Allow FSNamesystem lock fairness to be configurable (daryn)
|
HDFS-5239. Allow FSNamesystem lock fairness to be configurable (daryn)
|
||||||
@ -529,13 +548,20 @@ Release 2.3.0 - UNRELEASED
|
|||||||
HDFS-5476. Snapshot: clean the blocks/files/directories under a renamed
|
HDFS-5476. Snapshot: clean the blocks/files/directories under a renamed
|
||||||
file/directory while deletion. (jing9)
|
file/directory while deletion. (jing9)
|
||||||
|
|
||||||
HDFS-5325. Remove WebHdfsFileSystem#ConnRunner. (Haohui Mai via jing9)
|
|
||||||
|
|
||||||
HDFS-5488. Clean up TestHftpURLTimeout. (Haohui Mai via jing9)
|
|
||||||
|
|
||||||
HDFS-5425. Renaming underconstruction file with snapshots can make NN failure on
|
HDFS-5425. Renaming underconstruction file with snapshots can make NN failure on
|
||||||
restart. (jing9 and Vinay)
|
restart. (jing9 and Vinay)
|
||||||
|
|
||||||
|
HDFS-5474. Deletesnapshot can make Namenode in safemode on NN restarts.
|
||||||
|
(Sathish via jing9)
|
||||||
|
|
||||||
|
HDFS-5075. httpfs-config.sh calls out incorrect env script name
|
||||||
|
(Timothy St. Clair via stevel)
|
||||||
|
|
||||||
|
HDFS-5504. In HA mode, OP_DELETE_SNAPSHOT is not decrementing the safemode threshold,
|
||||||
|
leads to NN safemode. (Vinay via jing9)
|
||||||
|
|
||||||
|
HDFS-5438. Flaws in block report processing can cause data loss. (kihwal)
|
||||||
|
|
||||||
Release 2.2.1 - UNRELEASED
|
Release 2.2.1 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
@ -42,4 +42,8 @@ public boolean corruptPacket() {
|
|||||||
public boolean uncorruptPacket() {
|
public boolean uncorruptPacket() {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public boolean failPacket() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -198,6 +198,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||||||
|
|
||||||
public static final String DFS_LIST_LIMIT = "dfs.ls.limit";
|
public static final String DFS_LIST_LIMIT = "dfs.ls.limit";
|
||||||
public static final int DFS_LIST_LIMIT_DEFAULT = 1000;
|
public static final int DFS_LIST_LIMIT_DEFAULT = 1000;
|
||||||
|
public static final String DFS_CONTENT_SUMMARY_LIMIT_KEY = "dfs.content-summary.limit";
|
||||||
|
public static final int DFS_CONTENT_SUMMARY_LIMIT_DEFAULT = 0;
|
||||||
public static final String DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY = "dfs.datanode.failed.volumes.tolerated";
|
public static final String DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY = "dfs.datanode.failed.volumes.tolerated";
|
||||||
public static final int DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT = 0;
|
public static final int DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT = 0;
|
||||||
public static final String DFS_DATANODE_SYNCONCLOSE_KEY = "dfs.datanode.synconclose";
|
public static final String DFS_DATANODE_SYNCONCLOSE_KEY = "dfs.datanode.synconclose";
|
||||||
@ -342,6 +344,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||||||
public static final boolean DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT = true;
|
public static final boolean DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT = true;
|
||||||
public static final String DFS_HEARTBEAT_INTERVAL_KEY = "dfs.heartbeat.interval";
|
public static final String DFS_HEARTBEAT_INTERVAL_KEY = "dfs.heartbeat.interval";
|
||||||
public static final long DFS_HEARTBEAT_INTERVAL_DEFAULT = 3;
|
public static final long DFS_HEARTBEAT_INTERVAL_DEFAULT = 3;
|
||||||
|
public static final String DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS = "dfs.namenode.path.based.cache.retry.interval.ms";
|
||||||
|
public static final long DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS_DEFAULT = 60000L;
|
||||||
public static final String DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY = "dfs.namenode.decommission.interval";
|
public static final String DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY = "dfs.namenode.decommission.interval";
|
||||||
public static final int DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT = 30;
|
public static final int DFS_NAMENODE_DECOMMISSION_INTERVAL_DEFAULT = 30;
|
||||||
public static final String DFS_NAMENODE_DECOMMISSION_NODES_PER_INTERVAL_KEY = "dfs.namenode.decommission.nodes.per.interval";
|
public static final String DFS_NAMENODE_DECOMMISSION_NODES_PER_INTERVAL_KEY = "dfs.namenode.decommission.nodes.per.interval";
|
||||||
|
@ -151,6 +151,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||||||
private final short blockReplication; // replication factor of file
|
private final short blockReplication; // replication factor of file
|
||||||
private boolean shouldSyncBlock = false; // force blocks to disk upon close
|
private boolean shouldSyncBlock = false; // force blocks to disk upon close
|
||||||
private CachingStrategy cachingStrategy;
|
private CachingStrategy cachingStrategy;
|
||||||
|
private boolean failPacket = false;
|
||||||
|
|
||||||
private static class Packet {
|
private static class Packet {
|
||||||
private static final long HEART_BEAT_SEQNO = -1L;
|
private static final long HEART_BEAT_SEQNO = -1L;
|
||||||
@ -752,6 +753,16 @@ public void run() {
|
|||||||
one.seqno + " but received " + seqno);
|
one.seqno + " but received " + seqno);
|
||||||
}
|
}
|
||||||
isLastPacketInBlock = one.lastPacketInBlock;
|
isLastPacketInBlock = one.lastPacketInBlock;
|
||||||
|
|
||||||
|
// Fail the packet write for testing in order to force a
|
||||||
|
// pipeline recovery.
|
||||||
|
if (DFSClientFaultInjector.get().failPacket() &&
|
||||||
|
isLastPacketInBlock) {
|
||||||
|
failPacket = true;
|
||||||
|
throw new IOException(
|
||||||
|
"Failing the last packet for testing.");
|
||||||
|
}
|
||||||
|
|
||||||
// update bytesAcked
|
// update bytesAcked
|
||||||
block.setNumBytes(one.getLastByteOffsetBlock());
|
block.setNumBytes(one.getLastByteOffsetBlock());
|
||||||
|
|
||||||
@ -1044,8 +1055,19 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException {
|
|||||||
accessToken = lb.getBlockToken();
|
accessToken = lb.getBlockToken();
|
||||||
|
|
||||||
// set up the pipeline again with the remaining nodes
|
// set up the pipeline again with the remaining nodes
|
||||||
|
if (failPacket) { // for testing
|
||||||
|
success = createBlockOutputStream(nodes, newGS-1, isRecovery);
|
||||||
|
failPacket = false;
|
||||||
|
try {
|
||||||
|
// Give DNs time to send in bad reports. In real situations,
|
||||||
|
// good reports should follow bad ones, if client committed
|
||||||
|
// with those nodes.
|
||||||
|
Thread.sleep(2000);
|
||||||
|
} catch (InterruptedException ie) {}
|
||||||
|
} else {
|
||||||
success = createBlockOutputStream(nodes, newGS, isRecovery);
|
success = createBlockOutputStream(nodes, newGS, isRecovery);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
if (success) {
|
if (success) {
|
||||||
// update pipeline at the namenode
|
// update pipeline at the namenode
|
||||||
@ -1904,7 +1926,9 @@ public synchronized void close() throws IOException {
|
|||||||
// be called during unit tests
|
// be called during unit tests
|
||||||
private void completeFile(ExtendedBlock last) throws IOException {
|
private void completeFile(ExtendedBlock last) throws IOException {
|
||||||
long localstart = Time.now();
|
long localstart = Time.now();
|
||||||
|
long localTimeout = 400;
|
||||||
boolean fileComplete = false;
|
boolean fileComplete = false;
|
||||||
|
int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
|
||||||
while (!fileComplete) {
|
while (!fileComplete) {
|
||||||
fileComplete =
|
fileComplete =
|
||||||
dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
|
dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
|
||||||
@ -1920,7 +1944,13 @@ private void completeFile(ExtendedBlock last) throws IOException {
|
|||||||
throw new IOException(msg);
|
throw new IOException(msg);
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
Thread.sleep(400);
|
Thread.sleep(localTimeout);
|
||||||
|
if (retries == 0) {
|
||||||
|
throw new IOException("Unable to close file because the last block"
|
||||||
|
+ " does not have enough number of replicas.");
|
||||||
|
}
|
||||||
|
retries--;
|
||||||
|
localTimeout *= 2;
|
||||||
if (Time.now() - localstart > 5000) {
|
if (Time.now() - localstart > 5000) {
|
||||||
DFSClient.LOG.info("Could not complete " + src + " retrying...");
|
DFSClient.LOG.info("Could not complete " + src + " retrying...");
|
||||||
}
|
}
|
||||||
|
@ -229,6 +229,29 @@ public long getBlockRecoveryId() {
|
|||||||
return blockRecoveryId;
|
return blockRecoveryId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Process the recorded replicas. When about to commit or finish the
|
||||||
|
* pipeline recovery sort out bad replicas.
|
||||||
|
* @param genStamp The final generation stamp for the block.
|
||||||
|
*/
|
||||||
|
public void setGenerationStampAndVerifyReplicas(long genStamp) {
|
||||||
|
if (replicas == null)
|
||||||
|
return;
|
||||||
|
|
||||||
|
// Remove the replicas with wrong gen stamp.
|
||||||
|
// The replica list is unchanged.
|
||||||
|
for (ReplicaUnderConstruction r : replicas) {
|
||||||
|
if (genStamp != r.getGenerationStamp()) {
|
||||||
|
r.getExpectedStorageLocation().removeBlock(this);
|
||||||
|
NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
|
||||||
|
+ "from location: " + r);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Set the generation stamp for the block.
|
||||||
|
setGenerationStamp(genStamp);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commit block's length and generation stamp as reported by the client.
|
* Commit block's length and generation stamp as reported by the client.
|
||||||
* Set block state to {@link BlockUCState#COMMITTED}.
|
* Set block state to {@link BlockUCState#COMMITTED}.
|
||||||
@ -301,6 +324,8 @@ void addReplicaIfNotPresent(DatanodeStorageInfo storage,
|
|||||||
while (it.hasNext()) {
|
while (it.hasNext()) {
|
||||||
ReplicaUnderConstruction r = it.next();
|
ReplicaUnderConstruction r = it.next();
|
||||||
if(r.getExpectedStorageLocation() == storage) {
|
if(r.getExpectedStorageLocation() == storage) {
|
||||||
|
// Record the gen stamp from the report
|
||||||
|
r.setGenerationStamp(block.getGenerationStamp());
|
||||||
return;
|
return;
|
||||||
} else if (r.getExpectedStorageLocation().getDatanodeDescriptor() ==
|
} else if (r.getExpectedStorageLocation().getDatanodeDescriptor() ==
|
||||||
storage.getDatanodeDescriptor()) {
|
storage.getDatanodeDescriptor()) {
|
||||||
|
@ -59,6 +59,7 @@
|
|||||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
|
||||||
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
|
||||||
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
|
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
|
||||||
|
import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
|
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
|
||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
||||||
@ -1051,7 +1052,8 @@ public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
|
|||||||
+ blk + " not found");
|
+ blk + " not found");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn, storageID);
|
markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason,
|
||||||
|
Reason.CORRUPTION_REPORTED), dn, storageID);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void markBlockAsCorrupt(BlockToMarkCorrupt b,
|
private void markBlockAsCorrupt(BlockToMarkCorrupt b,
|
||||||
@ -1074,7 +1076,8 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b,
|
|||||||
node.addBlock(storageID, b.stored);
|
node.addBlock(storageID, b.stored);
|
||||||
|
|
||||||
// Add this replica to corruptReplicas Map
|
// Add this replica to corruptReplicas Map
|
||||||
corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason);
|
corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
|
||||||
|
b.reasonCode);
|
||||||
if (countNodes(b.stored).liveReplicas() >= bc.getBlockReplication()) {
|
if (countNodes(b.stored).liveReplicas() >= bc.getBlockReplication()) {
|
||||||
// the block is over-replicated so invalidate the replicas immediately
|
// the block is over-replicated so invalidate the replicas immediately
|
||||||
invalidateBlock(b, node);
|
invalidateBlock(b, node);
|
||||||
@ -1574,22 +1577,27 @@ private static class BlockToMarkCorrupt {
|
|||||||
final BlockInfo stored;
|
final BlockInfo stored;
|
||||||
/** The reason to mark corrupt. */
|
/** The reason to mark corrupt. */
|
||||||
final String reason;
|
final String reason;
|
||||||
|
/** The reason code to be stored */
|
||||||
|
final Reason reasonCode;
|
||||||
|
|
||||||
BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason) {
|
BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason,
|
||||||
|
Reason reasonCode) {
|
||||||
Preconditions.checkNotNull(corrupted, "corrupted is null");
|
Preconditions.checkNotNull(corrupted, "corrupted is null");
|
||||||
Preconditions.checkNotNull(stored, "stored is null");
|
Preconditions.checkNotNull(stored, "stored is null");
|
||||||
|
|
||||||
this.corrupted = corrupted;
|
this.corrupted = corrupted;
|
||||||
this.stored = stored;
|
this.stored = stored;
|
||||||
this.reason = reason;
|
this.reason = reason;
|
||||||
|
this.reasonCode = reasonCode;
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockToMarkCorrupt(BlockInfo stored, String reason) {
|
BlockToMarkCorrupt(BlockInfo stored, String reason, Reason reasonCode) {
|
||||||
this(stored, stored, reason);
|
this(stored, stored, reason, reasonCode);
|
||||||
}
|
}
|
||||||
|
|
||||||
BlockToMarkCorrupt(BlockInfo stored, long gs, String reason) {
|
BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
|
||||||
this(new BlockInfo(stored), stored, reason);
|
Reason reasonCode) {
|
||||||
|
this(new BlockInfo(stored), stored, reason, reasonCode);
|
||||||
//the corrupted block in datanode has a different generation stamp
|
//the corrupted block in datanode has a different generation stamp
|
||||||
corrupted.setGenerationStamp(gs);
|
corrupted.setGenerationStamp(gs);
|
||||||
}
|
}
|
||||||
@ -1946,9 +1954,11 @@ private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
|
|||||||
return storedBlock;
|
return storedBlock;
|
||||||
}
|
}
|
||||||
|
|
||||||
//add replica if appropriate
|
// Add replica if appropriate. If the replica was previously corrupt
|
||||||
|
// but now okay, it might need to be updated.
|
||||||
if (reportedState == ReplicaState.FINALIZED
|
if (reportedState == ReplicaState.FINALIZED
|
||||||
&& storedBlock.findDatanode(dn) < 0) {
|
&& (storedBlock.findDatanode(dn) < 0
|
||||||
|
|| corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
|
||||||
toAdd.add(storedBlock);
|
toAdd.add(storedBlock);
|
||||||
}
|
}
|
||||||
return storedBlock;
|
return storedBlock;
|
||||||
@ -2039,12 +2049,13 @@ private BlockToMarkCorrupt checkReplicaCorrupt(
|
|||||||
return new BlockToMarkCorrupt(storedBlock, reportedGS,
|
return new BlockToMarkCorrupt(storedBlock, reportedGS,
|
||||||
"block is " + ucState + " and reported genstamp " + reportedGS
|
"block is " + ucState + " and reported genstamp " + reportedGS
|
||||||
+ " does not match genstamp in block map "
|
+ " does not match genstamp in block map "
|
||||||
+ storedBlock.getGenerationStamp());
|
+ storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
|
||||||
} else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
|
} else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
|
||||||
return new BlockToMarkCorrupt(storedBlock,
|
return new BlockToMarkCorrupt(storedBlock,
|
||||||
"block is " + ucState + " and reported length " +
|
"block is " + ucState + " and reported length " +
|
||||||
reported.getNumBytes() + " does not match " +
|
reported.getNumBytes() + " does not match " +
|
||||||
"length in block map " + storedBlock.getNumBytes());
|
"length in block map " + storedBlock.getNumBytes(),
|
||||||
|
Reason.SIZE_MISMATCH);
|
||||||
} else {
|
} else {
|
||||||
return null; // not corrupt
|
return null; // not corrupt
|
||||||
}
|
}
|
||||||
@ -2060,7 +2071,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt(
|
|||||||
return new BlockToMarkCorrupt(storedBlock, reportedGS,
|
return new BlockToMarkCorrupt(storedBlock, reportedGS,
|
||||||
"reported " + reportedState + " replica with genstamp " + reportedGS
|
"reported " + reportedState + " replica with genstamp " + reportedGS
|
||||||
+ " does not match COMPLETE block's genstamp in block map "
|
+ " does not match COMPLETE block's genstamp in block map "
|
||||||
+ storedBlock.getGenerationStamp());
|
+ storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
|
||||||
} else { // COMPLETE block, same genstamp
|
} else { // COMPLETE block, same genstamp
|
||||||
if (reportedState == ReplicaState.RBW) {
|
if (reportedState == ReplicaState.RBW) {
|
||||||
// If it's a RBW report for a COMPLETE block, it may just be that
|
// If it's a RBW report for a COMPLETE block, it may just be that
|
||||||
@ -2073,7 +2084,8 @@ private BlockToMarkCorrupt checkReplicaCorrupt(
|
|||||||
return null;
|
return null;
|
||||||
} else {
|
} else {
|
||||||
return new BlockToMarkCorrupt(storedBlock,
|
return new BlockToMarkCorrupt(storedBlock,
|
||||||
"reported replica has invalid state " + reportedState);
|
"reported replica has invalid state " + reportedState,
|
||||||
|
Reason.INVALID_STATE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
case RUR: // should not be reported
|
case RUR: // should not be reported
|
||||||
@ -2084,7 +2096,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt(
|
|||||||
" on " + dn + " size " + storedBlock.getNumBytes();
|
" on " + dn + " size " + storedBlock.getNumBytes();
|
||||||
// log here at WARN level since this is really a broken HDFS invariant
|
// log here at WARN level since this is really a broken HDFS invariant
|
||||||
LOG.warn(msg);
|
LOG.warn(msg);
|
||||||
return new BlockToMarkCorrupt(storedBlock, msg);
|
return new BlockToMarkCorrupt(storedBlock, msg, Reason.INVALID_STATE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -2201,6 +2213,11 @@ private Block addStoredBlock(final BlockInfo block,
|
|||||||
logAddStoredBlock(storedBlock, node);
|
logAddStoredBlock(storedBlock, node);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
// if the same block is added again and the replica was corrupt
|
||||||
|
// previously because of a wrong gen stamp, remove it from the
|
||||||
|
// corrupt block list.
|
||||||
|
corruptReplicas.removeFromCorruptReplicasMap(block, node,
|
||||||
|
Reason.GENSTAMP_MISMATCH);
|
||||||
curReplicaDelta = 0;
|
curReplicaDelta = 0;
|
||||||
blockLog.warn("BLOCK* addStoredBlock: "
|
blockLog.warn("BLOCK* addStoredBlock: "
|
||||||
+ "Redundant addStoredBlock request received for " + storedBlock
|
+ "Redundant addStoredBlock request received for " + storedBlock
|
||||||
@ -2297,7 +2314,8 @@ private void invalidateCorruptReplicas(BlockInfo blk) {
|
|||||||
DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
|
DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
|
||||||
for (DatanodeDescriptor node : nodesCopy) {
|
for (DatanodeDescriptor node : nodesCopy) {
|
||||||
try {
|
try {
|
||||||
if (!invalidateBlock(new BlockToMarkCorrupt(blk, null), node)) {
|
if (!invalidateBlock(new BlockToMarkCorrupt(blk, null,
|
||||||
|
Reason.ANY), node)) {
|
||||||
removedFromBlocksMap = false;
|
removedFromBlocksMap = false;
|
||||||
}
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
|
@ -204,6 +204,7 @@ private void rescan() {
|
|||||||
namesystem.writeLock();
|
namesystem.writeLock();
|
||||||
try {
|
try {
|
||||||
rescanCachedBlockMap();
|
rescanCachedBlockMap();
|
||||||
|
blockManager.getDatanodeManager().resetLastCachingDirectiveSentTime();
|
||||||
} finally {
|
} finally {
|
||||||
namesystem.writeUnlock();
|
namesystem.writeUnlock();
|
||||||
}
|
}
|
||||||
@ -316,17 +317,21 @@ private void rescanCachedBlockMap() {
|
|||||||
int numCached = cached.size();
|
int numCached = cached.size();
|
||||||
if (numCached >= neededCached) {
|
if (numCached >= neededCached) {
|
||||||
// If we have enough replicas, drop all pending cached.
|
// If we have enough replicas, drop all pending cached.
|
||||||
for (DatanodeDescriptor datanode : pendingCached) {
|
for (Iterator<DatanodeDescriptor> iter = pendingCached.iterator();
|
||||||
|
iter.hasNext(); ) {
|
||||||
|
DatanodeDescriptor datanode = iter.next();
|
||||||
datanode.getPendingCached().remove(cblock);
|
datanode.getPendingCached().remove(cblock);
|
||||||
|
iter.remove();
|
||||||
}
|
}
|
||||||
pendingCached.clear();
|
|
||||||
}
|
}
|
||||||
if (numCached < neededCached) {
|
if (numCached < neededCached) {
|
||||||
// If we don't have enough replicas, drop all pending uncached.
|
// If we don't have enough replicas, drop all pending uncached.
|
||||||
for (DatanodeDescriptor datanode : pendingUncached) {
|
for (Iterator<DatanodeDescriptor> iter = pendingUncached.iterator();
|
||||||
|
iter.hasNext(); ) {
|
||||||
|
DatanodeDescriptor datanode = iter.next();
|
||||||
datanode.getPendingUncached().remove(cblock);
|
datanode.getPendingUncached().remove(cblock);
|
||||||
|
iter.remove();
|
||||||
}
|
}
|
||||||
pendingUncached.clear();
|
|
||||||
}
|
}
|
||||||
int neededUncached = numCached -
|
int neededUncached = numCached -
|
||||||
(pendingUncached.size() + neededCached);
|
(pendingUncached.size() + neededCached);
|
||||||
|
@ -36,8 +36,18 @@
|
|||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class CorruptReplicasMap{
|
public class CorruptReplicasMap{
|
||||||
|
|
||||||
private SortedMap<Block, Collection<DatanodeDescriptor>> corruptReplicasMap =
|
/** The corruption reason code */
|
||||||
new TreeMap<Block, Collection<DatanodeDescriptor>>();
|
public static enum Reason {
|
||||||
|
NONE, // not specified.
|
||||||
|
ANY, // wildcard reason
|
||||||
|
GENSTAMP_MISMATCH, // mismatch in generation stamps
|
||||||
|
SIZE_MISMATCH, // mismatch in sizes
|
||||||
|
INVALID_STATE, // invalid state
|
||||||
|
CORRUPTION_REPORTED // client or datanode reported the corruption
|
||||||
|
}
|
||||||
|
|
||||||
|
private SortedMap<Block, Map<DatanodeDescriptor, Reason>> corruptReplicasMap =
|
||||||
|
new TreeMap<Block, Map<DatanodeDescriptor, Reason>>();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mark the block belonging to datanode as corrupt.
|
* Mark the block belonging to datanode as corrupt.
|
||||||
@ -48,9 +58,22 @@ public class CorruptReplicasMap{
|
|||||||
*/
|
*/
|
||||||
public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
|
public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
|
||||||
String reason) {
|
String reason) {
|
||||||
Collection<DatanodeDescriptor> nodes = getNodes(blk);
|
addToCorruptReplicasMap(blk, dn, reason, Reason.NONE);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mark the block belonging to datanode as corrupt.
|
||||||
|
*
|
||||||
|
* @param blk Block to be added to CorruptReplicasMap
|
||||||
|
* @param dn DatanodeDescriptor which holds the corrupt replica
|
||||||
|
* @param reason a textual reason (for logging purposes)
|
||||||
|
* @param reasonCode the enum representation of the reason
|
||||||
|
*/
|
||||||
|
public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
|
||||||
|
String reason, Reason reasonCode) {
|
||||||
|
Map <DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
|
||||||
if (nodes == null) {
|
if (nodes == null) {
|
||||||
nodes = new TreeSet<DatanodeDescriptor>();
|
nodes = new HashMap<DatanodeDescriptor, Reason>();
|
||||||
corruptReplicasMap.put(blk, nodes);
|
corruptReplicasMap.put(blk, nodes);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -61,8 +84,7 @@ public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
|
|||||||
reasonText = "";
|
reasonText = "";
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!nodes.contains(dn)) {
|
if (!nodes.keySet().contains(dn)) {
|
||||||
nodes.add(dn);
|
|
||||||
NameNode.blockStateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
|
NameNode.blockStateChangeLog.info("BLOCK NameSystem.addToCorruptReplicasMap: "+
|
||||||
blk.getBlockName() +
|
blk.getBlockName() +
|
||||||
" added as corrupt on " + dn +
|
" added as corrupt on " + dn +
|
||||||
@ -76,6 +98,8 @@ public void addToCorruptReplicasMap(Block blk, DatanodeDescriptor dn,
|
|||||||
" by " + Server.getRemoteIp() +
|
" by " + Server.getRemoteIp() +
|
||||||
reasonText);
|
reasonText);
|
||||||
}
|
}
|
||||||
|
// Add the node or update the reason.
|
||||||
|
nodes.put(dn, reasonCode);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -97,10 +121,24 @@ void removeFromCorruptReplicasMap(Block blk) {
|
|||||||
false if the replica is not in the map
|
false if the replica is not in the map
|
||||||
*/
|
*/
|
||||||
boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode) {
|
boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode) {
|
||||||
Collection<DatanodeDescriptor> datanodes = corruptReplicasMap.get(blk);
|
return removeFromCorruptReplicasMap(blk, datanode, Reason.ANY);
|
||||||
|
}
|
||||||
|
|
||||||
|
boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode,
|
||||||
|
Reason reason) {
|
||||||
|
Map <DatanodeDescriptor, Reason> datanodes = corruptReplicasMap.get(blk);
|
||||||
|
boolean removed = false;
|
||||||
if (datanodes==null)
|
if (datanodes==null)
|
||||||
return false;
|
return false;
|
||||||
if (datanodes.remove(datanode)) { // remove the replicas
|
|
||||||
|
// if reasons can be compared but don't match, return false.
|
||||||
|
Reason storedReason = datanodes.get(datanode);
|
||||||
|
if (reason != Reason.ANY && storedReason != null &&
|
||||||
|
reason != storedReason) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (datanodes.remove(datanode) != null) { // remove the replicas
|
||||||
if (datanodes.isEmpty()) {
|
if (datanodes.isEmpty()) {
|
||||||
// remove the block if there is no more corrupted replicas
|
// remove the block if there is no more corrupted replicas
|
||||||
corruptReplicasMap.remove(blk);
|
corruptReplicasMap.remove(blk);
|
||||||
@ -118,7 +156,10 @@ boolean removeFromCorruptReplicasMap(Block blk, DatanodeDescriptor datanode) {
|
|||||||
* @return collection of nodes. Null if does not exists
|
* @return collection of nodes. Null if does not exists
|
||||||
*/
|
*/
|
||||||
Collection<DatanodeDescriptor> getNodes(Block blk) {
|
Collection<DatanodeDescriptor> getNodes(Block blk) {
|
||||||
return corruptReplicasMap.get(blk);
|
Map <DatanodeDescriptor, Reason> nodes = corruptReplicasMap.get(blk);
|
||||||
|
if (nodes == null)
|
||||||
|
return null;
|
||||||
|
return nodes.keySet();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -170,6 +170,21 @@ public CachedBlocksList getPendingUncached() {
|
|||||||
return pendingUncached;
|
return pendingUncached;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The time when the last batch of caching directives was sent, in
|
||||||
|
* monotonic milliseconds.
|
||||||
|
*/
|
||||||
|
private long lastCachingDirectiveSentTimeMs;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Head of the list of blocks on the datanode
|
||||||
|
*/
|
||||||
|
private volatile BlockInfo blockList = null;
|
||||||
|
/**
|
||||||
|
* Number of blocks on the datanode
|
||||||
|
*/
|
||||||
|
private int numBlocks = 0;
|
||||||
|
|
||||||
// isAlive == heartbeats.contains(this)
|
// isAlive == heartbeats.contains(this)
|
||||||
// This is an optimization, because contains takes O(n) time on Arraylist
|
// This is an optimization, because contains takes O(n) time on Arraylist
|
||||||
public boolean isAlive = false;
|
public boolean isAlive = false;
|
||||||
@ -661,5 +676,21 @@ public DatanodeStorageInfo updateStorage(DatanodeStorage s) {
|
|||||||
return storage;
|
return storage;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The time at which we last sent caching directives to this
|
||||||
|
* DataNode, in monotonic milliseconds.
|
||||||
|
*/
|
||||||
|
public long getLastCachingDirectiveSentTimeMs() {
|
||||||
|
return this.lastCachingDirectiveSentTimeMs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param time The time at which we last sent caching directives to this
|
||||||
|
* DataNode, in monotonic milliseconds.
|
||||||
|
*/
|
||||||
|
public void setLastCachingDirectiveSentTimeMs(long time) {
|
||||||
|
this.lastCachingDirectiveSentTimeMs = time;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -149,7 +149,7 @@ public class DatanodeManager {
|
|||||||
* Whether we should tell datanodes what to cache in replies to
|
* Whether we should tell datanodes what to cache in replies to
|
||||||
* heartbeat messages.
|
* heartbeat messages.
|
||||||
*/
|
*/
|
||||||
private boolean sendCachingCommands = false;
|
private boolean shouldSendCachingCommands = false;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The number of datanodes for each software version. This list should change
|
* The number of datanodes for each software version. This list should change
|
||||||
@ -159,6 +159,16 @@ public class DatanodeManager {
|
|||||||
private HashMap<String, Integer> datanodesSoftwareVersions =
|
private HashMap<String, Integer> datanodesSoftwareVersions =
|
||||||
new HashMap<String, Integer>(4, 0.75f);
|
new HashMap<String, Integer>(4, 0.75f);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The minimum time between resending caching directives to Datanodes,
|
||||||
|
* in milliseconds.
|
||||||
|
*
|
||||||
|
* Note that when a rescan happens, we will send the new directives
|
||||||
|
* as soon as possible. This timeout only applies to resending
|
||||||
|
* directives that we've already sent.
|
||||||
|
*/
|
||||||
|
private final long timeBetweenResendingCachingDirectivesMs;
|
||||||
|
|
||||||
DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
|
DatanodeManager(final BlockManager blockManager, final Namesystem namesystem,
|
||||||
final Configuration conf) throws IOException {
|
final Configuration conf) throws IOException {
|
||||||
this.namesystem = namesystem;
|
this.namesystem = namesystem;
|
||||||
@ -241,6 +251,9 @@ public class DatanodeManager {
|
|||||||
DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY +
|
DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY +
|
||||||
" = '" + ratioUseStaleDataNodesForWrite + "' is invalid. " +
|
" = '" + ratioUseStaleDataNodesForWrite + "' is invalid. " +
|
||||||
"It should be a positive non-zero float value, not greater than 1.0f.");
|
"It should be a positive non-zero float value, not greater than 1.0f.");
|
||||||
|
this.timeBetweenResendingCachingDirectivesMs = conf.getLong(
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS,
|
||||||
|
DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS_DEFAULT);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static long getStaleIntervalFromConf(Configuration conf,
|
private static long getStaleIntervalFromConf(Configuration conf,
|
||||||
@ -1297,17 +1310,28 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
|
|||||||
cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
|
cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
|
||||||
blockPoolId, blks));
|
blockPoolId, blks));
|
||||||
}
|
}
|
||||||
|
boolean sendingCachingCommands = false;
|
||||||
|
long nowMs = Time.monotonicNow();
|
||||||
|
if (shouldSendCachingCommands &&
|
||||||
|
((nowMs - nodeinfo.getLastCachingDirectiveSentTimeMs()) >=
|
||||||
|
timeBetweenResendingCachingDirectivesMs)) {
|
||||||
DatanodeCommand pendingCacheCommand =
|
DatanodeCommand pendingCacheCommand =
|
||||||
getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
|
getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
|
||||||
DatanodeProtocol.DNA_CACHE, blockPoolId);
|
DatanodeProtocol.DNA_CACHE, blockPoolId);
|
||||||
if (pendingCacheCommand != null) {
|
if (pendingCacheCommand != null) {
|
||||||
cmds.add(pendingCacheCommand);
|
cmds.add(pendingCacheCommand);
|
||||||
|
sendingCachingCommands = true;
|
||||||
}
|
}
|
||||||
DatanodeCommand pendingUncacheCommand =
|
DatanodeCommand pendingUncacheCommand =
|
||||||
getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
|
getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
|
||||||
DatanodeProtocol.DNA_UNCACHE, blockPoolId);
|
DatanodeProtocol.DNA_UNCACHE, blockPoolId);
|
||||||
if (pendingUncacheCommand != null) {
|
if (pendingUncacheCommand != null) {
|
||||||
cmds.add(pendingUncacheCommand);
|
cmds.add(pendingUncacheCommand);
|
||||||
|
sendingCachingCommands = true;
|
||||||
|
}
|
||||||
|
if (sendingCachingCommands) {
|
||||||
|
nodeinfo.setLastCachingDirectiveSentTimeMs(nowMs);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
blockManager.addKeyUpdateCommand(cmds, nodeinfo);
|
blockManager.addKeyUpdateCommand(cmds, nodeinfo);
|
||||||
@ -1345,19 +1369,13 @@ private DatanodeCommand getCacheCommand(CachedBlocksList list,
|
|||||||
if (length == 0) {
|
if (length == 0) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
// Read and clear the existing cache commands.
|
// Read the existing cache commands.
|
||||||
long[] blockIds = new long[length];
|
long[] blockIds = new long[length];
|
||||||
int i = 0;
|
int i = 0;
|
||||||
for (Iterator<CachedBlock> iter = list.iterator();
|
for (Iterator<CachedBlock> iter = list.iterator();
|
||||||
iter.hasNext(); ) {
|
iter.hasNext(); ) {
|
||||||
CachedBlock cachedBlock = iter.next();
|
CachedBlock cachedBlock = iter.next();
|
||||||
blockIds[i++] = cachedBlock.getBlockId();
|
blockIds[i++] = cachedBlock.getBlockId();
|
||||||
iter.remove();
|
|
||||||
}
|
|
||||||
if (!sendCachingCommands) {
|
|
||||||
// Do not send caching commands unless the FSNamesystem told us we
|
|
||||||
// should.
|
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
return new BlockIdCommand(action, poolId, blockIds);
|
return new BlockIdCommand(action, poolId, blockIds);
|
||||||
}
|
}
|
||||||
@ -1408,13 +1426,25 @@ public void clearPendingQueues() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reset the lastCachingDirectiveSentTimeMs field of all the DataNodes we
|
||||||
|
* know about.
|
||||||
|
*/
|
||||||
|
public void resetLastCachingDirectiveSentTime() {
|
||||||
|
synchronized (datanodeMap) {
|
||||||
|
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
||||||
|
dn.setLastCachingDirectiveSentTimeMs(0L);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return getClass().getSimpleName() + ": " + host2DatanodeMap;
|
return getClass().getSimpleName() + ": " + host2DatanodeMap;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setSendCachingCommands(boolean sendCachingCommands) {
|
public void setShouldSendCachingCommands(boolean shouldSendCachingCommands) {
|
||||||
this.sendCachingCommands = sendCachingCommands;
|
this.shouldSendCachingCommands = shouldSendCachingCommands;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -289,6 +289,10 @@ synchronized void cacheBlock(long blockId, String bpid,
|
|||||||
mappableBlockMap.put(key, new Value(null, State.CACHING));
|
mappableBlockMap.put(key, new Value(null, State.CACHING));
|
||||||
volumeExecutor.execute(
|
volumeExecutor.execute(
|
||||||
new CachingTask(key, blockFileName, length, genstamp));
|
new CachingTask(key, blockFileName, length, genstamp));
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Initiating caching for Block with id " + blockId +
|
||||||
|
", pool " + bpid);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized void uncacheBlock(String bpid, long blockId) {
|
synchronized void uncacheBlock(String bpid, long blockId) {
|
||||||
@ -427,6 +431,10 @@ public void run() {
|
|||||||
mappableBlock.close();
|
mappableBlock.close();
|
||||||
}
|
}
|
||||||
numBlocksFailedToCache.incrementAndGet();
|
numBlocksFailedToCache.incrementAndGet();
|
||||||
|
|
||||||
|
synchronized (FsDatasetCache.this) {
|
||||||
|
mappableBlockMap.remove(key);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -44,20 +44,6 @@
|
|||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Unstable
|
@InterfaceStability.Unstable
|
||||||
public class MappableBlock implements Closeable {
|
public class MappableBlock implements Closeable {
|
||||||
public static interface Mlocker {
|
|
||||||
void mlock(MappedByteBuffer mmap, long length) throws IOException;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class PosixMlocker implements Mlocker {
|
|
||||||
public void mlock(MappedByteBuffer mmap, long length)
|
|
||||||
throws IOException {
|
|
||||||
NativeIO.POSIX.mlock(mmap, length);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
public static Mlocker mlocker = new PosixMlocker();
|
|
||||||
|
|
||||||
private MappedByteBuffer mmap;
|
private MappedByteBuffer mmap;
|
||||||
private final long length;
|
private final long length;
|
||||||
|
|
||||||
@ -96,7 +82,7 @@ public static MappableBlock load(long length,
|
|||||||
throw new IOException("Block InputStream has no FileChannel.");
|
throw new IOException("Block InputStream has no FileChannel.");
|
||||||
}
|
}
|
||||||
mmap = blockChannel.map(MapMode.READ_ONLY, 0, length);
|
mmap = blockChannel.map(MapMode.READ_ONLY, 0, length);
|
||||||
mlocker.mlock(mmap, length);
|
NativeIO.POSIX.cacheManipulator.mlock(blockFileName, mmap, length);
|
||||||
verifyChecksum(length, metaIn, blockChannel, blockFileName);
|
verifyChecksum(length, metaIn, blockChannel, blockFileName);
|
||||||
mappableBlock = new MappableBlock(mmap, length);
|
mappableBlock = new MappableBlock(mmap, length);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -0,0 +1,119 @@
|
|||||||
|
/**
|
||||||
|
* 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 org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Unstable
|
||||||
|
public class ContentSummaryComputationContext {
|
||||||
|
private FSDirectory dir = null;
|
||||||
|
private FSNamesystem fsn = null;
|
||||||
|
private Content.Counts counts = null;
|
||||||
|
private long nextCountLimit = 0;
|
||||||
|
private long limitPerRun = 0;
|
||||||
|
private long yieldCount = 0;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructor
|
||||||
|
*
|
||||||
|
* @param dir The FSDirectory instance
|
||||||
|
* @param fsn The FSNamesystem instance
|
||||||
|
* @param limitPerRun allowed number of operations in one
|
||||||
|
* locking period. 0 or a negative number means
|
||||||
|
* no limit (i.e. no yielding)
|
||||||
|
*/
|
||||||
|
public ContentSummaryComputationContext(FSDirectory dir,
|
||||||
|
FSNamesystem fsn, long limitPerRun) {
|
||||||
|
this.dir = dir;
|
||||||
|
this.fsn = fsn;
|
||||||
|
this.limitPerRun = limitPerRun;
|
||||||
|
this.nextCountLimit = limitPerRun;
|
||||||
|
this.counts = Content.Counts.newInstance();
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Constructor for blocking computation. */
|
||||||
|
public ContentSummaryComputationContext() {
|
||||||
|
this(null, null, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Return current yield count */
|
||||||
|
public long getYieldCount() {
|
||||||
|
return yieldCount;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Relinquish locks held during computation for a short while
|
||||||
|
* and reacquire them. This will give other threads a chance
|
||||||
|
* to acquire the contended locks and run.
|
||||||
|
*
|
||||||
|
* @return true if locks were released and reacquired.
|
||||||
|
*/
|
||||||
|
public boolean yield() {
|
||||||
|
// Are we set up to do this?
|
||||||
|
if (limitPerRun <= 0 || dir == null || fsn == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Have we reached the limit?
|
||||||
|
long currentCount = counts.get(Content.FILE) +
|
||||||
|
counts.get(Content.SYMLINK) +
|
||||||
|
counts.get(Content.DIRECTORY) +
|
||||||
|
counts.get(Content.SNAPSHOTTABLE_DIRECTORY);
|
||||||
|
if (currentCount <= nextCountLimit) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Update the next limit
|
||||||
|
nextCountLimit = currentCount + limitPerRun;
|
||||||
|
|
||||||
|
boolean hadDirReadLock = dir.hasReadLock();
|
||||||
|
boolean hadDirWriteLock = dir.hasWriteLock();
|
||||||
|
boolean hadFsnReadLock = fsn.hasReadLock();
|
||||||
|
boolean hadFsnWriteLock = fsn.hasWriteLock();
|
||||||
|
|
||||||
|
// sanity check.
|
||||||
|
if (!hadDirReadLock || !hadFsnReadLock || hadDirWriteLock ||
|
||||||
|
hadFsnWriteLock || dir.getReadHoldCount() != 1 ||
|
||||||
|
fsn.getReadHoldCount() != 1) {
|
||||||
|
// cannot relinquish
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// unlock
|
||||||
|
dir.readUnlock();
|
||||||
|
fsn.readUnlock();
|
||||||
|
|
||||||
|
try {
|
||||||
|
Thread.sleep(1);
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
} finally {
|
||||||
|
// reacquire
|
||||||
|
fsn.readLock();
|
||||||
|
dir.readLock();
|
||||||
|
}
|
||||||
|
yieldCount++;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Get the content counts */
|
||||||
|
public Content.Counts getCounts() {
|
||||||
|
return counts;
|
||||||
|
}
|
||||||
|
}
|
@ -114,7 +114,9 @@ private static INodeDirectoryWithQuota createRoot(FSNamesystem namesystem) {
|
|||||||
private final int maxComponentLength;
|
private final int maxComponentLength;
|
||||||
private final int maxDirItems;
|
private final int maxDirItems;
|
||||||
private final int lsLimit; // max list limit
|
private final int lsLimit; // max list limit
|
||||||
|
private final int contentCountLimit; // max content summary counts per run
|
||||||
private final INodeMap inodeMap; // Synchronized by dirLock
|
private final INodeMap inodeMap; // Synchronized by dirLock
|
||||||
|
private long yieldCount = 0; // keep track of lock yield count.
|
||||||
|
|
||||||
// lock to protect the directory and BlockMap
|
// lock to protect the directory and BlockMap
|
||||||
private ReentrantReadWriteLock dirLock;
|
private ReentrantReadWriteLock dirLock;
|
||||||
@ -145,6 +147,14 @@ boolean hasReadLock() {
|
|||||||
return this.dirLock.getReadHoldCount() > 0;
|
return this.dirLock.getReadHoldCount() > 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int getReadHoldCount() {
|
||||||
|
return this.dirLock.getReadHoldCount();
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getWriteHoldCount() {
|
||||||
|
return this.dirLock.getWriteHoldCount();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Caches frequently used file names used in {@link INode} to reuse
|
* Caches frequently used file names used in {@link INode} to reuse
|
||||||
* byte[] objects and reduce heap usage.
|
* byte[] objects and reduce heap usage.
|
||||||
@ -162,6 +172,10 @@ boolean hasReadLock() {
|
|||||||
this.lsLimit = configuredLimit>0 ?
|
this.lsLimit = configuredLimit>0 ?
|
||||||
configuredLimit : DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT;
|
configuredLimit : DFSConfigKeys.DFS_LIST_LIMIT_DEFAULT;
|
||||||
|
|
||||||
|
this.contentCountLimit = conf.getInt(
|
||||||
|
DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY,
|
||||||
|
DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_DEFAULT);
|
||||||
|
|
||||||
// filesystem limits
|
// filesystem limits
|
||||||
this.maxComponentLength = conf.getInt(
|
this.maxComponentLength = conf.getInt(
|
||||||
DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY,
|
DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY,
|
||||||
@ -2296,13 +2310,26 @@ ContentSummary getContentSummary(String src)
|
|||||||
throw new FileNotFoundException("File does not exist: " + srcs);
|
throw new FileNotFoundException("File does not exist: " + srcs);
|
||||||
}
|
}
|
||||||
else {
|
else {
|
||||||
return targetNode.computeContentSummary();
|
// Make it relinquish locks everytime contentCountLimit entries are
|
||||||
|
// processed. 0 means disabled. I.e. blocking for the entire duration.
|
||||||
|
ContentSummaryComputationContext cscc =
|
||||||
|
|
||||||
|
new ContentSummaryComputationContext(this, getFSNamesystem(),
|
||||||
|
contentCountLimit);
|
||||||
|
ContentSummary cs = targetNode.computeAndConvertContentSummary(cscc);
|
||||||
|
yieldCount += cscc.getYieldCount();
|
||||||
|
return cs;
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
readUnlock();
|
readUnlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public long getYieldCount() {
|
||||||
|
return yieldCount;
|
||||||
|
}
|
||||||
|
|
||||||
public INodeMap getINodeMap() {
|
public INodeMap getINodeMap() {
|
||||||
return inodeMap;
|
return inodeMap;
|
||||||
}
|
}
|
||||||
|
@ -593,7 +593,7 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
|||||||
fsNamesys.getSnapshotManager().deleteSnapshot(
|
fsNamesys.getSnapshotManager().deleteSnapshot(
|
||||||
deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
|
deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
|
||||||
collectedBlocks, removedINodes);
|
collectedBlocks, removedINodes);
|
||||||
fsNamesys.removeBlocks(collectedBlocks);
|
fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
|
||||||
collectedBlocks.clear();
|
collectedBlocks.clear();
|
||||||
fsNamesys.dir.removeFromInodeMap(removedINodes);
|
fsNamesys.dir.removeFromInodeMap(removedINodes);
|
||||||
removedINodes.clear();
|
removedINodes.clear();
|
||||||
|
@ -1009,7 +1009,7 @@ void startActiveServices() throws IOException {
|
|||||||
nnEditLogRoller.start();
|
nnEditLogRoller.start();
|
||||||
|
|
||||||
cacheManager.activate();
|
cacheManager.activate();
|
||||||
blockManager.getDatanodeManager().setSendCachingCommands(true);
|
blockManager.getDatanodeManager().setShouldSendCachingCommands(true);
|
||||||
} finally {
|
} finally {
|
||||||
writeUnlock();
|
writeUnlock();
|
||||||
startingActiveService = false;
|
startingActiveService = false;
|
||||||
@ -1060,7 +1060,7 @@ void stopActiveServices() {
|
|||||||
dir.fsImage.updateLastAppliedTxIdFromWritten();
|
dir.fsImage.updateLastAppliedTxIdFromWritten();
|
||||||
}
|
}
|
||||||
cacheManager.deactivate();
|
cacheManager.deactivate();
|
||||||
blockManager.getDatanodeManager().setSendCachingCommands(false);
|
blockManager.getDatanodeManager().setShouldSendCachingCommands(false);
|
||||||
} finally {
|
} finally {
|
||||||
writeUnlock();
|
writeUnlock();
|
||||||
}
|
}
|
||||||
@ -1297,6 +1297,14 @@ public boolean hasReadOrWriteLock() {
|
|||||||
return hasReadLock() || hasWriteLock();
|
return hasReadLock() || hasWriteLock();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int getReadHoldCount() {
|
||||||
|
return this.fsLock.getReadHoldCount();
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getWriteHoldCount() {
|
||||||
|
return this.fsLock.getWriteHoldCount();
|
||||||
|
}
|
||||||
|
|
||||||
NamespaceInfo getNamespaceInfo() {
|
NamespaceInfo getNamespaceInfo() {
|
||||||
readLock();
|
readLock();
|
||||||
try {
|
try {
|
||||||
@ -3305,6 +3313,18 @@ void removePathAndBlocks(String src, BlocksMapUpdateInfo blocks,
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
removeBlocksAndUpdateSafemodeTotal(blocks);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes the blocks from blocksmap and updates the safemode blocks total
|
||||||
|
*
|
||||||
|
* @param blocks
|
||||||
|
* An instance of {@link BlocksMapUpdateInfo} which contains a list
|
||||||
|
* of blocks that need to be removed from blocksMap
|
||||||
|
*/
|
||||||
|
void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) {
|
||||||
|
assert hasWriteLock();
|
||||||
// In the case that we are a Standby tailing edits from the
|
// In the case that we are a Standby tailing edits from the
|
||||||
// active while in safe-mode, we need to track the total number
|
// active while in safe-mode, we need to track the total number
|
||||||
// of blocks and safe blocks in the system.
|
// of blocks and safe blocks in the system.
|
||||||
@ -3325,9 +3345,9 @@ void removePathAndBlocks(String src, BlocksMapUpdateInfo blocks,
|
|||||||
}
|
}
|
||||||
if (trackBlockCounts) {
|
if (trackBlockCounts) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Adjusting safe-mode totals for deletion of " + src + ":" +
|
LOG.debug("Adjusting safe-mode totals for deletion."
|
||||||
"decreasing safeBlocks by " + numRemovedSafe +
|
+ "decreasing safeBlocks by " + numRemovedSafe
|
||||||
", totalBlocks by " + numRemovedComplete);
|
+ ", totalBlocks by " + numRemovedComplete);
|
||||||
}
|
}
|
||||||
adjustSafeModeBlockTotals(-numRemovedSafe, -numRemovedComplete);
|
adjustSafeModeBlockTotals(-numRemovedSafe, -numRemovedComplete);
|
||||||
}
|
}
|
||||||
@ -5883,8 +5903,8 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Update old block with the new generation stamp and new length
|
// Update old block with the new generation stamp and new length
|
||||||
blockinfo.setGenerationStamp(newBlock.getGenerationStamp());
|
|
||||||
blockinfo.setNumBytes(newBlock.getNumBytes());
|
blockinfo.setNumBytes(newBlock.getNumBytes());
|
||||||
|
blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
|
||||||
|
|
||||||
// find the DatanodeDescriptor objects
|
// find the DatanodeDescriptor objects
|
||||||
final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
|
final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
|
||||||
@ -6953,6 +6973,7 @@ void deleteSnapshot(String snapshotRoot, String snapshotName)
|
|||||||
return; // Return previous response
|
return; // Return previous response
|
||||||
}
|
}
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
|
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
||||||
writeLock();
|
writeLock();
|
||||||
try {
|
try {
|
||||||
checkOperation(OperationCategory.WRITE);
|
checkOperation(OperationCategory.WRITE);
|
||||||
@ -6961,7 +6982,6 @@ void deleteSnapshot(String snapshotRoot, String snapshotName)
|
|||||||
checkOwner(pc, snapshotRoot);
|
checkOwner(pc, snapshotRoot);
|
||||||
}
|
}
|
||||||
|
|
||||||
BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
|
|
||||||
List<INode> removedINodes = new ChunkedArrayList<INode>();
|
List<INode> removedINodes = new ChunkedArrayList<INode>();
|
||||||
dir.writeLock();
|
dir.writeLock();
|
||||||
try {
|
try {
|
||||||
@ -6972,8 +6992,6 @@ void deleteSnapshot(String snapshotRoot, String snapshotName)
|
|||||||
dir.writeUnlock();
|
dir.writeUnlock();
|
||||||
}
|
}
|
||||||
removedINodes.clear();
|
removedINodes.clear();
|
||||||
this.removeBlocks(collectedBlocks);
|
|
||||||
collectedBlocks.clear();
|
|
||||||
getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName,
|
getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName,
|
||||||
cacheEntry != null);
|
cacheEntry != null);
|
||||||
success = true;
|
success = true;
|
||||||
@ -6983,6 +7001,9 @@ void deleteSnapshot(String snapshotRoot, String snapshotName)
|
|||||||
}
|
}
|
||||||
getEditLog().logSync();
|
getEditLog().logSync();
|
||||||
|
|
||||||
|
removeBlocks(collectedBlocks);
|
||||||
|
collectedBlocks.clear();
|
||||||
|
|
||||||
if (auditLog.isInfoEnabled() && isExternalInvocation()) {
|
if (auditLog.isInfoEnabled() && isExternalInvocation()) {
|
||||||
String rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
|
String rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
|
||||||
logAuditEvent(true, "deleteSnapshot", rootPath, null, null);
|
logAuditEvent(true, "deleteSnapshot", rootPath, null, null);
|
||||||
|
@ -371,10 +371,18 @@ public abstract Quota.Counts cleanSubtree(final Snapshot snapshot,
|
|||||||
public abstract void destroyAndCollectBlocks(
|
public abstract void destroyAndCollectBlocks(
|
||||||
BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
|
BlocksMapUpdateInfo collectedBlocks, List<INode> removedINodes);
|
||||||
|
|
||||||
/** Compute {@link ContentSummary}. */
|
/** Compute {@link ContentSummary}. Blocking call */
|
||||||
public final ContentSummary computeContentSummary() {
|
public final ContentSummary computeContentSummary() {
|
||||||
final Content.Counts counts = computeContentSummary(
|
return computeAndConvertContentSummary(
|
||||||
Content.Counts.newInstance());
|
new ContentSummaryComputationContext());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Compute {@link ContentSummary}.
|
||||||
|
*/
|
||||||
|
public final ContentSummary computeAndConvertContentSummary(
|
||||||
|
ContentSummaryComputationContext summary) {
|
||||||
|
Content.Counts counts = computeContentSummary(summary).getCounts();
|
||||||
return new ContentSummary(counts.get(Content.LENGTH),
|
return new ContentSummary(counts.get(Content.LENGTH),
|
||||||
counts.get(Content.FILE) + counts.get(Content.SYMLINK),
|
counts.get(Content.FILE) + counts.get(Content.SYMLINK),
|
||||||
counts.get(Content.DIRECTORY), getNsQuota(),
|
counts.get(Content.DIRECTORY), getNsQuota(),
|
||||||
@ -384,10 +392,12 @@ public final ContentSummary computeContentSummary() {
|
|||||||
/**
|
/**
|
||||||
* Count subtree content summary with a {@link Content.Counts}.
|
* Count subtree content summary with a {@link Content.Counts}.
|
||||||
*
|
*
|
||||||
* @param counts The subtree counts for returning.
|
* @param summary the context object holding counts for the subtree.
|
||||||
* @return The same objects as the counts parameter.
|
* @return The same objects as summary.
|
||||||
*/
|
*/
|
||||||
public abstract Content.Counts computeContentSummary(Content.Counts counts);
|
public abstract ContentSummaryComputationContext computeContentSummary(
|
||||||
|
ContentSummaryComputationContext summary);
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check and add namespace/diskspace consumed to itself and the ancestors.
|
* Check and add namespace/diskspace consumed to itself and the ancestors.
|
||||||
|
@ -466,12 +466,45 @@ public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Content.Counts computeContentSummary(final Content.Counts counts) {
|
public ContentSummaryComputationContext computeContentSummary(
|
||||||
for (INode child : getChildrenList(null)) {
|
ContentSummaryComputationContext summary) {
|
||||||
child.computeContentSummary(counts);
|
ReadOnlyList<INode> childrenList = getChildrenList(null);
|
||||||
|
// Explicit traversing is done to enable repositioning after relinquishing
|
||||||
|
// and reacquiring locks.
|
||||||
|
for (int i = 0; i < childrenList.size(); i++) {
|
||||||
|
INode child = childrenList.get(i);
|
||||||
|
byte[] childName = child.getLocalNameBytes();
|
||||||
|
|
||||||
|
long lastYieldCount = summary.getYieldCount();
|
||||||
|
child.computeContentSummary(summary);
|
||||||
|
|
||||||
|
// Check whether the computation was paused in the subtree.
|
||||||
|
// The counts may be off, but traversing the rest of children
|
||||||
|
// should be made safe.
|
||||||
|
if (lastYieldCount == summary.getYieldCount()) {
|
||||||
|
continue;
|
||||||
}
|
}
|
||||||
counts.add(Content.DIRECTORY, 1);
|
|
||||||
return counts;
|
// The locks were released and reacquired. Check parent first.
|
||||||
|
if (getParent() == null) {
|
||||||
|
// Stop further counting and return whatever we have so far.
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Obtain the children list again since it may have been modified.
|
||||||
|
childrenList = getChildrenList(null);
|
||||||
|
// Reposition in case the children list is changed. Decrement by 1
|
||||||
|
// since it will be incremented when loops.
|
||||||
|
i = nextChild(childrenList, childName) - 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Increment the directory count for this directory.
|
||||||
|
summary.getCounts().add(Content.DIRECTORY, 1);
|
||||||
|
|
||||||
|
// Relinquish and reacquire locks if necessary.
|
||||||
|
summary.yield();
|
||||||
|
|
||||||
|
return summary;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -107,12 +107,16 @@ public Quota.Counts computeQuotaUsage(Quota.Counts counts, boolean useCache,
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Content.Counts computeContentSummary(
|
public ContentSummaryComputationContext computeContentSummary(
|
||||||
final Content.Counts counts) {
|
final ContentSummaryComputationContext summary) {
|
||||||
final long original = counts.get(Content.DISKSPACE);
|
final long original = summary.getCounts().get(Content.DISKSPACE);
|
||||||
super.computeContentSummary(counts);
|
long oldYieldCount = summary.getYieldCount();
|
||||||
checkDiskspace(counts.get(Content.DISKSPACE) - original);
|
super.computeContentSummary(summary);
|
||||||
return counts;
|
// Check only when the content has not changed in the middle.
|
||||||
|
if (oldYieldCount == summary.getYieldCount()) {
|
||||||
|
checkDiskspace(summary.getCounts().get(Content.DISKSPACE) - original);
|
||||||
|
}
|
||||||
|
return summary;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void checkDiskspace(final long computed) {
|
private void checkDiskspace(final long computed) {
|
||||||
|
@ -342,11 +342,11 @@ public final Quota.Counts computeQuotaUsage(Quota.Counts counts,
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public final Content.Counts computeContentSummary(
|
public final ContentSummaryComputationContext computeContentSummary(
|
||||||
final Content.Counts counts) {
|
final ContentSummaryComputationContext summary) {
|
||||||
computeContentSummary4Snapshot(counts);
|
computeContentSummary4Snapshot(summary.getCounts());
|
||||||
computeContentSummary4Current(counts);
|
computeContentSummary4Current(summary.getCounts());
|
||||||
return counts;
|
return summary;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void computeContentSummary4Snapshot(final Content.Counts counts) {
|
private void computeContentSummary4Snapshot(final Content.Counts counts) {
|
||||||
|
@ -107,7 +107,8 @@ public Counts computeQuotaUsage(Counts counts, boolean useCache,
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Content.Counts computeContentSummary(Content.Counts counts) {
|
public ContentSummaryComputationContext computeContentSummary(
|
||||||
|
ContentSummaryComputationContext summary) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -278,8 +278,9 @@ public void destroyAndCollectBlocks(
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Content.Counts computeContentSummary(Content.Counts counts) {
|
public ContentSummaryComputationContext computeContentSummary(
|
||||||
return referred.computeContentSummary(counts);
|
ContentSummaryComputationContext summary) {
|
||||||
|
return referred.computeContentSummary(summary);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -444,12 +445,13 @@ public int getLastSnapshotId() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public final Content.Counts computeContentSummary(Content.Counts counts) {
|
public final ContentSummaryComputationContext computeContentSummary(
|
||||||
|
ContentSummaryComputationContext summary) {
|
||||||
//only count diskspace for WithName
|
//only count diskspace for WithName
|
||||||
final Quota.Counts q = Quota.Counts.newInstance();
|
final Quota.Counts q = Quota.Counts.newInstance();
|
||||||
computeQuotaUsage(q, false, lastSnapshotId);
|
computeQuotaUsage(q, false, lastSnapshotId);
|
||||||
counts.add(Content.DISKSPACE, q.get(Quota.DISKSPACE));
|
summary.getCounts().add(Content.DISKSPACE, q.get(Quota.DISKSPACE));
|
||||||
return counts;
|
return summary;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -98,9 +98,10 @@ public Quota.Counts computeQuotaUsage(Quota.Counts counts,
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Content.Counts computeContentSummary(final Content.Counts counts) {
|
public ContentSummaryComputationContext computeContentSummary(
|
||||||
counts.add(Content.SYMLINK, 1);
|
final ContentSummaryComputationContext summary) {
|
||||||
return counts;
|
summary.getCounts().add(Content.SYMLINK, 1);
|
||||||
|
return summary;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -38,6 +38,7 @@
|
|||||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
|
||||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.Content;
|
import org.apache.hadoop.hdfs.server.namenode.Content;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
||||||
@ -342,11 +343,12 @@ Snapshot removeSnapshot(String snapshotName,
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Content.Counts computeContentSummary(final Content.Counts counts) {
|
public ContentSummaryComputationContext computeContentSummary(
|
||||||
super.computeContentSummary(counts);
|
final ContentSummaryComputationContext summary) {
|
||||||
counts.add(Content.SNAPSHOT, snapshotsByNames.size());
|
super.computeContentSummary(summary);
|
||||||
counts.add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
|
summary.getCounts().add(Content.SNAPSHOT, snapshotsByNames.size());
|
||||||
return counts;
|
summary.getCounts().add(Content.SNAPSHOTTABLE_DIRECTORY, 1);
|
||||||
|
return summary;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -32,6 +32,7 @@
|
|||||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
|
||||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
|
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.Content;
|
import org.apache.hadoop.hdfs.server.namenode.Content;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
|
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
||||||
@ -883,18 +884,27 @@ public Quota.Counts computeQuotaUsage4CurrentDirectory(Quota.Counts counts) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Content.Counts computeContentSummary(final Content.Counts counts) {
|
public ContentSummaryComputationContext computeContentSummary(
|
||||||
super.computeContentSummary(counts);
|
final ContentSummaryComputationContext summary) {
|
||||||
computeContentSummary4Snapshot(counts);
|
// Snapshot summary calc won't be relinquishing locks in the middle.
|
||||||
return counts;
|
// Do this first and handover to parent.
|
||||||
|
computeContentSummary4Snapshot(summary.getCounts());
|
||||||
|
super.computeContentSummary(summary);
|
||||||
|
return summary;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void computeContentSummary4Snapshot(final Content.Counts counts) {
|
private void computeContentSummary4Snapshot(final Content.Counts counts) {
|
||||||
|
// Create a new blank summary context for blocking processing of subtree.
|
||||||
|
ContentSummaryComputationContext summary =
|
||||||
|
new ContentSummaryComputationContext();
|
||||||
for(DirectoryDiff d : diffs) {
|
for(DirectoryDiff d : diffs) {
|
||||||
for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
|
for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
|
||||||
deleted.computeContentSummary(counts);
|
deleted.computeContentSummary(summary);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// Add the counts from deleted trees.
|
||||||
|
counts.add(summary.getCounts());
|
||||||
|
// Add the deleted directory count.
|
||||||
counts.add(Content.DIRECTORY, diffs.asList().size());
|
counts.add(Content.DIRECTORY, diffs.asList().size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -26,8 +26,8 @@
|
|||||||
import java.io.PrintStream;
|
import java.io.PrintStream;
|
||||||
import java.net.HttpURLConnection;
|
import java.net.HttpURLConnection;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
import java.net.URI;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.net.URLConnection;
|
|
||||||
import java.security.PrivilegedExceptionAction;
|
import java.security.PrivilegedExceptionAction;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
@ -47,11 +47,14 @@
|
|||||||
import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
|
import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
|
import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
|
||||||
import org.apache.hadoop.hdfs.web.HftpFileSystem;
|
import org.apache.hadoop.hdfs.web.HftpFileSystem;
|
||||||
|
import org.apache.hadoop.hdfs.web.HsftpFileSystem;
|
||||||
|
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
import org.apache.hadoop.security.SecurityUtil;
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.util.ExitUtil;
|
import org.apache.hadoop.util.ExitUtil;
|
||||||
import org.apache.hadoop.util.GenericOptionsParser;
|
import org.apache.hadoop.util.GenericOptionsParser;
|
||||||
@ -142,11 +145,11 @@ public static void main(final String[] args) throws Exception {
|
|||||||
// default to using the local file system
|
// default to using the local file system
|
||||||
FileSystem local = FileSystem.getLocal(conf);
|
FileSystem local = FileSystem.getLocal(conf);
|
||||||
final Path tokenFile = new Path(local.getWorkingDirectory(), remaining[0]);
|
final Path tokenFile = new Path(local.getWorkingDirectory(), remaining[0]);
|
||||||
|
final URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
|
||||||
|
|
||||||
// Login the current user
|
// Login the current user
|
||||||
UserGroupInformation.getCurrentUser().doAs(
|
UserGroupInformation.getCurrentUser().doAs(
|
||||||
new PrivilegedExceptionAction<Object>() {
|
new PrivilegedExceptionAction<Object>() {
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
@Override
|
@Override
|
||||||
public Object run() throws Exception {
|
public Object run() throws Exception {
|
||||||
|
|
||||||
@ -182,7 +185,8 @@ public Object run() throws Exception {
|
|||||||
} else {
|
} else {
|
||||||
// otherwise we are fetching
|
// otherwise we are fetching
|
||||||
if (webUrl != null) {
|
if (webUrl != null) {
|
||||||
Credentials creds = getDTfromRemote(webUrl, renewer);
|
Credentials creds = getDTfromRemote(connectionFactory, new URI(webUrl),
|
||||||
|
renewer);
|
||||||
creds.writeTokenStorageFile(tokenFile, conf);
|
creds.writeTokenStorageFile(tokenFile, conf);
|
||||||
for (Token<?> token : creds.getAllTokens()) {
|
for (Token<?> token : creds.getAllTokens()) {
|
||||||
if(LOG.isDebugEnabled()) {
|
if(LOG.isDebugEnabled()) {
|
||||||
@ -208,32 +212,31 @@ public Object run() throws Exception {
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
static public Credentials getDTfromRemote(String nnAddr,
|
static public Credentials getDTfromRemote(URLConnectionFactory factory,
|
||||||
String renewer) throws IOException {
|
URI nnUri, String renewer) throws IOException {
|
||||||
|
StringBuilder buf = new StringBuilder(nnUri.toString())
|
||||||
|
.append(GetDelegationTokenServlet.PATH_SPEC);
|
||||||
|
if (renewer != null) {
|
||||||
|
buf.append("?").append(GetDelegationTokenServlet.RENEWER).append("=")
|
||||||
|
.append(renewer);
|
||||||
|
}
|
||||||
|
|
||||||
|
HttpURLConnection conn = null;
|
||||||
DataInputStream dis = null;
|
DataInputStream dis = null;
|
||||||
InetSocketAddress serviceAddr = NetUtils.createSocketAddr(nnAddr);
|
InetSocketAddress serviceAddr = NetUtils.createSocketAddr(nnUri
|
||||||
|
.getAuthority());
|
||||||
|
|
||||||
try {
|
try {
|
||||||
StringBuffer url = new StringBuffer();
|
|
||||||
if (renewer != null) {
|
|
||||||
url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC)
|
|
||||||
.append("?").append(GetDelegationTokenServlet.RENEWER).append("=")
|
|
||||||
.append(renewer);
|
|
||||||
} else {
|
|
||||||
url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC);
|
|
||||||
}
|
|
||||||
|
|
||||||
if(LOG.isDebugEnabled()) {
|
if(LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Retrieving token from: " + url);
|
LOG.debug("Retrieving token from: " + buf);
|
||||||
}
|
}
|
||||||
|
|
||||||
URL remoteURL = new URL(url.toString());
|
conn = run(factory, new URL(buf.toString()));
|
||||||
URLConnection connection = SecurityUtil.openSecureHttpConnection(remoteURL);
|
InputStream in = conn.getInputStream();
|
||||||
InputStream in = connection.getInputStream();
|
|
||||||
Credentials ts = new Credentials();
|
Credentials ts = new Credentials();
|
||||||
dis = new DataInputStream(in);
|
dis = new DataInputStream(in);
|
||||||
ts.readFields(dis);
|
ts.readFields(dis);
|
||||||
for(Token<?> token: ts.getAllTokens()) {
|
for (Token<?> token : ts.getAllTokens()) {
|
||||||
token.setKind(HftpFileSystem.TOKEN_KIND);
|
token.setKind(HftpFileSystem.TOKEN_KIND);
|
||||||
SecurityUtil.setTokenService(token, serviceAddr);
|
SecurityUtil.setTokenService(token, serviceAddr);
|
||||||
}
|
}
|
||||||
@ -241,9 +244,30 @@ static public Credentials getDTfromRemote(String nnAddr,
|
|||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new IOException("Unable to obtain remote token", e);
|
throw new IOException("Unable to obtain remote token", e);
|
||||||
} finally {
|
} finally {
|
||||||
if(dis != null) dis.close();
|
IOUtils.cleanup(LOG, dis);
|
||||||
|
if (conn != null) {
|
||||||
|
conn.disconnect();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Cancel a Delegation Token.
|
||||||
|
* @param nnAddr the NameNode's address
|
||||||
|
* @param tok the token to cancel
|
||||||
|
* @throws IOException
|
||||||
|
* @throws AuthenticationException
|
||||||
|
*/
|
||||||
|
static public void cancelDelegationToken(URLConnectionFactory factory,
|
||||||
|
URI nnAddr, Token<DelegationTokenIdentifier> tok) throws IOException,
|
||||||
|
AuthenticationException {
|
||||||
|
StringBuilder buf = new StringBuilder(nnAddr.toString())
|
||||||
|
.append(CancelDelegationTokenServlet.PATH_SPEC).append("?")
|
||||||
|
.append(CancelDelegationTokenServlet.TOKEN).append("=")
|
||||||
|
.append(tok.encodeToUrlString());
|
||||||
|
HttpURLConnection conn = run(factory, new URL(buf.toString()));
|
||||||
|
conn.disconnect();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Renew a Delegation Token.
|
* Renew a Delegation Token.
|
||||||
@ -251,43 +275,39 @@ static public Credentials getDTfromRemote(String nnAddr,
|
|||||||
* @param tok the token to renew
|
* @param tok the token to renew
|
||||||
* @return the Date that the token will expire next.
|
* @return the Date that the token will expire next.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
|
* @throws AuthenticationException
|
||||||
*/
|
*/
|
||||||
static public long renewDelegationToken(String nnAddr,
|
static public long renewDelegationToken(URLConnectionFactory factory,
|
||||||
Token<DelegationTokenIdentifier> tok
|
URI nnAddr, Token<DelegationTokenIdentifier> tok) throws IOException,
|
||||||
) throws IOException {
|
AuthenticationException {
|
||||||
StringBuilder buf = new StringBuilder();
|
StringBuilder buf = new StringBuilder(nnAddr.toString())
|
||||||
buf.append(nnAddr);
|
.append(RenewDelegationTokenServlet.PATH_SPEC).append("?")
|
||||||
buf.append(RenewDelegationTokenServlet.PATH_SPEC);
|
.append(RenewDelegationTokenServlet.TOKEN).append("=")
|
||||||
buf.append("?");
|
.append(tok.encodeToUrlString());
|
||||||
buf.append(RenewDelegationTokenServlet.TOKEN);
|
|
||||||
buf.append("=");
|
|
||||||
buf.append(tok.encodeToUrlString());
|
|
||||||
BufferedReader in = null;
|
|
||||||
HttpURLConnection connection = null;
|
|
||||||
|
|
||||||
|
HttpURLConnection connection = null;
|
||||||
|
BufferedReader in = null;
|
||||||
try {
|
try {
|
||||||
URL url = new URL(buf.toString());
|
connection = run(factory, new URL(buf.toString()));
|
||||||
connection = (HttpURLConnection) SecurityUtil.openSecureHttpConnection(url);
|
in = new BufferedReader(new InputStreamReader(
|
||||||
if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
|
connection.getInputStream(), Charsets.UTF_8));
|
||||||
throw new IOException("Error renewing token: " +
|
|
||||||
connection.getResponseMessage());
|
|
||||||
}
|
|
||||||
in = new BufferedReader(
|
|
||||||
new InputStreamReader(connection.getInputStream(), Charsets.UTF_8));
|
|
||||||
long result = Long.parseLong(in.readLine());
|
long result = Long.parseLong(in.readLine());
|
||||||
in.close();
|
|
||||||
return result;
|
return result;
|
||||||
} catch (IOException ie) {
|
} catch (IOException ie) {
|
||||||
LOG.info("error in renew over HTTP", ie);
|
LOG.info("error in renew over HTTP", ie);
|
||||||
IOException e = getExceptionFromResponse(connection);
|
IOException e = getExceptionFromResponse(connection);
|
||||||
|
|
||||||
IOUtils.cleanup(LOG, in);
|
if (e != null) {
|
||||||
if(e!=null) {
|
LOG.info("rethrowing exception from HTTP request: "
|
||||||
LOG.info("rethrowing exception from HTTP request: " +
|
+ e.getLocalizedMessage());
|
||||||
e.getLocalizedMessage());
|
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
throw ie;
|
throw ie;
|
||||||
|
} finally {
|
||||||
|
IOUtils.cleanup(LOG, in);
|
||||||
|
if (connection != null) {
|
||||||
|
connection.disconnect();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -339,43 +359,28 @@ static private IOException getExceptionFromResponse(HttpURLConnection con) {
|
|||||||
return e;
|
return e;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static HttpURLConnection run(URLConnectionFactory factory, URL url)
|
||||||
|
throws IOException, AuthenticationException {
|
||||||
|
HttpURLConnection conn = null;
|
||||||
|
|
||||||
/**
|
|
||||||
* Cancel a Delegation Token.
|
|
||||||
* @param nnAddr the NameNode's address
|
|
||||||
* @param tok the token to cancel
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
static public void cancelDelegationToken(String nnAddr,
|
|
||||||
Token<DelegationTokenIdentifier> tok
|
|
||||||
) throws IOException {
|
|
||||||
StringBuilder buf = new StringBuilder();
|
|
||||||
buf.append(nnAddr);
|
|
||||||
buf.append(CancelDelegationTokenServlet.PATH_SPEC);
|
|
||||||
buf.append("?");
|
|
||||||
buf.append(CancelDelegationTokenServlet.TOKEN);
|
|
||||||
buf.append("=");
|
|
||||||
buf.append(tok.encodeToUrlString());
|
|
||||||
BufferedReader in = null;
|
|
||||||
HttpURLConnection connection=null;
|
|
||||||
try {
|
try {
|
||||||
URL url = new URL(buf.toString());
|
conn = (HttpURLConnection) factory.openConnection(url, true);
|
||||||
connection = (HttpURLConnection) SecurityUtil.openSecureHttpConnection(url);
|
if (conn.getResponseCode() != HttpURLConnection.HTTP_OK) {
|
||||||
if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
|
String msg = conn.getResponseMessage();
|
||||||
throw new IOException("Error cancelling token: " +
|
|
||||||
connection.getResponseMessage());
|
throw new IOException("Error when dealing remote token: " + msg);
|
||||||
}
|
}
|
||||||
} catch (IOException ie) {
|
} catch (IOException ie) {
|
||||||
LOG.info("error in cancel over HTTP", ie);
|
LOG.info("Error when dealing remote token:", ie);
|
||||||
IOException e = getExceptionFromResponse(connection);
|
IOException e = getExceptionFromResponse(conn);
|
||||||
|
|
||||||
IOUtils.cleanup(LOG, in);
|
if (e != null) {
|
||||||
if(e!=null) {
|
LOG.info("rethrowing exception from HTTP request: "
|
||||||
LOG.info("rethrowing exception from HTTP request: " +
|
+ e.getLocalizedMessage());
|
||||||
e.getLocalizedMessage());
|
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
throw ie;
|
throw ie;
|
||||||
}
|
}
|
||||||
|
return conn;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -31,7 +31,6 @@
|
|||||||
import java.text.ParseException;
|
import java.text.ParseException;
|
||||||
import java.text.SimpleDateFormat;
|
import java.text.SimpleDateFormat;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.TimeZone;
|
import java.util.TimeZone;
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
@ -50,20 +49,17 @@
|
|||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
|
|
||||||
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
||||||
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
||||||
import org.apache.hadoop.hdfs.web.ByteRangeInputStream.URLOpener;
|
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
import org.apache.hadoop.security.SecurityUtil;
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||||
import org.apache.hadoop.security.token.TokenRenewer;
|
|
||||||
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
import org.apache.hadoop.util.ServletUtil;
|
import org.apache.hadoop.util.ServletUtil;
|
||||||
import org.xml.sax.Attributes;
|
import org.xml.sax.Attributes;
|
||||||
@ -83,7 +79,9 @@
|
|||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
@InterfaceStability.Evolving
|
@InterfaceStability.Evolving
|
||||||
public class HftpFileSystem extends FileSystem
|
public class HftpFileSystem extends FileSystem
|
||||||
implements DelegationTokenRenewer.Renewable {
|
implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
|
||||||
|
public static final String SCHEME = "hftp";
|
||||||
|
|
||||||
static {
|
static {
|
||||||
HttpURLConnection.setFollowRedirects(true);
|
HttpURLConnection.setFollowRedirects(true);
|
||||||
}
|
}
|
||||||
@ -100,19 +98,13 @@ public class HftpFileSystem extends FileSystem
|
|||||||
public static final String HFTP_TIMEZONE = "UTC";
|
public static final String HFTP_TIMEZONE = "UTC";
|
||||||
public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
|
public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
|
||||||
|
|
||||||
|
private TokenAspect<HftpFileSystem> tokenAspect = new TokenAspect<HftpFileSystem>(this, TOKEN_KIND);
|
||||||
private Token<?> delegationToken;
|
private Token<?> delegationToken;
|
||||||
private Token<?> renewToken;
|
private Token<?> renewToken;
|
||||||
private static final HftpDelegationTokenSelector hftpTokenSelector =
|
|
||||||
new HftpDelegationTokenSelector();
|
|
||||||
|
|
||||||
private DelegationTokenRenewer dtRenewer = null;
|
@Override
|
||||||
|
public URI getCanonicalUri() {
|
||||||
private synchronized void addRenewAction(final HftpFileSystem hftpFs) {
|
return super.getCanonicalUri();
|
||||||
if (dtRenewer == null) {
|
|
||||||
dtRenewer = DelegationTokenRenewer.getInstance();
|
|
||||||
}
|
|
||||||
|
|
||||||
dtRenewer.addRenewAction(hftpFs);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final SimpleDateFormat getDateFormat() {
|
public static final SimpleDateFormat getDateFormat() {
|
||||||
@ -177,7 +169,7 @@ protected URI canonicalizeUri(URI uri) {
|
|||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public String getScheme() {
|
public String getScheme() {
|
||||||
return "hftp";
|
return SCHEME;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -195,39 +187,10 @@ public void initialize(final URI name, final Configuration conf)
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (UserGroupInformation.isSecurityEnabled()) {
|
if (UserGroupInformation.isSecurityEnabled()) {
|
||||||
initDelegationToken();
|
tokenAspect.initDelegationToken(ugi);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void initDelegationToken() throws IOException {
|
|
||||||
// look for hftp token, then try hdfs
|
|
||||||
Token<?> token = selectDelegationToken(ugi);
|
|
||||||
|
|
||||||
// if we don't already have a token, go get one over https
|
|
||||||
boolean createdToken = false;
|
|
||||||
if (token == null) {
|
|
||||||
token = getDelegationToken(null);
|
|
||||||
createdToken = (token != null);
|
|
||||||
}
|
|
||||||
|
|
||||||
// we already had a token or getDelegationToken() didn't fail.
|
|
||||||
if (token != null) {
|
|
||||||
setDelegationToken(token);
|
|
||||||
if (createdToken) {
|
|
||||||
addRenewAction(this);
|
|
||||||
LOG.debug("Created new DT for " + token.getService());
|
|
||||||
} else {
|
|
||||||
LOG.debug("Found existing DT for " + token.getService());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
protected Token<DelegationTokenIdentifier> selectDelegationToken(
|
|
||||||
UserGroupInformation ugi) {
|
|
||||||
return hftpTokenSelector.selectToken(nnUri, ugi.getTokens(), getConf());
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Token<?> getRenewToken() {
|
public Token<?> getRenewToken() {
|
||||||
return renewToken;
|
return renewToken;
|
||||||
@ -242,16 +205,19 @@ protected String getUnderlyingProtocol() {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
|
public synchronized <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
|
||||||
|
/**
|
||||||
|
* XXX The kind of the token has been changed by DelegationTokenFetcher. We
|
||||||
|
* use the token for renewal, since the reflection utilities needs the value
|
||||||
|
* of the kind field to correctly renew the token.
|
||||||
|
*
|
||||||
|
* For other operations, however, the client has to send a
|
||||||
|
* HDFS_DELEGATION_KIND token over the wire so that it can talk to Hadoop
|
||||||
|
* 0.20.3 clusters. Later releases fix this problem. See HDFS-5440 for more
|
||||||
|
* details.
|
||||||
|
*/
|
||||||
renewToken = token;
|
renewToken = token;
|
||||||
// emulate the 203 usage of the tokens
|
|
||||||
// by setting the kind and service as if they were hdfs tokens
|
|
||||||
delegationToken = new Token<T>(token);
|
delegationToken = new Token<T>(token);
|
||||||
// NOTE: the remote nn must be configured to use hdfs
|
|
||||||
delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
|
delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
|
||||||
// no need to change service because we aren't exactly sure what it
|
|
||||||
// should be. we can guess, but it might be wrong if the local conf
|
|
||||||
// value is incorrect. the service is a client side field, so the remote
|
|
||||||
// end does not care about the value
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -266,7 +232,7 @@ public Token<?> run() throws IOException {
|
|||||||
final String nnHttpUrl = nnUri.toString();
|
final String nnHttpUrl = nnUri.toString();
|
||||||
Credentials c;
|
Credentials c;
|
||||||
try {
|
try {
|
||||||
c = DelegationTokenFetcher.getDTfromRemote(nnHttpUrl, renewer);
|
c = DelegationTokenFetcher.getDTfromRemote(connectionFactory, nnUri, renewer);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
if (e.getCause() instanceof ConnectException) {
|
if (e.getCause() instanceof ConnectException) {
|
||||||
LOG.warn("Couldn't connect to " + nnHttpUrl +
|
LOG.warn("Couldn't connect to " + nnHttpUrl +
|
||||||
@ -350,6 +316,7 @@ protected String addDelegationTokenParam(String query) throws IOException {
|
|||||||
String tokenString = null;
|
String tokenString = null;
|
||||||
if (UserGroupInformation.isSecurityEnabled()) {
|
if (UserGroupInformation.isSecurityEnabled()) {
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
|
tokenAspect.ensureTokenInitialized();
|
||||||
if (delegationToken != null) {
|
if (delegationToken != null) {
|
||||||
tokenString = delegationToken.encodeToUrlString();
|
tokenString = delegationToken.encodeToUrlString();
|
||||||
return (query + JspHelper.getDelegationTokenUrlParam(tokenString));
|
return (query + JspHelper.getDelegationTokenUrlParam(tokenString));
|
||||||
@ -419,9 +386,7 @@ public FSDataInputStream open(Path f, int buffersize) throws IOException {
|
|||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
super.close();
|
super.close();
|
||||||
if (dtRenewer != null) {
|
tokenAspect.removeRenewAction();
|
||||||
dtRenewer.removeRenewAction(this); // blocks
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Class to parse and store a listing reply from the server. */
|
/** Class to parse and store a listing reply from the server. */
|
||||||
@ -696,67 +661,33 @@ public ContentSummary getContentSummary(Path f) throws IOException {
|
|||||||
return cs != null? cs: super.getContentSummary(f);
|
return cs != null? cs: super.getContentSummary(f);
|
||||||
}
|
}
|
||||||
|
|
||||||
@InterfaceAudience.Private
|
@SuppressWarnings("unchecked")
|
||||||
public static class TokenManager extends TokenRenewer {
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean handleKind(Text kind) {
|
public long renewDelegationToken(Token<?> token) throws IOException {
|
||||||
return kind.equals(TOKEN_KIND);
|
// update the kerberos credentials, if they are coming from a keytab
|
||||||
|
UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
|
||||||
|
InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
|
||||||
|
try {
|
||||||
|
return DelegationTokenFetcher.renewDelegationToken(connectionFactory,
|
||||||
|
DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr),
|
||||||
|
(Token<DelegationTokenIdentifier>) token);
|
||||||
|
} catch (AuthenticationException e) {
|
||||||
|
throw new IOException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isManaged(Token<?> token) throws IOException {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
protected String getUnderlyingProtocol() {
|
|
||||||
return "http";
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Override
|
@Override
|
||||||
public long renew(Token<?> token,
|
public void cancelDelegationToken(Token<?> token) throws IOException {
|
||||||
Configuration conf) throws IOException {
|
|
||||||
// update the kerberos credentials, if they are coming from a keytab
|
// update the kerberos credentials, if they are coming from a keytab
|
||||||
UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
|
UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
|
||||||
InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
|
InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
|
||||||
return
|
try {
|
||||||
DelegationTokenFetcher.renewDelegationToken
|
DelegationTokenFetcher.cancelDelegationToken(connectionFactory, DFSUtil
|
||||||
(DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
|
.createUri(getUnderlyingProtocol(), serviceAddr),
|
||||||
(Token<DelegationTokenIdentifier>) token);
|
(Token<DelegationTokenIdentifier>) token);
|
||||||
}
|
} catch (AuthenticationException e) {
|
||||||
|
throw new IOException(e);
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
@Override
|
|
||||||
public void cancel(Token<?> token,
|
|
||||||
Configuration conf) throws IOException {
|
|
||||||
// update the kerberos credentials, if they are coming from a keytab
|
|
||||||
UserGroupInformation.getLoginUser().checkTGTAndReloginFromKeytab();
|
|
||||||
InetSocketAddress serviceAddr = SecurityUtil.getTokenServiceAddr(token);
|
|
||||||
DelegationTokenFetcher.cancelDelegationToken
|
|
||||||
(DFSUtil.createUri(getUnderlyingProtocol(), serviceAddr).toString(),
|
|
||||||
(Token<DelegationTokenIdentifier>) token);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class HftpDelegationTokenSelector
|
|
||||||
extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
|
|
||||||
private static final DelegationTokenSelector hdfsTokenSelector =
|
|
||||||
new DelegationTokenSelector();
|
|
||||||
|
|
||||||
public HftpDelegationTokenSelector() {
|
|
||||||
super(TOKEN_KIND);
|
|
||||||
}
|
|
||||||
|
|
||||||
Token<DelegationTokenIdentifier> selectToken(URI nnUri,
|
|
||||||
Collection<Token<?>> tokens, Configuration conf) {
|
|
||||||
Token<DelegationTokenIdentifier> token =
|
|
||||||
selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
|
|
||||||
if (token == null) {
|
|
||||||
// try to get a HDFS token
|
|
||||||
token = hdfsTokenSelector.selectToken(nnUri, tokens, conf);
|
|
||||||
}
|
|
||||||
return token;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,177 @@
|
|||||||
|
/**
|
||||||
|
* 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.web;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.InetSocketAddress;
|
||||||
|
import java.net.URI;
|
||||||
|
import java.util.Collection;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.DelegationTokenRenewer;
|
||||||
|
import org.apache.hadoop.fs.DelegationTokenRenewer.Renewable;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||||
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.security.SecurityUtil;
|
||||||
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
import org.apache.hadoop.security.token.TokenRenewer;
|
||||||
|
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class implements the aspects that relate to delegation tokens for all
|
||||||
|
* HTTP-based file system.
|
||||||
|
*/
|
||||||
|
final class TokenAspect<T extends FileSystem & Renewable> {
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public static class TokenManager extends TokenRenewer {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cancel(Token<?> token, Configuration conf) throws IOException {
|
||||||
|
getInstance(token, conf).cancelDelegationToken(token);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean handleKind(Text kind) {
|
||||||
|
return kind.equals(HftpFileSystem.TOKEN_KIND)
|
||||||
|
|| kind.equals(WebHdfsFileSystem.TOKEN_KIND);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isManaged(Token<?> token) throws IOException {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long renew(Token<?> token, Configuration conf) throws IOException {
|
||||||
|
return getInstance(token, conf).renewDelegationToken(token);
|
||||||
|
}
|
||||||
|
|
||||||
|
private TokenManagementDelegator getInstance(Token<?> token,
|
||||||
|
Configuration conf) throws IOException {
|
||||||
|
final InetSocketAddress address = SecurityUtil.getTokenServiceAddr(token);
|
||||||
|
Text kind = token.getKind();
|
||||||
|
final URI uri;
|
||||||
|
if (kind.equals(HftpFileSystem.TOKEN_KIND)) {
|
||||||
|
uri = DFSUtil.createUri(HftpFileSystem.SCHEME, address);
|
||||||
|
} else if (kind.equals(WebHdfsFileSystem.TOKEN_KIND)) {
|
||||||
|
uri = DFSUtil.createUri(WebHdfsFileSystem.SCHEME, address);
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException("Unsupported scheme");
|
||||||
|
}
|
||||||
|
return (TokenManagementDelegator) FileSystem.get(uri, conf);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static class DTSelecorByKind extends
|
||||||
|
AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
|
||||||
|
private static final DelegationTokenSelector selector = new DelegationTokenSelector();
|
||||||
|
|
||||||
|
public DTSelecorByKind(final Text kind) {
|
||||||
|
super(kind);
|
||||||
|
}
|
||||||
|
|
||||||
|
Token<DelegationTokenIdentifier> selectToken(URI nnUri,
|
||||||
|
Collection<Token<?>> tokens, Configuration conf) {
|
||||||
|
Token<DelegationTokenIdentifier> token = selectToken(
|
||||||
|
SecurityUtil.buildTokenService(nnUri), tokens);
|
||||||
|
if (token == null) {
|
||||||
|
token = selector.selectToken(nnUri, tokens, conf);
|
||||||
|
}
|
||||||
|
return token;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Callbacks for token management
|
||||||
|
*/
|
||||||
|
interface TokenManagementDelegator {
|
||||||
|
void cancelDelegationToken(final Token<?> token) throws IOException;
|
||||||
|
|
||||||
|
URI getCanonicalUri();
|
||||||
|
|
||||||
|
long renewDelegationToken(final Token<?> token) throws IOException;
|
||||||
|
}
|
||||||
|
|
||||||
|
private DelegationTokenRenewer.RenewAction<?> action;
|
||||||
|
private DelegationTokenRenewer dtRenewer = null;
|
||||||
|
private final DTSelecorByKind dtSelector;
|
||||||
|
private final T fs;
|
||||||
|
private boolean hasInitedToken;
|
||||||
|
private final Log LOG;
|
||||||
|
|
||||||
|
TokenAspect(T fs, final Text kind) {
|
||||||
|
this.LOG = LogFactory.getLog(fs.getClass());
|
||||||
|
this.fs = fs;
|
||||||
|
this.dtSelector = new DTSelecorByKind(kind);
|
||||||
|
}
|
||||||
|
|
||||||
|
synchronized void ensureTokenInitialized() throws IOException {
|
||||||
|
// we haven't inited yet, or we used to have a token but it expired
|
||||||
|
if (!hasInitedToken || (action != null && !action.isValid())) {
|
||||||
|
//since we don't already have a token, go get one
|
||||||
|
Token<?> token = fs.getDelegationToken(null);
|
||||||
|
// security might be disabled
|
||||||
|
if (token != null) {
|
||||||
|
fs.setDelegationToken(token);
|
||||||
|
addRenewAction(fs);
|
||||||
|
LOG.debug("Created new DT for " + token.getService());
|
||||||
|
}
|
||||||
|
hasInitedToken = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
synchronized void initDelegationToken(UserGroupInformation ugi) {
|
||||||
|
Token<?> token = selectDelegationToken(ugi);
|
||||||
|
if (token != null) {
|
||||||
|
LOG.debug("Found existing DT for " + token.getService());
|
||||||
|
fs.setDelegationToken(token);
|
||||||
|
hasInitedToken = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
synchronized void removeRenewAction() throws IOException {
|
||||||
|
if (dtRenewer != null) {
|
||||||
|
dtRenewer.removeRenewAction(fs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
Token<DelegationTokenIdentifier> selectDelegationToken(
|
||||||
|
UserGroupInformation ugi) {
|
||||||
|
return dtSelector.selectToken(
|
||||||
|
((TokenManagementDelegator)fs).getCanonicalUri(), ugi.getTokens(),
|
||||||
|
fs.getConf());
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized void addRenewAction(final T webhdfs) {
|
||||||
|
if (dtRenewer == null) {
|
||||||
|
dtRenewer = DelegationTokenRenewer.getInstance();
|
||||||
|
}
|
||||||
|
|
||||||
|
action = dtRenewer.addRenewAction(webhdfs);
|
||||||
|
}
|
||||||
|
}
|
@ -27,7 +27,7 @@
|
|||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
||||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||||
@ -77,24 +77,28 @@ public URLConnectionFactory(int socketTimeout) {
|
|||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public URLConnection openConnection(URL url) throws IOException {
|
public URLConnection openConnection(URL url) throws IOException {
|
||||||
URLConnection connection = url.openConnection();
|
try {
|
||||||
if (connection instanceof HttpURLConnection) {
|
return openConnection(url, false);
|
||||||
connConfigurator.configure((HttpURLConnection) connection);
|
} catch (AuthenticationException e) {
|
||||||
|
// Unreachable
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
return connection;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Opens a url with read and connect timeouts
|
* Opens a url with read and connect timeouts
|
||||||
*
|
*
|
||||||
* @param url URL to open
|
* @param url
|
||||||
|
* URL to open
|
||||||
|
* @param isSpnego
|
||||||
|
* whether the url should be authenticated via SPNEGO
|
||||||
* @return URLConnection
|
* @return URLConnection
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @throws AuthenticationException
|
* @throws AuthenticationException
|
||||||
*/
|
*/
|
||||||
public URLConnection openConnection(HttpOpParam.Op op, URL url)
|
public URLConnection openConnection(URL url, boolean isSpnego)
|
||||||
throws IOException, AuthenticationException {
|
throws IOException, AuthenticationException {
|
||||||
if (op.getRequireAuth()) {
|
if (isSpnego) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("open AuthenticatedURL connection" + url);
|
LOG.debug("open AuthenticatedURL connection" + url);
|
||||||
}
|
}
|
||||||
@ -106,7 +110,11 @@ public URLConnection openConnection(HttpOpParam.Op op, URL url)
|
|||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("open URL connection");
|
LOG.debug("open URL connection");
|
||||||
}
|
}
|
||||||
return openConnection(url);
|
URLConnection connection = url.openConnection();
|
||||||
|
if (connection instanceof HttpURLConnection) {
|
||||||
|
connConfigurator.configure((HttpURLConnection) connection);
|
||||||
|
}
|
||||||
|
return connection;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -30,7 +30,6 @@
|
|||||||
import java.net.URISyntaxException;
|
import java.net.URISyntaxException;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.security.PrivilegedExceptionAction;
|
import java.security.PrivilegedExceptionAction;
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.StringTokenizer;
|
import java.util.StringTokenizer;
|
||||||
@ -56,8 +55,8 @@
|
|||||||
import org.apache.hadoop.hdfs.HAUtil;
|
import org.apache.hadoop.hdfs.HAUtil;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
|
|
||||||
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
|
||||||
|
import org.apache.hadoop.hdfs.web.TokenAspect.DTSelecorByKind;
|
||||||
import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
|
import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
|
||||||
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
|
import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
|
||||||
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
|
import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
|
||||||
@ -96,8 +95,6 @@
|
|||||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||||
import org.apache.hadoop.security.token.TokenRenewer;
|
|
||||||
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
|
|
||||||
import org.apache.hadoop.util.Progressable;
|
import org.apache.hadoop.util.Progressable;
|
||||||
import org.mortbay.util.ajax.JSON;
|
import org.mortbay.util.ajax.JSON;
|
||||||
|
|
||||||
@ -107,7 +104,7 @@
|
|||||||
|
|
||||||
/** A FileSystem for HDFS over the web. */
|
/** A FileSystem for HDFS over the web. */
|
||||||
public class WebHdfsFileSystem extends FileSystem
|
public class WebHdfsFileSystem extends FileSystem
|
||||||
implements DelegationTokenRenewer.Renewable {
|
implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
|
||||||
public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
|
public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
|
||||||
/** File System URI: {SCHEME}://namenode:port/path/to/file */
|
/** File System URI: {SCHEME}://namenode:port/path/to/file */
|
||||||
public static final String SCHEME = "webhdfs";
|
public static final String SCHEME = "webhdfs";
|
||||||
@ -122,13 +119,18 @@ public class WebHdfsFileSystem extends FileSystem
|
|||||||
/** Delegation token kind */
|
/** Delegation token kind */
|
||||||
public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
|
public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
|
||||||
/** Token selector */
|
/** Token selector */
|
||||||
public static final WebHdfsDelegationTokenSelector DT_SELECTOR
|
public static final DTSelecorByKind DT_SELECTOR
|
||||||
= new WebHdfsDelegationTokenSelector();
|
= new DTSelecorByKind(TOKEN_KIND);
|
||||||
|
|
||||||
private DelegationTokenRenewer dtRenewer = null;
|
private DelegationTokenRenewer dtRenewer = null;
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
DelegationTokenRenewer.RenewAction<?> action;
|
DelegationTokenRenewer.RenewAction<?> action;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public URI getCanonicalUri() {
|
||||||
|
return super.getCanonicalUri();
|
||||||
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
protected synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
|
protected synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
|
||||||
if (dtRenewer == null) {
|
if (dtRenewer == null) {
|
||||||
@ -142,7 +144,6 @@ protected synchronized void addRenewAction(final WebHdfsFileSystem webhdfs) {
|
|||||||
public static boolean isEnabled(final Configuration conf, final Log log) {
|
public static boolean isEnabled(final Configuration conf, final Log log) {
|
||||||
final boolean b = conf.getBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY,
|
final boolean b = conf.getBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY,
|
||||||
DFSConfigKeys.DFS_WEBHDFS_ENABLED_DEFAULT);
|
DFSConfigKeys.DFS_WEBHDFS_ENABLED_DEFAULT);
|
||||||
log.info(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY + " = " + b);
|
|
||||||
return b;
|
return b;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -490,7 +491,8 @@ private HttpURLConnection openHttpUrlConnection(final URL url)
|
|||||||
throws IOException {
|
throws IOException {
|
||||||
final HttpURLConnection conn;
|
final HttpURLConnection conn;
|
||||||
try {
|
try {
|
||||||
conn = (HttpURLConnection) connectionFactory.openConnection(op, url);
|
conn = (HttpURLConnection) connectionFactory.openConnection(url,
|
||||||
|
op.getRequireAuth());
|
||||||
} catch (AuthenticationException e) {
|
} catch (AuthenticationException e) {
|
||||||
throw new IOException(e);
|
throw new IOException(e);
|
||||||
}
|
}
|
||||||
@ -986,7 +988,8 @@ public <T extends TokenIdentifier> void setDelegationToken(
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized long renewDelegationToken(final Token<?> token
|
@Override
|
||||||
|
public synchronized long renewDelegationToken(final Token<?> token
|
||||||
) throws IOException {
|
) throws IOException {
|
||||||
final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
|
final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
|
||||||
TokenArgumentParam dtargParam = new TokenArgumentParam(
|
TokenArgumentParam dtargParam = new TokenArgumentParam(
|
||||||
@ -995,7 +998,8 @@ private synchronized long renewDelegationToken(final Token<?> token
|
|||||||
return (Long) m.get("long");
|
return (Long) m.get("long");
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void cancelDelegationToken(final Token<?> token
|
@Override
|
||||||
|
public synchronized void cancelDelegationToken(final Token<?> token
|
||||||
) throws IOException {
|
) throws IOException {
|
||||||
final HttpOpParam.Op op = PutOpParam.Op.CANCELDELEGATIONTOKEN;
|
final HttpOpParam.Op op = PutOpParam.Op.CANCELDELEGATIONTOKEN;
|
||||||
TokenArgumentParam dtargParam = new TokenArgumentParam(
|
TokenArgumentParam dtargParam = new TokenArgumentParam(
|
||||||
@ -1041,57 +1045,4 @@ public MD5MD5CRC32FileChecksum getFileChecksum(final Path p
|
|||||||
final Map<?, ?> m = run(op, p);
|
final Map<?, ?> m = run(op, p);
|
||||||
return JsonUtil.toMD5MD5CRC32FileChecksum(m);
|
return JsonUtil.toMD5MD5CRC32FileChecksum(m);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Delegation token renewer. */
|
|
||||||
public static class DtRenewer extends TokenRenewer {
|
|
||||||
@Override
|
|
||||||
public boolean handleKind(Text kind) {
|
|
||||||
return kind.equals(TOKEN_KIND);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean isManaged(Token<?> token) throws IOException {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static WebHdfsFileSystem getWebHdfs(
|
|
||||||
final Token<?> token, final Configuration conf) throws IOException {
|
|
||||||
|
|
||||||
final InetSocketAddress nnAddr = SecurityUtil.getTokenServiceAddr(token);
|
|
||||||
final URI uri = DFSUtil.createUri(WebHdfsFileSystem.SCHEME, nnAddr);
|
|
||||||
return (WebHdfsFileSystem)FileSystem.get(uri, conf);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long renew(final Token<?> token, final Configuration conf
|
|
||||||
) throws IOException, InterruptedException {
|
|
||||||
return getWebHdfs(token, conf).renewDelegationToken(token);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void cancel(final Token<?> token, final Configuration conf
|
|
||||||
) throws IOException, InterruptedException {
|
|
||||||
getWebHdfs(token, conf).cancelDelegationToken(token);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class WebHdfsDelegationTokenSelector
|
|
||||||
extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
|
|
||||||
private static final DelegationTokenSelector hdfsTokenSelector =
|
|
||||||
new DelegationTokenSelector();
|
|
||||||
|
|
||||||
public WebHdfsDelegationTokenSelector() {
|
|
||||||
super(TOKEN_KIND);
|
|
||||||
}
|
|
||||||
|
|
||||||
Token<DelegationTokenIdentifier> selectToken(URI nnUri,
|
|
||||||
Collection<Token<?>> tokens, Configuration conf) {
|
|
||||||
Token<DelegationTokenIdentifier> token =
|
|
||||||
selectToken(SecurityUtil.buildTokenService(nnUri), tokens);
|
|
||||||
if (token == null) {
|
|
||||||
token = hdfsTokenSelector.selectToken(nnUri, tokens, conf);
|
|
||||||
}
|
|
||||||
return token;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -13,5 +13,4 @@
|
|||||||
#
|
#
|
||||||
org.apache.hadoop.hdfs.DFSClient$Renewer
|
org.apache.hadoop.hdfs.DFSClient$Renewer
|
||||||
org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer
|
org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer
|
||||||
org.apache.hadoop.hdfs.web.HftpFileSystem$TokenManager
|
org.apache.hadoop.hdfs.web.TokenAspect$TokenManager
|
||||||
org.apache.hadoop.hdfs.web.WebHdfsFileSystem$DtRenewer
|
|
||||||
|
@ -1516,6 +1516,18 @@
|
|||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>dfs.namenode.path.based.cache.retry.interval.ms</name>
|
||||||
|
<value>60000</value>
|
||||||
|
<description>
|
||||||
|
When the NameNode needs to uncache something that is cached, or cache
|
||||||
|
something that is not cached, it must direct the DataNodes to do so by
|
||||||
|
sending a DNA_CACHE or DNA_UNCACHE command in response to a DataNode
|
||||||
|
heartbeat. This parameter controls how frequently the NameNode will
|
||||||
|
resend these commands.
|
||||||
|
</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>dfs.datanode.fsdatasetcache.max.threads.per.volume</name>
|
<name>dfs.datanode.fsdatasetcache.max.threads.per.volume</name>
|
||||||
<value>4</value>
|
<value>4</value>
|
||||||
|
@ -30,6 +30,12 @@
|
|||||||
</div>
|
</div>
|
||||||
<div id="panel"></div>
|
<div id="panel"></div>
|
||||||
</div>
|
</div>
|
||||||
|
<div class="row">
|
||||||
|
<hr />
|
||||||
|
<div class="col-xs-2"><p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p></div>
|
||||||
|
<div class="col-xs-1 pull-right"><a style="color: #ddd" href="dfshealth.jsp">Legacy UI</a></div>
|
||||||
|
</div>
|
||||||
|
|
||||||
<script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
|
<script type="text/javascript" src="//ajax.googleapis.com/ajax/libs/jquery/2.0.3/jquery.min.js">
|
||||||
</script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
|
</script><script type="text/javascript" src="//netdna.bootstrapcdn.com/bootstrap/3.0.0/js/bootstrap.min.js">
|
||||||
</script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
|
</script><script type="text/javascript" src="/static/dust-full-2.0.0.min.js">
|
||||||
@ -37,7 +43,5 @@
|
|||||||
</script><script type="text/javascript" src="dfs-dust.js">
|
</script><script type="text/javascript" src="dfs-dust.js">
|
||||||
</script><script type="text/javascript" src="dfshealth.js">
|
</script><script type="text/javascript" src="dfshealth.js">
|
||||||
</script>
|
</script>
|
||||||
<hr />
|
|
||||||
<p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
|
|
||||||
</body>
|
</body>
|
||||||
</html>
|
</html>
|
||||||
|
@ -70,6 +70,6 @@
|
|||||||
<hr/>
|
<hr/>
|
||||||
<h3>Startup Progress</h3>
|
<h3>Startup Progress</h3>
|
||||||
<% healthjsp.generateStartupProgress(out, nn.getStartupProgress()); %>
|
<% healthjsp.generateStartupProgress(out, nn.getStartupProgress()); %>
|
||||||
<%
|
<hr/><p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013. <a href="dfshealth.html">New UI</a></p>
|
||||||
out.println(ServletUtil.htmlFooter());
|
</body>
|
||||||
%>
|
</html>
|
||||||
|
@ -14,22 +14,22 @@
|
|||||||
See the License for the specific language governing permissions and
|
See the License for the specific language governing permissions and
|
||||||
limitations under the License.
|
limitations under the License.
|
||||||
-->
|
-->
|
||||||
<meta HTTP-EQUIV="REFRESH" content="0;url=dfshealth.jsp"/>
|
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
|
||||||
<html>
|
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
|
||||||
|
<html xmlns="http://www.w3.org/1999/xhtml">
|
||||||
<head>
|
<head>
|
||||||
|
<meta http-equiv="REFRESH" content="1;url=dfshealth.jsp" />
|
||||||
<title>Hadoop Administration</title>
|
<title>Hadoop Administration</title>
|
||||||
</head>
|
</head>
|
||||||
|
|
||||||
<body>
|
<body>
|
||||||
|
<script type="text/javascript">
|
||||||
|
//<![CDATA[
|
||||||
|
window.location.href='dfshealth.html';
|
||||||
|
//]]>
|
||||||
|
</script>
|
||||||
<h1>Hadoop Administration</h1>
|
<h1>Hadoop Administration</h1>
|
||||||
|
|
||||||
<ul>
|
<ul>
|
||||||
|
|
||||||
<li><a href="dfshealth.jsp">DFS Health/Status</a></li>
|
<li><a href="dfshealth.jsp">DFS Health/Status</a></li>
|
||||||
|
|
||||||
</ul>
|
</ul>
|
||||||
|
|
||||||
</body>
|
</body>
|
||||||
|
|
||||||
</html>
|
</html>
|
||||||
|
@ -26,10 +26,14 @@
|
|||||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
|
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This tests pipeline recovery related client protocol works correct or not.
|
* This tests pipeline recovery related client protocol works correct or not.
|
||||||
*/
|
*/
|
||||||
@ -112,4 +116,55 @@ public class TestClientProtocolForPipelineRecovery {
|
|||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Test whether corrupt replicas are detected correctly during pipeline
|
||||||
|
* recoveries.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testPipelineRecoveryForLastBlock() throws IOException {
|
||||||
|
DFSClientFaultInjector faultInjector
|
||||||
|
= Mockito.mock(DFSClientFaultInjector.class);
|
||||||
|
DFSClientFaultInjector oldInjector = DFSClientFaultInjector.instance;
|
||||||
|
DFSClientFaultInjector.instance = faultInjector;
|
||||||
|
Configuration conf = new HdfsConfiguration();
|
||||||
|
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 3);
|
||||||
|
MiniDFSCluster cluster = null;
|
||||||
|
|
||||||
|
try {
|
||||||
|
int numDataNodes = 3;
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
FileSystem fileSys = cluster.getFileSystem();
|
||||||
|
|
||||||
|
Path file = new Path("dataprotocol1.dat");
|
||||||
|
Mockito.when(faultInjector.failPacket()).thenReturn(true);
|
||||||
|
try {
|
||||||
|
DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
|
||||||
|
} catch (IOException e) {
|
||||||
|
// completeFile() should fail.
|
||||||
|
Assert.assertTrue(e.getMessage().startsWith("Unable to close file"));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// At this point, NN let data corruption to happen.
|
||||||
|
// Before failing test, try reading the file. It should fail.
|
||||||
|
FSDataInputStream in = fileSys.open(file);
|
||||||
|
try {
|
||||||
|
int c = in.read();
|
||||||
|
// Test will fail with BlockMissingException if NN does not update the
|
||||||
|
// replica state based on the latest report.
|
||||||
|
} catch (org.apache.hadoop.hdfs.BlockMissingException bme) {
|
||||||
|
Assert.fail("Block is missing because the file was closed with"
|
||||||
|
+ " corrupt replicas.");
|
||||||
|
}
|
||||||
|
Assert.fail("The file was closed with corrupt replicas, but read still"
|
||||||
|
+ " works!");
|
||||||
|
} finally {
|
||||||
|
DFSClientFaultInjector.instance = oldInjector;
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -86,6 +86,9 @@ public void testQuotaCommands() throws Exception {
|
|||||||
// Space quotas
|
// Space quotas
|
||||||
final int DEFAULT_BLOCK_SIZE = 512;
|
final int DEFAULT_BLOCK_SIZE = 512;
|
||||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE);
|
||||||
|
// Make it relinquish locks. When run serially, the result should
|
||||||
|
// be identical.
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
|
||||||
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||||
final FileSystem fs = cluster.getFileSystem();
|
final FileSystem fs = cluster.getFileSystem();
|
||||||
assertTrue("Not a HDFS: "+fs.getUri(),
|
assertTrue("Not a HDFS: "+fs.getUri(),
|
||||||
@ -350,6 +353,7 @@ public Object run() throws Exception {
|
|||||||
}
|
}
|
||||||
assertTrue(hasException);
|
assertTrue(hasException);
|
||||||
|
|
||||||
|
assertEquals(4, cluster.getNamesystem().getFSDirectory().getYieldCount());
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
@ -360,6 +364,9 @@ public Object run() throws Exception {
|
|||||||
@Test
|
@Test
|
||||||
public void testNamespaceCommands() throws Exception {
|
public void testNamespaceCommands() throws Exception {
|
||||||
final Configuration conf = new HdfsConfiguration();
|
final Configuration conf = new HdfsConfiguration();
|
||||||
|
// Make it relinquish locks. When run serially, the result should
|
||||||
|
// be identical.
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
|
||||||
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||||
final FileSystem fs = cluster.getFileSystem();
|
final FileSystem fs = cluster.getFileSystem();
|
||||||
assertTrue("Not a HDFS: "+fs.getUri(),
|
assertTrue("Not a HDFS: "+fs.getUri(),
|
||||||
@ -515,6 +522,7 @@ public void testNamespaceCommands() throws Exception {
|
|||||||
c = dfs.getContentSummary(quotaDir1);
|
c = dfs.getContentSummary(quotaDir1);
|
||||||
assertEquals(c.getDirectoryCount(), 6);
|
assertEquals(c.getDirectoryCount(), 6);
|
||||||
assertEquals(c.getQuota(), 6);
|
assertEquals(c.getQuota(), 6);
|
||||||
|
assertEquals(14, cluster.getNamesystem().getFSDirectory().getYieldCount());
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
@ -532,6 +540,9 @@ public void testSpaceCommands() throws Exception {
|
|||||||
// set a smaller block size so that we can test with smaller
|
// set a smaller block size so that we can test with smaller
|
||||||
// diskspace quotas
|
// diskspace quotas
|
||||||
conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
|
conf.set(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, "512");
|
||||||
|
// Make it relinquish locks. When run serially, the result should
|
||||||
|
// be identical.
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
|
||||||
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
||||||
final FileSystem fs = cluster.getFileSystem();
|
final FileSystem fs = cluster.getFileSystem();
|
||||||
assertTrue("Not a HDFS: "+fs.getUri(),
|
assertTrue("Not a HDFS: "+fs.getUri(),
|
||||||
@ -764,6 +775,7 @@ public void testSpaceCommands() throws Exception {
|
|||||||
assertEquals(c.getSpaceConsumed(),
|
assertEquals(c.getSpaceConsumed(),
|
||||||
(sizeFactorA + sizeFactorB + sizeFactorC) * fileSpace);
|
(sizeFactorA + sizeFactorB + sizeFactorC) * fileSpace);
|
||||||
|
|
||||||
|
assertEquals(20, cluster.getNamesystem().getFSDirectory().getYieldCount());
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
@ -905,6 +917,9 @@ public void testMultipleFilesSmallerThanOneBlock() throws Exception {
|
|||||||
final int BLOCK_SIZE = 6 * 1024;
|
final int BLOCK_SIZE = 6 * 1024;
|
||||||
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
|
conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
|
||||||
|
// Make it relinquish locks. When run serially, the result should
|
||||||
|
// be identical.
|
||||||
|
conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);
|
||||||
MiniDFSCluster cluster =
|
MiniDFSCluster cluster =
|
||||||
new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
@ -971,6 +986,7 @@ public void testMultipleFilesSmallerThanOneBlock() throws Exception {
|
|||||||
exceededQuota = true;
|
exceededQuota = true;
|
||||||
}
|
}
|
||||||
assertTrue("Quota not exceeded", exceededQuota);
|
assertTrue("Quota not exceeded", exceededQuota);
|
||||||
|
assertEquals(2, cluster.getNamesystem().getFSDirectory().getYieldCount());
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
|
@ -28,8 +28,10 @@
|
|||||||
|
|
||||||
import java.io.FileInputStream;
|
import java.io.FileInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.MappedByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.channels.FileChannel;
|
import java.nio.channels.FileChannel;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
@ -49,7 +51,6 @@
|
|||||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.PageRounder;
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.PageRounder;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
|
||||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock.Mlocker;
|
|
||||||
import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
|
import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
|
||||||
@ -60,6 +61,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
|
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
||||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||||
|
import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
|
||||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
import org.apache.hadoop.test.MetricsAsserts;
|
import org.apache.hadoop.test.MetricsAsserts;
|
||||||
@ -87,8 +89,7 @@ public class TestFsDatasetCache {
|
|||||||
private static FsDatasetSpi<?> fsd;
|
private static FsDatasetSpi<?> fsd;
|
||||||
private static DatanodeProtocolClientSideTranslatorPB spyNN;
|
private static DatanodeProtocolClientSideTranslatorPB spyNN;
|
||||||
private static PageRounder rounder = new PageRounder();
|
private static PageRounder rounder = new PageRounder();
|
||||||
|
private static CacheManipulator prevCacheManipulator;
|
||||||
private Mlocker mlocker;
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
@ -96,6 +97,8 @@ public void setUp() throws Exception {
|
|||||||
assumeTrue(NativeIO.getMemlockLimit() >= CACHE_CAPACITY);
|
assumeTrue(NativeIO.getMemlockLimit() >= CACHE_CAPACITY);
|
||||||
conf = new HdfsConfiguration();
|
conf = new HdfsConfiguration();
|
||||||
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY, true);
|
conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY, true);
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_RETRY_INTERVAL_MS,
|
||||||
|
500);
|
||||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
||||||
conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||||
CACHE_CAPACITY);
|
CACHE_CAPACITY);
|
||||||
@ -113,8 +116,19 @@ public void setUp() throws Exception {
|
|||||||
fsd = dn.getFSDataset();
|
fsd = dn.getFSDataset();
|
||||||
|
|
||||||
spyNN = DataNodeTestUtils.spyOnBposToNN(dn, nn);
|
spyNN = DataNodeTestUtils.spyOnBposToNN(dn, nn);
|
||||||
// Save the current mlocker and replace it at the end of the test
|
|
||||||
mlocker = MappableBlock.mlocker;
|
prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
|
||||||
|
|
||||||
|
// Save the current CacheManipulator and replace it at the end of the test
|
||||||
|
// Stub out mlock calls to avoid failing when not enough memory is lockable
|
||||||
|
// by the operating system.
|
||||||
|
NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
|
||||||
|
@Override
|
||||||
|
public void mlock(String identifier,
|
||||||
|
ByteBuffer mmap, long length) throws IOException {
|
||||||
|
LOG.info("mlocking " + identifier);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
@ -125,8 +139,8 @@ public void tearDown() throws Exception {
|
|||||||
if (cluster != null) {
|
if (cluster != null) {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
// Restore the original mlocker
|
// Restore the original CacheManipulator
|
||||||
MappableBlock.mlocker = mlocker;
|
NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void setHeartbeatResponse(DatanodeCommand[] cmds)
|
private static void setHeartbeatResponse(DatanodeCommand[] cmds)
|
||||||
@ -214,8 +228,7 @@ public Boolean get() {
|
|||||||
return expected;
|
return expected;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=600000)
|
private void testCacheAndUncacheBlock() throws Exception {
|
||||||
public void testCacheAndUncacheBlock() throws Exception {
|
|
||||||
LOG.info("beginning testCacheAndUncacheBlock");
|
LOG.info("beginning testCacheAndUncacheBlock");
|
||||||
final int NUM_BLOCKS = 5;
|
final int NUM_BLOCKS = 5;
|
||||||
|
|
||||||
@ -268,6 +281,42 @@ public void testCacheAndUncacheBlock() throws Exception {
|
|||||||
LOG.info("finishing testCacheAndUncacheBlock");
|
LOG.info("finishing testCacheAndUncacheBlock");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test(timeout=600000)
|
||||||
|
public void testCacheAndUncacheBlockSimple() throws Exception {
|
||||||
|
testCacheAndUncacheBlock();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run testCacheAndUncacheBlock with some failures injected into the mlock
|
||||||
|
* call. This tests the ability of the NameNode to resend commands.
|
||||||
|
*/
|
||||||
|
@Test(timeout=600000)
|
||||||
|
public void testCacheAndUncacheBlockWithRetries() throws Exception {
|
||||||
|
CacheManipulator prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
|
||||||
|
|
||||||
|
try {
|
||||||
|
NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
|
||||||
|
private final Set<String> seenIdentifiers = new HashSet<String>();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void mlock(String identifier,
|
||||||
|
ByteBuffer mmap, long length) throws IOException {
|
||||||
|
if (seenIdentifiers.contains(identifier)) {
|
||||||
|
// mlock succeeds the second time.
|
||||||
|
LOG.info("mlocking " + identifier);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
seenIdentifiers.add(identifier);
|
||||||
|
throw new IOException("injecting IOException during mlock of " +
|
||||||
|
identifier);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
testCacheAndUncacheBlock();
|
||||||
|
} finally {
|
||||||
|
NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test(timeout=600000)
|
@Test(timeout=600000)
|
||||||
public void testFilesExceedMaxLockedMemory() throws Exception {
|
public void testFilesExceedMaxLockedMemory() throws Exception {
|
||||||
LOG.info("beginning testFilesExceedMaxLockedMemory");
|
LOG.info("beginning testFilesExceedMaxLockedMemory");
|
||||||
@ -357,10 +406,11 @@ public void testUncachingBlocksBeforeCachingFinishes() throws Exception {
|
|||||||
assertEquals("Unexpected cache capacity", CACHE_CAPACITY, cacheCapacity);
|
assertEquals("Unexpected cache capacity", CACHE_CAPACITY, cacheCapacity);
|
||||||
assertEquals("Unexpected amount of cache used", current, cacheUsed);
|
assertEquals("Unexpected amount of cache used", current, cacheUsed);
|
||||||
|
|
||||||
MappableBlock.mlocker = new MappableBlock.Mlocker() {
|
NativeIO.POSIX.cacheManipulator = new NativeIO.POSIX.CacheManipulator() {
|
||||||
@Override
|
@Override
|
||||||
public void mlock(MappedByteBuffer mmap, long length) throws IOException {
|
public void mlock(String identifier,
|
||||||
LOG.info("An mlock operation is starting.");
|
ByteBuffer mmap, long length) throws IOException {
|
||||||
|
LOG.info("An mlock operation is starting on " + identifier);
|
||||||
try {
|
try {
|
||||||
Thread.sleep(3000);
|
Thread.sleep(3000);
|
||||||
} catch (InterruptedException e) {
|
} catch (InterruptedException e) {
|
||||||
|
@ -93,6 +93,10 @@ public void testCorruptFilesJsp() throws Exception {
|
|||||||
in.close();
|
in.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
Thread.sleep(3000); // Wait for block reports. They shouldn't matter.
|
||||||
|
} catch (InterruptedException ie) {}
|
||||||
|
|
||||||
// verify if all corrupt files were reported to NN
|
// verify if all corrupt files were reported to NN
|
||||||
badFiles = namenode.getNamesystem().listCorruptFileBlocks("/", null);
|
badFiles = namenode.getNamesystem().listCorruptFileBlocks("/", null);
|
||||||
assertTrue("Expecting 3 corrupt files, but got " + badFiles.size(),
|
assertTrue("Expecting 3 corrupt files, but got " + badFiles.size(),
|
||||||
|
@ -33,6 +33,7 @@
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.MappedByteBuffer;
|
import java.nio.MappedByteBuffer;
|
||||||
import java.security.PrivilegedExceptionAction;
|
import java.security.PrivilegedExceptionAction;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.LinkedList;
|
import java.util.LinkedList;
|
||||||
@ -60,6 +61,7 @@
|
|||||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.MappableBlock;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||||
|
import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
|
||||||
import org.apache.hadoop.security.AccessControlException;
|
import org.apache.hadoop.security.AccessControlException;
|
||||||
import org.apache.hadoop.security.UserGroupInformation;
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
import org.apache.hadoop.test.GenericTestUtils;
|
import org.apache.hadoop.test.GenericTestUtils;
|
||||||
@ -81,15 +83,7 @@ public class TestPathBasedCacheRequests {
|
|||||||
static private MiniDFSCluster cluster;
|
static private MiniDFSCluster cluster;
|
||||||
static private DistributedFileSystem dfs;
|
static private DistributedFileSystem dfs;
|
||||||
static private NamenodeProtocols proto;
|
static private NamenodeProtocols proto;
|
||||||
|
static private CacheManipulator prevCacheManipulator;
|
||||||
static {
|
|
||||||
MappableBlock.mlocker = new MappableBlock.Mlocker() {
|
|
||||||
@Override
|
|
||||||
public void mlock(MappedByteBuffer mmap, long length) throws IOException {
|
|
||||||
// Stubbed out for testing
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup() throws Exception {
|
public void setup() throws Exception {
|
||||||
@ -101,6 +95,18 @@ public void setup() throws Exception {
|
|||||||
cluster.waitActive();
|
cluster.waitActive();
|
||||||
dfs = cluster.getFileSystem();
|
dfs = cluster.getFileSystem();
|
||||||
proto = cluster.getNameNodeRpc();
|
proto = cluster.getNameNodeRpc();
|
||||||
|
prevCacheManipulator = NativeIO.POSIX.cacheManipulator;
|
||||||
|
|
||||||
|
// Save the current CacheManipulator and replace it at the end of the test
|
||||||
|
// Stub out mlock calls to avoid failing when not enough memory is lockable
|
||||||
|
// by the operating system.
|
||||||
|
NativeIO.POSIX.cacheManipulator = new CacheManipulator() {
|
||||||
|
@Override
|
||||||
|
public void mlock(String identifier,
|
||||||
|
ByteBuffer mmap, long length) throws IOException {
|
||||||
|
LOG.info("mlocking " + identifier);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
@ -108,6 +114,8 @@ public void teardown() throws Exception {
|
|||||||
if (cluster != null) {
|
if (cluster != null) {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
|
// Restore the original CacheManipulator
|
||||||
|
NativeIO.POSIX.cacheManipulator = prevCacheManipulator;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout=60000)
|
@Test(timeout=60000)
|
||||||
@ -552,8 +560,8 @@ public void testCacheManagerRestart() throws Exception {
|
|||||||
* @throws Exception
|
* @throws Exception
|
||||||
*/
|
*/
|
||||||
private static void waitForCachedBlocks(NameNode nn,
|
private static void waitForCachedBlocks(NameNode nn,
|
||||||
final int expectedCachedBlocks, final int expectedCachedReplicas)
|
final int expectedCachedBlocks, final int expectedCachedReplicas,
|
||||||
throws Exception {
|
final String logString) throws Exception {
|
||||||
final FSNamesystem namesystem = nn.getNamesystem();
|
final FSNamesystem namesystem = nn.getNamesystem();
|
||||||
final CacheManager cacheManager = namesystem.getCacheManager();
|
final CacheManager cacheManager = namesystem.getCacheManager();
|
||||||
LOG.info("Waiting for " + expectedCachedBlocks + " blocks with " +
|
LOG.info("Waiting for " + expectedCachedBlocks + " blocks with " +
|
||||||
@ -581,9 +589,9 @@ public Boolean get() {
|
|||||||
(numCachedReplicas == expectedCachedReplicas)) {
|
(numCachedReplicas == expectedCachedReplicas)) {
|
||||||
return true;
|
return true;
|
||||||
} else {
|
} else {
|
||||||
LOG.info("cached blocks: have " + numCachedBlocks +
|
LOG.info(logString + " cached blocks: have " + numCachedBlocks +
|
||||||
" / " + expectedCachedBlocks);
|
" / " + expectedCachedBlocks + ". " +
|
||||||
LOG.info("cached replicas: have " + numCachedReplicas +
|
"cached replicas: have " + numCachedReplicas +
|
||||||
" / " + expectedCachedReplicas);
|
" / " + expectedCachedReplicas);
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
@ -681,7 +689,7 @@ public void testWaitForCachedReplicas() throws Exception {
|
|||||||
paths.add(p.toUri().getPath());
|
paths.add(p.toUri().getPath());
|
||||||
}
|
}
|
||||||
// Check the initial statistics at the namenode
|
// Check the initial statistics at the namenode
|
||||||
waitForCachedBlocks(namenode, 0, 0);
|
waitForCachedBlocks(namenode, 0, 0, "testWaitForCachedReplicas:0");
|
||||||
// Cache and check each path in sequence
|
// Cache and check each path in sequence
|
||||||
int expected = 0;
|
int expected = 0;
|
||||||
for (int i=0; i<numFiles; i++) {
|
for (int i=0; i<numFiles; i++) {
|
||||||
@ -692,7 +700,8 @@ public void testWaitForCachedReplicas() throws Exception {
|
|||||||
build();
|
build();
|
||||||
nnRpc.addPathBasedCacheDirective(directive);
|
nnRpc.addPathBasedCacheDirective(directive);
|
||||||
expected += numBlocksPerFile;
|
expected += numBlocksPerFile;
|
||||||
waitForCachedBlocks(namenode, expected, expected);
|
waitForCachedBlocks(namenode, expected, expected,
|
||||||
|
"testWaitForCachedReplicas:1");
|
||||||
}
|
}
|
||||||
// Uncache and check each path in sequence
|
// Uncache and check each path in sequence
|
||||||
RemoteIterator<PathBasedCacheDirective> entries =
|
RemoteIterator<PathBasedCacheDirective> entries =
|
||||||
@ -701,7 +710,8 @@ public void testWaitForCachedReplicas() throws Exception {
|
|||||||
PathBasedCacheDirective directive = entries.next();
|
PathBasedCacheDirective directive = entries.next();
|
||||||
nnRpc.removePathBasedCacheDirective(directive.getId());
|
nnRpc.removePathBasedCacheDirective(directive.getId());
|
||||||
expected -= numBlocksPerFile;
|
expected -= numBlocksPerFile;
|
||||||
waitForCachedBlocks(namenode, expected, expected);
|
waitForCachedBlocks(namenode, expected, expected,
|
||||||
|
"testWaitForCachedReplicas:2");
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
@ -735,7 +745,8 @@ public void testAddingPathBasedCacheDirectivesWhenCachingIsDisabled()
|
|||||||
paths.add(p.toUri().getPath());
|
paths.add(p.toUri().getPath());
|
||||||
}
|
}
|
||||||
// Check the initial statistics at the namenode
|
// Check the initial statistics at the namenode
|
||||||
waitForCachedBlocks(namenode, 0, 0);
|
waitForCachedBlocks(namenode, 0, 0,
|
||||||
|
"testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:0");
|
||||||
// Cache and check each path in sequence
|
// Cache and check each path in sequence
|
||||||
int expected = 0;
|
int expected = 0;
|
||||||
for (int i=0; i<numFiles; i++) {
|
for (int i=0; i<numFiles; i++) {
|
||||||
@ -745,10 +756,12 @@ public void testAddingPathBasedCacheDirectivesWhenCachingIsDisabled()
|
|||||||
setPool(pool).
|
setPool(pool).
|
||||||
build();
|
build();
|
||||||
dfs.addPathBasedCacheDirective(directive);
|
dfs.addPathBasedCacheDirective(directive);
|
||||||
waitForCachedBlocks(namenode, expected, 0);
|
waitForCachedBlocks(namenode, expected, 0,
|
||||||
|
"testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:1");
|
||||||
}
|
}
|
||||||
Thread.sleep(20000);
|
Thread.sleep(20000);
|
||||||
waitForCachedBlocks(namenode, expected, 0);
|
waitForCachedBlocks(namenode, expected, 0,
|
||||||
|
"testAddingPathBasedCacheDirectivesWhenCachingIsDisabled:2");
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
@ -781,7 +794,8 @@ public void testWaitForCachedReplicasInDirectory() throws Exception {
|
|||||||
FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
|
FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
|
||||||
(int)BLOCK_SIZE, (short)3, false);
|
(int)BLOCK_SIZE, (short)3, false);
|
||||||
}
|
}
|
||||||
waitForCachedBlocks(namenode, 0, 0);
|
waitForCachedBlocks(namenode, 0, 0,
|
||||||
|
"testWaitForCachedReplicasInDirectory:0");
|
||||||
// cache entire directory
|
// cache entire directory
|
||||||
long id = dfs.addPathBasedCacheDirective(
|
long id = dfs.addPathBasedCacheDirective(
|
||||||
new PathBasedCacheDirective.Builder().
|
new PathBasedCacheDirective.Builder().
|
||||||
@ -789,10 +803,12 @@ public void testWaitForCachedReplicasInDirectory() throws Exception {
|
|||||||
setReplication((short)2).
|
setReplication((short)2).
|
||||||
setPool(pool).
|
setPool(pool).
|
||||||
build());
|
build());
|
||||||
waitForCachedBlocks(namenode, 4, 8);
|
waitForCachedBlocks(namenode, 4, 8,
|
||||||
|
"testWaitForCachedReplicasInDirectory:1");
|
||||||
// remove and watch numCached go to 0
|
// remove and watch numCached go to 0
|
||||||
dfs.removePathBasedCacheDirective(id);
|
dfs.removePathBasedCacheDirective(id);
|
||||||
waitForCachedBlocks(namenode, 0, 0);
|
waitForCachedBlocks(namenode, 0, 0,
|
||||||
|
"testWaitForCachedReplicasInDirectory:2");
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
@ -830,7 +846,7 @@ public void testReplicationFactor() throws Exception {
|
|||||||
FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
|
FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
|
||||||
(int)BLOCK_SIZE, (short)3, false);
|
(int)BLOCK_SIZE, (short)3, false);
|
||||||
}
|
}
|
||||||
waitForCachedBlocks(namenode, 0, 0);
|
waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:0");
|
||||||
checkNumCachedReplicas(dfs, paths, 0, 0);
|
checkNumCachedReplicas(dfs, paths, 0, 0);
|
||||||
// cache directory
|
// cache directory
|
||||||
long id = dfs.addPathBasedCacheDirective(
|
long id = dfs.addPathBasedCacheDirective(
|
||||||
@ -839,7 +855,7 @@ public void testReplicationFactor() throws Exception {
|
|||||||
setReplication((short)1).
|
setReplication((short)1).
|
||||||
setPool(pool).
|
setPool(pool).
|
||||||
build());
|
build());
|
||||||
waitForCachedBlocks(namenode, 4, 4);
|
waitForCachedBlocks(namenode, 4, 4, "testReplicationFactor:1");
|
||||||
checkNumCachedReplicas(dfs, paths, 4, 4);
|
checkNumCachedReplicas(dfs, paths, 4, 4);
|
||||||
// step up the replication factor
|
// step up the replication factor
|
||||||
for (int i=2; i<=3; i++) {
|
for (int i=2; i<=3; i++) {
|
||||||
@ -848,7 +864,7 @@ public void testReplicationFactor() throws Exception {
|
|||||||
setId(id).
|
setId(id).
|
||||||
setReplication((short)i).
|
setReplication((short)i).
|
||||||
build());
|
build());
|
||||||
waitForCachedBlocks(namenode, 4, 4*i);
|
waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:2");
|
||||||
checkNumCachedReplicas(dfs, paths, 4, 4*i);
|
checkNumCachedReplicas(dfs, paths, 4, 4*i);
|
||||||
}
|
}
|
||||||
// step it down
|
// step it down
|
||||||
@ -858,12 +874,12 @@ public void testReplicationFactor() throws Exception {
|
|||||||
setId(id).
|
setId(id).
|
||||||
setReplication((short)i).
|
setReplication((short)i).
|
||||||
build());
|
build());
|
||||||
waitForCachedBlocks(namenode, 4, 4*i);
|
waitForCachedBlocks(namenode, 4, 4*i, "testReplicationFactor:3");
|
||||||
checkNumCachedReplicas(dfs, paths, 4, 4*i);
|
checkNumCachedReplicas(dfs, paths, 4, 4*i);
|
||||||
}
|
}
|
||||||
// remove and watch numCached go to 0
|
// remove and watch numCached go to 0
|
||||||
dfs.removePathBasedCacheDirective(id);
|
dfs.removePathBasedCacheDirective(id);
|
||||||
waitForCachedBlocks(namenode, 0, 0);
|
waitForCachedBlocks(namenode, 0, 0, "testReplicationFactor:4");
|
||||||
checkNumCachedReplicas(dfs, paths, 0, 0);
|
checkNumCachedReplicas(dfs, paths, 0, 0);
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
|
@ -38,6 +38,7 @@
|
|||||||
import org.apache.hadoop.hdfs.DFSUtil;
|
import org.apache.hadoop.hdfs.DFSUtil;
|
||||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
||||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
||||||
@ -47,7 +48,10 @@
|
|||||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
|
import org.apache.hadoop.hdfs.server.namenode.INodeDirectoryWithQuota;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.Quota;
|
import org.apache.hadoop.hdfs.server.namenode.Quota;
|
||||||
|
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
|
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectoryWithSnapshot.DirectoryDiffList;
|
||||||
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
@ -949,4 +953,54 @@ public void testDeleteSnapshotCommandWithIllegalArguments() throws Exception {
|
|||||||
psOut.close();
|
psOut.close();
|
||||||
out.close();
|
out.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* OP_DELETE_SNAPSHOT edits op was not decrementing the safemode threshold on
|
||||||
|
* restart in HA mode. HDFS-5504
|
||||||
|
*/
|
||||||
|
@Test(timeout = 60000)
|
||||||
|
public void testHANNRestartAfterSnapshotDeletion() throws Exception {
|
||||||
|
hdfs.close();
|
||||||
|
cluster.shutdown();
|
||||||
|
conf = new Configuration();
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf)
|
||||||
|
.nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(1)
|
||||||
|
.build();
|
||||||
|
cluster.transitionToActive(0);
|
||||||
|
// stop the standby namenode
|
||||||
|
NameNode snn = cluster.getNameNode(1);
|
||||||
|
snn.stop();
|
||||||
|
|
||||||
|
hdfs = (DistributedFileSystem) HATestUtil
|
||||||
|
.configureFailoverFs(cluster, conf);
|
||||||
|
Path dir = new Path("/dir");
|
||||||
|
Path subDir = new Path(dir, "sub");
|
||||||
|
hdfs.mkdirs(dir);
|
||||||
|
hdfs.allowSnapshot(dir);
|
||||||
|
for (int i = 0; i < 5; i++) {
|
||||||
|
DFSTestUtil.createFile(hdfs, new Path(subDir, "" + i), 100, (short) 1,
|
||||||
|
1024L);
|
||||||
|
}
|
||||||
|
|
||||||
|
// take snapshot
|
||||||
|
hdfs.createSnapshot(dir, "s0");
|
||||||
|
|
||||||
|
// delete the subdir
|
||||||
|
hdfs.delete(subDir, true);
|
||||||
|
|
||||||
|
// roll the edit log
|
||||||
|
NameNode ann = cluster.getNameNode(0);
|
||||||
|
ann.getRpcServer().rollEditLog();
|
||||||
|
|
||||||
|
hdfs.deleteSnapshot(dir, "s0");
|
||||||
|
// wait for the blocks deletion at namenode
|
||||||
|
Thread.sleep(2000);
|
||||||
|
|
||||||
|
NameNodeAdapter.abortEditLogs(ann);
|
||||||
|
cluster.restartNameNode(0, false);
|
||||||
|
cluster.transitionToActive(0);
|
||||||
|
|
||||||
|
// wait till the cluster becomes active
|
||||||
|
cluster.waitClusterUp();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -22,7 +22,6 @@
|
|||||||
import static org.junit.Assert.*;
|
import static org.junit.Assert.*;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.lang.reflect.Field;
|
|
||||||
import java.net.ServerSocket;
|
import java.net.ServerSocket;
|
||||||
import java.net.Socket;
|
import java.net.Socket;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
@ -40,6 +39,7 @@
|
|||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.mockito.internal.util.reflection.Whitebox;
|
||||||
|
|
||||||
public class TestHftpDelegationToken {
|
public class TestHftpDelegationToken {
|
||||||
|
|
||||||
@ -71,9 +71,8 @@ public FileSystem run() throws Exception {
|
|||||||
});
|
});
|
||||||
assertSame("wrong kind of file system", HftpFileSystem.class,
|
assertSame("wrong kind of file system", HftpFileSystem.class,
|
||||||
fs.getClass());
|
fs.getClass());
|
||||||
Field renewToken = HftpFileSystem.class.getDeclaredField("renewToken");
|
assertSame("wrong token", token,
|
||||||
renewToken.setAccessible(true);
|
Whitebox.getInternalState(fs, "renewToken"));
|
||||||
assertSame("wrong token", token, renewToken.get(fs));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -81,7 +80,7 @@ public void testSelectHftpDelegationToken() throws Exception {
|
|||||||
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
||||||
|
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
conf.setClass("fs.hftp.impl", MyHftpFileSystem.class, FileSystem.class);
|
conf.setClass("fs.hftp.impl", HftpFileSystem.class, FileSystem.class);
|
||||||
|
|
||||||
int httpPort = 80;
|
int httpPort = 80;
|
||||||
int httpsPort = 443;
|
int httpsPort = 443;
|
||||||
@ -90,21 +89,21 @@ public void testSelectHftpDelegationToken() throws Exception {
|
|||||||
|
|
||||||
// test with implicit default port
|
// test with implicit default port
|
||||||
URI fsUri = URI.create("hftp://localhost");
|
URI fsUri = URI.create("hftp://localhost");
|
||||||
MyHftpFileSystem fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
HftpFileSystem fs = (HftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
||||||
assertEquals(httpPort, fs.getCanonicalUri().getPort());
|
assertEquals(httpPort, fs.getCanonicalUri().getPort());
|
||||||
checkTokenSelection(fs, httpPort, conf);
|
checkTokenSelection(fs, httpPort, conf);
|
||||||
|
|
||||||
// test with explicit default port
|
// test with explicit default port
|
||||||
// Make sure it uses the port from the hftp URI.
|
// Make sure it uses the port from the hftp URI.
|
||||||
fsUri = URI.create("hftp://localhost:"+httpPort);
|
fsUri = URI.create("hftp://localhost:"+httpPort);
|
||||||
fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
fs = (HftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
||||||
assertEquals(httpPort, fs.getCanonicalUri().getPort());
|
assertEquals(httpPort, fs.getCanonicalUri().getPort());
|
||||||
checkTokenSelection(fs, httpPort, conf);
|
checkTokenSelection(fs, httpPort, conf);
|
||||||
|
|
||||||
// test with non-default port
|
// test with non-default port
|
||||||
// Make sure it uses the port from the hftp URI.
|
// Make sure it uses the port from the hftp URI.
|
||||||
fsUri = URI.create("hftp://localhost:"+(httpPort+1));
|
fsUri = URI.create("hftp://localhost:"+(httpPort+1));
|
||||||
fs = (MyHftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
fs = (HftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
||||||
assertEquals(httpPort+1, fs.getCanonicalUri().getPort());
|
assertEquals(httpPort+1, fs.getCanonicalUri().getPort());
|
||||||
checkTokenSelection(fs, httpPort + 1, conf);
|
checkTokenSelection(fs, httpPort + 1, conf);
|
||||||
|
|
||||||
@ -116,7 +115,7 @@ public void testSelectHsftpDelegationToken() throws Exception {
|
|||||||
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
||||||
|
|
||||||
Configuration conf = new Configuration();
|
Configuration conf = new Configuration();
|
||||||
conf.setClass("fs.hsftp.impl", MyHsftpFileSystem.class, FileSystem.class);
|
conf.setClass("fs.hsftp.impl", HsftpFileSystem.class, FileSystem.class);
|
||||||
|
|
||||||
int httpPort = 80;
|
int httpPort = 80;
|
||||||
int httpsPort = 443;
|
int httpsPort = 443;
|
||||||
@ -125,19 +124,19 @@ public void testSelectHsftpDelegationToken() throws Exception {
|
|||||||
|
|
||||||
// test with implicit default port
|
// test with implicit default port
|
||||||
URI fsUri = URI.create("hsftp://localhost");
|
URI fsUri = URI.create("hsftp://localhost");
|
||||||
MyHsftpFileSystem fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
HsftpFileSystem fs = (HsftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
||||||
assertEquals(httpsPort, fs.getCanonicalUri().getPort());
|
assertEquals(httpsPort, fs.getCanonicalUri().getPort());
|
||||||
checkTokenSelection(fs, httpsPort, conf);
|
checkTokenSelection(fs, httpsPort, conf);
|
||||||
|
|
||||||
// test with explicit default port
|
// test with explicit default port
|
||||||
fsUri = URI.create("hsftp://localhost:"+httpsPort);
|
fsUri = URI.create("hsftp://localhost:"+httpsPort);
|
||||||
fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
fs = (HsftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
||||||
assertEquals(httpsPort, fs.getCanonicalUri().getPort());
|
assertEquals(httpsPort, fs.getCanonicalUri().getPort());
|
||||||
checkTokenSelection(fs, httpsPort, conf);
|
checkTokenSelection(fs, httpsPort, conf);
|
||||||
|
|
||||||
// test with non-default port
|
// test with non-default port
|
||||||
fsUri = URI.create("hsftp://localhost:"+(httpsPort+1));
|
fsUri = URI.create("hsftp://localhost:"+(httpsPort+1));
|
||||||
fs = (MyHsftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
fs = (HsftpFileSystem) FileSystem.newInstance(fsUri, conf);
|
||||||
assertEquals(httpsPort+1, fs.getCanonicalUri().getPort());
|
assertEquals(httpsPort+1, fs.getCanonicalUri().getPort());
|
||||||
checkTokenSelection(fs, httpsPort+1, conf);
|
checkTokenSelection(fs, httpsPort+1, conf);
|
||||||
|
|
||||||
@ -197,6 +196,9 @@ private void checkTokenSelection(HftpFileSystem fs,
|
|||||||
UserGroupInformation ugi =
|
UserGroupInformation ugi =
|
||||||
UserGroupInformation.createUserForTesting(fs.getUri().getAuthority(), new String[]{});
|
UserGroupInformation.createUserForTesting(fs.getUri().getAuthority(), new String[]{});
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
TokenAspect<HftpFileSystem> aspect = (TokenAspect<HftpFileSystem>) Whitebox.getInternalState(fs, "tokenAspect");
|
||||||
|
|
||||||
// use ip-based tokens
|
// use ip-based tokens
|
||||||
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
||||||
|
|
||||||
@ -208,7 +210,7 @@ private void checkTokenSelection(HftpFileSystem fs,
|
|||||||
ugi.addToken(hdfsToken);
|
ugi.addToken(hdfsToken);
|
||||||
|
|
||||||
// test fallback to hdfs token
|
// test fallback to hdfs token
|
||||||
Token<?> token = fs.selectDelegationToken(ugi);
|
Token<?> token = aspect.selectDelegationToken(ugi);
|
||||||
assertNotNull(token);
|
assertNotNull(token);
|
||||||
assertEquals(hdfsToken, token);
|
assertEquals(hdfsToken, token);
|
||||||
|
|
||||||
@ -217,13 +219,13 @@ private void checkTokenSelection(HftpFileSystem fs,
|
|||||||
new byte[0], new byte[0],
|
new byte[0], new byte[0],
|
||||||
HftpFileSystem.TOKEN_KIND, new Text("127.0.0.1:"+port));
|
HftpFileSystem.TOKEN_KIND, new Text("127.0.0.1:"+port));
|
||||||
ugi.addToken(hftpToken);
|
ugi.addToken(hftpToken);
|
||||||
token = fs.selectDelegationToken(ugi);
|
token = aspect.selectDelegationToken(ugi);
|
||||||
assertNotNull(token);
|
assertNotNull(token);
|
||||||
assertEquals(hftpToken, token);
|
assertEquals(hftpToken, token);
|
||||||
|
|
||||||
// switch to using host-based tokens, no token should match
|
// switch to using host-based tokens, no token should match
|
||||||
SecurityUtilTestHelper.setTokenServiceUseIp(false);
|
SecurityUtilTestHelper.setTokenServiceUseIp(false);
|
||||||
token = fs.selectDelegationToken(ugi);
|
token = aspect.selectDelegationToken(ugi);
|
||||||
assertNull(token);
|
assertNull(token);
|
||||||
|
|
||||||
// test fallback to hdfs token
|
// test fallback to hdfs token
|
||||||
@ -232,7 +234,7 @@ private void checkTokenSelection(HftpFileSystem fs,
|
|||||||
DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
|
DelegationTokenIdentifier.HDFS_DELEGATION_KIND,
|
||||||
new Text("localhost:8020"));
|
new Text("localhost:8020"));
|
||||||
ugi.addToken(hdfsToken);
|
ugi.addToken(hdfsToken);
|
||||||
token = fs.selectDelegationToken(ugi);
|
token = aspect.selectDelegationToken(ugi);
|
||||||
assertNotNull(token);
|
assertNotNull(token);
|
||||||
assertEquals(hdfsToken, token);
|
assertEquals(hdfsToken, token);
|
||||||
|
|
||||||
@ -241,36 +243,8 @@ private void checkTokenSelection(HftpFileSystem fs,
|
|||||||
new byte[0], new byte[0],
|
new byte[0], new byte[0],
|
||||||
HftpFileSystem.TOKEN_KIND, new Text("localhost:"+port));
|
HftpFileSystem.TOKEN_KIND, new Text("localhost:"+port));
|
||||||
ugi.addToken(hftpToken);
|
ugi.addToken(hftpToken);
|
||||||
token = fs.selectDelegationToken(ugi);
|
token = aspect.selectDelegationToken(ugi);
|
||||||
assertNotNull(token);
|
assertNotNull(token);
|
||||||
assertEquals(hftpToken, token);
|
assertEquals(hftpToken, token);
|
||||||
}
|
}
|
||||||
|
|
||||||
static class MyHftpFileSystem extends HftpFileSystem {
|
|
||||||
@Override
|
|
||||||
public URI getCanonicalUri() {
|
|
||||||
return super.getCanonicalUri();
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
public int getDefaultPort() {
|
|
||||||
return super.getDefaultPort();
|
|
||||||
}
|
|
||||||
// don't automatically get a token
|
|
||||||
@Override
|
|
||||||
protected void initDelegationToken() throws IOException {}
|
|
||||||
}
|
|
||||||
|
|
||||||
static class MyHsftpFileSystem extends HsftpFileSystem {
|
|
||||||
@Override
|
|
||||||
public URI getCanonicalUri() {
|
|
||||||
return super.getCanonicalUri();
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
public int getDefaultPort() {
|
|
||||||
return super.getDefaultPort();
|
|
||||||
}
|
|
||||||
// don't automatically get a token
|
|
||||||
@Override
|
|
||||||
protected void initDelegationToken() throws IOException {}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,298 @@
|
|||||||
|
/**
|
||||||
|
* 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.web;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
|
import static org.mockito.Matchers.anyString;
|
||||||
|
import static org.mockito.Mockito.doReturn;
|
||||||
|
import static org.mockito.Mockito.doThrow;
|
||||||
|
import static org.mockito.Mockito.never;
|
||||||
|
import static org.mockito.Mockito.spy;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.URI;
|
||||||
|
import java.net.URISyntaxException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.DelegationTokenRenewer;
|
||||||
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.permission.FsPermission;
|
||||||
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.security.SecurityUtilTestHelper;
|
||||||
|
import org.apache.hadoop.security.UserGroupInformation;
|
||||||
|
import org.apache.hadoop.security.token.Token;
|
||||||
|
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||||
|
import org.apache.hadoop.util.Progressable;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
import org.mockito.internal.util.reflection.Whitebox;
|
||||||
|
|
||||||
|
public class TestTokenAspect {
|
||||||
|
|
||||||
|
private static class DummyFs extends FileSystem implements
|
||||||
|
DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
|
||||||
|
|
||||||
|
private static final Text TOKEN_KIND = new Text("DummyFS Token");
|
||||||
|
private boolean emulateSecurityEnabled;
|
||||||
|
private TokenAspect<DummyFs> tokenAspect;
|
||||||
|
private UserGroupInformation ugi = UserGroupInformation
|
||||||
|
.createUserForTesting("foo", new String[] { "bar" });
|
||||||
|
private URI uri;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FSDataOutputStream append(Path f, int bufferSize,
|
||||||
|
Progressable progress) throws IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cancelDelegationToken(Token<?> token) throws IOException {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FSDataOutputStream create(Path f, FsPermission permission,
|
||||||
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||||
|
Progressable progress) throws IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean delete(Path f, boolean recursive) throws IOException {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public URI getCanonicalUri() {
|
||||||
|
return super.getCanonicalUri();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FileStatus getFileStatus(Path f) throws IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Token<?> getRenewToken() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public URI getUri() {
|
||||||
|
return uri;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Path getWorkingDirectory() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void initialize(URI name, Configuration conf) throws IOException {
|
||||||
|
super.initialize(name, conf);
|
||||||
|
setConf(conf);
|
||||||
|
try {
|
||||||
|
this.uri = new URI(name.getScheme(), name.getAuthority(), null, null,
|
||||||
|
null);
|
||||||
|
} catch (URISyntaxException e) {
|
||||||
|
throw new IllegalArgumentException(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
tokenAspect = new TokenAspect<DummyFs>(this, DummyFs.TOKEN_KIND);
|
||||||
|
if (emulateSecurityEnabled || UserGroupInformation.isSecurityEnabled()) {
|
||||||
|
tokenAspect.initDelegationToken(ugi);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FileStatus[] listStatus(Path f) throws FileNotFoundException,
|
||||||
|
IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean rename(Path src, Path dst) throws IOException {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long renewDelegationToken(Token<?> token) throws IOException {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setWorkingDirectory(Path new_dir) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetRemoteToken() throws IOException, URISyntaxException {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
UserGroupInformation.setConfiguration(conf);
|
||||||
|
DummyFs fs = spy(new DummyFs());
|
||||||
|
Token<TokenIdentifier> token = new Token<TokenIdentifier>(new byte[0],
|
||||||
|
new byte[0], DummyFs.TOKEN_KIND, new Text("127.0.0.1:1234"));
|
||||||
|
|
||||||
|
doReturn(token).when(fs).getDelegationToken(anyString());
|
||||||
|
fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
|
||||||
|
|
||||||
|
fs.tokenAspect.ensureTokenInitialized();
|
||||||
|
|
||||||
|
// Select a token, store and renew it
|
||||||
|
verify(fs).setDelegationToken(token);
|
||||||
|
assertNotNull(Whitebox.getInternalState(fs.tokenAspect, "dtRenewer"));
|
||||||
|
assertNotNull(Whitebox.getInternalState(fs.tokenAspect, "action"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetRemoteTokenFailure() throws IOException,
|
||||||
|
URISyntaxException {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
UserGroupInformation.setConfiguration(conf);
|
||||||
|
DummyFs fs = spy(new DummyFs());
|
||||||
|
IOException e = new IOException();
|
||||||
|
doThrow(e).when(fs).getDelegationToken(anyString());
|
||||||
|
|
||||||
|
fs.emulateSecurityEnabled = true;
|
||||||
|
fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
|
||||||
|
try {
|
||||||
|
fs.tokenAspect.ensureTokenInitialized();
|
||||||
|
} catch (IOException exc) {
|
||||||
|
assertEquals(e, exc);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInitWithNoTokens() throws IOException, URISyntaxException {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
UserGroupInformation.setConfiguration(conf);
|
||||||
|
DummyFs fs = spy(new DummyFs());
|
||||||
|
doReturn(null).when(fs).getDelegationToken(anyString());
|
||||||
|
fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
|
||||||
|
|
||||||
|
fs.tokenAspect.ensureTokenInitialized();
|
||||||
|
|
||||||
|
// No token will be selected.
|
||||||
|
verify(fs, never()).setDelegationToken(
|
||||||
|
Mockito.<Token<? extends TokenIdentifier>> any());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testInitWithUGIToken() throws IOException, URISyntaxException {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
UserGroupInformation.setConfiguration(conf);
|
||||||
|
DummyFs fs = spy(new DummyFs());
|
||||||
|
doReturn(null).when(fs).getDelegationToken(anyString());
|
||||||
|
|
||||||
|
Token<TokenIdentifier> token = new Token<TokenIdentifier>(new byte[0],
|
||||||
|
new byte[0], DummyFs.TOKEN_KIND, new Text("127.0.0.1:1234"));
|
||||||
|
fs.ugi.addToken(token);
|
||||||
|
fs.ugi.addToken(new Token<TokenIdentifier>(new byte[0], new byte[0],
|
||||||
|
new Text("Other token"), new Text("127.0.0.1:8021")));
|
||||||
|
assertEquals("wrong tokens in user", 2, fs.ugi.getTokens().size());
|
||||||
|
|
||||||
|
fs.emulateSecurityEnabled = true;
|
||||||
|
fs.initialize(new URI("dummyfs://127.0.0.1:1234"), conf);
|
||||||
|
fs.tokenAspect.ensureTokenInitialized();
|
||||||
|
|
||||||
|
// Select a token from ugi (not from the remote host), store it but don't
|
||||||
|
// renew it
|
||||||
|
verify(fs).setDelegationToken(token);
|
||||||
|
verify(fs, never()).getDelegationToken(anyString());
|
||||||
|
assertNull(Whitebox.getInternalState(fs.tokenAspect, "dtRenewer"));
|
||||||
|
assertNull(Whitebox.getInternalState(fs.tokenAspect, "action"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTokenSelectionPreferences() throws IOException,
|
||||||
|
URISyntaxException {
|
||||||
|
Configuration conf = new Configuration();
|
||||||
|
DummyFs fs = spy(new DummyFs());
|
||||||
|
doReturn(null).when(fs).getDelegationToken(anyString());
|
||||||
|
fs.initialize(new URI("dummyfs://localhost:1234"), conf);
|
||||||
|
TokenAspect<DummyFs> aspect = new TokenAspect<DummyFs>(fs,
|
||||||
|
DummyFs.TOKEN_KIND);
|
||||||
|
UserGroupInformation ugi = UserGroupInformation.createUserForTesting("foo",
|
||||||
|
new String[] { "bar" });
|
||||||
|
UserGroupInformation.setConfiguration(conf);
|
||||||
|
|
||||||
|
// use ip-based tokens
|
||||||
|
SecurityUtilTestHelper.setTokenServiceUseIp(true);
|
||||||
|
|
||||||
|
// test fallback to hdfs token
|
||||||
|
Token<TokenIdentifier> hdfsToken = new Token<TokenIdentifier>(new byte[0],
|
||||||
|
new byte[0], DelegationTokenIdentifier.HDFS_DELEGATION_KIND, new Text(
|
||||||
|
"127.0.0.1:8020"));
|
||||||
|
ugi.addToken(hdfsToken);
|
||||||
|
|
||||||
|
// test fallback to hdfs token
|
||||||
|
Token<?> token = aspect.selectDelegationToken(ugi);
|
||||||
|
assertEquals(hdfsToken, token);
|
||||||
|
|
||||||
|
// test dummyfs is favored over hdfs
|
||||||
|
Token<TokenIdentifier> dummyFsToken = new Token<TokenIdentifier>(
|
||||||
|
new byte[0], new byte[0], DummyFs.TOKEN_KIND,
|
||||||
|
new Text("127.0.0.1:1234"));
|
||||||
|
ugi.addToken(dummyFsToken);
|
||||||
|
token = aspect.selectDelegationToken(ugi);
|
||||||
|
assertEquals(dummyFsToken, token);
|
||||||
|
|
||||||
|
// switch to using host-based tokens, no token should match
|
||||||
|
SecurityUtilTestHelper.setTokenServiceUseIp(false);
|
||||||
|
token = aspect.selectDelegationToken(ugi);
|
||||||
|
assertNull(token);
|
||||||
|
|
||||||
|
// test fallback to hdfs token
|
||||||
|
hdfsToken = new Token<TokenIdentifier>(new byte[0], new byte[0],
|
||||||
|
DelegationTokenIdentifier.HDFS_DELEGATION_KIND, new Text(
|
||||||
|
"localhost:8020"));
|
||||||
|
ugi.addToken(hdfsToken);
|
||||||
|
token = aspect.selectDelegationToken(ugi);
|
||||||
|
assertEquals(hdfsToken, token);
|
||||||
|
|
||||||
|
// test dummyfs is favored over hdfs
|
||||||
|
dummyFsToken = new Token<TokenIdentifier>(new byte[0], new byte[0],
|
||||||
|
DummyFs.TOKEN_KIND, new Text("localhost:1234"));
|
||||||
|
ugi.addToken(dummyFsToken);
|
||||||
|
token = aspect.selectDelegationToken(ugi);
|
||||||
|
assertEquals(dummyFsToken, token);
|
||||||
|
}
|
||||||
|
}
|
@ -26,6 +26,8 @@
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
import java.net.URI;
|
||||||
|
import java.net.URISyntaxException;
|
||||||
import java.nio.charset.Charset;
|
import java.nio.charset.Charset;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -37,10 +39,12 @@
|
|||||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||||
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
||||||
import org.apache.hadoop.hdfs.web.HftpFileSystem;
|
import org.apache.hadoop.hdfs.web.HftpFileSystem;
|
||||||
|
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
|
||||||
import org.apache.hadoop.io.DataOutputBuffer;
|
import org.apache.hadoop.io.DataOutputBuffer;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||||
import org.apache.hadoop.security.token.Token;
|
import org.apache.hadoop.security.token.Token;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.jboss.netty.bootstrap.ServerBootstrap;
|
import org.jboss.netty.bootstrap.ServerBootstrap;
|
||||||
@ -59,6 +63,7 @@
|
|||||||
import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
|
import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
|
||||||
import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
|
import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
|
||||||
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
||||||
|
import org.jboss.netty.handler.codec.http.HttpMethod;
|
||||||
import org.jboss.netty.handler.codec.http.HttpRequest;
|
import org.jboss.netty.handler.codec.http.HttpRequest;
|
||||||
import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
|
import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
|
||||||
import org.jboss.netty.handler.codec.http.HttpResponse;
|
import org.jboss.netty.handler.codec.http.HttpResponse;
|
||||||
@ -78,9 +83,10 @@ public class TestDelegationTokenRemoteFetcher {
|
|||||||
|
|
||||||
private static final String EXP_DATE = "124123512361236";
|
private static final String EXP_DATE = "124123512361236";
|
||||||
private static final String tokenFile = "http.file.dta";
|
private static final String tokenFile = "http.file.dta";
|
||||||
|
private static final URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
|
||||||
|
|
||||||
private int httpPort;
|
private int httpPort;
|
||||||
private String serviceUrl;
|
private URI serviceUrl;
|
||||||
private FileSystem fileSys;
|
private FileSystem fileSys;
|
||||||
private Configuration conf;
|
private Configuration conf;
|
||||||
private ServerBootstrap bootstrap;
|
private ServerBootstrap bootstrap;
|
||||||
@ -92,7 +98,7 @@ public void init() throws Exception {
|
|||||||
conf = new Configuration();
|
conf = new Configuration();
|
||||||
fileSys = FileSystem.getLocal(conf);
|
fileSys = FileSystem.getLocal(conf);
|
||||||
httpPort = NetUtils.getFreeSocketPort();
|
httpPort = NetUtils.getFreeSocketPort();
|
||||||
serviceUrl = "http://localhost:" + httpPort;
|
serviceUrl = new URI("http://localhost:" + httpPort);
|
||||||
testToken = createToken(serviceUrl);
|
testToken = createToken(serviceUrl);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -121,9 +127,9 @@ public void testTokenFetchFail() throws Exception {
|
|||||||
* try to fetch token without http server with IOException
|
* try to fetch token without http server with IOException
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testTokenRenewFail() {
|
public void testTokenRenewFail() throws AuthenticationException {
|
||||||
try {
|
try {
|
||||||
DelegationTokenFetcher.renewDelegationToken(serviceUrl, testToken);
|
DelegationTokenFetcher.renewDelegationToken(connectionFactory, serviceUrl, testToken);
|
||||||
fail("Token fetcher shouldn't be able to renew tokens in absense of NN");
|
fail("Token fetcher shouldn't be able to renew tokens in absense of NN");
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
}
|
}
|
||||||
@ -133,9 +139,9 @@ public void testTokenRenewFail() {
|
|||||||
* try cancel token without http server with IOException
|
* try cancel token without http server with IOException
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void expectedTokenCancelFail() {
|
public void expectedTokenCancelFail() throws AuthenticationException {
|
||||||
try {
|
try {
|
||||||
DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
|
DelegationTokenFetcher.cancelDelegationToken(connectionFactory, serviceUrl, testToken);
|
||||||
fail("Token fetcher shouldn't be able to cancel tokens in absense of NN");
|
fail("Token fetcher shouldn't be able to cancel tokens in absense of NN");
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
}
|
}
|
||||||
@ -145,11 +151,12 @@ public void expectedTokenCancelFail() {
|
|||||||
* try fetch token and get http response with error
|
* try fetch token and get http response with error
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void expectedTokenRenewErrorHttpResponse() {
|
public void expectedTokenRenewErrorHttpResponse()
|
||||||
|
throws AuthenticationException, URISyntaxException {
|
||||||
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
||||||
try {
|
try {
|
||||||
DelegationTokenFetcher.renewDelegationToken(serviceUrl + "/exception",
|
DelegationTokenFetcher.renewDelegationToken(connectionFactory, new URI(
|
||||||
createToken(serviceUrl));
|
serviceUrl.toString() + "/exception"), createToken(serviceUrl));
|
||||||
fail("Token fetcher shouldn't be able to renew tokens using an invalid"
|
fail("Token fetcher shouldn't be able to renew tokens using an invalid"
|
||||||
+ " NN URL");
|
+ " NN URL");
|
||||||
} catch (IOException ex) {
|
} catch (IOException ex) {
|
||||||
@ -159,13 +166,14 @@ public void expectedTokenRenewErrorHttpResponse() {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testCancelTokenFromHttp() throws IOException {
|
public void testCancelTokenFromHttp() throws IOException,
|
||||||
|
AuthenticationException {
|
||||||
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
||||||
DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
|
DelegationTokenFetcher.cancelDelegationToken(connectionFactory, serviceUrl,
|
||||||
|
testToken);
|
||||||
if (assertionError != null)
|
if (assertionError != null)
|
||||||
throw assertionError;
|
throw assertionError;
|
||||||
}
|
}
|
||||||
@ -174,11 +182,12 @@ public void testCancelTokenFromHttp() throws IOException {
|
|||||||
* Call renew token using http server return new expiration time
|
* Call renew token using http server return new expiration time
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testRenewTokenFromHttp() throws IOException {
|
public void testRenewTokenFromHttp() throws IOException,
|
||||||
|
NumberFormatException, AuthenticationException {
|
||||||
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
||||||
assertTrue("testRenewTokenFromHttp error",
|
assertTrue("testRenewTokenFromHttp error",
|
||||||
Long.valueOf(EXP_DATE) == DelegationTokenFetcher.renewDelegationToken(
|
Long.valueOf(EXP_DATE) == DelegationTokenFetcher.renewDelegationToken(
|
||||||
serviceUrl, testToken));
|
connectionFactory, serviceUrl, testToken));
|
||||||
if (assertionError != null)
|
if (assertionError != null)
|
||||||
throw assertionError;
|
throw assertionError;
|
||||||
}
|
}
|
||||||
@ -204,11 +213,11 @@ public void expectedTokenIsRetrievedFromHttp() throws Exception {
|
|||||||
throw assertionError;
|
throw assertionError;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Token<DelegationTokenIdentifier> createToken(String serviceUri) {
|
private static Token<DelegationTokenIdentifier> createToken(URI serviceUri) {
|
||||||
byte[] pw = "hadoop".getBytes();
|
byte[] pw = "hadoop".getBytes();
|
||||||
byte[] ident = new DelegationTokenIdentifier(new Text("owner"), new Text(
|
byte[] ident = new DelegationTokenIdentifier(new Text("owner"), new Text(
|
||||||
"renewer"), new Text("realuser")).getBytes();
|
"renewer"), new Text("realuser")).getBytes();
|
||||||
Text service = new Text(serviceUri);
|
Text service = new Text(serviceUri.toString());
|
||||||
return new Token<DelegationTokenIdentifier>(ident, pw,
|
return new Token<DelegationTokenIdentifier>(ident, pw,
|
||||||
HftpFileSystem.TOKEN_KIND, service);
|
HftpFileSystem.TOKEN_KIND, service);
|
||||||
}
|
}
|
||||||
@ -301,8 +310,15 @@ public CredentialsLogicHandler(Token<DelegationTokenIdentifier> token,
|
|||||||
public void messageReceived(ChannelHandlerContext ctx, final MessageEvent e)
|
public void messageReceived(ChannelHandlerContext ctx, final MessageEvent e)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
HttpRequest request = (HttpRequest) e.getMessage();
|
HttpRequest request = (HttpRequest) e.getMessage();
|
||||||
if (request.getMethod() != GET) {
|
|
||||||
return;
|
if (request.getMethod() == HttpMethod.OPTIONS) {
|
||||||
|
// Mimic SPNEGO authentication
|
||||||
|
HttpResponse response = new DefaultHttpResponse(HTTP_1_1,
|
||||||
|
HttpResponseStatus.OK);
|
||||||
|
response.addHeader("Set-Cookie", "hadoop-auth=1234");
|
||||||
|
e.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
|
||||||
|
} else if (request.getMethod() != GET) {
|
||||||
|
e.getChannel().close();
|
||||||
}
|
}
|
||||||
UnmodifiableIterator<Map.Entry<String, Handler>> iter = routes.entrySet()
|
UnmodifiableIterator<Map.Entry<String, Handler>> iter = routes.entrySet()
|
||||||
.iterator();
|
.iterator();
|
||||||
@ -338,7 +354,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
|
|||||||
}
|
}
|
||||||
|
|
||||||
private ServerBootstrap startHttpServer(int port,
|
private ServerBootstrap startHttpServer(int port,
|
||||||
final Token<DelegationTokenIdentifier> token, final String url) {
|
final Token<DelegationTokenIdentifier> token, final URI url) {
|
||||||
ServerBootstrap bootstrap = new ServerBootstrap(
|
ServerBootstrap bootstrap = new ServerBootstrap(
|
||||||
new NioServerSocketChannelFactory(Executors.newCachedThreadPool(),
|
new NioServerSocketChannelFactory(Executors.newCachedThreadPool(),
|
||||||
Executors.newCachedThreadPool()));
|
Executors.newCachedThreadPool()));
|
||||||
@ -348,7 +364,7 @@ private ServerBootstrap startHttpServer(int port,
|
|||||||
public ChannelPipeline getPipeline() throws Exception {
|
public ChannelPipeline getPipeline() throws Exception {
|
||||||
return Channels.pipeline(new HttpRequestDecoder(),
|
return Channels.pipeline(new HttpRequestDecoder(),
|
||||||
new HttpChunkAggregator(65536), new HttpResponseEncoder(),
|
new HttpChunkAggregator(65536), new HttpResponseEncoder(),
|
||||||
new CredentialsLogicHandler(token, url));
|
new CredentialsLogicHandler(token, url.toString()));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
bootstrap.bind(new InetSocketAddress("localhost", port));
|
bootstrap.bind(new InetSocketAddress("localhost", port));
|
||||||
|
@ -170,6 +170,14 @@ Release 2.3.0 - UNRELEASED
|
|||||||
MAPREDUCE-5613. DefaultSpeculator holds and checks hashmap that is always
|
MAPREDUCE-5613. DefaultSpeculator holds and checks hashmap that is always
|
||||||
empty (Gera Shegalov via Sandy Ryza)
|
empty (Gera Shegalov via Sandy Ryza)
|
||||||
|
|
||||||
|
MAPREDUCE-5431. Missing pom dependency in MR-client (Timothy St. Clair
|
||||||
|
via stevel)
|
||||||
|
|
||||||
|
MAPREDUCE-5624 Move grizzly-test and junit dependencies to test scope
|
||||||
|
(Ted Yu via stevel)
|
||||||
|
|
||||||
|
MAPREDUCE-5481. Enable uber jobs to have multiple reducers (Sandy Ryza)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
|
MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
|
||||||
@ -209,6 +217,9 @@ Release 2.3.0 - UNRELEASED
|
|||||||
|
|
||||||
MAPREDUCE-5610. TestSleepJob fails in jdk7 (Jonathan Eagles via jlowe)
|
MAPREDUCE-5610. TestSleepJob fails in jdk7 (Jonathan Eagles via jlowe)
|
||||||
|
|
||||||
|
MAPREDUCE-5616. MR Client-AppMaster RPC max retries on socket timeout is too
|
||||||
|
high. (cnauroth)
|
||||||
|
|
||||||
Release 2.2.1 - UNRELEASED
|
Release 2.2.1 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
@ -20,7 +20,9 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
|
||||||
@ -35,6 +37,7 @@
|
|||||||
import org.apache.hadoop.mapreduce.JobContext;
|
import org.apache.hadoop.mapreduce.JobContext;
|
||||||
import org.apache.hadoop.mapreduce.JobCounter;
|
import org.apache.hadoop.mapreduce.JobCounter;
|
||||||
import org.apache.hadoop.mapreduce.MRConfig;
|
import org.apache.hadoop.mapreduce.MRConfig;
|
||||||
|
import org.apache.hadoop.mapreduce.TaskID;
|
||||||
import org.apache.hadoop.mapreduce.TypeConverter;
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
|
||||||
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
|
||||||
@ -168,6 +171,10 @@ private class SubtaskRunner implements Runnable {
|
|||||||
public void run() {
|
public void run() {
|
||||||
ContainerLauncherEvent event = null;
|
ContainerLauncherEvent event = null;
|
||||||
|
|
||||||
|
// Collect locations of map outputs to give to reduces
|
||||||
|
Map<TaskAttemptID, MapOutputFile> localMapFiles =
|
||||||
|
new HashMap<TaskAttemptID, MapOutputFile>();
|
||||||
|
|
||||||
// _must_ either run subtasks sequentially or accept expense of new JVMs
|
// _must_ either run subtasks sequentially or accept expense of new JVMs
|
||||||
// (i.e., fork()), else will get weird failures when maps try to create/
|
// (i.e., fork()), else will get weird failures when maps try to create/
|
||||||
// write same dirname or filename: no chdir() in Java
|
// write same dirname or filename: no chdir() in Java
|
||||||
@ -223,7 +230,7 @@ public void run() {
|
|||||||
context.getEventHandler().handle(jce);
|
context.getEventHandler().handle(jce);
|
||||||
}
|
}
|
||||||
runSubtask(remoteTask, ytask.getType(), attemptID, numMapTasks,
|
runSubtask(remoteTask, ytask.getType(), attemptID, numMapTasks,
|
||||||
(numReduceTasks > 0));
|
(numReduceTasks > 0), localMapFiles);
|
||||||
|
|
||||||
} catch (RuntimeException re) {
|
} catch (RuntimeException re) {
|
||||||
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptID.getTaskId().getJobId());
|
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptID.getTaskId().getJobId());
|
||||||
@ -265,7 +272,8 @@ private void runSubtask(org.apache.hadoop.mapred.Task task,
|
|||||||
final TaskType taskType,
|
final TaskType taskType,
|
||||||
TaskAttemptId attemptID,
|
TaskAttemptId attemptID,
|
||||||
final int numMapTasks,
|
final int numMapTasks,
|
||||||
boolean renameOutputs)
|
boolean renameOutputs,
|
||||||
|
Map<TaskAttemptID, MapOutputFile> localMapFiles)
|
||||||
throws RuntimeException, IOException {
|
throws RuntimeException, IOException {
|
||||||
org.apache.hadoop.mapred.TaskAttemptID classicAttemptID =
|
org.apache.hadoop.mapred.TaskAttemptID classicAttemptID =
|
||||||
TypeConverter.fromYarn(attemptID);
|
TypeConverter.fromYarn(attemptID);
|
||||||
@ -309,7 +317,9 @@ private void runSubtask(org.apache.hadoop.mapred.Task task,
|
|||||||
map.run(conf, umbilical);
|
map.run(conf, umbilical);
|
||||||
|
|
||||||
if (renameOutputs) {
|
if (renameOutputs) {
|
||||||
renameMapOutputForReduce(conf, attemptID, map.getMapOutputFile());
|
MapOutputFile renamed = renameMapOutputForReduce(conf, attemptID,
|
||||||
|
map.getMapOutputFile());
|
||||||
|
localMapFiles.put(classicAttemptID, renamed);
|
||||||
}
|
}
|
||||||
relocalize();
|
relocalize();
|
||||||
|
|
||||||
@ -335,10 +345,11 @@ private void runSubtask(org.apache.hadoop.mapred.Task task,
|
|||||||
conf.set(MRConfig.MASTER_ADDRESS, "local"); // bypass shuffle
|
conf.set(MRConfig.MASTER_ADDRESS, "local"); // bypass shuffle
|
||||||
|
|
||||||
ReduceTask reduce = (ReduceTask)task;
|
ReduceTask reduce = (ReduceTask)task;
|
||||||
|
reduce.setLocalMapFiles(localMapFiles);
|
||||||
reduce.setConf(conf);
|
reduce.setConf(conf);
|
||||||
|
|
||||||
reduce.run(conf, umbilical);
|
reduce.run(conf, umbilical);
|
||||||
//relocalize(); // needed only if more than one reducer supported (is MAPREDUCE-434 fixed yet?)
|
relocalize();
|
||||||
}
|
}
|
||||||
|
|
||||||
} catch (FSError e) {
|
} catch (FSError e) {
|
||||||
@ -387,15 +398,16 @@ private void runSubtask(org.apache.hadoop.mapred.Task task,
|
|||||||
* so there are no particular compatibility issues.)
|
* so there are no particular compatibility issues.)
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
private void renameMapOutputForReduce(JobConf conf, TaskAttemptId mapId,
|
private MapOutputFile renameMapOutputForReduce(JobConf conf,
|
||||||
MapOutputFile subMapOutputFile)
|
TaskAttemptId mapId, MapOutputFile subMapOutputFile) throws IOException {
|
||||||
throws IOException {
|
|
||||||
FileSystem localFs = FileSystem.getLocal(conf);
|
FileSystem localFs = FileSystem.getLocal(conf);
|
||||||
// move map output to reduce input
|
// move map output to reduce input
|
||||||
Path mapOut = subMapOutputFile.getOutputFile();
|
Path mapOut = subMapOutputFile.getOutputFile();
|
||||||
FileStatus mStatus = localFs.getFileStatus(mapOut);
|
FileStatus mStatus = localFs.getFileStatus(mapOut);
|
||||||
Path reduceIn = subMapOutputFile.getInputFileForWrite(
|
Path reduceIn = subMapOutputFile.getInputFileForWrite(
|
||||||
TypeConverter.fromYarn(mapId).getTaskID(), mStatus.getLen());
|
TypeConverter.fromYarn(mapId).getTaskID(), mStatus.getLen());
|
||||||
|
Path mapOutIndex = new Path(mapOut.toString() + ".index");
|
||||||
|
Path reduceInIndex = new Path(reduceIn.toString() + ".index");
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Renaming map output file for task attempt "
|
LOG.debug("Renaming map output file for task attempt "
|
||||||
+ mapId.toString() + " from original location " + mapOut.toString()
|
+ mapId.toString() + " from original location " + mapOut.toString()
|
||||||
@ -407,6 +419,10 @@ private void renameMapOutputForReduce(JobConf conf, TaskAttemptId mapId,
|
|||||||
}
|
}
|
||||||
if (!localFs.rename(mapOut, reduceIn))
|
if (!localFs.rename(mapOut, reduceIn))
|
||||||
throw new IOException("Couldn't rename " + mapOut);
|
throw new IOException("Couldn't rename " + mapOut);
|
||||||
|
if (!localFs.rename(mapOutIndex, reduceInIndex))
|
||||||
|
throw new IOException("Couldn't rename " + mapOutIndex);
|
||||||
|
|
||||||
|
return new RenamedMapOutputFile(reduceIn);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -442,4 +458,69 @@ private void relocalize() {
|
|||||||
|
|
||||||
} // end SubtaskRunner
|
} // end SubtaskRunner
|
||||||
|
|
||||||
|
private static class RenamedMapOutputFile extends MapOutputFile {
|
||||||
|
private Path path;
|
||||||
|
|
||||||
|
public RenamedMapOutputFile(Path path) {
|
||||||
|
this.path = path;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Path getOutputFile() throws IOException {
|
||||||
|
return path;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Path getOutputFileForWrite(long size) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getOutputFileForWriteInVolume(Path existing) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getOutputIndexFile() throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getOutputIndexFileForWrite(long size) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getOutputIndexFileForWriteInVolume(Path existing) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getSpillFile(int spillNumber) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getSpillFileForWrite(int spillNumber, long size)
|
||||||
|
throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getSpillIndexFile(int spillNumber) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getSpillIndexFileForWrite(int spillNumber, long size)
|
||||||
|
throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getInputFile(int mapId) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public Path getInputFileForWrite(TaskID mapId, long size)
|
||||||
|
throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
@Override
|
||||||
|
public void removeAll() throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1173,11 +1173,7 @@ private void makeUberDecision(long dataInputLength) {
|
|||||||
// these are no longer "system" settings, necessarily; user may override
|
// these are no longer "system" settings, necessarily; user may override
|
||||||
int sysMaxMaps = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
|
int sysMaxMaps = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
|
||||||
|
|
||||||
//FIXME: handling multiple reduces within a single AM does not seem to
|
|
||||||
//work.
|
|
||||||
int sysMaxReduces = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
|
int sysMaxReduces = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
|
||||||
boolean isValidUberMaxReduces = (sysMaxReduces == 0)
|
|
||||||
|| (sysMaxReduces == 1);
|
|
||||||
|
|
||||||
long sysMaxBytes = conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
|
long sysMaxBytes = conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
|
||||||
fs.getDefaultBlockSize(this.remoteJobSubmitDir)); // FIXME: this is wrong; get FS from
|
fs.getDefaultBlockSize(this.remoteJobSubmitDir)); // FIXME: this is wrong; get FS from
|
||||||
@ -1225,7 +1221,7 @@ private void makeUberDecision(long dataInputLength) {
|
|||||||
// and thus requires sequential execution.
|
// and thus requires sequential execution.
|
||||||
isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
|
isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
|
||||||
&& smallInput && smallMemory && smallCpu
|
&& smallInput && smallMemory && smallCpu
|
||||||
&& notChainJob && isValidUberMaxReduces;
|
&& notChainJob;
|
||||||
|
|
||||||
if (isUber) {
|
if (isUber) {
|
||||||
LOG.info("Uberizing job " + jobId + ": " + numMapTasks + "m+"
|
LOG.info("Uberizing job " + jobId + ": " + numMapTasks + "m+"
|
||||||
@ -1259,8 +1255,6 @@ private void makeUberDecision(long dataInputLength) {
|
|||||||
msg.append(" too much RAM;");
|
msg.append(" too much RAM;");
|
||||||
if (!notChainJob)
|
if (!notChainJob)
|
||||||
msg.append(" chainjob;");
|
msg.append(" chainjob;");
|
||||||
if (!isValidUberMaxReduces)
|
|
||||||
msg.append(" not supported uber max reduces");
|
|
||||||
LOG.info(msg.toString());
|
LOG.info(msg.toString());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -357,13 +357,21 @@ public interface MRJobConfig {
|
|||||||
|
|
||||||
public static final String MR_AM_PREFIX = MR_PREFIX + "am.";
|
public static final String MR_AM_PREFIX = MR_PREFIX + "am.";
|
||||||
|
|
||||||
/** The number of client retires to the AM - before reconnecting to the RM
|
/** The number of client retries to the AM - before reconnecting to the RM
|
||||||
* to fetch Application State.
|
* to fetch Application State.
|
||||||
*/
|
*/
|
||||||
public static final String MR_CLIENT_TO_AM_IPC_MAX_RETRIES =
|
public static final String MR_CLIENT_TO_AM_IPC_MAX_RETRIES =
|
||||||
MR_PREFIX + "client-am.ipc.max-retries";
|
MR_PREFIX + "client-am.ipc.max-retries";
|
||||||
public static final int DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES = 3;
|
public static final int DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES = 3;
|
||||||
|
|
||||||
|
/** The number of client retries on socket timeouts to the AM - before
|
||||||
|
* reconnecting to the RM to fetch Application Status.
|
||||||
|
*/
|
||||||
|
public static final String MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS =
|
||||||
|
MR_PREFIX + "yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts";
|
||||||
|
public static final int
|
||||||
|
DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS = 3;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The number of client retries to the RM/HS before throwing exception.
|
* The number of client retries to the RM/HS before throwing exception.
|
||||||
*/
|
*/
|
||||||
|
@ -1010,6 +1010,13 @@
|
|||||||
to the RM to fetch Application Status.</description>
|
to the RM to fetch Application Status.</description>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<name>yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts</name>
|
||||||
|
<value>3</value>
|
||||||
|
<description>The number of client retries on socket timeouts to the AM - before
|
||||||
|
reconnecting to the RM to fetch Application Status.</description>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<name>yarn.app.mapreduce.client.max-retries</name>
|
<name>yarn.app.mapreduce.client.max-retries</name>
|
||||||
<value>3</value>
|
<value>3</value>
|
||||||
|
@ -107,6 +107,10 @@ public ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
|
|||||||
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
|
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY,
|
||||||
this.conf.getInt(MRJobConfig.MR_CLIENT_TO_AM_IPC_MAX_RETRIES,
|
this.conf.getInt(MRJobConfig.MR_CLIENT_TO_AM_IPC_MAX_RETRIES,
|
||||||
MRJobConfig.DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES));
|
MRJobConfig.DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES));
|
||||||
|
this.conf.setInt(
|
||||||
|
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
|
||||||
|
this.conf.getInt(MRJobConfig.MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS,
|
||||||
|
MRJobConfig.DEFAULT_MR_CLIENT_TO_AM_IPC_MAX_RETRIES_ON_TIMEOUTS));
|
||||||
this.rm = rm;
|
this.rm = rm;
|
||||||
this.jobId = jobId;
|
this.jobId = jobId;
|
||||||
this.historyServerProxy = historyServerProxy;
|
this.historyServerProxy = historyServerProxy;
|
||||||
|
@ -20,6 +20,7 @@
|
|||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
@ -39,12 +40,14 @@
|
|||||||
public class TestUberAM extends TestMRJobs {
|
public class TestUberAM extends TestMRJobs {
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(TestUberAM.class);
|
private static final Log LOG = LogFactory.getLog(TestUberAM.class);
|
||||||
|
private int numSleepReducers;
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setup() throws IOException {
|
public static void setup() throws IOException {
|
||||||
TestMRJobs.setup();
|
TestMRJobs.setup();
|
||||||
if (mrCluster != null) {
|
if (mrCluster != null) {
|
||||||
mrCluster.getConfig().setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
|
mrCluster.getConfig().setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, true);
|
||||||
|
mrCluster.getConfig().setInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 3);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -52,8 +55,19 @@ public static void setup() throws IOException {
|
|||||||
@Test
|
@Test
|
||||||
public void testSleepJob()
|
public void testSleepJob()
|
||||||
throws IOException, InterruptedException, ClassNotFoundException {
|
throws IOException, InterruptedException, ClassNotFoundException {
|
||||||
|
numSleepReducers = 1;
|
||||||
if (mrCluster != null) {
|
if (mrCluster != null) {
|
||||||
mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", 1);
|
mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", numSleepReducers);
|
||||||
|
}
|
||||||
|
super.testSleepJob();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSleepJobWithMultipleReducers()
|
||||||
|
throws IOException, InterruptedException, ClassNotFoundException {
|
||||||
|
numSleepReducers = 3;
|
||||||
|
if (mrCluster != null) {
|
||||||
|
mrCluster.getConfig().setInt("TestMRJobs.testSleepJob.reduces", numSleepReducers);
|
||||||
}
|
}
|
||||||
super.testSleepJob();
|
super.testSleepJob();
|
||||||
}
|
}
|
||||||
@ -67,7 +81,7 @@ protected void verifySleepJobCounters(Job job) throws InterruptedException,
|
|||||||
.getValue());
|
.getValue());
|
||||||
Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
|
Assert.assertEquals(3, counters.findCounter(JobCounter.TOTAL_LAUNCHED_MAPS)
|
||||||
.getValue());
|
.getValue());
|
||||||
Assert.assertEquals(1,
|
Assert.assertEquals(numSleepReducers,
|
||||||
counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue());
|
counters.findCounter(JobCounter.TOTAL_LAUNCHED_REDUCES).getValue());
|
||||||
Assert
|
Assert
|
||||||
.assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
|
.assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
|
||||||
@ -76,11 +90,11 @@ protected void verifySleepJobCounters(Job job) throws InterruptedException,
|
|||||||
.assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
|
.assertTrue(counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS) != null
|
||||||
&& counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
|
&& counters.findCounter(JobCounter.SLOTS_MILLIS_MAPS).getValue() != 0);
|
||||||
|
|
||||||
Assert.assertEquals(3, counters.findCounter(JobCounter.NUM_UBER_SUBMAPS)
|
Assert.assertEquals(3,
|
||||||
.getValue());
|
counters.findCounter(JobCounter.NUM_UBER_SUBMAPS).getValue());
|
||||||
Assert.assertEquals(1, counters.findCounter(JobCounter.NUM_UBER_SUBREDUCES)
|
Assert.assertEquals(numSleepReducers,
|
||||||
.getValue());
|
counters.findCounter(JobCounter.NUM_UBER_SUBREDUCES).getValue());
|
||||||
Assert.assertEquals(4,
|
Assert.assertEquals(3 + numSleepReducers,
|
||||||
counters.findCounter(JobCounter.TOTAL_LAUNCHED_UBERTASKS).getValue());
|
counters.findCounter(JobCounter.TOTAL_LAUNCHED_UBERTASKS).getValue());
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -138,8 +152,10 @@ public void testFailingMapper()
|
|||||||
|
|
||||||
TaskCompletionEvent[] events = job.getTaskCompletionEvents(0, 2);
|
TaskCompletionEvent[] events = job.getTaskCompletionEvents(0, 2);
|
||||||
Assert.assertEquals(1, events.length);
|
Assert.assertEquals(1, events.length);
|
||||||
Assert.assertEquals(TaskCompletionEvent.Status.TIPFAILED,
|
// TIPFAILED if it comes from the AM, FAILED if it comes from the JHS
|
||||||
events[0].getStatus());
|
TaskCompletionEvent.Status status = events[0].getStatus();
|
||||||
|
Assert.assertTrue(status == TaskCompletionEvent.Status.FAILED ||
|
||||||
|
status == TaskCompletionEvent.Status.TIPFAILED);
|
||||||
Assert.assertEquals(JobStatus.State.FAILED, job.getJobState());
|
Assert.assertEquals(JobStatus.State.FAILED, job.getJobState());
|
||||||
|
|
||||||
//Disabling till UberAM honors MRJobConfig.MAP_MAX_ATTEMPTS
|
//Disabling till UberAM honors MRJobConfig.MAP_MAX_ATTEMPTS
|
||||||
|
@ -158,6 +158,16 @@
|
|||||||
<artifactId>commons-lang</artifactId>
|
<artifactId>commons-lang</artifactId>
|
||||||
<scope>provided</scope>
|
<scope>provided</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>commons-collections</groupId>
|
||||||
|
<artifactId>commons-collections</artifactId>
|
||||||
|
<scope>provided</scope>
|
||||||
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.sun.jersey.jersey-test-framework</groupId>
|
||||||
|
<artifactId>jersey-test-framework-grizzly2</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
<build>
|
<build>
|
||||||
|
@ -93,6 +93,11 @@
|
|||||||
<type>test-jar</type>
|
<type>test-jar</type>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.sun.jersey.jersey-test-framework</groupId>
|
||||||
|
<artifactId>jersey-test-framework-grizzly2</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-mapreduce-client-hs</artifactId>
|
<artifactId>hadoop-mapreduce-client-hs</artifactId>
|
||||||
|
@ -324,8 +324,8 @@
|
|||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.commons</groupId>
|
<groupId>org.apache.commons</groupId>
|
||||||
<artifactId>commons-math</artifactId>
|
<artifactId>commons-math3</artifactId>
|
||||||
<version>2.1</version>
|
<version>3.1.1</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.commons</groupId>
|
<groupId>org.apache.commons</groupId>
|
||||||
@ -745,6 +745,11 @@
|
|||||||
<artifactId>hadoop-sls</artifactId>
|
<artifactId>hadoop-sls</artifactId>
|
||||||
<version>${project.version}</version>
|
<version>${project.version}</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>com.google.code.findbugs</groupId>
|
||||||
|
<artifactId>jsr305</artifactId>
|
||||||
|
<version>1.3.9</version>
|
||||||
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
</dependencyManagement>
|
</dependencyManagement>
|
||||||
|
|
||||||
|
@ -35,6 +35,11 @@
|
|||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-annotations</artifactId>
|
<artifactId>hadoop-annotations</artifactId>
|
||||||
|
@ -32,6 +32,11 @@
|
|||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-annotations</artifactId>
|
<artifactId>hadoop-annotations</artifactId>
|
||||||
|
@ -37,6 +37,11 @@
|
|||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-common</artifactId>
|
<artifactId>hadoop-common</artifactId>
|
||||||
|
@ -35,6 +35,11 @@
|
|||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-annotations</artifactId>
|
<artifactId>hadoop-annotations</artifactId>
|
||||||
|
@ -32,6 +32,11 @@
|
|||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-annotations</artifactId>
|
<artifactId>hadoop-annotations</artifactId>
|
||||||
|
@ -35,6 +35,11 @@
|
|||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-annotations</artifactId>
|
<artifactId>hadoop-annotations</artifactId>
|
||||||
|
@ -36,6 +36,11 @@
|
|||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
<dependency>
|
||||||
|
<groupId>junit</groupId>
|
||||||
|
<artifactId>junit</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-annotations</artifactId>
|
<artifactId>hadoop-annotations</artifactId>
|
||||||
|
@ -40,6 +40,9 @@ Release 2.3.0 - UNRELEASED
|
|||||||
YARN-311. RM/scheduler support for dynamic resource configuration.
|
YARN-311. RM/scheduler support for dynamic resource configuration.
|
||||||
(Junping Du via llu)
|
(Junping Du via llu)
|
||||||
|
|
||||||
|
YARN-1392. Allow sophisticated app-to-queue placement policies in the Fair
|
||||||
|
Scheduler (Sandy Ryza)
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
|
|
||||||
YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
|
YARN-7. Support CPU resource for DistributedShell. (Junping Du via llu)
|
||||||
@ -94,6 +97,9 @@ Release 2.3.0 - UNRELEASED
|
|||||||
YARN-1387. RMWebServices should use ClientRMService for filtering
|
YARN-1387. RMWebServices should use ClientRMService for filtering
|
||||||
applications (Karthik Kambatla via Sandy Ryza)
|
applications (Karthik Kambatla via Sandy Ryza)
|
||||||
|
|
||||||
|
YARN-1222. Make improvements in ZKRMStateStore for fencing (Karthik
|
||||||
|
Kambatla via bikas)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
@ -127,6 +133,9 @@ Release 2.3.0 - UNRELEASED
|
|||||||
YARN-1400. yarn.cmd uses HADOOP_RESOURCEMANAGER_OPTS. Should be
|
YARN-1400. yarn.cmd uses HADOOP_RESOURCEMANAGER_OPTS. Should be
|
||||||
YARN_RESOURCEMANAGER_OPTS. (Raja Aluri via cnauroth)
|
YARN_RESOURCEMANAGER_OPTS. (Raja Aluri via cnauroth)
|
||||||
|
|
||||||
|
YARN-1401. With zero sleep-delay-before-sigkill.ms, no signal is ever sent
|
||||||
|
(Gera Shegalov via Sandy Ryza)
|
||||||
|
|
||||||
Release 2.2.1 - UNRELEASED
|
Release 2.2.1 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
@ -178,6 +178,12 @@
|
|||||||
<Field name="minimumAllocation" />
|
<Field name="minimumAllocation" />
|
||||||
<Bug pattern="IS2_INCONSISTENT_SYNC" />
|
<Bug pattern="IS2_INCONSISTENT_SYNC" />
|
||||||
</Match>
|
</Match>
|
||||||
|
<!-- Inconsistent sync warning - numRetries is only initialized once and never changed -->
|
||||||
|
<Match>
|
||||||
|
<Class name="org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore" />
|
||||||
|
<Field name="numRetries" />
|
||||||
|
<Bug pattern="IS2_INCONSISTENT_SYNC" />
|
||||||
|
</Match>
|
||||||
<Match>
|
<Match>
|
||||||
<Class name="org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer"/>
|
<Class name="org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer"/>
|
||||||
<Field name="renewalTimer" />
|
<Field name="renewalTimer" />
|
||||||
|
@ -193,7 +193,7 @@ private static String getConfKeyForRMInstance(String prefix,
|
|||||||
return addSuffix(prefix, getRMHAId(conf));
|
return addSuffix(prefix, getRMHAId(conf));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static String getConfValueForRMInstance(String prefix,
|
public static String getConfValueForRMInstance(String prefix,
|
||||||
Configuration conf) {
|
Configuration conf) {
|
||||||
String confKey = getConfKeyForRMInstance(prefix, conf);
|
String confKey = getConfKeyForRMInstance(prefix, conf);
|
||||||
String retVal = conf.getTrimmed(confKey);
|
String retVal = conf.getTrimmed(confKey);
|
||||||
@ -205,8 +205,8 @@ private static String getConfValueForRMInstance(String prefix,
|
|||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
static String getConfValueForRMInstance(String prefix, String defaultValue,
|
public static String getConfValueForRMInstance(
|
||||||
Configuration conf) {
|
String prefix, String defaultValue, Configuration conf) {
|
||||||
String value = getConfValueForRMInstance(prefix, conf);
|
String value = getConfValueForRMInstance(prefix, conf);
|
||||||
return (value == null) ? defaultValue : value;
|
return (value == null) ? defaultValue : value;
|
||||||
}
|
}
|
||||||
|
@ -328,6 +328,8 @@ public class YarnConfiguration extends Configuration {
|
|||||||
ZK_STATE_STORE_PREFIX + "acl";
|
ZK_STATE_STORE_PREFIX + "acl";
|
||||||
public static final String DEFAULT_ZK_RM_STATE_STORE_ACL =
|
public static final String DEFAULT_ZK_RM_STATE_STORE_ACL =
|
||||||
"world:anyone:rwcda";
|
"world:anyone:rwcda";
|
||||||
|
public static final String ZK_RM_STATE_STORE_ROOT_NODE_ACL =
|
||||||
|
ZK_STATE_STORE_PREFIX + "root-node.acl";
|
||||||
|
|
||||||
/** The maximum number of completed applications RM keeps. */
|
/** The maximum number of completed applications RM keeps. */
|
||||||
public static final String RM_MAX_COMPLETED_APPLICATIONS =
|
public static final String RM_MAX_COMPLETED_APPLICATIONS =
|
||||||
|
@ -279,7 +279,11 @@
|
|||||||
<description>Host:Port of the ZooKeeper server where RM state will
|
<description>Host:Port of the ZooKeeper server where RM state will
|
||||||
be stored. This must be supplied when using
|
be stored. This must be supplied when using
|
||||||
org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
|
org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore
|
||||||
as the value for yarn.resourcemanager.store.class</description>
|
as the value for yarn.resourcemanager.store.class. ZKRMStateStore
|
||||||
|
is implicitly fenced, meaning a single ResourceManager is
|
||||||
|
able to use the store at any point in time. More details on this, along
|
||||||
|
with setting up appropriate ACLs is discussed under the description for
|
||||||
|
yarn.resourcemanager.zk.state-store.root-node.acl.</description>
|
||||||
<name>yarn.resourcemanager.zk.state-store.address</name>
|
<name>yarn.resourcemanager.zk.state-store.address</name>
|
||||||
<!--value>127.0.0.1:2181</value-->
|
<!--value>127.0.0.1:2181</value-->
|
||||||
</property>
|
</property>
|
||||||
@ -320,6 +324,31 @@
|
|||||||
<value>world:anyone:rwcda</value>
|
<value>world:anyone:rwcda</value>
|
||||||
</property>
|
</property>
|
||||||
|
|
||||||
|
<property>
|
||||||
|
<description>
|
||||||
|
ACLs to be used for the root znode when using ZKRMStateStore in a HA
|
||||||
|
scenario for fencing.
|
||||||
|
|
||||||
|
ZKRMStateStore supports implicit fencing to allow a single
|
||||||
|
ResourceManager write-access to the store. For fencing, the
|
||||||
|
ResourceManagers in the cluster share read-write-admin privileges on the
|
||||||
|
root node, but the Active ResourceManager claims exclusive create-delete
|
||||||
|
permissions.
|
||||||
|
|
||||||
|
By default, when this property is not set, we use the ACLs from
|
||||||
|
yarn.resourcemanager.zk.state-store.acl for shared admin access and
|
||||||
|
rm-address:cluster-timestamp for username-based exclusive create-delete
|
||||||
|
access.
|
||||||
|
|
||||||
|
This property allows users to set ACLs of their choice instead of using
|
||||||
|
the default mechanism. For fencing to work, the ACLs should be
|
||||||
|
carefully set differently on each ResourceManger such that all the
|
||||||
|
ResourceManagers have shared admin access and the Active ResourceManger
|
||||||
|
takes over (exclusively) the create-delete access.
|
||||||
|
</description>
|
||||||
|
<name>yarn.resourcemanager.zk.state-store.root-node.acl</name>
|
||||||
|
</property>
|
||||||
|
|
||||||
<property>
|
<property>
|
||||||
<description>URI pointing to the location of the FileSystem path where
|
<description>URI pointing to the location of the FileSystem path where
|
||||||
RM state will be stored. This must be supplied when using
|
RM state will be stored. This must be supplied when using
|
||||||
|
@ -375,13 +375,19 @@ public void cleanupContainer() throws IOException {
|
|||||||
LOG.debug("Sending signal to pid " + processId
|
LOG.debug("Sending signal to pid " + processId
|
||||||
+ " as user " + user
|
+ " as user " + user
|
||||||
+ " for container " + containerIdStr);
|
+ " for container " + containerIdStr);
|
||||||
if (sleepDelayBeforeSigKill > 0) {
|
|
||||||
boolean result = exec.signalContainer(user,
|
final Signal signal = sleepDelayBeforeSigKill > 0
|
||||||
processId, Signal.TERM);
|
? Signal.TERM
|
||||||
LOG.debug("Sent signal to pid " + processId
|
: Signal.KILL;
|
||||||
|
|
||||||
|
boolean result = exec.signalContainer(user, processId, signal);
|
||||||
|
|
||||||
|
LOG.debug("Sent signal " + signal + " to pid " + processId
|
||||||
+ " as user " + user
|
+ " as user " + user
|
||||||
+ " for container " + containerIdStr
|
+ " for container " + containerIdStr
|
||||||
+ ", result=" + (result? "success" : "failed"));
|
+ ", result=" + (result? "success" : "failed"));
|
||||||
|
|
||||||
|
if (sleepDelayBeforeSigKill > 0) {
|
||||||
new DelayedProcessKiller(container, user,
|
new DelayedProcessKiller(container, user,
|
||||||
processId, sleepDelayBeforeSigKill, Signal.KILL, exec).start();
|
processId, sleepDelayBeforeSigKill, Signal.KILL, exec).start();
|
||||||
}
|
}
|
||||||
|
@ -97,7 +97,6 @@ public void setup() throws IOException {
|
|||||||
conf.setClass(
|
conf.setClass(
|
||||||
YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
|
YarnConfiguration.NM_CONTAINER_MON_RESOURCE_CALCULATOR,
|
||||||
LinuxResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
|
LinuxResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
|
||||||
conf.setLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS, 1000);
|
|
||||||
super.setup();
|
super.setup();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -590,8 +589,9 @@ public void testAuxiliaryServiceHelper() throws Exception {
|
|||||||
AuxiliaryServiceHelper.getServiceDataFromEnv(serviceName, env));
|
AuxiliaryServiceHelper.getServiceDataFromEnv(serviceName, env));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
private void internalKillTest(boolean delayed) throws Exception {
|
||||||
public void testDelayedKill() throws Exception {
|
conf.setLong(YarnConfiguration.NM_SLEEP_DELAY_BEFORE_SIGKILL_MS,
|
||||||
|
delayed ? 1000 : 0);
|
||||||
containerManager.start();
|
containerManager.start();
|
||||||
|
|
||||||
// ////// Construct the Container-id
|
// ////// Construct the Container-id
|
||||||
@ -675,7 +675,8 @@ public void testDelayedKill() throws Exception {
|
|||||||
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
BaseContainerManagerTest.waitForContainerState(containerManager, cId,
|
||||||
ContainerState.COMPLETE);
|
ContainerState.COMPLETE);
|
||||||
|
|
||||||
// container stop sends a sigterm followed by a sigkill
|
// if delayed container stop sends a sigterm followed by a sigkill
|
||||||
|
// otherwise sigkill is sent immediately
|
||||||
GetContainerStatusesRequest gcsRequest =
|
GetContainerStatusesRequest gcsRequest =
|
||||||
GetContainerStatusesRequest.newInstance(containerIds);
|
GetContainerStatusesRequest.newInstance(containerIds);
|
||||||
|
|
||||||
@ -690,7 +691,7 @@ public void testDelayedKill() throws Exception {
|
|||||||
// Windows, because the process is not notified when killed by winutils.
|
// Windows, because the process is not notified when killed by winutils.
|
||||||
// There is no way for the process to trap and respond. Instead, we can
|
// There is no way for the process to trap and respond. Instead, we can
|
||||||
// verify that the job object with ID matching container ID no longer exists.
|
// verify that the job object with ID matching container ID no longer exists.
|
||||||
if (Shell.WINDOWS) {
|
if (Shell.WINDOWS || !delayed) {
|
||||||
Assert.assertFalse("Process is still alive!",
|
Assert.assertFalse("Process is still alive!",
|
||||||
DefaultContainerExecutor.containerIsAlive(cId.toString()));
|
DefaultContainerExecutor.containerIsAlive(cId.toString()));
|
||||||
} else {
|
} else {
|
||||||
@ -713,6 +714,16 @@ public void testDelayedKill() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDelayedKill() throws Exception {
|
||||||
|
internalKillTest(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testImmediateKill() throws Exception {
|
||||||
|
internalKillTest(false);
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("rawtypes")
|
@SuppressWarnings("rawtypes")
|
||||||
@Test
|
@Test
|
||||||
public void testCallFailureWithNullLocalizedResources() {
|
public void testCallFailureWithNullLocalizedResources() {
|
||||||
|
@ -67,7 +67,9 @@ public class RMHAProtocolService extends AbstractService implements
|
|||||||
protected HAServiceState haState = HAServiceState.INITIALIZING;
|
protected HAServiceState haState = HAServiceState.INITIALIZING;
|
||||||
private AccessControlList adminAcl;
|
private AccessControlList adminAcl;
|
||||||
private Server haAdminServer;
|
private Server haAdminServer;
|
||||||
private boolean haEnabled;
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
boolean haEnabled;
|
||||||
|
|
||||||
public RMHAProtocolService(ResourceManager resourceManager) {
|
public RMHAProtocolService(ResourceManager resourceManager) {
|
||||||
super("RMHAProtocolService");
|
super("RMHAProtocolService");
|
||||||
@ -174,7 +176,8 @@ public synchronized void monitorHealth()
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void transitionToActive() throws Exception {
|
@InterfaceAudience.Private
|
||||||
|
synchronized void transitionToActive() throws Exception {
|
||||||
if (haState == HAServiceState.ACTIVE) {
|
if (haState == HAServiceState.ACTIVE) {
|
||||||
LOG.info("Already in active state");
|
LOG.info("Already in active state");
|
||||||
return;
|
return;
|
||||||
@ -205,7 +208,8 @@ public synchronized void transitionToActive(StateChangeRequestInfo reqInfo)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void transitionToStandby(boolean initialize)
|
@InterfaceAudience.Private
|
||||||
|
synchronized void transitionToStandby(boolean initialize)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
if (haState == HAServiceState.STANDBY) {
|
if (haState == HAServiceState.STANDBY) {
|
||||||
LOG.info("Already in standby state");
|
LOG.info("Already in standby state");
|
||||||
|
@ -56,6 +56,8 @@
|
|||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreOperationFailedEvent;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreOperationFailedEventType;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
|
||||||
@ -164,6 +166,10 @@ public ResourceManager() {
|
|||||||
super("ResourceManager");
|
super("ResourceManager");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public RMHAProtocolService getHAService() {
|
||||||
|
return this.haService;
|
||||||
|
}
|
||||||
|
|
||||||
public RMContext getRMContext() {
|
public RMContext getRMContext() {
|
||||||
return this.rmContext;
|
return this.rmContext;
|
||||||
}
|
}
|
||||||
@ -216,6 +222,11 @@ protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
|
|||||||
return new SchedulerEventDispatcher(this.scheduler);
|
return new SchedulerEventDispatcher(this.scheduler);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected RMStateStoreOperationFailedEventDispatcher
|
||||||
|
createRMStateStoreOperationFailedEventDispatcher() {
|
||||||
|
return new RMStateStoreOperationFailedEventDispatcher(haService);
|
||||||
|
}
|
||||||
|
|
||||||
protected Dispatcher createDispatcher() {
|
protected Dispatcher createDispatcher() {
|
||||||
return new AsyncDispatcher();
|
return new AsyncDispatcher();
|
||||||
}
|
}
|
||||||
@ -339,6 +350,8 @@ protected void serviceInit(Configuration configuration) throws Exception {
|
|||||||
try {
|
try {
|
||||||
rmStore.init(conf);
|
rmStore.init(conf);
|
||||||
rmStore.setRMDispatcher(rmDispatcher);
|
rmStore.setRMDispatcher(rmDispatcher);
|
||||||
|
rmDispatcher.register(RMStateStoreOperationFailedEventType.class,
|
||||||
|
createRMStateStoreOperationFailedEventDispatcher());
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
// the Exception from stateStore.init() needs to be handled for
|
// the Exception from stateStore.init() needs to be handled for
|
||||||
// HA and we need to give up master status if we got fenced
|
// HA and we need to give up master status if we got fenced
|
||||||
@ -632,6 +645,46 @@ public void handle(SchedulerEvent event) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Private
|
||||||
|
public static class RMStateStoreOperationFailedEventDispatcher implements
|
||||||
|
EventHandler<RMStateStoreOperationFailedEvent> {
|
||||||
|
private final RMHAProtocolService haService;
|
||||||
|
|
||||||
|
public RMStateStoreOperationFailedEventDispatcher(
|
||||||
|
RMHAProtocolService haService) {
|
||||||
|
this.haService = haService;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void handle(RMStateStoreOperationFailedEvent event) {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Received a " +
|
||||||
|
RMStateStoreOperationFailedEvent.class.getName() + " of type " +
|
||||||
|
event.getType().name());
|
||||||
|
}
|
||||||
|
if (event.getType() == RMStateStoreOperationFailedEventType.FENCED) {
|
||||||
|
LOG.info("RMStateStore has been fenced");
|
||||||
|
synchronized(haService) {
|
||||||
|
if (haService.haEnabled) {
|
||||||
|
try {
|
||||||
|
// Transition to standby and reinit active services
|
||||||
|
LOG.info("Transitioning RM to Standby mode");
|
||||||
|
haService.transitionToStandby(true);
|
||||||
|
return;
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("Failed to transition RM to Standby mode.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
LOG.error("Shutting down RM on receiving a " +
|
||||||
|
RMStateStoreOperationFailedEvent.class.getName() + " of type " +
|
||||||
|
event.getType().name());
|
||||||
|
ExitUtil.terminate(1, event.getCause());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
public static final class ApplicationEventDispatcher implements
|
public static final class ApplicationEventDispatcher implements
|
||||||
EventHandler<RMAppEvent> {
|
EventHandler<RMAppEvent> {
|
||||||
|
@ -43,6 +43,7 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.Container;
|
import org.apache.hadoop.yarn.api.records.Container;
|
||||||
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
|
||||||
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
|
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
||||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
import org.apache.hadoop.yarn.event.EventHandler;
|
import org.apache.hadoop.yarn.event.EventHandler;
|
||||||
@ -388,9 +389,13 @@ protected abstract void updateApplicationAttemptStateInternal(String attemptId,
|
|||||||
*/
|
*/
|
||||||
public synchronized void storeRMDelegationTokenAndSequenceNumber(
|
public synchronized void storeRMDelegationTokenAndSequenceNumber(
|
||||||
RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
|
RMDelegationTokenIdentifier rmDTIdentifier, Long renewDate,
|
||||||
int latestSequenceNumber) throws Exception {
|
int latestSequenceNumber) {
|
||||||
|
try {
|
||||||
storeRMDelegationTokenAndSequenceNumberState(rmDTIdentifier, renewDate,
|
storeRMDelegationTokenAndSequenceNumberState(rmDTIdentifier, renewDate,
|
||||||
latestSequenceNumber);
|
latestSequenceNumber);
|
||||||
|
} catch (Exception e) {
|
||||||
|
notifyStoreOperationFailed(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -406,9 +411,12 @@ protected abstract void storeRMDelegationTokenAndSequenceNumberState(
|
|||||||
* RMDTSecretManager call this to remove the state of a delegation token
|
* RMDTSecretManager call this to remove the state of a delegation token
|
||||||
*/
|
*/
|
||||||
public synchronized void removeRMDelegationToken(
|
public synchronized void removeRMDelegationToken(
|
||||||
RMDelegationTokenIdentifier rmDTIdentifier, int sequenceNumber)
|
RMDelegationTokenIdentifier rmDTIdentifier, int sequenceNumber) {
|
||||||
throws Exception {
|
try {
|
||||||
removeRMDelegationTokenState(rmDTIdentifier);
|
removeRMDelegationTokenState(rmDTIdentifier);
|
||||||
|
} catch (Exception e) {
|
||||||
|
notifyStoreOperationFailed(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -421,9 +429,12 @@ protected abstract void removeRMDelegationTokenState(
|
|||||||
/**
|
/**
|
||||||
* RMDTSecretManager call this to store the state of a master key
|
* RMDTSecretManager call this to store the state of a master key
|
||||||
*/
|
*/
|
||||||
public synchronized void storeRMDTMasterKey(DelegationKey delegationKey)
|
public synchronized void storeRMDTMasterKey(DelegationKey delegationKey) {
|
||||||
throws Exception {
|
try {
|
||||||
storeRMDTMasterKeyState(delegationKey);
|
storeRMDTMasterKeyState(delegationKey);
|
||||||
|
} catch (Exception e) {
|
||||||
|
notifyStoreOperationFailed(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -437,9 +448,12 @@ protected abstract void storeRMDTMasterKeyState(DelegationKey delegationKey)
|
|||||||
/**
|
/**
|
||||||
* RMDTSecretManager call this to remove the state of a master key
|
* RMDTSecretManager call this to remove the state of a master key
|
||||||
*/
|
*/
|
||||||
public synchronized void removeRMDTMasterKey(DelegationKey delegationKey)
|
public synchronized void removeRMDTMasterKey(DelegationKey delegationKey) {
|
||||||
throws Exception {
|
try {
|
||||||
removeRMDTMasterKeyState(delegationKey);
|
removeRMDTMasterKeyState(delegationKey);
|
||||||
|
} catch (Exception e) {
|
||||||
|
notifyStoreOperationFailed(e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -539,19 +553,15 @@ protected void handleStoreEvent(RMStateStoreEvent event) {
|
|||||||
try {
|
try {
|
||||||
if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
|
if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
|
||||||
storeApplicationStateInternal(appId.toString(), appStateData);
|
storeApplicationStateInternal(appId.toString(), appStateData);
|
||||||
|
notifyDoneStoringApplication(appId, storedException);
|
||||||
} else {
|
} else {
|
||||||
assert event.getType().equals(RMStateStoreEventType.UPDATE_APP);
|
assert event.getType().equals(RMStateStoreEventType.UPDATE_APP);
|
||||||
updateApplicationStateInternal(appId.toString(), appStateData);
|
updateApplicationStateInternal(appId.toString(), appStateData);
|
||||||
|
notifyDoneUpdatingApplication(appId, storedException);
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOG.error("Error storing app: " + appId, e);
|
LOG.error("Error storing app: " + appId, e);
|
||||||
storedException = e;
|
notifyStoreOperationFailed(e);
|
||||||
} finally {
|
|
||||||
if (event.getType().equals(RMStateStoreEventType.STORE_APP)) {
|
|
||||||
notifyDoneStoringApplication(appId, storedException);
|
|
||||||
} else {
|
|
||||||
notifyDoneUpdatingApplication(appId, storedException);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
} else if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)
|
} else if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)
|
||||||
|| event.getType().equals(RMStateStoreEventType.UPDATE_APP_ATTEMPT)) {
|
|| event.getType().equals(RMStateStoreEventType.UPDATE_APP_ATTEMPT)) {
|
||||||
@ -589,24 +599,20 @@ protected void handleStoreEvent(RMStateStoreEvent event) {
|
|||||||
if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
|
if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
|
||||||
storeApplicationAttemptStateInternal(attemptState.getAttemptId()
|
storeApplicationAttemptStateInternal(attemptState.getAttemptId()
|
||||||
.toString(), attemptStateData);
|
.toString(), attemptStateData);
|
||||||
|
notifyDoneStoringApplicationAttempt(attemptState.getAttemptId(),
|
||||||
|
storedException);
|
||||||
} else {
|
} else {
|
||||||
assert event.getType().equals(
|
assert event.getType().equals(
|
||||||
RMStateStoreEventType.UPDATE_APP_ATTEMPT);
|
RMStateStoreEventType.UPDATE_APP_ATTEMPT);
|
||||||
updateApplicationAttemptStateInternal(attemptState.getAttemptId()
|
updateApplicationAttemptStateInternal(attemptState.getAttemptId()
|
||||||
.toString(), attemptStateData);
|
.toString(), attemptStateData);
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
LOG
|
|
||||||
.error("Error storing appAttempt: " + attemptState.getAttemptId(), e);
|
|
||||||
storedException = e;
|
|
||||||
} finally {
|
|
||||||
if (event.getType().equals(RMStateStoreEventType.STORE_APP_ATTEMPT)) {
|
|
||||||
notifyDoneStoringApplicationAttempt(attemptState.getAttemptId(),
|
|
||||||
storedException);
|
|
||||||
} else {
|
|
||||||
notifyDoneUpdatingApplicationAttempt(attemptState.getAttemptId(),
|
notifyDoneUpdatingApplicationAttempt(attemptState.getAttemptId(),
|
||||||
storedException);
|
storedException);
|
||||||
}
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error(
|
||||||
|
"Error storing appAttempt: " + attemptState.getAttemptId(), e);
|
||||||
|
notifyStoreOperationFailed(e);
|
||||||
}
|
}
|
||||||
} else if (event.getType().equals(RMStateStoreEventType.REMOVE_APP)) {
|
} else if (event.getType().equals(RMStateStoreEventType.REMOVE_APP)) {
|
||||||
ApplicationState appState =
|
ApplicationState appState =
|
||||||
@ -616,17 +622,34 @@ protected void handleStoreEvent(RMStateStoreEvent event) {
|
|||||||
LOG.info("Removing info for app: " + appId);
|
LOG.info("Removing info for app: " + appId);
|
||||||
try {
|
try {
|
||||||
removeApplicationState(appState);
|
removeApplicationState(appState);
|
||||||
|
notifyDoneRemovingApplcation(appId, removedException);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOG.error("Error removing app: " + appId, e);
|
LOG.error("Error removing app: " + appId, e);
|
||||||
removedException = e;
|
notifyStoreOperationFailed(e);
|
||||||
} finally {
|
|
||||||
notifyDoneRemovingApplcation(appId, removedException);
|
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
LOG.error("Unknown RMStateStoreEvent type: " + event.getType());
|
LOG.error("Unknown RMStateStoreEvent type: " + event.getType());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
/**
|
||||||
|
* In {#handleStoreEvent}, this method is called to notify the
|
||||||
|
* ResourceManager that the store operation has failed.
|
||||||
|
* @param failureCause the exception due to which the operation failed
|
||||||
|
*/
|
||||||
|
private void notifyStoreOperationFailed(Exception failureCause) {
|
||||||
|
RMStateStoreOperationFailedEventType type;
|
||||||
|
if (failureCause instanceof StoreFencedException) {
|
||||||
|
type = RMStateStoreOperationFailedEventType.FENCED;
|
||||||
|
} else {
|
||||||
|
type = RMStateStoreOperationFailedEventType.FAILED;
|
||||||
|
}
|
||||||
|
|
||||||
|
rmDispatcher.getEventHandler().handle(
|
||||||
|
new RMStateStoreOperationFailedEvent(type, failureCause));
|
||||||
|
}
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
/**
|
/**
|
||||||
* In (@link handleStoreEvent}, this method is called to notify the
|
* In (@link handleStoreEvent}, this method is called to notify the
|
||||||
|
@ -0,0 +1,36 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.event.AbstractEvent;
|
||||||
|
|
||||||
|
public class RMStateStoreOperationFailedEvent
|
||||||
|
extends AbstractEvent<RMStateStoreOperationFailedEventType> {
|
||||||
|
private Exception cause;
|
||||||
|
|
||||||
|
RMStateStoreOperationFailedEvent(
|
||||||
|
RMStateStoreOperationFailedEventType type, Exception cause) {
|
||||||
|
super(type);
|
||||||
|
this.cause = cause;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Exception getCause() {
|
||||||
|
return this.cause;
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,23 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
|
||||||
|
|
||||||
|
public enum RMStateStoreOperationFailedEventType {
|
||||||
|
FENCED, // Store operation failed because it was fenced
|
||||||
|
FAILED // Store operation failed for no known reason
|
||||||
|
}
|
@ -0,0 +1,28 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
|
||||||
|
|
||||||
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
|
|
||||||
|
public class StoreFencedException extends YarnException {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
|
||||||
|
public StoreFencedException() {
|
||||||
|
super("RMStateStore has been fenced");
|
||||||
|
}
|
||||||
|
}
|
@ -23,7 +23,9 @@
|
|||||||
import java.io.DataInputStream;
|
import java.io.DataInputStream;
|
||||||
import java.io.DataOutputStream;
|
import java.io.DataOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.security.NoSuchAlgorithmException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
@ -31,6 +33,7 @@
|
|||||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.ha.HAServiceProtocol;
|
||||||
import org.apache.hadoop.io.DataInputByteBuffer;
|
import org.apache.hadoop.io.DataInputByteBuffer;
|
||||||
import org.apache.hadoop.security.Credentials;
|
import org.apache.hadoop.security.Credentials;
|
||||||
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
||||||
@ -38,11 +41,14 @@
|
|||||||
import org.apache.hadoop.util.ZKUtil;
|
import org.apache.hadoop.util.ZKUtil;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.client.RMHAServiceTarget;
|
||||||
|
import org.apache.hadoop.yarn.conf.HAUtil;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
|
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationAttemptStateDataProto;
|
||||||
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
|
import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ApplicationStateDataProto;
|
||||||
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
|
import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
|
||||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||||
@ -53,11 +59,14 @@
|
|||||||
import org.apache.zookeeper.WatchedEvent;
|
import org.apache.zookeeper.WatchedEvent;
|
||||||
import org.apache.zookeeper.Watcher;
|
import org.apache.zookeeper.Watcher;
|
||||||
import org.apache.zookeeper.Watcher.Event;
|
import org.apache.zookeeper.Watcher.Event;
|
||||||
|
import org.apache.zookeeper.ZooDefs;
|
||||||
import org.apache.zookeeper.ZooKeeper;
|
import org.apache.zookeeper.ZooKeeper;
|
||||||
import org.apache.zookeeper.data.ACL;
|
import org.apache.zookeeper.data.ACL;
|
||||||
|
import org.apache.zookeeper.data.Id;
|
||||||
import org.apache.zookeeper.data.Stat;
|
import org.apache.zookeeper.data.Stat;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
|
||||||
|
|
||||||
@Private
|
@Private
|
||||||
@Unstable
|
@Unstable
|
||||||
@ -83,6 +92,55 @@ public class ZKRMStateStore extends RMStateStore {
|
|||||||
protected ZooKeeper zkClient;
|
protected ZooKeeper zkClient;
|
||||||
private ZooKeeper oldZkClient;
|
private ZooKeeper oldZkClient;
|
||||||
|
|
||||||
|
/** Fencing related variables */
|
||||||
|
private static final String FENCING_LOCK = "RM_ZK_FENCING_LOCK";
|
||||||
|
private String fencingNodePath;
|
||||||
|
private Op createFencingNodePathOp;
|
||||||
|
private Op deleteFencingNodePathOp;
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
List<ACL> zkRootNodeAcl;
|
||||||
|
private boolean useDefaultFencingScheme = false;
|
||||||
|
public static final int CREATE_DELETE_PERMS =
|
||||||
|
ZooDefs.Perms.CREATE | ZooDefs.Perms.DELETE;
|
||||||
|
private final String zkRootNodeAuthScheme =
|
||||||
|
new DigestAuthenticationProvider().getScheme();
|
||||||
|
|
||||||
|
private String zkRootNodeUsername;
|
||||||
|
private String zkRootNodePassword;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Given the {@link Configuration} and {@link ACL}s used (zkAcl) for
|
||||||
|
* ZooKeeper access, construct the {@link ACL}s for the store's root node.
|
||||||
|
* In the constructed {@link ACL}, all the users allowed by zkAcl are given
|
||||||
|
* rwa access, while the current RM has exclude create-delete access.
|
||||||
|
*
|
||||||
|
* To be called only when HA is enabled and the configuration doesn't set ACL
|
||||||
|
* for the root node.
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
|
protected List<ACL> constructZkRootNodeACL(
|
||||||
|
Configuration conf, List<ACL> sourceACLs) throws NoSuchAlgorithmException {
|
||||||
|
List<ACL> zkRootNodeAcl = new ArrayList<ACL>();
|
||||||
|
for (ACL acl : sourceACLs) {
|
||||||
|
zkRootNodeAcl.add(new ACL(
|
||||||
|
ZKUtil.removeSpecificPerms(acl.getPerms(), CREATE_DELETE_PERMS),
|
||||||
|
acl.getId()));
|
||||||
|
}
|
||||||
|
|
||||||
|
zkRootNodeUsername = HAUtil.getConfValueForRMInstance(
|
||||||
|
YarnConfiguration.RM_ADDRESS,
|
||||||
|
YarnConfiguration.DEFAULT_RM_ADDRESS, conf);
|
||||||
|
zkRootNodePassword = Long.toString(ResourceManager.getClusterTimeStamp());
|
||||||
|
Id rmId = new Id(zkRootNodeAuthScheme,
|
||||||
|
DigestAuthenticationProvider.generateDigest(
|
||||||
|
zkRootNodeUsername + ":" + zkRootNodePassword));
|
||||||
|
zkRootNodeAcl.add(new ACL(CREATE_DELETE_PERMS, rmId));
|
||||||
|
return zkRootNodeAcl;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void initInternal(Configuration conf) throws Exception {
|
public synchronized void initInternal(Configuration conf) throws Exception {
|
||||||
zkHostPort = conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS);
|
zkHostPort = conf.get(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS);
|
||||||
@ -116,6 +174,29 @@ public synchronized void initInternal(Configuration conf) throws Exception {
|
|||||||
zkRootNodePath = znodeWorkingPath + "/" + ROOT_ZNODE_NAME;
|
zkRootNodePath = znodeWorkingPath + "/" + ROOT_ZNODE_NAME;
|
||||||
rmDTSecretManagerRoot = zkRootNodePath + "/" + RM_DT_SECRET_MANAGER_ROOT;
|
rmDTSecretManagerRoot = zkRootNodePath + "/" + RM_DT_SECRET_MANAGER_ROOT;
|
||||||
rmAppRoot = zkRootNodePath + "/" + RM_APP_ROOT;
|
rmAppRoot = zkRootNodePath + "/" + RM_APP_ROOT;
|
||||||
|
|
||||||
|
/* Initialize fencing related paths, acls, and ops */
|
||||||
|
fencingNodePath = zkRootNodePath + "/" + FENCING_LOCK;
|
||||||
|
createFencingNodePathOp = Op.create(fencingNodePath, new byte[0], zkAcl,
|
||||||
|
CreateMode.PERSISTENT);
|
||||||
|
deleteFencingNodePathOp = Op.delete(fencingNodePath, -1);
|
||||||
|
if (HAUtil.isHAEnabled(conf)) {
|
||||||
|
String zkRootNodeAclConf = HAUtil.getConfValueForRMInstance
|
||||||
|
(YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL, conf);
|
||||||
|
if (zkRootNodeAclConf != null) {
|
||||||
|
zkRootNodeAclConf = ZKUtil.resolveConfIndirection(zkRootNodeAclConf);
|
||||||
|
try {
|
||||||
|
zkRootNodeAcl = ZKUtil.parseACLs(zkRootNodeAclConf);
|
||||||
|
} catch (ZKUtil.BadAclFormatException bafe) {
|
||||||
|
LOG.error("Invalid format for " +
|
||||||
|
YarnConfiguration.ZK_RM_STATE_STORE_ROOT_NODE_ACL);
|
||||||
|
throw bafe;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
useDefaultFencingScheme = true;
|
||||||
|
zkRootNodeAcl = constructZkRootNodeACL(conf, zkAcl);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -126,20 +207,76 @@ public synchronized void startInternal() throws Exception {
|
|||||||
// ensure root dirs exist
|
// ensure root dirs exist
|
||||||
createRootDir(znodeWorkingPath);
|
createRootDir(znodeWorkingPath);
|
||||||
createRootDir(zkRootNodePath);
|
createRootDir(zkRootNodePath);
|
||||||
|
if (HAUtil.isHAEnabled(getConfig())){
|
||||||
|
fence();
|
||||||
|
}
|
||||||
createRootDir(rmDTSecretManagerRoot);
|
createRootDir(rmDTSecretManagerRoot);
|
||||||
createRootDir(rmAppRoot);
|
createRootDir(rmAppRoot);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void createRootDir(String rootPath) throws Exception {
|
private void createRootDir(final String rootPath) throws Exception {
|
||||||
|
// For root dirs, we shouldn't use the doMulti helper methods
|
||||||
try {
|
try {
|
||||||
createWithRetries(rootPath, null, zkAcl, CreateMode.PERSISTENT);
|
new ZKAction<String>() {
|
||||||
|
@Override
|
||||||
|
public String run() throws KeeperException, InterruptedException {
|
||||||
|
return zkClient.create(rootPath, null, zkAcl, CreateMode.PERSISTENT);
|
||||||
|
}
|
||||||
|
}.runWithRetries();
|
||||||
} catch (KeeperException ke) {
|
} catch (KeeperException ke) {
|
||||||
if (ke.code() != Code.NODEEXISTS) {
|
if (ke.code() == Code.NODEEXISTS) {
|
||||||
|
LOG.debug(rootPath + "znode already exists!");
|
||||||
|
} else {
|
||||||
throw ke;
|
throw ke;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void logRootNodeAcls(String prefix) throws KeeperException,
|
||||||
|
InterruptedException {
|
||||||
|
Stat getStat = new Stat();
|
||||||
|
List<ACL> getAcls = zkClient.getACL(zkRootNodePath, getStat);
|
||||||
|
|
||||||
|
StringBuilder builder = new StringBuilder();
|
||||||
|
builder.append(prefix);
|
||||||
|
for (ACL acl : getAcls) {
|
||||||
|
builder.append(acl.toString());
|
||||||
|
}
|
||||||
|
builder.append(getStat.toString());
|
||||||
|
LOG.debug(builder.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized void fence() throws Exception {
|
||||||
|
if (LOG.isTraceEnabled()) {
|
||||||
|
logRootNodeAcls("Before fencing\n");
|
||||||
|
}
|
||||||
|
|
||||||
|
new ZKAction<Void>() {
|
||||||
|
@Override
|
||||||
|
public Void run() throws KeeperException, InterruptedException {
|
||||||
|
zkClient.setACL(zkRootNodePath, zkRootNodeAcl, -1);
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}.runWithRetries();
|
||||||
|
|
||||||
|
// delete fencingnodepath
|
||||||
|
new ZKAction<Void>() {
|
||||||
|
@Override
|
||||||
|
public Void run() throws KeeperException, InterruptedException {
|
||||||
|
try {
|
||||||
|
zkClient.multi(Collections.singletonList(deleteFencingNodePathOp));
|
||||||
|
} catch (KeeperException.NoNodeException nne) {
|
||||||
|
LOG.info("Fencing node " + fencingNodePath + " doesn't exist to delete");
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}.runWithRetries();
|
||||||
|
|
||||||
|
if (LOG.isTraceEnabled()) {
|
||||||
|
logRootNodeAcls("After fencing\n");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private synchronized void closeZkClients() throws IOException {
|
private synchronized void closeZkClients() throws IOException {
|
||||||
if (zkClient != null) {
|
if (zkClient != null) {
|
||||||
try {
|
try {
|
||||||
@ -176,7 +313,8 @@ public synchronized RMState loadState() throws Exception {
|
|||||||
|
|
||||||
private synchronized void loadRMDTSecretManagerState(RMState rmState)
|
private synchronized void loadRMDTSecretManagerState(RMState rmState)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
List<String> childNodes = zkClient.getChildren(rmDTSecretManagerRoot, true);
|
List<String> childNodes =
|
||||||
|
getChildrenWithRetries(rmDTSecretManagerRoot, true);
|
||||||
|
|
||||||
for (String childNodeName : childNodes) {
|
for (String childNodeName : childNodes) {
|
||||||
if (childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
|
if (childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
|
||||||
@ -209,7 +347,7 @@ private synchronized void loadRMDTSecretManagerState(RMState rmState)
|
|||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void loadRMAppState(RMState rmState) throws Exception {
|
private synchronized void loadRMAppState(RMState rmState) throws Exception {
|
||||||
List<String> childNodes = zkClient.getChildren(rmAppRoot, true);
|
List<String> childNodes = getChildrenWithRetries(rmAppRoot, true);
|
||||||
List<ApplicationAttemptState> attempts =
|
List<ApplicationAttemptState> attempts =
|
||||||
new ArrayList<ApplicationAttemptState>();
|
new ArrayList<ApplicationAttemptState>();
|
||||||
for (String childNodeName : childNodes) {
|
for (String childNodeName : childNodes) {
|
||||||
@ -466,6 +604,8 @@ public void process(WatchedEvent event) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
public synchronized void processWatchEvent(WatchedEvent event)
|
public synchronized void processWatchEvent(WatchedEvent event)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
Event.EventType eventType = event.getType();
|
Event.EventType eventType = event.getType();
|
||||||
@ -506,65 +646,71 @@ public synchronized void processWatchEvent(WatchedEvent event)
|
|||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
String getNodePath(String root, String nodeName) {
|
String getNodePath(String root, String nodeName) {
|
||||||
return (root + "/" + nodeName);
|
return (root + "/" + nodeName);
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
/**
|
||||||
public String createWithRetries(
|
* Helper method that creates fencing node, executes the passed operations,
|
||||||
final String path, final byte[] data, final List<ACL> acl,
|
* and deletes the fencing node.
|
||||||
final CreateMode mode) throws Exception {
|
*/
|
||||||
return new ZKAction<String>() {
|
private synchronized void doMultiWithRetries(
|
||||||
|
final List<Op> opList) throws Exception {
|
||||||
|
final List<Op> execOpList = new ArrayList<Op>(opList.size() + 2);
|
||||||
|
execOpList.add(createFencingNodePathOp);
|
||||||
|
execOpList.addAll(opList);
|
||||||
|
execOpList.add(deleteFencingNodePathOp);
|
||||||
|
new ZKAction<Void>() {
|
||||||
@Override
|
@Override
|
||||||
public String run() throws KeeperException, InterruptedException {
|
public Void run() throws KeeperException, InterruptedException {
|
||||||
return zkClient.create(path, data, acl, mode);
|
zkClient.multi(execOpList);
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
}.runWithRetries();
|
}.runWithRetries();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper method that creates fencing node, executes the passed operation,
|
||||||
|
* and deletes the fencing node.
|
||||||
|
*/
|
||||||
|
private void doMultiWithRetries(final Op op) throws Exception {
|
||||||
|
doMultiWithRetries(Collections.singletonList(op));
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
|
public void createWithRetries(
|
||||||
|
final String path, final byte[] data, final List<ACL> acl,
|
||||||
|
final CreateMode mode) throws Exception {
|
||||||
|
doMultiWithRetries(Op.create(path, data, acl, mode));
|
||||||
|
}
|
||||||
|
|
||||||
private void deleteWithRetries(final String path, final int version)
|
private void deleteWithRetries(final String path, final int version)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
new ZKAction<Void>() {
|
try {
|
||||||
@Override
|
doMultiWithRetries(Op.delete(path, version));
|
||||||
public Void run() throws KeeperException, InterruptedException {
|
} catch (KeeperException.NoNodeException nne) {
|
||||||
/**
|
// We tried to delete a node that doesn't exist
|
||||||
* Call exists() to leave a watch on the node denoted by path.
|
if (LOG.isDebugEnabled()) {
|
||||||
* Delete node if exists. To pass the existence information to the
|
LOG.debug("Attempted to delete a non-existing znode " + path);
|
||||||
* caller, call delete irrespective of whether node exists or not.
|
|
||||||
*/
|
|
||||||
if (zkClient.exists(path, true) == null) {
|
|
||||||
LOG.error("Trying to delete a path (" + path
|
|
||||||
+ ") that doesn't exist.");
|
|
||||||
}
|
}
|
||||||
zkClient.delete(path, version);
|
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
}.runWithRetries();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void doMultiWithRetries(final ArrayList<Op> opList) throws Exception {
|
|
||||||
new ZKAction<Void>() {
|
|
||||||
@Override
|
|
||||||
public Void run() throws KeeperException, InterruptedException {
|
|
||||||
zkClient.multi(opList);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}.runWithRetries();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
public void setDataWithRetries(final String path, final byte[] data,
|
public void setDataWithRetries(final String path, final byte[] data,
|
||||||
final int version) throws Exception {
|
final int version) throws Exception {
|
||||||
new ZKAction<Void>() {
|
doMultiWithRetries(Op.setData(path, data, version));
|
||||||
@Override
|
|
||||||
public Void run() throws KeeperException, InterruptedException {
|
|
||||||
zkClient.setData(path, data, version);
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}.runWithRetries();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
public byte[] getDataWithRetries(final String path, final boolean watch)
|
public byte[] getDataWithRetries(final String path, final boolean watch)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
return new ZKAction<byte[]>() {
|
return new ZKAction<byte[]>() {
|
||||||
@ -576,6 +722,16 @@ public byte[] run() throws KeeperException, InterruptedException {
|
|||||||
}.runWithRetries();
|
}.runWithRetries();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private List<String> getChildrenWithRetries(
|
||||||
|
final String path, final boolean watch) throws Exception {
|
||||||
|
return new ZKAction<List<String>>() {
|
||||||
|
@Override
|
||||||
|
List<String> run() throws KeeperException, InterruptedException {
|
||||||
|
return zkClient.getChildren(path, watch);
|
||||||
|
}
|
||||||
|
}.runWithRetries();
|
||||||
|
}
|
||||||
|
|
||||||
private abstract class ZKAction<T> {
|
private abstract class ZKAction<T> {
|
||||||
// run() expects synchronization on ZKRMStateStore.this
|
// run() expects synchronization on ZKRMStateStore.this
|
||||||
abstract T run() throws KeeperException, InterruptedException;
|
abstract T run() throws KeeperException, InterruptedException;
|
||||||
@ -596,22 +752,7 @@ T runWithCheck() throws Exception {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
T runWithRetries() throws Exception {
|
private boolean shouldRetry(Code code) {
|
||||||
int retry = 0;
|
|
||||||
while (true) {
|
|
||||||
try {
|
|
||||||
return runWithCheck();
|
|
||||||
} catch (KeeperException ke) {
|
|
||||||
if (shouldRetry(ke.code()) && ++retry < numRetries) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
throw ke;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static boolean shouldRetry(Code code) {
|
|
||||||
switch (code) {
|
switch (code) {
|
||||||
case CONNECTIONLOSS:
|
case CONNECTIONLOSS:
|
||||||
case OPERATIONTIMEOUT:
|
case OPERATIONTIMEOUT:
|
||||||
@ -622,6 +763,28 @@ private static boolean shouldRetry(Code code) {
|
|||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
T runWithRetries() throws Exception {
|
||||||
|
int retry = 0;
|
||||||
|
while (true) {
|
||||||
|
try {
|
||||||
|
return runWithCheck();
|
||||||
|
} catch (KeeperException.NoAuthException nae) {
|
||||||
|
if (HAUtil.isHAEnabled(getConfig())) {
|
||||||
|
// NoAuthException possibly means that this store is fenced due to
|
||||||
|
// another RM becoming active. Even if not,
|
||||||
|
// it is safer to assume we have been fenced
|
||||||
|
throw new StoreFencedException();
|
||||||
|
}
|
||||||
|
} catch (KeeperException ke) {
|
||||||
|
if (shouldRetry(ke.code()) && ++retry < numRetries) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
throw ke;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private synchronized void createConnection()
|
private synchronized void createConnection()
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
closeZkClients();
|
closeZkClients();
|
||||||
@ -629,6 +792,10 @@ private synchronized void createConnection()
|
|||||||
retries++) {
|
retries++) {
|
||||||
try {
|
try {
|
||||||
zkClient = getNewZooKeeper();
|
zkClient = getNewZooKeeper();
|
||||||
|
if (useDefaultFencingScheme) {
|
||||||
|
zkClient.addAuthInfo(zkRootNodeAuthScheme,
|
||||||
|
(zkRootNodeUsername + ":" + zkRootNodePassword).getBytes());
|
||||||
|
}
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
// Retry in case of network failures
|
// Retry in case of network failures
|
||||||
LOG.info("Failed to connect to the ZooKeeper on attempt - " +
|
LOG.info("Failed to connect to the ZooKeeper on attempt - " +
|
||||||
@ -646,6 +813,8 @@ private synchronized void createConnection()
|
|||||||
|
|
||||||
// protected to mock for testing
|
// protected to mock for testing
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
|
@Private
|
||||||
|
@Unstable
|
||||||
protected synchronized ZooKeeper getNewZooKeeper()
|
protected synchronized ZooKeeper getNewZooKeeper()
|
||||||
throws IOException, InterruptedException {
|
throws IOException, InterruptedException {
|
||||||
ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, null);
|
ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, null);
|
||||||
|
@ -136,9 +136,6 @@ public class FairScheduler implements ResourceScheduler {
|
|||||||
// How often fair shares are re-calculated (ms)
|
// How often fair shares are re-calculated (ms)
|
||||||
protected long UPDATE_INTERVAL = 500;
|
protected long UPDATE_INTERVAL = 500;
|
||||||
|
|
||||||
// Whether to use username in place of "default" queue name
|
|
||||||
private volatile boolean userAsDefaultQueue = false;
|
|
||||||
|
|
||||||
private final static List<Container> EMPTY_CONTAINER_LIST =
|
private final static List<Container> EMPTY_CONTAINER_LIST =
|
||||||
new ArrayList<Container>();
|
new ArrayList<Container>();
|
||||||
|
|
||||||
@ -640,6 +637,12 @@ protected synchronized void addApplication(
|
|||||||
RMApp rmApp = rmContext.getRMApps().get(
|
RMApp rmApp = rmContext.getRMApps().get(
|
||||||
applicationAttemptId.getApplicationId());
|
applicationAttemptId.getApplicationId());
|
||||||
FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
|
FSLeafQueue queue = assignToQueue(rmApp, queueName, user);
|
||||||
|
if (queue == null) {
|
||||||
|
rmContext.getDispatcher().getEventHandler().handle(
|
||||||
|
new RMAppAttemptRejectedEvent(applicationAttemptId,
|
||||||
|
"Application rejected by queue placement policy"));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
FSSchedulerApp schedulerApp =
|
FSSchedulerApp schedulerApp =
|
||||||
new FSSchedulerApp(applicationAttemptId, user,
|
new FSSchedulerApp(applicationAttemptId, user,
|
||||||
@ -675,17 +678,16 @@ queue, new ActiveUsersManager(getRootQueueMetrics()),
|
|||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
FSLeafQueue assignToQueue(RMApp rmApp, String queueName, String user) {
|
FSLeafQueue assignToQueue(RMApp rmApp, String queueName, String user) {
|
||||||
// Potentially set queue to username if configured to do so
|
FSLeafQueue queue = null;
|
||||||
if (queueName.equals(YarnConfiguration.DEFAULT_QUEUE_NAME) &&
|
try {
|
||||||
userAsDefaultQueue) {
|
QueuePlacementPolicy policy = queueMgr.getPlacementPolicy();
|
||||||
queueName = user;
|
queueName = policy.assignAppToQueue(queueName, user);
|
||||||
|
if (queueName == null) {
|
||||||
|
return null;
|
||||||
}
|
}
|
||||||
|
queue = queueMgr.getLeafQueue(queueName, true);
|
||||||
FSLeafQueue queue = queueMgr.getLeafQueue(queueName,
|
} catch (IOException ex) {
|
||||||
conf.getAllowUndeclaredPools());
|
LOG.error("Error assigning app to queue, rejecting", ex);
|
||||||
if (queue == null) {
|
|
||||||
// queue is not an existing or createable leaf queue
|
|
||||||
queue = queueMgr.getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, false);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (rmApp != null) {
|
if (rmApp != null) {
|
||||||
@ -1155,7 +1157,6 @@ public synchronized void reinitialize(Configuration conf, RMContext rmContext)
|
|||||||
minimumAllocation = this.conf.getMinimumAllocation();
|
minimumAllocation = this.conf.getMinimumAllocation();
|
||||||
maximumAllocation = this.conf.getMaximumAllocation();
|
maximumAllocation = this.conf.getMaximumAllocation();
|
||||||
incrAllocation = this.conf.getIncrementAllocation();
|
incrAllocation = this.conf.getIncrementAllocation();
|
||||||
userAsDefaultQueue = this.conf.getUserAsDefaultQueue();
|
|
||||||
continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
|
continuousSchedulingEnabled = this.conf.isContinuousSchedulingEnabled();
|
||||||
continuousSchedulingSleepMs =
|
continuousSchedulingSleepMs =
|
||||||
this.conf.getContinuousSchedulingSleepMs();
|
this.conf.getContinuousSchedulingSleepMs();
|
||||||
|
@ -25,6 +25,7 @@
|
|||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
@ -51,6 +52,8 @@
|
|||||||
import org.w3c.dom.Text;
|
import org.w3c.dom.Text;
|
||||||
import org.xml.sax.SAXException;
|
import org.xml.sax.SAXException;
|
||||||
|
|
||||||
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Maintains a list of queues as well as scheduling parameters for each queue,
|
* Maintains a list of queues as well as scheduling parameters for each queue,
|
||||||
* such as guaranteed share allocations, from the fair scheduler config file.
|
* such as guaranteed share allocations, from the fair scheduler config file.
|
||||||
@ -87,6 +90,8 @@ public class QueueManager {
|
|||||||
private FSParentQueue rootQueue;
|
private FSParentQueue rootQueue;
|
||||||
|
|
||||||
private volatile QueueManagerInfo info = new QueueManagerInfo();
|
private volatile QueueManagerInfo info = new QueueManagerInfo();
|
||||||
|
@VisibleForTesting
|
||||||
|
volatile QueuePlacementPolicy placementPolicy;
|
||||||
|
|
||||||
private long lastReloadAttempt; // Last time we tried to reload the queues file
|
private long lastReloadAttempt; // Last time we tried to reload the queues file
|
||||||
private long lastSuccessfulReload; // Last time we successfully reloaded queues
|
private long lastSuccessfulReload; // Last time we successfully reloaded queues
|
||||||
@ -107,6 +112,8 @@ public void initialize() throws IOException, SAXException,
|
|||||||
queues.put(rootQueue.getName(), rootQueue);
|
queues.put(rootQueue.getName(), rootQueue);
|
||||||
|
|
||||||
this.allocFile = conf.getAllocationFile();
|
this.allocFile = conf.getAllocationFile();
|
||||||
|
placementPolicy = new QueuePlacementPolicy(getSimplePlacementRules(),
|
||||||
|
new HashSet<String>(), conf);
|
||||||
|
|
||||||
reloadAllocs();
|
reloadAllocs();
|
||||||
lastSuccessfulReload = scheduler.getClock().getTime();
|
lastSuccessfulReload = scheduler.getClock().getTime();
|
||||||
@ -115,6 +122,28 @@ public void initialize() throws IOException, SAXException,
|
|||||||
getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
|
getLeafQueue(YarnConfiguration.DEFAULT_QUEUE_NAME, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void updatePlacementPolicy(FairSchedulerConfiguration conf) {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Construct simple queue placement policy from allow-undeclared-pools and
|
||||||
|
* user-as-default-queue.
|
||||||
|
*/
|
||||||
|
private List<QueuePlacementRule> getSimplePlacementRules() {
|
||||||
|
boolean create = scheduler.getConf().getAllowUndeclaredPools();
|
||||||
|
boolean userAsDefaultQueue = scheduler.getConf().getUserAsDefaultQueue();
|
||||||
|
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
|
||||||
|
rules.add(new QueuePlacementRule.Specified().initialize(create, null));
|
||||||
|
if (userAsDefaultQueue) {
|
||||||
|
rules.add(new QueuePlacementRule.User().initialize(create, null));
|
||||||
|
}
|
||||||
|
if (!userAsDefaultQueue || !create) {
|
||||||
|
rules.add(new QueuePlacementRule.Default().initialize(true, null));
|
||||||
|
}
|
||||||
|
return rules;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a queue by name, creating it if the create param is true and is necessary.
|
* Get a queue by name, creating it if the create param is true and is necessary.
|
||||||
* If the queue is not or can not be a leaf queue, i.e. it already exists as a
|
* If the queue is not or can not be a leaf queue, i.e. it already exists as a
|
||||||
@ -227,6 +256,10 @@ public boolean exists(String name) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public QueuePlacementPolicy getPlacementPolicy() {
|
||||||
|
return placementPolicy;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reload allocations file if it hasn't been loaded in a while
|
* Reload allocations file if it hasn't been loaded in a while
|
||||||
*/
|
*/
|
||||||
@ -291,6 +324,8 @@ public void reloadAllocs() throws IOException, ParserConfigurationException,
|
|||||||
long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
|
long defaultMinSharePreemptionTimeout = Long.MAX_VALUE;
|
||||||
SchedulingPolicy defaultSchedPolicy = SchedulingPolicy.getDefault();
|
SchedulingPolicy defaultSchedPolicy = SchedulingPolicy.getDefault();
|
||||||
|
|
||||||
|
QueuePlacementPolicy newPlacementPolicy = null;
|
||||||
|
|
||||||
// Remember all queue names so we can display them on web UI, etc.
|
// Remember all queue names so we can display them on web UI, etc.
|
||||||
List<String> queueNamesInAllocFile = new ArrayList<String>();
|
List<String> queueNamesInAllocFile = new ArrayList<String>();
|
||||||
|
|
||||||
@ -306,6 +341,7 @@ public void reloadAllocs() throws IOException, ParserConfigurationException,
|
|||||||
"file: top-level element not <allocations>");
|
"file: top-level element not <allocations>");
|
||||||
NodeList elements = root.getChildNodes();
|
NodeList elements = root.getChildNodes();
|
||||||
List<Element> queueElements = new ArrayList<Element>();
|
List<Element> queueElements = new ArrayList<Element>();
|
||||||
|
Element placementPolicyElement = null;
|
||||||
for (int i = 0; i < elements.getLength(); i++) {
|
for (int i = 0; i < elements.getLength(); i++) {
|
||||||
Node node = elements.item(i);
|
Node node = elements.item(i);
|
||||||
if (node instanceof Element) {
|
if (node instanceof Element) {
|
||||||
@ -348,6 +384,8 @@ public void reloadAllocs() throws IOException, ParserConfigurationException,
|
|||||||
String text = ((Text)element.getFirstChild()).getData().trim();
|
String text = ((Text)element.getFirstChild()).getData().trim();
|
||||||
SchedulingPolicy.setDefault(text);
|
SchedulingPolicy.setDefault(text);
|
||||||
defaultSchedPolicy = SchedulingPolicy.getDefault();
|
defaultSchedPolicy = SchedulingPolicy.getDefault();
|
||||||
|
} else if ("queuePlacementPolicy".equals(element.getTagName())) {
|
||||||
|
placementPolicyElement = element;
|
||||||
} else {
|
} else {
|
||||||
LOG.warn("Bad element in allocations file: " + element.getTagName());
|
LOG.warn("Bad element in allocations file: " + element.getTagName());
|
||||||
}
|
}
|
||||||
@ -370,6 +408,15 @@ public void reloadAllocs() throws IOException, ParserConfigurationException,
|
|||||||
queueAcls, queueNamesInAllocFile);
|
queueAcls, queueNamesInAllocFile);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Load placement policy and pass it configured queues
|
||||||
|
if (placementPolicyElement != null) {
|
||||||
|
newPlacementPolicy = QueuePlacementPolicy.fromXml(placementPolicyElement,
|
||||||
|
new HashSet<String>(queueNamesInAllocFile), scheduler.getConf());
|
||||||
|
} else {
|
||||||
|
newPlacementPolicy = new QueuePlacementPolicy(getSimplePlacementRules(),
|
||||||
|
new HashSet<String>(queueNamesInAllocFile), scheduler.getConf());
|
||||||
|
}
|
||||||
|
|
||||||
// Commit the reload; also create any queue defined in the alloc file
|
// Commit the reload; also create any queue defined in the alloc file
|
||||||
// if it does not already exist, so it can be displayed on the web UI.
|
// if it does not already exist, so it can be displayed on the web UI.
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
@ -377,6 +424,7 @@ public void reloadAllocs() throws IOException, ParserConfigurationException,
|
|||||||
queueMaxApps, userMaxApps, queueWeights, userMaxAppsDefault,
|
queueMaxApps, userMaxApps, queueWeights, userMaxAppsDefault,
|
||||||
queueMaxAppsDefault, defaultSchedPolicy, minSharePreemptionTimeouts,
|
queueMaxAppsDefault, defaultSchedPolicy, minSharePreemptionTimeouts,
|
||||||
queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout);
|
queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout);
|
||||||
|
placementPolicy = newPlacementPolicy;
|
||||||
|
|
||||||
// Make sure all queues exist
|
// Make sure all queues exist
|
||||||
for (String name: queueNamesInAllocFile) {
|
for (String name: queueNamesInAllocFile) {
|
||||||
|
@ -0,0 +1,121 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.security.Groups;
|
||||||
|
import org.apache.hadoop.util.ReflectionUtils;
|
||||||
|
import org.w3c.dom.Element;
|
||||||
|
import org.w3c.dom.Node;
|
||||||
|
import org.w3c.dom.NodeList;
|
||||||
|
|
||||||
|
public class QueuePlacementPolicy {
|
||||||
|
private static final Map<String, Class<? extends QueuePlacementRule>> ruleClasses;
|
||||||
|
static {
|
||||||
|
Map<String, Class<? extends QueuePlacementRule>> map =
|
||||||
|
new HashMap<String, Class<? extends QueuePlacementRule>>();
|
||||||
|
map.put("user", QueuePlacementRule.User.class);
|
||||||
|
map.put("primaryGroup", QueuePlacementRule.PrimaryGroup.class);
|
||||||
|
map.put("specified", QueuePlacementRule.Specified.class);
|
||||||
|
map.put("default", QueuePlacementRule.Default.class);
|
||||||
|
map.put("reject", QueuePlacementRule.Reject.class);
|
||||||
|
ruleClasses = Collections.unmodifiableMap(map);
|
||||||
|
}
|
||||||
|
|
||||||
|
private final List<QueuePlacementRule> rules;
|
||||||
|
private final Set<String> configuredQueues;
|
||||||
|
private final Groups groups;
|
||||||
|
|
||||||
|
public QueuePlacementPolicy(List<QueuePlacementRule> rules,
|
||||||
|
Set<String> configuredQueues, Configuration conf)
|
||||||
|
throws AllocationConfigurationException {
|
||||||
|
for (int i = 0; i < rules.size()-1; i++) {
|
||||||
|
if (rules.get(i).isTerminal()) {
|
||||||
|
throw new AllocationConfigurationException("Rules after rule "
|
||||||
|
+ i + " in queue placement policy can never be reached");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (!rules.get(rules.size()-1).isTerminal()) {
|
||||||
|
throw new AllocationConfigurationException(
|
||||||
|
"Could get past last queue placement rule without assigning");
|
||||||
|
}
|
||||||
|
this.rules = rules;
|
||||||
|
this.configuredQueues = configuredQueues;
|
||||||
|
groups = new Groups(conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Builds a QueuePlacementPolicy from an xml element.
|
||||||
|
*/
|
||||||
|
public static QueuePlacementPolicy fromXml(Element el, Set<String> configuredQueues,
|
||||||
|
Configuration conf) throws AllocationConfigurationException {
|
||||||
|
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
|
||||||
|
NodeList elements = el.getChildNodes();
|
||||||
|
for (int i = 0; i < elements.getLength(); i++) {
|
||||||
|
Node node = elements.item(i);
|
||||||
|
if (node instanceof Element) {
|
||||||
|
Element element = (Element)node;
|
||||||
|
String ruleName = element.getTagName();
|
||||||
|
Class<? extends QueuePlacementRule> clazz = ruleClasses.get(ruleName);
|
||||||
|
if (clazz == null) {
|
||||||
|
throw new AllocationConfigurationException("No rule class found for "
|
||||||
|
+ ruleName);
|
||||||
|
}
|
||||||
|
QueuePlacementRule rule = ReflectionUtils.newInstance(clazz, null);
|
||||||
|
rule.initializeFromXml(element);
|
||||||
|
rules.add(rule);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new QueuePlacementPolicy(rules, configuredQueues, conf);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Applies this rule to an app with the given requested queue and user/group
|
||||||
|
* information.
|
||||||
|
*
|
||||||
|
* @param requestedQueue
|
||||||
|
* The queue specified in the ApplicationSubmissionContext
|
||||||
|
* @param user
|
||||||
|
* The user submitting the app
|
||||||
|
* @return
|
||||||
|
* The name of the queue to assign the app to. Or null if the app should
|
||||||
|
* be rejected.
|
||||||
|
* @throws IOException
|
||||||
|
* If an exception is encountered while getting the user's groups
|
||||||
|
*/
|
||||||
|
public String assignAppToQueue(String requestedQueue, String user)
|
||||||
|
throws IOException {
|
||||||
|
for (QueuePlacementRule rule : rules) {
|
||||||
|
String queue = rule.assignAppToQueue(requestedQueue, user, groups,
|
||||||
|
configuredQueues);
|
||||||
|
if (queue == null || !queue.isEmpty()) {
|
||||||
|
return queue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
throw new IllegalStateException("Should have applied a rule before " +
|
||||||
|
"reaching here");
|
||||||
|
}
|
||||||
|
}
|
@ -0,0 +1,200 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.hadoop.security.Groups;
|
||||||
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
import org.w3c.dom.Element;
|
||||||
|
import org.w3c.dom.NamedNodeMap;
|
||||||
|
import org.w3c.dom.Node;
|
||||||
|
|
||||||
|
public abstract class QueuePlacementRule {
|
||||||
|
protected boolean create;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initializes the rule with any arguments.
|
||||||
|
*
|
||||||
|
* @param args
|
||||||
|
* Additional attributes of the rule's xml element other than create.
|
||||||
|
*/
|
||||||
|
public QueuePlacementRule initialize(boolean create, Map<String, String> args) {
|
||||||
|
this.create = create;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* @param requestedQueue
|
||||||
|
* The queue explicitly requested.
|
||||||
|
* @param user
|
||||||
|
* The user submitting the app.
|
||||||
|
* @param groups
|
||||||
|
* The groups of the user submitting the app.
|
||||||
|
* @param configuredQueues
|
||||||
|
* The queues specified in the scheduler configuration.
|
||||||
|
* @return
|
||||||
|
* The queue to place the app into. An empty string indicates that we should
|
||||||
|
* continue to the next rule, and null indicates that the app should be rejected.
|
||||||
|
*/
|
||||||
|
public String assignAppToQueue(String requestedQueue, String user,
|
||||||
|
Groups groups, Collection<String> configuredQueues) throws IOException {
|
||||||
|
String queue = getQueueForApp(requestedQueue, user, groups);
|
||||||
|
if (create || configuredQueues.contains(queue)) {
|
||||||
|
return queue;
|
||||||
|
} else {
|
||||||
|
return "";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void initializeFromXml(Element el) {
|
||||||
|
boolean create = true;
|
||||||
|
NamedNodeMap attributes = el.getAttributes();
|
||||||
|
Map<String, String> args = new HashMap<String, String>();
|
||||||
|
for (int i = 0; i < attributes.getLength(); i++) {
|
||||||
|
Node node = attributes.item(i);
|
||||||
|
String key = node.getNodeName();
|
||||||
|
String value = node.getNodeValue();
|
||||||
|
if (key.equals("create")) {
|
||||||
|
create = Boolean.parseBoolean(value);
|
||||||
|
} else {
|
||||||
|
args.put(key, value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
initialize(create, args);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if this rule never tells the policy to continue.
|
||||||
|
*/
|
||||||
|
public abstract boolean isTerminal();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Applies this rule to an app with the given requested queue and user/group
|
||||||
|
* information.
|
||||||
|
*
|
||||||
|
* @param requestedQueue
|
||||||
|
* The queue specified in the ApplicationSubmissionContext
|
||||||
|
* @param user
|
||||||
|
* The user submitting the app.
|
||||||
|
* @param groups
|
||||||
|
* The groups of the user submitting the app.
|
||||||
|
* @return
|
||||||
|
* The name of the queue to assign the app to, or null to empty string
|
||||||
|
* continue to the next rule.
|
||||||
|
*/
|
||||||
|
protected abstract String getQueueForApp(String requestedQueue, String user,
|
||||||
|
Groups groups) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Places apps in queues by username of the submitter
|
||||||
|
*/
|
||||||
|
public static class User extends QueuePlacementRule {
|
||||||
|
@Override
|
||||||
|
protected String getQueueForApp(String requestedQueue,
|
||||||
|
String user, Groups groups) {
|
||||||
|
return "root." + user;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isTerminal() {
|
||||||
|
return create;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Places apps in queues by primary group of the submitter
|
||||||
|
*/
|
||||||
|
public static class PrimaryGroup extends QueuePlacementRule {
|
||||||
|
@Override
|
||||||
|
protected String getQueueForApp(String requestedQueue,
|
||||||
|
String user, Groups groups) throws IOException {
|
||||||
|
return "root." + groups.getGroups(user).get(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isTerminal() {
|
||||||
|
return create;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Places apps in queues by requested queue of the submitter
|
||||||
|
*/
|
||||||
|
public static class Specified extends QueuePlacementRule {
|
||||||
|
@Override
|
||||||
|
protected String getQueueForApp(String requestedQueue,
|
||||||
|
String user, Groups groups) {
|
||||||
|
if (requestedQueue.equals(YarnConfiguration.DEFAULT_QUEUE_NAME)) {
|
||||||
|
return "";
|
||||||
|
} else {
|
||||||
|
if (!requestedQueue.startsWith("root.")) {
|
||||||
|
requestedQueue = "root." + requestedQueue;
|
||||||
|
}
|
||||||
|
return requestedQueue;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isTerminal() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Places all apps in the default queue
|
||||||
|
*/
|
||||||
|
public static class Default extends QueuePlacementRule {
|
||||||
|
@Override
|
||||||
|
protected String getQueueForApp(String requestedQueue, String user,
|
||||||
|
Groups groups) {
|
||||||
|
return "root." + YarnConfiguration.DEFAULT_QUEUE_NAME;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isTerminal() {
|
||||||
|
return create;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Rejects all apps
|
||||||
|
*/
|
||||||
|
public static class Reject extends QueuePlacementRule {
|
||||||
|
@Override
|
||||||
|
public String assignAppToQueue(String requestedQueue, String user,
|
||||||
|
Groups groups, Collection<String> configuredQueues) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected String getQueueForApp(String requestedQueue, String user,
|
||||||
|
Groups groups) {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isTerminal() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -18,15 +18,32 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
|
package org.apache.hadoop.yarn.server.resourcemanager.recovery;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.ha.HAServiceProtocol;
|
||||||
|
import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
|
||||||
|
import org.apache.hadoop.service.Service;
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||||
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Priority;
|
||||||
|
import org.apache.hadoop.yarn.api.records.Resource;
|
||||||
|
import org.apache.hadoop.yarn.conf.HAUtil;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
|
||||||
|
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||||
import org.apache.zookeeper.ZooKeeper;
|
import org.apache.zookeeper.ZooKeeper;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
@ -56,7 +73,7 @@ public ZooKeeper getNewZooKeeper() throws IOException {
|
|||||||
|
|
||||||
public RMStateStore getRMStateStore() throws Exception {
|
public RMStateStore getRMStateStore() throws Exception {
|
||||||
String workingZnode = "/Test";
|
String workingZnode = "/Test";
|
||||||
YarnConfiguration conf = new YarnConfiguration();
|
Configuration conf = new YarnConfiguration();
|
||||||
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
|
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
|
||||||
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
|
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
|
||||||
this.client = createClient();
|
this.client = createClient();
|
||||||
@ -77,4 +94,81 @@ public void testZKRMStateStoreRealZK() throws Exception {
|
|||||||
testRMAppStateStore(zkTester);
|
testRMAppStateStore(zkTester);
|
||||||
testRMDTSecretManagerStateStore(zkTester);
|
testRMDTSecretManagerStateStore(zkTester);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Configuration createHARMConf(
|
||||||
|
String rmIds, String rmId, int adminPort) {
|
||||||
|
Configuration conf = new YarnConfiguration();
|
||||||
|
conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
|
||||||
|
conf.set(YarnConfiguration.RM_HA_IDS, rmIds);
|
||||||
|
conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
|
||||||
|
conf.set(YarnConfiguration.RM_STORE, ZKRMStateStore.class.getName());
|
||||||
|
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
|
||||||
|
conf.set(YarnConfiguration.RM_HA_ID, rmId);
|
||||||
|
for (String rpcAddress : HAUtil.RPC_ADDRESS_CONF_KEYS) {
|
||||||
|
conf.set(HAUtil.addSuffix(rpcAddress, rmId), "localhost:0");
|
||||||
|
}
|
||||||
|
conf.set(YarnConfiguration.RM_HA_ADMIN_ADDRESS, "localhost:" + adminPort);
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
@Test
|
||||||
|
public void testFencing() throws Exception {
|
||||||
|
StateChangeRequestInfo req = new StateChangeRequestInfo(
|
||||||
|
HAServiceProtocol.RequestSource.REQUEST_BY_USER);
|
||||||
|
|
||||||
|
Configuration conf1 = createHARMConf("rm1,rm2", "rm1", 1234);
|
||||||
|
ResourceManager rm1 = new ResourceManager();
|
||||||
|
rm1.init(conf1);
|
||||||
|
rm1.start();
|
||||||
|
rm1.getHAService().transitionToActive(req);
|
||||||
|
assertEquals("RM with ZKStore didn't start",
|
||||||
|
Service.STATE.STARTED, rm1.getServiceState());
|
||||||
|
assertEquals("RM should be Active",
|
||||||
|
HAServiceProtocol.HAServiceState.ACTIVE,
|
||||||
|
rm1.getHAService().getServiceStatus().getState());
|
||||||
|
|
||||||
|
Configuration conf2 = createHARMConf("rm1,rm2", "rm2", 5678);
|
||||||
|
ResourceManager rm2 = new ResourceManager();
|
||||||
|
rm2.init(conf2);
|
||||||
|
rm2.start();
|
||||||
|
rm2.getHAService().transitionToActive(req);
|
||||||
|
assertEquals("RM with ZKStore didn't start",
|
||||||
|
Service.STATE.STARTED, rm2.getServiceState());
|
||||||
|
assertEquals("RM should be Active",
|
||||||
|
HAServiceProtocol.HAServiceState.ACTIVE,
|
||||||
|
rm2.getHAService().getServiceStatus().getState());
|
||||||
|
|
||||||
|
// Submitting an application to RM1 to trigger a state store operation.
|
||||||
|
// RM1 should realize that it got fenced and is not the Active RM anymore.
|
||||||
|
Map mockMap = mock(Map.class);
|
||||||
|
ApplicationSubmissionContext asc =
|
||||||
|
ApplicationSubmissionContext.newInstance(
|
||||||
|
ApplicationId.newInstance(1000, 1),
|
||||||
|
"testApplication", // app Name
|
||||||
|
"default", // queue name
|
||||||
|
Priority.newInstance(0),
|
||||||
|
ContainerLaunchContext.newInstance(mockMap, mockMap,
|
||||||
|
new ArrayList<String>(), mockMap, mock(ByteBuffer.class),
|
||||||
|
mockMap),
|
||||||
|
false, // unmanaged AM
|
||||||
|
true, // cancelTokens
|
||||||
|
1, // max app attempts
|
||||||
|
Resource.newInstance(1024, 1));
|
||||||
|
ClientRMService rmService = rm1.getClientRMService();
|
||||||
|
rmService.submitApplication(SubmitApplicationRequest.newInstance(asc));
|
||||||
|
|
||||||
|
for (int i = 0; i < 30; i++) {
|
||||||
|
if (HAServiceProtocol.HAServiceState.ACTIVE == rm1.getHAService()
|
||||||
|
.getServiceStatus().getState()) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
assertEquals("RM should have been fenced",
|
||||||
|
HAServiceProtocol.HAServiceState.STANDBY,
|
||||||
|
rm1.getHAService().getServiceStatus().getState());
|
||||||
|
assertEquals("RM should be Active",
|
||||||
|
HAServiceProtocol.HAServiceState.ACTIVE,
|
||||||
|
rm2.getHAService().getServiceStatus().getState());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,44 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.security.GroupMappingServiceProvider;
|
||||||
|
|
||||||
|
public class SimpleGroupsMapping implements GroupMappingServiceProvider {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> getGroups(String user) {
|
||||||
|
return Arrays.asList(user + "group");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cacheGroupsRefresh() throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cacheGroupsAdd(List<String> groups) throws IOException {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -44,7 +44,9 @@
|
|||||||
import junit.framework.Assert;
|
import junit.framework.Assert;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||||
|
import org.apache.hadoop.security.GroupMappingServiceProvider;
|
||||||
import org.apache.hadoop.yarn.MockApps;
|
import org.apache.hadoop.yarn.MockApps;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||||
@ -94,6 +96,8 @@
|
|||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.xml.sax.SAXException;
|
import org.xml.sax.SAXException;
|
||||||
|
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
public class TestFairScheduler {
|
public class TestFairScheduler {
|
||||||
|
|
||||||
private class MockClock implements Clock {
|
private class MockClock implements Clock {
|
||||||
@ -616,6 +620,7 @@ public void testUserAsDefaultQueue() throws Exception {
|
|||||||
|
|
||||||
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
|
conf.set(FairSchedulerConfiguration.USER_AS_DEFAULT_QUEUE, "false");
|
||||||
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
scheduler.reinitialize(conf, resourceManager.getRMContext());
|
||||||
|
scheduler.getQueueManager().initialize();
|
||||||
AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent(
|
AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent(
|
||||||
createAppAttemptId(2, 1), "default", "user2");
|
createAppAttemptId(2, 1), "default", "user2");
|
||||||
scheduler.handle(appAddedEvent2);
|
scheduler.handle(appAddedEvent2);
|
||||||
@ -665,6 +670,46 @@ public void testAssignToQueue() throws Exception {
|
|||||||
assertEquals("root.notdefault", rmApp2.getQueue());
|
assertEquals("root.notdefault", rmApp2.getQueue());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testQueuePlacementWithPolicy() throws Exception {
|
||||||
|
Configuration conf = createConfiguration();
|
||||||
|
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
|
||||||
|
SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
|
||||||
|
ApplicationAttemptId appId;
|
||||||
|
Map<ApplicationAttemptId, FSSchedulerApp> apps = scheduler.applications;
|
||||||
|
|
||||||
|
List<QueuePlacementRule> rules = new ArrayList<QueuePlacementRule>();
|
||||||
|
rules.add(new QueuePlacementRule.Specified().initialize(true, null));
|
||||||
|
rules.add(new QueuePlacementRule.User().initialize(false, null));
|
||||||
|
rules.add(new QueuePlacementRule.PrimaryGroup().initialize(false, null));
|
||||||
|
rules.add(new QueuePlacementRule.Default().initialize(true, null));
|
||||||
|
Set<String> queues = Sets.newHashSet("root.user1", "root.user3group");
|
||||||
|
scheduler.getQueueManager().placementPolicy = new QueuePlacementPolicy(
|
||||||
|
rules, queues, conf);
|
||||||
|
appId = createSchedulingRequest(1024, "somequeue", "user1");
|
||||||
|
assertEquals("root.somequeue", apps.get(appId).getQueueName());
|
||||||
|
appId = createSchedulingRequest(1024, "default", "user1");
|
||||||
|
assertEquals("root.user1", apps.get(appId).getQueueName());
|
||||||
|
appId = createSchedulingRequest(1024, "default", "user3");
|
||||||
|
assertEquals("root.user3group", apps.get(appId).getQueueName());
|
||||||
|
appId = createSchedulingRequest(1024, "default", "otheruser");
|
||||||
|
assertEquals("root.default", apps.get(appId).getQueueName());
|
||||||
|
|
||||||
|
// test without specified as first rule
|
||||||
|
rules = new ArrayList<QueuePlacementRule>();
|
||||||
|
rules.add(new QueuePlacementRule.User().initialize(false, null));
|
||||||
|
rules.add(new QueuePlacementRule.Specified().initialize(true, null));
|
||||||
|
rules.add(new QueuePlacementRule.Default().initialize(true, null));
|
||||||
|
scheduler.getQueueManager().placementPolicy = new QueuePlacementPolicy(
|
||||||
|
rules, queues, conf);
|
||||||
|
appId = createSchedulingRequest(1024, "somequeue", "user1");
|
||||||
|
assertEquals("root.user1", apps.get(appId).getQueueName());
|
||||||
|
appId = createSchedulingRequest(1024, "somequeue", "otheruser");
|
||||||
|
assertEquals("root.somequeue", apps.get(appId).getQueueName());
|
||||||
|
appId = createSchedulingRequest(1024, "default", "otheruser");
|
||||||
|
assertEquals("root.default", apps.get(appId).getQueueName());
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testFairShareWithMinAlloc() throws Exception {
|
public void testFairShareWithMinAlloc() throws Exception {
|
||||||
Configuration conf = createConfiguration();
|
Configuration conf = createConfiguration();
|
||||||
|
@ -0,0 +1,119 @@
|
|||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import javax.xml.parsers.DocumentBuilder;
|
||||||
|
import javax.xml.parsers.DocumentBuilderFactory;
|
||||||
|
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||||
|
import org.apache.hadoop.security.GroupMappingServiceProvider;
|
||||||
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.w3c.dom.Document;
|
||||||
|
import org.w3c.dom.Element;
|
||||||
|
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
|
public class TestQueuePlacementPolicy {
|
||||||
|
private final static Configuration conf = new Configuration();
|
||||||
|
private final static Set<String> configuredQueues = Sets.newHashSet("root.someuser");
|
||||||
|
|
||||||
|
@BeforeClass
|
||||||
|
public static void setup() {
|
||||||
|
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
|
||||||
|
SimpleGroupsMapping.class, GroupMappingServiceProvider.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSpecifiedUserPolicy() throws Exception {
|
||||||
|
StringBuffer sb = new StringBuffer();
|
||||||
|
sb.append("<queuePlacementPolicy>");
|
||||||
|
sb.append(" <specified />");
|
||||||
|
sb.append(" <user />");
|
||||||
|
sb.append("</queuePlacementPolicy>");
|
||||||
|
QueuePlacementPolicy policy = parse(sb.toString());
|
||||||
|
assertEquals("root.specifiedq",policy.assignAppToQueue("specifiedq", "someuser"));
|
||||||
|
assertEquals("root.someuser", policy.assignAppToQueue("default", "someuser"));
|
||||||
|
assertEquals("root.otheruser", policy.assignAppToQueue("default", "otheruser"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNoCreate() throws Exception {
|
||||||
|
StringBuffer sb = new StringBuffer();
|
||||||
|
sb.append("<queuePlacementPolicy>");
|
||||||
|
sb.append(" <specified />");
|
||||||
|
sb.append(" <user create=\"false\" />");
|
||||||
|
sb.append(" <default />");
|
||||||
|
sb.append("</queuePlacementPolicy>");
|
||||||
|
QueuePlacementPolicy policy = parse(sb.toString());
|
||||||
|
assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "someuser"));
|
||||||
|
assertEquals("root.someuser", policy.assignAppToQueue("default", "someuser"));
|
||||||
|
assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "otheruser"));
|
||||||
|
assertEquals("root.default", policy.assignAppToQueue("default", "otheruser"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSpecifiedThenReject() throws Exception {
|
||||||
|
StringBuffer sb = new StringBuffer();
|
||||||
|
sb.append("<queuePlacementPolicy>");
|
||||||
|
sb.append(" <specified />");
|
||||||
|
sb.append(" <reject />");
|
||||||
|
sb.append("</queuePlacementPolicy>");
|
||||||
|
QueuePlacementPolicy policy = parse(sb.toString());
|
||||||
|
assertEquals("root.specifiedq", policy.assignAppToQueue("specifiedq", "someuser"));
|
||||||
|
assertEquals(null, policy.assignAppToQueue("default", "someuser"));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test (expected = AllocationConfigurationException.class)
|
||||||
|
public void testOmittedTerminalRule() throws Exception {
|
||||||
|
StringBuffer sb = new StringBuffer();
|
||||||
|
sb.append("<queuePlacementPolicy>");
|
||||||
|
sb.append(" <specified />");
|
||||||
|
sb.append(" <user create=\"false\" />");
|
||||||
|
sb.append("</queuePlacementPolicy>");
|
||||||
|
parse(sb.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test (expected = AllocationConfigurationException.class)
|
||||||
|
public void testTerminalRuleInMiddle() throws Exception {
|
||||||
|
StringBuffer sb = new StringBuffer();
|
||||||
|
sb.append("<queuePlacementPolicy>");
|
||||||
|
sb.append(" <specified />");
|
||||||
|
sb.append(" <default />");
|
||||||
|
sb.append(" <user />");
|
||||||
|
sb.append("</queuePlacementPolicy>");
|
||||||
|
parse(sb.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
private QueuePlacementPolicy parse(String str) throws Exception {
|
||||||
|
// Read and parse the allocations file.
|
||||||
|
DocumentBuilderFactory docBuilderFactory =
|
||||||
|
DocumentBuilderFactory.newInstance();
|
||||||
|
docBuilderFactory.setIgnoringComments(true);
|
||||||
|
DocumentBuilder builder = docBuilderFactory.newDocumentBuilder();
|
||||||
|
Document doc = builder.parse(IOUtils.toInputStream(str));
|
||||||
|
Element root = doc.getDocumentElement();
|
||||||
|
return QueuePlacementPolicy.fromXml(root, configuredQueues, conf);
|
||||||
|
}
|
||||||
|
}
|
@ -101,6 +101,16 @@ Hadoop MapReduce Next Generation - Fair Scheduler
|
|||||||
Fair Scheduler. Among them, is the use of a custom policies governing
|
Fair Scheduler. Among them, is the use of a custom policies governing
|
||||||
priority “boosting” over certain apps.
|
priority “boosting” over certain apps.
|
||||||
|
|
||||||
|
* {Automatically placing applications in queues}
|
||||||
|
|
||||||
|
The Fair Scheduler allows administrators to configure policies that
|
||||||
|
automatically place submitted applications into appropriate queues. Placement
|
||||||
|
can depend on the user and groups of the submitter and the requested queue
|
||||||
|
passed by the application. A policy consists of a set of rules that are applied
|
||||||
|
sequentially to classify an incoming application. Each rule either places the
|
||||||
|
app into a queue, rejects it, or continues on to the next rule. Refer to the
|
||||||
|
allocation file format below for how to configure these policies.
|
||||||
|
|
||||||
* {Installation}
|
* {Installation}
|
||||||
|
|
||||||
To use the Fair Scheduler first assign the appropriate scheduler class in
|
To use the Fair Scheduler first assign the appropriate scheduler class in
|
||||||
@ -138,7 +148,8 @@ Properties that can be placed in yarn-site.xml
|
|||||||
* Whether to use the username associated with the allocation as the default
|
* Whether to use the username associated with the allocation as the default
|
||||||
queue name, in the event that a queue name is not specified. If this is set
|
queue name, in the event that a queue name is not specified. If this is set
|
||||||
to "false" or unset, all jobs have a shared default queue, named "default".
|
to "false" or unset, all jobs have a shared default queue, named "default".
|
||||||
Defaults to true.
|
Defaults to true. If a queue placement policy is given in the allocations
|
||||||
|
file, this property is ignored.
|
||||||
|
|
||||||
* <<<yarn.scheduler.fair.preemption>>>
|
* <<<yarn.scheduler.fair.preemption>>>
|
||||||
|
|
||||||
@ -180,6 +191,16 @@ Properties that can be placed in yarn-site.xml
|
|||||||
opportunities to pass up. The default value of -1.0 means don't pass up any
|
opportunities to pass up. The default value of -1.0 means don't pass up any
|
||||||
scheduling opportunities.
|
scheduling opportunities.
|
||||||
|
|
||||||
|
* <<<yarn.scheduler.fair.allow-undeclared-pools>>>
|
||||||
|
|
||||||
|
* If this is true, new queues can be created at application submission time,
|
||||||
|
whether because they are specified as the application's queue by the
|
||||||
|
submitter or because they are placed there by the user-as-default-queue
|
||||||
|
property. If this is false, any time an app would be placed in a queue that
|
||||||
|
is not specified in the allocations file, it is placed in the "default" queue
|
||||||
|
instead. Defaults to true. If a queue placement policy is given in the
|
||||||
|
allocations file, this property is ignored.
|
||||||
|
|
||||||
Allocation file format
|
Allocation file format
|
||||||
|
|
||||||
The allocation file must be in XML format. The format contains five types of
|
The allocation file must be in XML format. The format contains five types of
|
||||||
@ -248,8 +269,61 @@ Allocation file format
|
|||||||
policy for queues; overriden by the schedulingPolicy element in each queue
|
policy for queues; overriden by the schedulingPolicy element in each queue
|
||||||
if specified. Defaults to "fair".
|
if specified. Defaults to "fair".
|
||||||
|
|
||||||
|
* <<A queuePlacementPolicy element>>, which contains a list of rule elements
|
||||||
|
that tell the scheduler how to place incoming apps into queues. Rules
|
||||||
|
are applied in the order that they are listed. Rules may take arguments. All
|
||||||
|
rules accept the "create" argument, which indicates whether the rule can create
|
||||||
|
a new queue. "Create" defaults to true; if set to false and the rule would
|
||||||
|
place the app in a queue that is not configured in the allocations file, we
|
||||||
|
continue on to the next rule. The last rule must be one that can never issue a
|
||||||
|
continue. Valid rules are:
|
||||||
|
|
||||||
|
* specified: the app is placed into the queue it requested. If the app
|
||||||
|
requested no queue, i.e. it specified "default", we continue.
|
||||||
|
|
||||||
|
* user: the app is placed into a queue with the name of the user who
|
||||||
|
submitted it.
|
||||||
|
|
||||||
|
* primaryGroup: the app is placed into a queue with the name of the
|
||||||
|
primary group of the user who submitted it.
|
||||||
|
|
||||||
|
* default: the app is placed into the queue named "default".
|
||||||
|
|
||||||
|
* reject: the app is rejected.
|
||||||
|
|
||||||
An example allocation file is given here:
|
An example allocation file is given here:
|
||||||
|
|
||||||
|
---
|
||||||
|
<?xml version="1.0"?>
|
||||||
|
<allocations>
|
||||||
|
<queue name="sample_queue">
|
||||||
|
<minResources>10000 mb,0vcores</minResources>
|
||||||
|
<maxResources>90000 mb,0vcores</maxResources>
|
||||||
|
<maxRunningApps>50</maxRunningApps>
|
||||||
|
<weight>2.0</weight>
|
||||||
|
<schedulingPolicy>fair</schedulingPolicy>
|
||||||
|
<queue name="sample_sub_queue">
|
||||||
|
<aclSubmitApps>charlie</aclSubmitApps>
|
||||||
|
<minResources>5000 mb,0vcores</minResources>
|
||||||
|
</queue>
|
||||||
|
</queue>
|
||||||
|
|
||||||
|
<user name="sample_user">
|
||||||
|
<maxRunningApps>30</maxRunningApps>
|
||||||
|
</user>
|
||||||
|
<userMaxAppsDefault>5</userMaxAppsDefault>
|
||||||
|
|
||||||
|
<queuePlacementPolicy>
|
||||||
|
<specified />
|
||||||
|
<primarygroup create="false" />
|
||||||
|
<default />
|
||||||
|
</queuePlacementPolicy>
|
||||||
|
</allocations>
|
||||||
|
---
|
||||||
|
|
||||||
|
Note that for backwards compatibility with the original FairScheduler, "queue" elements can instead be named as "pool" elements.
|
||||||
|
|
||||||
|
|
||||||
Queue Access Control Lists (ACLs)
|
Queue Access Control Lists (ACLs)
|
||||||
|
|
||||||
Queue Access Control Lists (ACLs) allow administrators to control who may
|
Queue Access Control Lists (ACLs) allow administrators to control who may
|
||||||
@ -268,28 +342,6 @@ Queue Access Control Lists (ACLs)
|
|||||||
To start restricting access, change the root queue's ACLs to something other
|
To start restricting access, change the root queue's ACLs to something other
|
||||||
than "*".
|
than "*".
|
||||||
|
|
||||||
---
|
|
||||||
<?xml version="1.0"?>
|
|
||||||
<allocations>
|
|
||||||
<queue name="sample_queue">
|
|
||||||
<minResources>10000 mb,0vcores</minResources>
|
|
||||||
<maxResources>90000 mb,0vcores</maxResources>
|
|
||||||
<maxRunningApps>50</maxRunningApps>
|
|
||||||
<weight>2.0</weight>
|
|
||||||
<schedulingPolicy>fair</schedulingPolicy>
|
|
||||||
<queue name="sample_sub_queue">
|
|
||||||
<aclSubmitApps>charlie</aclSubmitApps>
|
|
||||||
<minResources>5000 mb,0vcores</minResources>
|
|
||||||
</queue>
|
|
||||||
</queue>
|
|
||||||
<user name="sample_user">
|
|
||||||
<maxRunningApps>30</maxRunningApps>
|
|
||||||
</user>
|
|
||||||
<userMaxAppsDefault>5</userMaxAppsDefault>
|
|
||||||
</allocations>
|
|
||||||
---
|
|
||||||
|
|
||||||
Note that for backwards compatibility with the original FairScheduler, "queue" elements can instead be named as "pool" elements.
|
|
||||||
|
|
||||||
* {Administration}
|
* {Administration}
|
||||||
|
|
||||||
|
@ -119,6 +119,7 @@
|
|||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.sun.jersey.jersey-test-framework</groupId>
|
<groupId>com.sun.jersey.jersey-test-framework</groupId>
|
||||||
<artifactId>jersey-test-framework-grizzly2</artifactId>
|
<artifactId>jersey-test-framework-grizzly2</artifactId>
|
||||||
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.sun.jersey</groupId>
|
<groupId>com.sun.jersey</groupId>
|
||||||
|
Loading…
x
Reference in New Issue
Block a user