merge trunk into HDFS-4949 branch
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4949@1535563 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
e87b2a3684
|
@ -363,6 +363,15 @@ Release 2.3.0 - UNRELEASED
|
|||
HADOOP-9078. enhance unit-test coverage of class
|
||||
org.apache.hadoop.fs.FileContext (Ivan A. Veselovsky via jeagles)
|
||||
|
||||
HDFS-5276. FileSystem.Statistics should use thread-local counters to avoid
|
||||
multi-threaded performance issues on read/write. (Colin Patrick McCabe)
|
||||
|
||||
HADOOP-9291. enhance unit-test coverage of package o.a.h.metrics2 (Ivan A.
|
||||
Veselovsky via jeagles)
|
||||
|
||||
HADOOP-10064. Upgrade to maven antrun plugin version 1.7 (Arpit Agarwal via
|
||||
jeagles)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)
|
||||
|
@ -405,6 +414,9 @@ Release 2.3.0 - UNRELEASED
|
|||
HADOOP-9981. globStatus should minimize its listStatus and getFileStatus
|
||||
calls. (Contributed by Colin Patrick McCabe)
|
||||
|
||||
HADOOP-9016. HarFsInputStream.skip(long) must never return negative value.
|
||||
(Ivan A. Veselovsky via jeagles)
|
||||
|
||||
Release 2.2.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -434,6 +446,9 @@ Release 2.2.1 - UNRELEASED
|
|||
HADOOP-10040. hadoop.cmd in UNIX format and would not run by default on
|
||||
Windows. (cnauroth)
|
||||
|
||||
HADOOP-10055. FileSystemShell.apt.vm doc has typo "numRepicas".
|
||||
(Akira Ajisaka via cnauroth)
|
||||
|
||||
Release 2.2.0 - 2013-10-13
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
|
|||
import java.io.Closeable;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.lang.ref.WeakReference;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
@ -31,6 +32,7 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NoSuchElementException;
|
||||
|
@ -2501,28 +2503,149 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tracks statistics about how many reads, writes, and so forth have been
|
||||
* done in a FileSystem.
|
||||
*
|
||||
* Since there is only one of these objects per FileSystem, there will
|
||||
* typically be many threads writing to this object. Almost every operation
|
||||
* on an open file will involve a write to this object. In contrast, reading
|
||||
* statistics is done infrequently by most programs, and not at all by others.
|
||||
* Hence, this is optimized for writes.
|
||||
*
|
||||
* Each thread writes to its own thread-local area of memory. This removes
|
||||
* contention and allows us to scale up to many, many threads. To read
|
||||
* statistics, the reader thread totals up the contents of all of the
|
||||
* thread-local data areas.
|
||||
*/
|
||||
public static final class Statistics {
|
||||
/**
|
||||
* Statistics data.
|
||||
*
|
||||
* There is only a single writer to thread-local StatisticsData objects.
|
||||
* Hence, volatile is adequate here-- we do not need AtomicLong or similar
|
||||
* to prevent lost updates.
|
||||
* The Java specification guarantees that updates to volatile longs will
|
||||
* be perceived as atomic with respect to other threads, which is all we
|
||||
* need.
|
||||
*/
|
||||
private static class StatisticsData {
|
||||
volatile long bytesRead;
|
||||
volatile long bytesWritten;
|
||||
volatile int readOps;
|
||||
volatile int largeReadOps;
|
||||
volatile int writeOps;
|
||||
/**
|
||||
* Stores a weak reference to the thread owning this StatisticsData.
|
||||
* This allows us to remove StatisticsData objects that pertain to
|
||||
* threads that no longer exist.
|
||||
*/
|
||||
final WeakReference<Thread> owner;
|
||||
|
||||
StatisticsData(WeakReference<Thread> owner) {
|
||||
this.owner = owner;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add another StatisticsData object to this one.
|
||||
*/
|
||||
void add(StatisticsData other) {
|
||||
this.bytesRead += other.bytesRead;
|
||||
this.bytesWritten += other.bytesWritten;
|
||||
this.readOps += other.readOps;
|
||||
this.largeReadOps += other.largeReadOps;
|
||||
this.writeOps += other.writeOps;
|
||||
}
|
||||
|
||||
/**
|
||||
* Negate the values of all statistics.
|
||||
*/
|
||||
void negate() {
|
||||
this.bytesRead = -this.bytesRead;
|
||||
this.bytesWritten = -this.bytesWritten;
|
||||
this.readOps = -this.readOps;
|
||||
this.largeReadOps = -this.largeReadOps;
|
||||
this.writeOps = -this.writeOps;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return bytesRead + " bytes read, " + bytesWritten + " bytes written, "
|
||||
+ readOps + " read ops, " + largeReadOps + " large read ops, "
|
||||
+ writeOps + " write ops";
|
||||
}
|
||||
}
|
||||
|
||||
private interface StatisticsAggregator<T> {
|
||||
void accept(StatisticsData data);
|
||||
T aggregate();
|
||||
}
|
||||
|
||||
private final String scheme;
|
||||
private AtomicLong bytesRead = new AtomicLong();
|
||||
private AtomicLong bytesWritten = new AtomicLong();
|
||||
private AtomicInteger readOps = new AtomicInteger();
|
||||
private AtomicInteger largeReadOps = new AtomicInteger();
|
||||
private AtomicInteger writeOps = new AtomicInteger();
|
||||
|
||||
/**
|
||||
* rootData is data that doesn't belong to any thread, but will be added
|
||||
* to the totals. This is useful for making copies of Statistics objects,
|
||||
* and for storing data that pertains to threads that have been garbage
|
||||
* collected. Protected by the Statistics lock.
|
||||
*/
|
||||
private final StatisticsData rootData;
|
||||
|
||||
/**
|
||||
* Thread-local data.
|
||||
*/
|
||||
private final ThreadLocal<StatisticsData> threadData;
|
||||
|
||||
/**
|
||||
* List of all thread-local data areas. Protected by the Statistics lock.
|
||||
*/
|
||||
private LinkedList<StatisticsData> allData;
|
||||
|
||||
public Statistics(String scheme) {
|
||||
this.scheme = scheme;
|
||||
this.rootData = new StatisticsData(null);
|
||||
this.threadData = new ThreadLocal<StatisticsData>();
|
||||
this.allData = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy constructor.
|
||||
*
|
||||
* @param st
|
||||
* The input Statistics object which is cloned.
|
||||
* @param other The input Statistics object which is cloned.
|
||||
*/
|
||||
public Statistics(Statistics st) {
|
||||
this.scheme = st.scheme;
|
||||
this.bytesRead = new AtomicLong(st.bytesRead.longValue());
|
||||
this.bytesWritten = new AtomicLong(st.bytesWritten.longValue());
|
||||
public Statistics(Statistics other) {
|
||||
this.scheme = other.scheme;
|
||||
this.rootData = new StatisticsData(null);
|
||||
other.visitAll(new StatisticsAggregator<Void>() {
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
rootData.add(data);
|
||||
}
|
||||
|
||||
public Void aggregate() {
|
||||
return null;
|
||||
}
|
||||
});
|
||||
this.threadData = new ThreadLocal<StatisticsData>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get or create the thread-local data associated with the current thread.
|
||||
*/
|
||||
private StatisticsData getThreadData() {
|
||||
StatisticsData data = threadData.get();
|
||||
if (data == null) {
|
||||
data = new StatisticsData(
|
||||
new WeakReference<Thread>(Thread.currentThread()));
|
||||
threadData.set(data);
|
||||
synchronized(this) {
|
||||
if (allData == null) {
|
||||
allData = new LinkedList<StatisticsData>();
|
||||
}
|
||||
allData.add(data);
|
||||
}
|
||||
}
|
||||
return data;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2530,7 +2653,7 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @param newBytes the additional bytes read
|
||||
*/
|
||||
public void incrementBytesRead(long newBytes) {
|
||||
bytesRead.getAndAdd(newBytes);
|
||||
getThreadData().bytesRead += newBytes;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2538,7 +2661,7 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @param newBytes the additional bytes written
|
||||
*/
|
||||
public void incrementBytesWritten(long newBytes) {
|
||||
bytesWritten.getAndAdd(newBytes);
|
||||
getThreadData().bytesWritten += newBytes;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2546,7 +2669,7 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @param count number of read operations
|
||||
*/
|
||||
public void incrementReadOps(int count) {
|
||||
readOps.getAndAdd(count);
|
||||
getThreadData().readOps += count;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2554,7 +2677,7 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @param count number of large read operations
|
||||
*/
|
||||
public void incrementLargeReadOps(int count) {
|
||||
largeReadOps.getAndAdd(count);
|
||||
getThreadData().largeReadOps += count;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2562,7 +2685,38 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @param count number of write operations
|
||||
*/
|
||||
public void incrementWriteOps(int count) {
|
||||
writeOps.getAndAdd(count);
|
||||
getThreadData().writeOps += count;
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply the given aggregator to all StatisticsData objects associated with
|
||||
* this Statistics object.
|
||||
*
|
||||
* For each StatisticsData object, we will call accept on the visitor.
|
||||
* Finally, at the end, we will call aggregate to get the final total.
|
||||
*
|
||||
* @param The visitor to use.
|
||||
* @return The total.
|
||||
*/
|
||||
private synchronized <T> T visitAll(StatisticsAggregator<T> visitor) {
|
||||
visitor.accept(rootData);
|
||||
if (allData != null) {
|
||||
for (Iterator<StatisticsData> iter = allData.iterator();
|
||||
iter.hasNext(); ) {
|
||||
StatisticsData data = iter.next();
|
||||
visitor.accept(data);
|
||||
if (data.owner.get() == null) {
|
||||
/*
|
||||
* If the thread that created this thread-local data no
|
||||
* longer exists, remove the StatisticsData from our list
|
||||
* and fold the values into rootData.
|
||||
*/
|
||||
rootData.add(data);
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
return visitor.aggregate();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2570,7 +2724,18 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @return the number of bytes
|
||||
*/
|
||||
public long getBytesRead() {
|
||||
return bytesRead.get();
|
||||
return visitAll(new StatisticsAggregator<Long>() {
|
||||
private long bytesRead = 0;
|
||||
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
bytesRead += data.bytesRead;
|
||||
}
|
||||
|
||||
public Long aggregate() {
|
||||
return bytesRead;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2578,7 +2743,18 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @return the number of bytes
|
||||
*/
|
||||
public long getBytesWritten() {
|
||||
return bytesWritten.get();
|
||||
return visitAll(new StatisticsAggregator<Long>() {
|
||||
private long bytesWritten = 0;
|
||||
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
bytesWritten += data.bytesWritten;
|
||||
}
|
||||
|
||||
public Long aggregate() {
|
||||
return bytesWritten;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2586,7 +2762,19 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @return number of read operations
|
||||
*/
|
||||
public int getReadOps() {
|
||||
return readOps.get() + largeReadOps.get();
|
||||
return visitAll(new StatisticsAggregator<Integer>() {
|
||||
private int readOps = 0;
|
||||
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
readOps += data.readOps;
|
||||
readOps += data.largeReadOps;
|
||||
}
|
||||
|
||||
public Integer aggregate() {
|
||||
return readOps;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2595,7 +2783,18 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @return number of large read operations
|
||||
*/
|
||||
public int getLargeReadOps() {
|
||||
return largeReadOps.get();
|
||||
return visitAll(new StatisticsAggregator<Integer>() {
|
||||
private int largeReadOps = 0;
|
||||
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
largeReadOps += data.largeReadOps;
|
||||
}
|
||||
|
||||
public Integer aggregate() {
|
||||
return largeReadOps;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2604,22 +2803,70 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* @return number of write operations
|
||||
*/
|
||||
public int getWriteOps() {
|
||||
return writeOps.get();
|
||||
return visitAll(new StatisticsAggregator<Integer>() {
|
||||
private int writeOps = 0;
|
||||
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
writeOps += data.writeOps;
|
||||
}
|
||||
|
||||
public Integer aggregate() {
|
||||
return writeOps;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return bytesRead + " bytes read, " + bytesWritten + " bytes written, "
|
||||
+ readOps + " read ops, " + largeReadOps + " large read ops, "
|
||||
+ writeOps + " write ops";
|
||||
return visitAll(new StatisticsAggregator<String>() {
|
||||
private StatisticsData total = new StatisticsData(null);
|
||||
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
total.add(data);
|
||||
}
|
||||
|
||||
public String aggregate() {
|
||||
return total.toString();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Reset the counts of bytes to 0.
|
||||
* Resets all statistics to 0.
|
||||
*
|
||||
* In order to reset, we add up all the thread-local statistics data, and
|
||||
* set rootData to the negative of that.
|
||||
*
|
||||
* This may seem like a counterintuitive way to reset the statsitics. Why
|
||||
* can't we just zero out all the thread-local data? Well, thread-local
|
||||
* data can only be modified by the thread that owns it. If we tried to
|
||||
* modify the thread-local data from this thread, our modification might get
|
||||
* interleaved with a read-modify-write operation done by the thread that
|
||||
* owns the data. That would result in our update getting lost.
|
||||
*
|
||||
* The approach used here avoids this problem because it only ever reads
|
||||
* (not writes) the thread-local data. Both reads and writes to rootData
|
||||
* are done under the lock, so we're free to modify rootData from any thread
|
||||
* that holds the lock.
|
||||
*/
|
||||
public void reset() {
|
||||
bytesWritten.set(0);
|
||||
bytesRead.set(0);
|
||||
visitAll(new StatisticsAggregator<Void>() {
|
||||
private StatisticsData total = new StatisticsData(null);
|
||||
|
||||
@Override
|
||||
public void accept(StatisticsData data) {
|
||||
total.add(data);
|
||||
}
|
||||
|
||||
public Void aggregate() {
|
||||
total.negate();
|
||||
rootData.add(total);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -898,11 +898,15 @@ public class HarFileSystem extends FileSystem {
|
|||
private long position, start, end;
|
||||
//The underlying data input stream that the
|
||||
// underlying filesystem will return.
|
||||
private FSDataInputStream underLyingStream;
|
||||
private final FSDataInputStream underLyingStream;
|
||||
//one byte buffer
|
||||
private byte[] oneBytebuff = new byte[1];
|
||||
private final byte[] oneBytebuff = new byte[1];
|
||||
|
||||
HarFsInputStream(FileSystem fs, Path path, long start,
|
||||
long length, int bufferSize) throws IOException {
|
||||
if (length < 0) {
|
||||
throw new IllegalArgumentException("Negative length ["+length+"]");
|
||||
}
|
||||
underLyingStream = fs.open(path, bufferSize);
|
||||
underLyingStream.seek(start);
|
||||
// the start of this file in the part file
|
||||
|
@ -916,7 +920,7 @@ public class HarFileSystem extends FileSystem {
|
|||
@Override
|
||||
public synchronized int available() throws IOException {
|
||||
long remaining = end - underLyingStream.getPos();
|
||||
if (remaining > (long)Integer.MAX_VALUE) {
|
||||
if (remaining > Integer.MAX_VALUE) {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
return (int) remaining;
|
||||
|
@ -948,10 +952,14 @@ public class HarFileSystem extends FileSystem {
|
|||
return (ret <= 0) ? -1: (oneBytebuff[0] & 0xff);
|
||||
}
|
||||
|
||||
// NB: currently this method actually never executed becusae
|
||||
// java.io.DataInputStream.read(byte[]) directly delegates to
|
||||
// method java.io.InputStream.read(byte[], int, int).
|
||||
// However, potentially it can be invoked, so leave it intact for now.
|
||||
@Override
|
||||
public synchronized int read(byte[] b) throws IOException {
|
||||
int ret = read(b, 0, b.length);
|
||||
if (ret != -1) {
|
||||
final int ret = read(b, 0, b.length);
|
||||
if (ret > 0) {
|
||||
position += ret;
|
||||
}
|
||||
return ret;
|
||||
|
@ -980,15 +988,19 @@ public class HarFileSystem extends FileSystem {
|
|||
public synchronized long skip(long n) throws IOException {
|
||||
long tmpN = n;
|
||||
if (tmpN > 0) {
|
||||
if (position + tmpN > end) {
|
||||
tmpN = end - position;
|
||||
}
|
||||
final long actualRemaining = end - position;
|
||||
if (tmpN > actualRemaining) {
|
||||
tmpN = actualRemaining;
|
||||
}
|
||||
underLyingStream.seek(tmpN + position);
|
||||
position += tmpN;
|
||||
return tmpN;
|
||||
}
|
||||
return (tmpN < 0)? -1 : 0;
|
||||
}
|
||||
}
|
||||
// NB: the contract is described in java.io.InputStream.skip(long):
|
||||
// this method returns the number of bytes actually skipped, so,
|
||||
// the return value should never be negative.
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getPos() throws IOException {
|
||||
|
@ -996,14 +1008,23 @@ public class HarFileSystem extends FileSystem {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized void seek(long pos) throws IOException {
|
||||
if (pos < 0 || (start + pos > end)) {
|
||||
throw new IOException("Failed to seek: EOF");
|
||||
}
|
||||
public synchronized void seek(final long pos) throws IOException {
|
||||
validatePosition(pos);
|
||||
position = start + pos;
|
||||
underLyingStream.seek(position);
|
||||
}
|
||||
|
||||
private void validatePosition(final long pos) throws IOException {
|
||||
if (pos < 0) {
|
||||
throw new IOException("Negative position: "+pos);
|
||||
}
|
||||
final long length = end - start;
|
||||
if (pos > length) {
|
||||
throw new IOException("Position behind the end " +
|
||||
"of the stream (length = "+length+"): " + pos);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean seekToNewSource(long targetPos) throws IOException {
|
||||
// do not need to implement this
|
||||
|
@ -1020,7 +1041,12 @@ public class HarFileSystem extends FileSystem {
|
|||
throws IOException {
|
||||
int nlength = length;
|
||||
if (start + nlength + pos > end) {
|
||||
nlength = (int) (end - (start + pos));
|
||||
// length corrected to the real remaining length:
|
||||
nlength = (int) (end - start - pos);
|
||||
}
|
||||
if (nlength <= 0) {
|
||||
// EOS:
|
||||
return -1;
|
||||
}
|
||||
return underLyingStream.read(pos + start , b, offset, nlength);
|
||||
}
|
||||
|
|
|
@ -112,7 +112,7 @@ public abstract class AbstractPatternFilter extends MetricsFilter {
|
|||
return false;
|
||||
}
|
||||
// Reject if no match in whitelist only mode
|
||||
if (ipat != null && epat == null) {
|
||||
if (!includeTagPatterns.isEmpty() && excludeTagPatterns.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
|
|
|
@ -234,7 +234,7 @@
|
|||
patterns.
|
||||
</p>
|
||||
<p>Similarly, you can specify the <code>record.filter</code> and
|
||||
<code>metrics.filter</code> options, which operate at record and metric
|
||||
<code>metric.filter</code> options, which operate at record and metric
|
||||
level, respectively. Filters can be combined to optimize
|
||||
the filtering efficiency.</p>
|
||||
|
||||
|
|
|
@ -381,7 +381,7 @@ rmr
|
|||
|
||||
setrep
|
||||
|
||||
Usage: <<<hdfs dfs -setrep [-R] [-w] <numRepicas> <path> >>>
|
||||
Usage: <<<hdfs dfs -setrep [-R] [-w] <numReplicas> <path> >>>
|
||||
|
||||
Changes the replication factor of a file. If <path> is a directory then
|
||||
the command recursively changes the replication factor of all files under
|
||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.hadoop.fs.FileSystem.Statistics;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.util.concurrent.Uninterruptibles;
|
||||
|
||||
import static org.apache.hadoop.fs.FileContextTestHelper.*;
|
||||
|
||||
/**
|
||||
|
@ -44,6 +46,38 @@ public abstract class FCStatisticsBaseTest {
|
|||
//fc should be set appropriately by the deriving test.
|
||||
protected static FileContext fc = null;
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testStatisticsOperations() throws Exception {
|
||||
final Statistics stats = new Statistics("file");
|
||||
Assert.assertEquals(0L, stats.getBytesRead());
|
||||
Assert.assertEquals(0L, stats.getBytesWritten());
|
||||
Assert.assertEquals(0, stats.getWriteOps());
|
||||
stats.incrementBytesWritten(1000);
|
||||
Assert.assertEquals(1000L, stats.getBytesWritten());
|
||||
Assert.assertEquals(0, stats.getWriteOps());
|
||||
stats.incrementWriteOps(123);
|
||||
Assert.assertEquals(123, stats.getWriteOps());
|
||||
|
||||
Thread thread = new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
stats.incrementWriteOps(1);
|
||||
}
|
||||
};
|
||||
thread.start();
|
||||
Uninterruptibles.joinUninterruptibly(thread);
|
||||
Assert.assertEquals(124, stats.getWriteOps());
|
||||
// Test copy constructor and reset function
|
||||
Statistics stats2 = new Statistics(stats);
|
||||
stats.reset();
|
||||
Assert.assertEquals(0, stats.getWriteOps());
|
||||
Assert.assertEquals(0L, stats.getBytesWritten());
|
||||
Assert.assertEquals(0L, stats.getBytesRead());
|
||||
Assert.assertEquals(124, stats2.getWriteOps());
|
||||
Assert.assertEquals(1000L, stats2.getBytesWritten());
|
||||
Assert.assertEquals(0L, stats2.getBytesRead());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStatistics() throws IOException, URISyntaxException {
|
||||
URI fsUri = getFsUri();
|
||||
|
|
|
@ -23,9 +23,11 @@ import java.util.List;
|
|||
|
||||
import org.apache.commons.configuration.SubsetConfiguration;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
import org.apache.hadoop.metrics2.MetricsFilter;
|
||||
import org.apache.hadoop.metrics2.MetricsRecord;
|
||||
import org.apache.hadoop.metrics2.MetricsTag;
|
||||
import org.apache.hadoop.metrics2.impl.ConfigBuilder;
|
||||
|
@ -53,7 +55,7 @@ public class TestPatternFilter {
|
|||
.add("p.include.tags", "foo:f").subset("p");
|
||||
shouldAccept(wl, "foo");
|
||||
shouldAccept(wl, Arrays.asList(tag("bar", "", ""),
|
||||
tag("foo", "", "f")));
|
||||
tag("foo", "", "f")), new boolean[] {false, true});
|
||||
shouldAccept(wl, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("bar", "", ""), tag("foo", "", "f"))));
|
||||
shouldReject(wl, "bar");
|
||||
|
@ -78,7 +80,7 @@ public class TestPatternFilter {
|
|||
tag("bar", "", ""))));
|
||||
shouldReject(bl, "foo");
|
||||
shouldReject(bl, Arrays.asList(tag("bar", "", ""),
|
||||
tag("foo", "", "f")));
|
||||
tag("foo", "", "f")), new boolean[] {true, false});
|
||||
shouldReject(bl, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("bar", "", ""))));
|
||||
shouldReject(bl, mockMetricsRecord("bar", Arrays.asList(
|
||||
|
@ -125,15 +127,61 @@ public class TestPatternFilter {
|
|||
shouldAccept(c, mockMetricsRecord("foo", Arrays.asList(
|
||||
tag("foo", "", "f"))));
|
||||
}
|
||||
|
||||
|
||||
static void shouldAccept(SubsetConfiguration conf, String s) {
|
||||
assertTrue("accepts "+ s, newGlobFilter(conf).accepts(s));
|
||||
assertTrue("accepts "+ s, newRegexFilter(conf).accepts(s));
|
||||
}
|
||||
|
||||
// Version for one tag:
|
||||
static void shouldAccept(SubsetConfiguration conf, List<MetricsTag> tags) {
|
||||
assertTrue("accepts "+ tags, newGlobFilter(conf).accepts(tags));
|
||||
assertTrue("accepts "+ tags, newRegexFilter(conf).accepts(tags));
|
||||
shouldAcceptImpl(true, conf, tags, new boolean[] {true});
|
||||
}
|
||||
// Version for multiple tags:
|
||||
static void shouldAccept(SubsetConfiguration conf, List<MetricsTag> tags,
|
||||
boolean[] expectedAcceptedSpec) {
|
||||
shouldAcceptImpl(true, conf, tags, expectedAcceptedSpec);
|
||||
}
|
||||
|
||||
// Version for one tag:
|
||||
static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags) {
|
||||
shouldAcceptImpl(false, conf, tags, new boolean[] {false});
|
||||
}
|
||||
// Version for multiple tags:
|
||||
static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags,
|
||||
boolean[] expectedAcceptedSpec) {
|
||||
shouldAcceptImpl(false, conf, tags, expectedAcceptedSpec);
|
||||
}
|
||||
|
||||
private static void shouldAcceptImpl(final boolean expectAcceptList,
|
||||
SubsetConfiguration conf, List<MetricsTag> tags, boolean[] expectedAcceptedSpec) {
|
||||
final MetricsFilter globFilter = newGlobFilter(conf);
|
||||
final MetricsFilter regexFilter = newRegexFilter(conf);
|
||||
|
||||
// Test acceptance of the tag list:
|
||||
assertEquals("accepts "+ tags, expectAcceptList, globFilter.accepts(tags));
|
||||
assertEquals("accepts "+ tags, expectAcceptList, regexFilter.accepts(tags));
|
||||
|
||||
// Test results on each of the individual tags:
|
||||
int acceptedCount = 0;
|
||||
for (int i=0; i<tags.size(); i++) {
|
||||
MetricsTag tag = tags.get(i);
|
||||
boolean actGlob = globFilter.accepts(tag);
|
||||
boolean actRegex = regexFilter.accepts(tag);
|
||||
assertEquals("accepts "+tag, expectedAcceptedSpec[i], actGlob);
|
||||
// Both the filters should give the same result:
|
||||
assertEquals(actGlob, actRegex);
|
||||
if (actGlob) {
|
||||
acceptedCount++;
|
||||
}
|
||||
}
|
||||
if (expectAcceptList) {
|
||||
// At least one individual tag should be accepted:
|
||||
assertTrue("No tag of the following accepted: " + tags, acceptedCount > 0);
|
||||
} else {
|
||||
// At least one individual tag should be rejected:
|
||||
assertTrue("No tag of the following rejected: " + tags, acceptedCount < tags.size());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -152,11 +200,6 @@ public class TestPatternFilter {
|
|||
assertTrue("rejects "+ s, !newRegexFilter(conf).accepts(s));
|
||||
}
|
||||
|
||||
static void shouldReject(SubsetConfiguration conf, List<MetricsTag> tags) {
|
||||
assertTrue("rejects "+ tags, !newGlobFilter(conf).accepts(tags));
|
||||
assertTrue("rejects "+ tags, !newRegexFilter(conf).accepts(tags));
|
||||
}
|
||||
|
||||
/**
|
||||
* Asserts that filters with the given configuration reject the given record.
|
||||
*
|
||||
|
|
|
@ -0,0 +1,138 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.metrics2.sink;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.metrics2.MetricsSystem;
|
||||
import org.apache.hadoop.metrics2.annotation.Metric;
|
||||
import org.apache.hadoop.metrics2.annotation.Metrics;
|
||||
import org.apache.hadoop.metrics2.annotation.Metric.Type;
|
||||
import org.apache.hadoop.metrics2.impl.ConfigBuilder;
|
||||
import org.apache.hadoop.metrics2.impl.MetricsSystemImpl;
|
||||
import org.apache.hadoop.metrics2.impl.TestMetricsConfig;
|
||||
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
|
||||
import org.junit.After;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class TestFileSink {
|
||||
|
||||
private File outFile;
|
||||
|
||||
// The 2 sample metric classes:
|
||||
@Metrics(name="testRecord1", context="test1")
|
||||
static class MyMetrics1 {
|
||||
@Metric(value={"testTag1", ""}, type=Type.TAG)
|
||||
String testTag1() { return "testTagValue1"; }
|
||||
|
||||
@Metric(value={"testTag2", ""}, type=Type.TAG)
|
||||
String gettestTag2() { return "testTagValue2"; }
|
||||
|
||||
@Metric(value={"testMetric1", "An integer gauge"},always=true)
|
||||
MutableGaugeInt testMetric1;
|
||||
|
||||
@Metric(value={"testMetric2", "An integer gauge"},always=true)
|
||||
MutableGaugeInt testMetric2;
|
||||
|
||||
public MyMetrics1 registerWith(MetricsSystem ms) {
|
||||
return ms.register("m1", null, this);
|
||||
}
|
||||
}
|
||||
|
||||
@Metrics(name="testRecord2", context="test1")
|
||||
static class MyMetrics2 {
|
||||
@Metric(value={"testTag22", ""}, type=Type.TAG)
|
||||
String testTag1() { return "testTagValue22"; }
|
||||
|
||||
public MyMetrics2 registerWith(MetricsSystem ms) {
|
||||
return ms.register("m2", null, this);
|
||||
}
|
||||
}
|
||||
|
||||
private File getTestTempFile(String prefix, String suffix) throws IOException {
|
||||
String tmpPath = System.getProperty("java.io.tmpdir", "/tmp");
|
||||
String user = System.getProperty("user.name", "unknown-user");
|
||||
File dir = new File(tmpPath + "/" + user);
|
||||
dir.mkdirs();
|
||||
return File.createTempFile(prefix, suffix, dir);
|
||||
}
|
||||
|
||||
@Test(timeout=6000)
|
||||
public void testFileSink() throws IOException {
|
||||
outFile = getTestTempFile("test-file-sink-", ".out");
|
||||
final String outPath = outFile.getAbsolutePath();
|
||||
|
||||
// NB: specify large period to avoid multiple metrics snapshotting:
|
||||
new ConfigBuilder().add("*.period", 10000)
|
||||
.add("test.sink.mysink0.class", FileSink.class.getName())
|
||||
.add("test.sink.mysink0.filename", outPath)
|
||||
// NB: we filter by context to exclude "metricssystem" context metrics:
|
||||
.add("test.sink.mysink0.context", "test1")
|
||||
.save(TestMetricsConfig.getTestFilename("hadoop-metrics2-test"));
|
||||
MetricsSystemImpl ms = new MetricsSystemImpl("test");
|
||||
ms.start();
|
||||
|
||||
final MyMetrics1 mm1
|
||||
= new MyMetrics1().registerWith(ms);
|
||||
new MyMetrics2().registerWith(ms);
|
||||
|
||||
mm1.testMetric1.incr();
|
||||
mm1.testMetric2.incr(2);
|
||||
|
||||
ms.publishMetricsNow(); // publish the metrics
|
||||
ms.stop();
|
||||
ms.shutdown();
|
||||
|
||||
InputStream is = new FileInputStream(outFile);
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream((int)outFile.length());
|
||||
IOUtils.copyBytes(is, baos, 1024, true);
|
||||
String outFileContent = new String(baos.toByteArray(), "UTF-8");
|
||||
|
||||
// Check the out file content. Should be something like the following:
|
||||
//1360244820087 test1.testRecord1: Context=test1, testTag1=testTagValue1, testTag2=testTagValue2, Hostname=myhost, testMetric1=1, testMetric2=2
|
||||
//1360244820089 test1.testRecord2: Context=test1, testTag22=testTagValue22, Hostname=myhost
|
||||
|
||||
// Note that in the below expression we allow tags and metrics to go in arbitrary order.
|
||||
Pattern expectedContentPattern = Pattern.compile(
|
||||
// line #1:
|
||||
"^\\d+\\s+test1.testRecord1:\\s+Context=test1,\\s+" +
|
||||
"(testTag1=testTagValue1,\\s+testTag2=testTagValue2|testTag2=testTagValue2,\\s+testTag1=testTagValue1)," +
|
||||
"\\s+Hostname=.*,\\s+(testMetric1=1,\\s+testMetric2=2|testMetric2=2,\\s+testMetric1=1)" +
|
||||
// line #2:
|
||||
"$[\\n\\r]*^\\d+\\s+test1.testRecord2:\\s+Context=test1," +
|
||||
"\\s+testTag22=testTagValue22,\\s+Hostname=.*$[\\n\\r]*",
|
||||
Pattern.MULTILINE);
|
||||
assertTrue(expectedContentPattern.matcher(outFileContent).matches());
|
||||
}
|
||||
|
||||
@After
|
||||
public void after() {
|
||||
if (outFile != null) {
|
||||
outFile.delete();
|
||||
assertTrue(!outFile.exists());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -255,7 +255,15 @@ Release 2.3.0 - UNRELEASED
|
|||
|
||||
HDFS-5342. Provide more information in the FSNamesystem JMX interfaces.
|
||||
(Haohui Mai via jing9)
|
||||
|
||||
|
||||
HDFS-5334. Implement dfshealth.jsp in HTML pages. (Haohui Mai via jing9)
|
||||
|
||||
HDFS-5379. Update links to datanode information in dfshealth.html. (Haohui
|
||||
Mai via jing9)
|
||||
|
||||
HDFS-5382. Implement the UI of browsing filesystems in HTML 5 page. (Haohui
|
||||
Mai via jing9)
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
|
||||
|
@ -319,10 +327,24 @@ Release 2.3.0 - UNRELEASED
|
|||
HDFS-5130. Add test for snapshot related FsShell and DFSAdmin commands.
|
||||
(Binglin Chang via jing9)
|
||||
|
||||
HDFS-5374. Remove deadcode in DFSOutputStream. (suresh)
|
||||
|
||||
HDFS-4511. Cover package org.apache.hadoop.hdfs.tools with unit test
|
||||
(Andrey Klochkov via jeagles)
|
||||
|
||||
HDFS-4885. Improve the verifyBlockPlacement() API in BlockPlacementPolicy.
|
||||
(Junping Du via szetszwo)
|
||||
|
||||
HDFS-5363. Refactor WebHdfsFileSystem: move SPENGO-authenticated connection
|
||||
creation to URLConnectionFactory. (Haohui Mai via jing9)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-5239. Allow FSNamesystem lock fairness to be configurable (daryn)
|
||||
|
||||
HDFS-5341. Reduce fsdataset lock duration during directory scanning.
|
||||
(Qus-Jiawei via kihwal)
|
||||
|
||||
BUG FIXES
|
||||
HDFS-5034. Remove debug prints from GetFileLinkInfo (Andrew Wang via Colin
|
||||
Patrick McCabe)
|
||||
|
@ -354,6 +376,14 @@ Release 2.3.0 - UNRELEASED
|
|||
|
||||
HDFS-4376. Fix race conditions in Balancer. (Junping Du via szetszwo)
|
||||
|
||||
HDFS-5375. hdfs.cmd does not expose several snapshot commands. (cnauroth)
|
||||
|
||||
HDFS-5336. DataNode should not output 'StartupProgress' metrics.
|
||||
(Akira Ajisaka via cnauroth)
|
||||
|
||||
HDFS-5400. DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT constant is set
|
||||
to the wrong value. (Colin Patrick McCabe)
|
||||
|
||||
Release 2.2.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -365,6 +395,9 @@ Release 2.2.1 - UNRELEASED
|
|||
HDFS-5360. Improvement of usage message of renameSnapshot and
|
||||
deleteSnapshot. (Shinichi Yamashita via wang)
|
||||
|
||||
HDFS-5331. make SnapshotDiff.java to a o.a.h.util.Tool interface implementation.
|
||||
(Vinayakumar B via umamahesh)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -400,6 +433,13 @@ Release 2.2.1 - UNRELEASED
|
|||
HDFS-5370. Typo in Error Message: different between range in condition
|
||||
and range in error message. (Kousuke Saruta via suresh)
|
||||
|
||||
HDFS-5365. Fix libhdfs compile error on FreeBSD9. (Radim Kolar via cnauroth)
|
||||
|
||||
HDFS-5347. Add HDFS NFS user guide. (brandonli)
|
||||
|
||||
HDFS-5403. WebHdfs client cannot communicate with older WebHdfs servers
|
||||
post HDFS-5306. (atm)
|
||||
|
||||
Release 2.2.0 - 2013-10-13
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -83,7 +83,7 @@
|
|||
<Class name="org.apache.hadoop.mapred.Task$TaskReporter" />
|
||||
<Method name="run" />
|
||||
<Bug pattern="DM_EXIT" />
|
||||
</Match>
|
||||
</Match>
|
||||
<!--
|
||||
We need to cast objects between old and new api objects
|
||||
-->
|
||||
|
@ -325,6 +325,12 @@
|
|||
<Field name="modification" />
|
||||
<Bug pattern="VO_VOLATILE_INCREMENT" />
|
||||
</Match>
|
||||
<!-- Replace System.exit() call with ExitUtil.terminate() -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.hdfs.tools.JMXGet"/>
|
||||
<Method name="main" />
|
||||
<Bug pattern="NP_NULL_ON_SOME_PATH" />
|
||||
</Match>
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.hdfs.server.datanode.ReplicaInfo" />
|
||||
<Method name="setDirInternal" />
|
||||
|
|
|
@ -176,6 +176,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|||
<artifactId>xmlenc</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.netty</groupId>
|
||||
<artifactId>netty</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
@ -542,6 +547,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|||
<exclude>src/main/docs/releasenotes.html</exclude>
|
||||
<exclude>src/contrib/**</exclude>
|
||||
<exclude>src/site/resources/images/*</exclude>
|
||||
<exclude>src/main/webapps/static/dust-full-2.0.0.min.js</exclude>
|
||||
<exclude>src/main/webapps/static/dust-helpers-1.1.1.min.js</exclude>
|
||||
<exclude>src/main/webapps/hdfs/dfshealth.dust.html</exclude>
|
||||
<exclude>src/main/webapps/hdfs/explorer-block-info.dust.html</exclude>
|
||||
<exclude>src/main/webapps/hdfs/explorer.dust.html</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
|
|
|
@ -62,6 +62,11 @@ endfunction()
|
|||
INCLUDE(CheckCSourceCompiles)
|
||||
CHECK_C_SOURCE_COMPILES("int main(void) { static __thread int i = 0; return 0; }" HAVE_BETTER_TLS)
|
||||
|
||||
# Check if we need to link dl library to get dlopen.
|
||||
# dlopen on Linux is in separate library but on FreeBSD its in libc
|
||||
INCLUDE(CheckLibraryExists)
|
||||
CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
|
||||
|
||||
find_package(JNI REQUIRED)
|
||||
if (NOT GENERATED_JAVAH)
|
||||
# Must identify where the generated headers have been placed
|
||||
|
@ -89,9 +94,13 @@ add_dual_library(hdfs
|
|||
main/native/libhdfs/jni_helper.c
|
||||
main/native/libhdfs/hdfs.c
|
||||
)
|
||||
if (NEED_LINK_DL)
|
||||
set(LIB_DL dl)
|
||||
endif(NEED_LINK_DL)
|
||||
|
||||
target_link_dual_libraries(hdfs
|
||||
${JAVA_JVM_LIBRARY}
|
||||
dl
|
||||
${LIB_DL}
|
||||
pthread
|
||||
)
|
||||
dual_output_directory(hdfs target/usr/local/lib)
|
||||
|
|
|
@ -123,6 +123,14 @@ goto :eof
|
|||
set CLASS=org.apache.hadoop.hdfs.tools.GetGroups
|
||||
goto :eof
|
||||
|
||||
:snapshotDiff
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff
|
||||
goto :eof
|
||||
|
||||
:lsSnapshottableDir
|
||||
set CLASS=org.apache.hadoop.hdfs.tools.snapshot.LsSnapshottableDir
|
||||
goto :eof
|
||||
|
||||
@rem This changes %1, %2 etc. Hence those cannot be used after calling this.
|
||||
:make_command_arguments
|
||||
if "%1" == "--config" (
|
||||
|
@ -164,7 +172,10 @@ goto :eof
|
|||
@echo fetchdt fetch a delegation token from the NameNode
|
||||
@echo getconf get config values from configuration
|
||||
@echo groups get the groups which users belong to
|
||||
@echo Use -help to see options
|
||||
@echo snapshotDiff diff two snapshots of a directory or diff the
|
||||
@echo current directory contents with a snapshot
|
||||
@echo lsSnapshottableDir list all snapshottable dirs owned by the current user
|
||||
@echo Use -help to see options
|
||||
@echo.
|
||||
@echo Most commands print help when invoked w/o parameters.
|
||||
|
||||
|
|
|
@ -401,7 +401,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final int DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT = 1024;
|
||||
public static final String DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS = "dfs.client.mmap.cache.timeout.ms";
|
||||
public static final long DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT = 15 * 60 * 1000;
|
||||
public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.timeout.ms";
|
||||
public static final String DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT = "dfs.client.mmap.cache.thread.runs.per.timeout";
|
||||
public static final int DFS_CLIENT_MMAP_CACHE_THREAD_RUNS_PER_TIMEOUT_DEFAULT = 4;
|
||||
|
||||
// property for fsimage compression
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.hadoop.fs.FSOutputSummer;
|
|||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.Syncable;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
||||
|
@ -138,7 +137,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
private long currentSeqno = 0;
|
||||
private long lastQueuedSeqno = -1;
|
||||
private long lastAckedSeqno = -1;
|
||||
private long bytesCurBlock = 0; // bytes writen in current block
|
||||
private long bytesCurBlock = 0; // bytes written in current block
|
||||
private int packetSize = 0; // write packet size, not including the header.
|
||||
private int chunksPerPacket = 0;
|
||||
private final AtomicReference<IOException> lastException = new AtomicReference<IOException>();
|
||||
|
@ -458,8 +457,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
}
|
||||
|
||||
Packet one = null;
|
||||
|
||||
Packet one;
|
||||
try {
|
||||
// process datanode IO errors if any
|
||||
boolean doSleep = false;
|
||||
|
@ -504,7 +502,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
if(DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("Allocating new block");
|
||||
}
|
||||
nodes = nextBlockOutputStream(src);
|
||||
nodes = nextBlockOutputStream();
|
||||
initDataStreaming();
|
||||
} else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
|
||||
if(DFSClient.LOG.isDebugEnabled()) {
|
||||
|
@ -569,9 +567,6 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
lastPacket = Time.now();
|
||||
|
||||
if (one.isHeartbeatPacket()) { //heartbeat packet
|
||||
}
|
||||
|
||||
// update bytesSent
|
||||
long tmpBytesSent = one.getLastByteOffsetBlock();
|
||||
if (bytesSent < tmpBytesSent) {
|
||||
|
@ -690,7 +685,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
|
||||
//
|
||||
// Processes reponses from the datanodes. A packet is removed
|
||||
// Processes responses from the datanodes. A packet is removed
|
||||
// from the ackQueue when its response arrives.
|
||||
//
|
||||
private class ResponseProcessor extends Daemon {
|
||||
|
@ -732,18 +727,18 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
|
||||
assert seqno != PipelineAck.UNKOWN_SEQNO :
|
||||
"Ack for unkown seqno should be a failed ack: " + ack;
|
||||
"Ack for unknown seqno should be a failed ack: " + ack;
|
||||
if (seqno == Packet.HEART_BEAT_SEQNO) { // a heartbeat ack
|
||||
continue;
|
||||
}
|
||||
|
||||
// a success ack for a data packet
|
||||
Packet one = null;
|
||||
Packet one;
|
||||
synchronized (dataQueue) {
|
||||
one = ackQueue.getFirst();
|
||||
}
|
||||
if (one.seqno != seqno) {
|
||||
throw new IOException("Responseprocessor: Expecting seqno " +
|
||||
throw new IOException("ResponseProcessor: Expecting seqno " +
|
||||
" for block " + block +
|
||||
one.seqno + " but received " + seqno);
|
||||
}
|
||||
|
@ -1052,7 +1047,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
* Must get block ID and the IDs of the destinations from the namenode.
|
||||
* Returns the list of target datanodes.
|
||||
*/
|
||||
private DatanodeInfo[] nextBlockOutputStream(String client) throws IOException {
|
||||
private DatanodeInfo[] nextBlockOutputStream() throws IOException {
|
||||
LocatedBlock lb = null;
|
||||
DatanodeInfo[] nodes = null;
|
||||
int count = dfsClient.getConf().nBlockWriteRetry;
|
||||
|
@ -1210,8 +1205,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
|
||||
private LocatedBlock locateFollowingBlock(long start,
|
||||
DatanodeInfo[] excludedNodes)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
DatanodeInfo[] excludedNodes) throws IOException {
|
||||
int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
|
||||
long sleeptime = 400;
|
||||
while (true) {
|
||||
|
@ -1283,7 +1277,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
* Create a socket for a write pipeline
|
||||
* @param first the first datanode
|
||||
* @param length the pipeline length
|
||||
* @param client
|
||||
* @param client client
|
||||
* @return the socket connected to the first datanode
|
||||
*/
|
||||
static Socket createSocketForPipeline(final DatanodeInfo first,
|
||||
|
@ -1475,7 +1469,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
//
|
||||
// Rather than wait around for space in the queue, we should instead try to
|
||||
// return to the caller as soon as possible, even though we slightly overrun
|
||||
// the MAX_PACKETS iength.
|
||||
// the MAX_PACKETS length.
|
||||
Thread.currentThread().interrupt();
|
||||
break;
|
||||
}
|
||||
|
@ -1696,7 +1690,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
}
|
||||
}
|
||||
// If 1) any new blocks were allocated since the last flush, or 2) to
|
||||
// update length in NN is requried, then persist block locations on
|
||||
// update length in NN is required, then persist block locations on
|
||||
// namenode.
|
||||
if (persistBlocks.getAndSet(false) || updateLength) {
|
||||
try {
|
||||
|
|
|
@ -95,18 +95,17 @@ public abstract class BlockPlacementPolicy {
|
|||
}
|
||||
|
||||
/**
|
||||
* Verify that the block is replicated on at least minRacks different racks
|
||||
* if there is more than minRacks rack in the system.
|
||||
* Verify if the block's placement meets requirement of placement policy,
|
||||
* i.e. replicas are placed on no less than minRacks racks in the system.
|
||||
*
|
||||
* @param srcPath the full pathname of the file to be verified
|
||||
* @param lBlk block with locations
|
||||
* @param minRacks number of racks the block should be replicated to
|
||||
* @return the difference between the required and the actual number of racks
|
||||
* the block is replicated to.
|
||||
* @param numOfReplicas replica number of file to be verified
|
||||
* @return the result of verification
|
||||
*/
|
||||
abstract public int verifyBlockPlacement(String srcPath,
|
||||
LocatedBlock lBlk,
|
||||
int minRacks);
|
||||
abstract public BlockPlacementStatus verifyBlockPlacement(String srcPath,
|
||||
LocatedBlock lBlk,
|
||||
int numOfReplicas);
|
||||
/**
|
||||
* Decide whether deleting the specified replica of the block still makes
|
||||
* the block conform to the configured block placement policy.
|
||||
|
|
|
@ -655,22 +655,22 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|||
}
|
||||
|
||||
@Override
|
||||
public int verifyBlockPlacement(String srcPath,
|
||||
LocatedBlock lBlk,
|
||||
int minRacks) {
|
||||
public BlockPlacementStatus verifyBlockPlacement(String srcPath,
|
||||
LocatedBlock lBlk, int numberOfReplicas) {
|
||||
DatanodeInfo[] locs = lBlk.getLocations();
|
||||
if (locs == null)
|
||||
locs = DatanodeDescriptor.EMPTY_ARRAY;
|
||||
int numRacks = clusterMap.getNumOfRacks();
|
||||
if(numRacks <= 1) // only one rack
|
||||
return 0;
|
||||
minRacks = Math.min(minRacks, numRacks);
|
||||
return new BlockPlacementStatusDefault(
|
||||
Math.min(numRacks, numberOfReplicas), numRacks);
|
||||
int minRacks = Math.min(2, numberOfReplicas);
|
||||
// 1. Check that all locations are different.
|
||||
// 2. Count locations on different racks.
|
||||
Set<String> racks = new TreeSet<String>();
|
||||
for (DatanodeInfo dn : locs)
|
||||
racks.add(dn.getNetworkLocation());
|
||||
return minRacks - racks.size();
|
||||
return new BlockPlacementStatusDefault(racks.size(), minRacks);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/**
|
||||
* 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.blockmanagement;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public interface BlockPlacementStatus {
|
||||
|
||||
/**
|
||||
* Boolean value to identify if replicas of this block satisfy requirement of
|
||||
* placement policy
|
||||
* @return if replicas satisfy placement policy's requirement
|
||||
*/
|
||||
public boolean isPlacementPolicySatisfied();
|
||||
|
||||
/**
|
||||
* Get description info for log or printed in case replicas are failed to meet
|
||||
* requirement of placement policy
|
||||
* @return description in case replicas are failed to meet requirement of
|
||||
* placement policy
|
||||
*/
|
||||
public String getErrorDescription();
|
||||
|
||||
}
|
|
@ -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.hdfs.server.blockmanagement;
|
||||
|
||||
public class BlockPlacementStatusDefault implements BlockPlacementStatus {
|
||||
|
||||
private int requiredRacks = 0;
|
||||
private int currentRacks = 0;
|
||||
|
||||
public BlockPlacementStatusDefault(int currentRacks, int requiredRacks){
|
||||
this.requiredRacks = requiredRacks;
|
||||
this.currentRacks = currentRacks;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPlacementPolicySatisfied() {
|
||||
return requiredRacks <= currentRacks;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getErrorDescription() {
|
||||
if (isPlacementPolicySatisfied()) {
|
||||
return null;
|
||||
}
|
||||
return "Block should be additionally replicated on " +
|
||||
(requiredRacks - currentRacks) + " more rack(s).";
|
||||
}
|
||||
|
||||
}
|
|
@ -191,6 +191,11 @@ public class DirectoryScanner implements Runnable {
|
|||
|
||||
private final FsVolumeSpi volume;
|
||||
|
||||
/**
|
||||
* Get the file's length in async block scan
|
||||
*/
|
||||
private final long blockFileLength;
|
||||
|
||||
private final static Pattern CONDENSED_PATH_REGEX =
|
||||
Pattern.compile("(?<!^)(\\\\|/){2,}");
|
||||
|
||||
|
@ -235,6 +240,7 @@ public class DirectoryScanner implements Runnable {
|
|||
getCondensedPath(vol.getBasePath());
|
||||
this.blockSuffix = blockFile == null ? null :
|
||||
getSuffix(blockFile, condensedVolPath);
|
||||
this.blockFileLength = (blockFile != null) ? blockFile.length() : 0;
|
||||
if (metaFile == null) {
|
||||
this.metaSuffix = null;
|
||||
} else if (blockFile == null) {
|
||||
|
@ -251,6 +257,10 @@ public class DirectoryScanner implements Runnable {
|
|||
new File(volume.getBasePath(), blockSuffix);
|
||||
}
|
||||
|
||||
long getBlockFileLength() {
|
||||
return blockFileLength;
|
||||
}
|
||||
|
||||
File getMetaFile() {
|
||||
if (metaSuffix == null) {
|
||||
return null;
|
||||
|
@ -458,7 +468,7 @@ public class DirectoryScanner implements Runnable {
|
|||
// Block metadata file exits and block file is missing
|
||||
addDifference(diffRecord, statsRecord, info);
|
||||
} else if (info.getGenStamp() != memBlock.getGenerationStamp()
|
||||
|| info.getBlockFile().length() != memBlock.getNumBytes()) {
|
||||
|| info.getBlockFileLength() != memBlock.getNumBytes()) {
|
||||
// Block metadata file is missing or has wrong generation stamp,
|
||||
// or block file length is different than expected
|
||||
statsRecord.mismatchBlocks++;
|
||||
|
|
|
@ -413,8 +413,15 @@ public class DatanodeWebHdfsMethods {
|
|||
final long n = length.getValue() != null ?
|
||||
Math.min(length.getValue(), in.getVisibleLength() - offset.getValue()) :
|
||||
in.getVisibleLength() - offset.getValue();
|
||||
return Response.ok(new OpenEntity(in, n, dfsclient)).type(
|
||||
MediaType.APPLICATION_OCTET_STREAM).build();
|
||||
|
||||
/**
|
||||
* Allow the Web UI to perform an AJAX request to get the data.
|
||||
*/
|
||||
return Response.ok(new OpenEntity(in, n, dfsclient))
|
||||
.type(MediaType.APPLICATION_OCTET_STREAM)
|
||||
.header("Access-Control-Allow-Methods", "GET")
|
||||
.header("Access-Control-Allow-Origin", "*")
|
||||
.build();
|
||||
}
|
||||
case GETFILECHECKSUM:
|
||||
{
|
||||
|
|
|
@ -270,10 +270,6 @@ public class NameNode implements NameNodeStatusMXBean {
|
|||
|
||||
static NameNodeMetrics metrics;
|
||||
private static final StartupProgress startupProgress = new StartupProgress();
|
||||
static {
|
||||
StartupProgressMetrics.register(startupProgress);
|
||||
}
|
||||
|
||||
/** Return the {@link FSNamesystem} object.
|
||||
* @return {@link FSNamesystem} object.
|
||||
*/
|
||||
|
@ -485,6 +481,7 @@ public class NameNode implements NameNodeStatusMXBean {
|
|||
loginAsNameNodeUser(conf);
|
||||
|
||||
NameNode.initMetrics(conf, this.getRole());
|
||||
StartupProgressMetrics.register(startupProgress);
|
||||
|
||||
if (NamenodeRole.NAMENODE == role) {
|
||||
startHttpServer(conf);
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
|
@ -374,9 +375,10 @@ public class NamenodeFsck {
|
|||
locs.length + " replica(s).");
|
||||
}
|
||||
// verify block placement policy
|
||||
int missingRacks = BlockPlacementPolicy.getInstance(conf, null, networktopology).
|
||||
verifyBlockPlacement(path, lBlk, Math.min(2,targetFileReplication));
|
||||
if (missingRacks > 0) {
|
||||
BlockPlacementStatus blockPlacementStatus =
|
||||
BlockPlacementPolicy.getInstance(conf, null, networktopology).
|
||||
verifyBlockPlacement(path, lBlk, targetFileReplication);
|
||||
if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
|
||||
res.numMisReplicatedBlocks++;
|
||||
misReplicatedPerFile++;
|
||||
if (!showFiles) {
|
||||
|
@ -385,9 +387,7 @@ public class NamenodeFsck {
|
|||
out.print(path + ": ");
|
||||
}
|
||||
out.println(" Replica placement policy is violated for " +
|
||||
block +
|
||||
". Block should be additionally replicated on " +
|
||||
missingRacks + " more rack(s).");
|
||||
block + ". " + blockPlacementStatus.getErrorDescription());
|
||||
}
|
||||
report.append(i + ". " + blkName + " len=" + block.getNumBytes());
|
||||
if (locs.length == 0) {
|
||||
|
|
|
@ -53,6 +53,7 @@ import org.apache.hadoop.security.Credentials;
|
|||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.ExitUtil;
|
||||
import org.apache.hadoop.util.GenericOptionsParser;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
|
@ -86,7 +87,7 @@ public class DelegationTokenFetcher {
|
|||
err.println(" --print Print the delegation token");
|
||||
err.println();
|
||||
GenericOptionsParser.printGenericCommandUsage(err);
|
||||
System.exit(1);
|
||||
ExitUtil.terminate(1);
|
||||
}
|
||||
|
||||
private static Collection<Token<?>> readTokens(Path file, Configuration conf)
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.commons.cli.Options;
|
|||
import org.apache.commons.cli.ParseException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.util.ExitUtil;
|
||||
|
||||
/**
|
||||
* tool to get data from NameNode or DataNode using MBeans currently the
|
||||
|
@ -295,7 +296,7 @@ public class JMXGet {
|
|||
// invalid arguments
|
||||
err("Invalid args");
|
||||
printUsage(opts);
|
||||
System.exit(-1);
|
||||
ExitUtil.terminate(-1);
|
||||
}
|
||||
|
||||
JMXGet jm = new JMXGet();
|
||||
|
@ -317,7 +318,7 @@ public class JMXGet {
|
|||
|
||||
if (commandLine.hasOption("help")) {
|
||||
printUsage(opts);
|
||||
System.exit(0);
|
||||
ExitUtil.terminate(0);
|
||||
}
|
||||
|
||||
// rest of args
|
||||
|
@ -342,6 +343,6 @@ public class JMXGet {
|
|||
res = -1;
|
||||
}
|
||||
|
||||
System.exit(res);
|
||||
ExitUtil.terminate(res);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,12 +20,14 @@ package org.apache.hadoop.hdfs.tools.snapshot;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
/**
|
||||
* A tool used to get the difference report between two snapshots, or between
|
||||
|
@ -38,7 +40,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|||
* </pre>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SnapshotDiff {
|
||||
public class SnapshotDiff extends Configured implements Tool {
|
||||
private static String getSnapshotName(String name) {
|
||||
if (Path.CUR_DIR.equals(name)) { // current directory
|
||||
return "";
|
||||
|
@ -57,7 +59,8 @@ public class SnapshotDiff {
|
|||
return name.substring(i + HdfsConstants.DOT_SNAPSHOT_DIR.length() + 1);
|
||||
}
|
||||
|
||||
public static void main(String[] argv) throws IOException {
|
||||
@Override
|
||||
public int run(String[] argv) throws Exception {
|
||||
String description = "SnapshotDiff <snapshotDir> <from> <to>:\n" +
|
||||
"\tGet the difference between two snapshots, \n" +
|
||||
"\tor between a snapshot and the current tree of a directory.\n" +
|
||||
|
@ -67,15 +70,14 @@ public class SnapshotDiff {
|
|||
|
||||
if(argv.length != 3) {
|
||||
System.err.println("Usage: \n" + description);
|
||||
System.exit(1);
|
||||
return 1;
|
||||
}
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
FileSystem fs = FileSystem.get(getConf());
|
||||
if (! (fs instanceof DistributedFileSystem)) {
|
||||
System.err.println(
|
||||
"SnapshotDiff can only be used in DistributedFileSystem");
|
||||
System.exit(1);
|
||||
return 1;
|
||||
}
|
||||
DistributedFileSystem dfs = (DistributedFileSystem) fs;
|
||||
|
||||
|
@ -89,7 +91,14 @@ public class SnapshotDiff {
|
|||
} catch (IOException e) {
|
||||
String[] content = e.getLocalizedMessage().split("\n");
|
||||
System.err.println("snapshotDiff: " + content[0]);
|
||||
return 1;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
public static void main(String[] argv) throws Exception {
|
||||
int rc = ToolRunner.run(new SnapshotDiff(), argv);
|
||||
System.exit(rc);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -299,10 +299,15 @@ public class JsonUtil {
|
|||
}
|
||||
|
||||
/** Convert a Json map to an DatanodeInfo object. */
|
||||
private static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
|
||||
static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
|
||||
if (m == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Object infoSecurePort = m.get("infoSecurePort");
|
||||
if (infoSecurePort == null) {
|
||||
infoSecurePort = 0l; // same as the default value in hdfs.proto
|
||||
}
|
||||
|
||||
return new DatanodeInfo(
|
||||
(String)m.get("ipAddr"),
|
||||
|
@ -310,7 +315,7 @@ public class JsonUtil {
|
|||
(String)m.get("storageID"),
|
||||
(int)(long)(Long)m.get("xferPort"),
|
||||
(int)(long)(Long)m.get("infoPort"),
|
||||
(int)(long)(Long)m.get("infoSecurePort"),
|
||||
(int)(long)(Long)infoSecurePort,
|
||||
(int)(long)(Long)m.get("ipcPort"),
|
||||
|
||||
(Long)m.get("capacity"),
|
||||
|
|
|
@ -19,49 +19,114 @@
|
|||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URL;
|
||||
import java.net.URLConnection;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
|
||||
|
||||
/**
|
||||
* Utilities for handling URLs
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||
@InterfaceAudience.LimitedPrivate({ "HDFS" })
|
||||
@InterfaceStability.Unstable
|
||||
public class URLConnectionFactory {
|
||||
private static final Log LOG = LogFactory.getLog(URLConnectionFactory.class);
|
||||
|
||||
/** SPNEGO authenticator */
|
||||
private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
|
||||
|
||||
/**
|
||||
* Timeout for socket connects and reads
|
||||
*/
|
||||
public final static int DEFAULT_SOCKET_TIMEOUT = 1*60*1000; // 1 minute
|
||||
public final static int DEFAULT_SOCKET_TIMEOUT = 1 * 60 * 1000; // 1 minute
|
||||
|
||||
public static final URLConnectionFactory DEFAULT_CONNECTION_FACTORY = new URLConnectionFactory(
|
||||
DEFAULT_SOCKET_TIMEOUT);
|
||||
|
||||
public static final URLConnectionFactory DEFAULT_CONNECTION_FACTORY = new URLConnectionFactory(DEFAULT_SOCKET_TIMEOUT);
|
||||
|
||||
private int socketTimeout;
|
||||
|
||||
/** Configure connections for AuthenticatedURL */
|
||||
private ConnectionConfigurator connConfigurator = new ConnectionConfigurator() {
|
||||
@Override
|
||||
public HttpURLConnection configure(HttpURLConnection conn)
|
||||
throws IOException {
|
||||
URLConnectionFactory.setTimeouts(conn, socketTimeout);
|
||||
return conn;
|
||||
}
|
||||
};
|
||||
|
||||
public URLConnectionFactory(int socketTimeout) {
|
||||
this.socketTimeout = socketTimeout;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Opens a url with read and connect timeouts
|
||||
* @param url to open
|
||||
*
|
||||
* @param url
|
||||
* to open
|
||||
* @return URLConnection
|
||||
* @throws IOException
|
||||
*/
|
||||
public URLConnection openConnection(URL url) throws IOException {
|
||||
URLConnection connection = url.openConnection();
|
||||
setTimeouts(connection);
|
||||
return connection;
|
||||
if (connection instanceof HttpURLConnection) {
|
||||
connConfigurator.configure((HttpURLConnection) connection);
|
||||
}
|
||||
return connection;
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens a url with read and connect timeouts
|
||||
*
|
||||
* @param url URL to open
|
||||
* @return URLConnection
|
||||
* @throws IOException
|
||||
* @throws AuthenticationException
|
||||
*/
|
||||
public URLConnection openConnection(HttpOpParam.Op op, URL url)
|
||||
throws IOException, AuthenticationException {
|
||||
if (op.getRequireAuth()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("open AuthenticatedURL connection" + url);
|
||||
}
|
||||
UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
|
||||
final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
|
||||
return new AuthenticatedURL(AUTH, connConfigurator).openConnection(url,
|
||||
authToken);
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("open URL connection");
|
||||
}
|
||||
return openConnection(url);
|
||||
}
|
||||
}
|
||||
|
||||
public ConnectionConfigurator getConnConfigurator() {
|
||||
return connConfigurator;
|
||||
}
|
||||
|
||||
public void setConnConfigurator(ConnectionConfigurator connConfigurator) {
|
||||
this.connConfigurator = connConfigurator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets timeout parameters on the given URLConnection.
|
||||
*
|
||||
* @param connection URLConnection to set
|
||||
* @param connection
|
||||
* URLConnection to set
|
||||
* @param socketTimeout
|
||||
* the connection and read timeout of the connection.
|
||||
*/
|
||||
public void setTimeouts(URLConnection connection) {
|
||||
static void setTimeouts(URLConnection connection, int socketTimeout) {
|
||||
connection.setConnectTimeout(socketTimeout);
|
||||
connection.setReadTimeout(socketTimeout);
|
||||
}
|
||||
|
|
|
@ -94,9 +94,7 @@ import org.apache.hadoop.ipc.RemoteException;
|
|||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.security.token.TokenRenewer;
|
||||
|
@ -119,20 +117,9 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
/** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
|
||||
public static final String PATH_PREFIX = "/" + SCHEME + "/v" + VERSION;
|
||||
|
||||
/** SPNEGO authenticator */
|
||||
private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
|
||||
/** Default connection factory may be overridden in tests to use smaller timeout values */
|
||||
URLConnectionFactory connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
|
||||
/** Configures connections for AuthenticatedURL */
|
||||
private final ConnectionConfigurator CONN_CONFIGURATOR =
|
||||
new ConnectionConfigurator() {
|
||||
@Override
|
||||
public HttpURLConnection configure(HttpURLConnection conn)
|
||||
throws IOException {
|
||||
connectionFactory.setTimeouts(conn);
|
||||
return conn;
|
||||
}
|
||||
};
|
||||
|
||||
/** Delegation token kind */
|
||||
public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
|
||||
/** Token selector */
|
||||
|
@ -504,16 +491,7 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
throws IOException {
|
||||
final HttpURLConnection conn;
|
||||
try {
|
||||
if (op.getRequireAuth()) {
|
||||
LOG.debug("open AuthenticatedURL connection");
|
||||
UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
|
||||
final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
|
||||
conn = new AuthenticatedURL(AUTH, CONN_CONFIGURATOR).openConnection(
|
||||
url, authToken);
|
||||
} else {
|
||||
LOG.debug("open URL connection");
|
||||
conn = (HttpURLConnection)connectionFactory.openConnection(url);
|
||||
}
|
||||
conn = (HttpURLConnection) connectionFactory.openConnection(op, url);
|
||||
} catch (AuthenticationException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
@ -635,8 +613,10 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
checkRetry = false;
|
||||
|
||||
//Step 2) Submit another Http request with the URL from the Location header with data.
|
||||
conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect));
|
||||
conn.setRequestProperty("Content-Type", MediaType.APPLICATION_OCTET_STREAM);
|
||||
conn = (HttpURLConnection) connectionFactory.openConnection(new URL(
|
||||
redirect));
|
||||
conn.setRequestProperty("Content-Type",
|
||||
MediaType.APPLICATION_OCTET_STREAM);
|
||||
conn.setChunkedStreamingMode(32 << 10); //32kB-chunk
|
||||
connect();
|
||||
return conn;
|
||||
|
@ -658,7 +638,8 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
disconnect();
|
||||
|
||||
checkRetry = false;
|
||||
conn = (HttpURLConnection)connectionFactory.openConnection(new URL(redirect));
|
||||
conn = (HttpURLConnection) connectionFactory.openConnection(new URL(
|
||||
redirect));
|
||||
connect();
|
||||
}
|
||||
|
||||
|
@ -892,12 +873,6 @@ public class WebHdfsFileSystem extends FileSystem
|
|||
.write(bufferSize);
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
@Override
|
||||
public boolean delete(final Path f) throws IOException {
|
||||
return delete(f, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean delete(Path f, boolean recursive) throws IOException {
|
||||
final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;
|
||||
|
|
|
@ -27,6 +27,7 @@
|
|||
#include <sys/stat.h>
|
||||
#include <sys/types.h>
|
||||
#include <unistd.h>
|
||||
#include <limits.h>
|
||||
|
||||
static pthread_mutex_t gTempdirLock = PTHREAD_MUTEX_INITIALIZER;
|
||||
|
||||
|
|
|
@ -0,0 +1,116 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
(function ($, dust, exports) {
|
||||
"use strict";
|
||||
|
||||
var filters = {
|
||||
'fmt_bytes': function (v) {
|
||||
var UNITS = ['B', 'KB', 'MB', 'GB', 'TB', 'PB', 'ZB'];
|
||||
var prev = 0, i = 0;
|
||||
while (Math.floor(v) > 0 && i < UNITS.length) {
|
||||
prev = v;
|
||||
v /= 1024;
|
||||
i += 1;
|
||||
}
|
||||
|
||||
if (i > 0 && i < UNITS.length) {
|
||||
v = prev;
|
||||
i -= 1;
|
||||
}
|
||||
return Math.round(v * 100) / 100 + ' ' + UNITS[i];
|
||||
},
|
||||
|
||||
'fmt_percentage': function (v) {
|
||||
return Math.round(v * 100) / 100 + '%';
|
||||
},
|
||||
|
||||
'fmt_time': function (v) {
|
||||
var s = Math.floor(v / 1000), h = Math.floor(s / 3600);
|
||||
s -= h * 3600;
|
||||
var m = Math.floor(s / 60);
|
||||
s -= m * 60;
|
||||
|
||||
var res = s + " sec";
|
||||
if (m !== 0) {
|
||||
res = m + " mins, " + res;
|
||||
}
|
||||
|
||||
if (h !== 0) {
|
||||
res = h + " hrs, " + res;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
};
|
||||
$.extend(dust.filters, filters);
|
||||
|
||||
/**
|
||||
* Load templates from external sources in sequential orders, and
|
||||
* compile them. The loading order is important to resolve dependency.
|
||||
*
|
||||
* The code compile the templates on the client sides, which should be
|
||||
* precompiled once we introduce the infrastructure in the building
|
||||
* system.
|
||||
*
|
||||
* templates is an array of tuples in the format of {url, name}.
|
||||
*/
|
||||
function load_templates(dust, templates, success_cb, error_cb) {
|
||||
if (templates.length === 0) {
|
||||
success_cb();
|
||||
return;
|
||||
}
|
||||
|
||||
var t = templates.shift();
|
||||
$.get(t.url, function (tmpl) {
|
||||
var c = dust.compile(tmpl, t.name);
|
||||
dust.loadSource(c);
|
||||
load_templates(dust, templates, success_cb, error_cb);
|
||||
}).error(function (jqxhr, text, err) {
|
||||
error_cb(t.url, jqxhr, text, err);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Load a sequence of JSON.
|
||||
*
|
||||
* beans is an array of tuples in the format of {url, name}.
|
||||
*/
|
||||
function load_json(beans, success_cb, error_cb) {
|
||||
var data = {}, error = false, to_be_completed = beans.length;
|
||||
|
||||
$.each(beans, function(idx, b) {
|
||||
if (error) {
|
||||
return false;
|
||||
}
|
||||
$.get(b.url, function (resp) {
|
||||
data[b.name] = resp;
|
||||
to_be_completed -= 1;
|
||||
if (to_be_completed === 0) {
|
||||
success_cb(data);
|
||||
}
|
||||
}).error(function (jqxhr, text, err) {
|
||||
error = true;
|
||||
error_cb(b.url, jqxhr, text, err);
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
exports.load_templates = load_templates;
|
||||
exports.load_json = load_json;
|
||||
|
||||
}($, dust, window));
|
|
@ -0,0 +1,265 @@
|
|||
<div class="page-header">
|
||||
{#nnstat}
|
||||
<h1>NameNode '{HostAndPort}' ({State})</h1>
|
||||
{/nnstat}
|
||||
</div>
|
||||
|
||||
{#nn}
|
||||
{@if cond="{DistinctVersionCount} > 1"}
|
||||
<div class="alert alert-dismissable alert-success">
|
||||
<button type="button" class="close" data-dismiss="alert" aria-hidden="true">×</button>
|
||||
|
||||
There are {DistinctVersionCount} versions of datanodes currently live:
|
||||
{#DistinctVersions}
|
||||
{key} ({value}) {@sep},{/sep}
|
||||
{/DistinctVersions}
|
||||
</div>
|
||||
{/if}
|
||||
|
||||
{@if cond="{NumberOfMissingBlocks} > 0"}
|
||||
<div class="alert alert-dismissable alert-warning">
|
||||
<button type="button" class="close" data-dismiss="alert" aria-hidden="true">×</button>
|
||||
|
||||
<p>There are {NumberOfMissingBlocks} missing blocks. The following files may be corrupted:</p>
|
||||
<br/>
|
||||
<div class="well">
|
||||
{#CorruptFiles}
|
||||
{.}<br/>
|
||||
{/CorruptFiles}
|
||||
</div>
|
||||
<p>Please check the logs or run fsck in order to identify the missing blocks. See the Hadoop FAQ for common causes and potential solutions.</p>
|
||||
</div>
|
||||
{/if}
|
||||
{/nn}
|
||||
|
||||
<div class="panel panel-primary">
|
||||
<div class="panel-heading">Overview</div>
|
||||
<div class="panel-body">
|
||||
{#nn}
|
||||
<table class="table table-bordered">
|
||||
<tr><th>Started:</th><td>{NNStarted}</td></tr>
|
||||
<tr><th>Version:</th><td>{Version}</td></tr>
|
||||
<tr><th>Compiled:</th><td>{CompileInfo}</td></tr>
|
||||
<tr><th>Cluster ID:</th><td>{ClusterId}</td></tr>
|
||||
<tr><th>Block Pool ID:</th><td>{BlockPoolId}</td></tr>
|
||||
</table>
|
||||
{/nn}
|
||||
</div>
|
||||
</div>
|
||||
|
||||
<a href="explorer.html">Browse the filesystem</a> <a href="/logs/">NameNode Logs</a>
|
||||
|
||||
<hr/>
|
||||
|
||||
<div class="panel panel-primary">
|
||||
<div class="panel-heading">Cluster Summary</div>
|
||||
<div class="panel-body">
|
||||
|
||||
<p>
|
||||
Security is {#nnstat}{#SecurityEnabled}on{:else}off{/SecurityEnabled}{/nnstat}.</p>
|
||||
<p>{#nn}{#Safemode}{.}{:else}Safemode is off.{/Safemode}{/nn}</p>
|
||||
|
||||
<p>
|
||||
{#fs}
|
||||
{TotalLoad} files and directories, {BlocksTotal} blocks = {FilesTotal} total filesystem object(s).
|
||||
{#helper_fs_max_objects/}
|
||||
{/fs}
|
||||
</p>
|
||||
{#mem.HeapMemoryUsage}
|
||||
<p>Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Heap Memory. Max Heap Memory is {max|fmt_bytes}. </p>
|
||||
{/mem.HeapMemoryUsage}
|
||||
|
||||
{#mem.NonHeapMemoryUsage}
|
||||
<p>Non Heap Memory used {used|fmt_bytes} of {committed|fmt_bytes} Commited Non Heap Memory. Max Non Heap Memory is {max|fmt_bytes}. </p>
|
||||
{/mem.NonHeapMemoryUsage}
|
||||
|
||||
{#nn}
|
||||
<table class="table table-bordered table-striped">
|
||||
<tr><th> Configured Capacity:</th><td>{Total|fmt_bytes}</td></tr>
|
||||
<tr><th> DFS Used:</th><td>{Used|fmt_bytes}</td></tr>
|
||||
<tr><th> Non DFS Used:</th><td>{NonDfsUsedSpace|fmt_bytes}</td></tr>
|
||||
<tr><th> DFS Remaining:</th><td>{Free|fmt_bytes}</td></tr>
|
||||
<tr><th> DFS Used%:</th><td>{PercentUsed|fmt_percentage}</td></tr>
|
||||
<tr><th> DFS Remaining%:</th><td>{PercentRemaining|fmt_percentage}</td></tr>
|
||||
<tr><th> Block Pool Used:</th><td>{BlockPoolUsedSpace|fmt_bytes}</td></tr>
|
||||
<tr><th> Block Pool Used%:</th><td>{PercentBlockPoolUsed|fmt_percentage}</td></tr>
|
||||
<tr><th> DataNodes usages% (Min/Median/Max/stdDev): </th>
|
||||
<td>{#NodeUsage.nodeUsage}{min} / {median} / {max} / {stdDev}{/NodeUsage.nodeUsage}</td></tr>
|
||||
{/nn}
|
||||
|
||||
{#fs}
|
||||
<tr><th><a href="#nodelist-operation">Live Nodes</a></th><td>{NumLiveDataNodes} (Decommissioned: {NumDecomLiveDataNodes})</td></tr>
|
||||
<tr><th><a href="#nodelist-operation">Dead Nodes</a></th><td>{NumDeadDataNodes} (Decommissioned: {NumDecomDeadDataNodes})</td></tr>
|
||||
<tr><th><a href="#nodelist-decom">Decommissioning Nodes</a></th><td>{NumDecommissioningDataNodes}</td></tr>
|
||||
<tr><th title="Excludes missing blocks.">Number of Under-Replicated Blocks</th><td>{UnderReplicatedBlocks}</td></tr>
|
||||
{/fs}
|
||||
</table>
|
||||
</div>
|
||||
</div>
|
||||
|
||||
<hr/>
|
||||
<div class="panel panel-primary">
|
||||
<div class="panel-heading">NameNode Journal Status</div>
|
||||
<div class="panel-body">
|
||||
<p><b>Current transaction ID:</b> {nn.JournalTransactionInfo.LastAppliedOrWrittenTxId}</p>
|
||||
<table class="table" title="NameNode Journals">
|
||||
<thead>
|
||||
<tr><th>Journal Manager</th><th>State</th></tr>
|
||||
</thead>
|
||||
<tbody>
|
||||
{#nn.NameJournalStatus}
|
||||
<tr><td>{manager}</td><td>{stream}</td></tr>
|
||||
{/nn.NameJournalStatus}
|
||||
</tbody>
|
||||
</table>
|
||||
</div>
|
||||
</div>
|
||||
|
||||
<hr/>
|
||||
<div class="panel panel-primary">
|
||||
<div class="panel-heading">NameNode Storage</div>
|
||||
<div class="panel-body">
|
||||
<table class="table" title="NameNode Storage">
|
||||
<thead><tr><td><b>Storage Directory</b></td><td><b>Type</b></td><td><b>State</b></td></tr></thead>
|
||||
{#nn.NameDirStatuses}
|
||||
{#active}{#helper_dir_status type="Active"/}{/active}
|
||||
{#failed}{#helper_dir_status type="Failed"/}{/failed}
|
||||
{/nn.NameDirStatuses}
|
||||
</table>
|
||||
</div>
|
||||
</div>
|
||||
<hr/>
|
||||
|
||||
<div class="panel panel-primary">
|
||||
<div class="panel-heading">Snapshot Summary</div>
|
||||
<div class="panel-body">
|
||||
{#fs.SnapshotStats}
|
||||
<table class="table" title="Snapshot Summary">
|
||||
<thead><tr><td><b>Snapshottable directories</b></td>
|
||||
<td><b>Snapshotted directories</b></td></tr>
|
||||
</thead>
|
||||
<tbody>
|
||||
<tr>
|
||||
<td>{SnapshottableDirectories}</td>
|
||||
<td>{Snapshots}</td>
|
||||
</tr>
|
||||
</tbody>
|
||||
</table>
|
||||
{/fs.SnapshotStats}
|
||||
</div>
|
||||
</div>
|
||||
<hr/>
|
||||
|
||||
{#startup}
|
||||
<div class="panel panel-primary">
|
||||
<div class="panel-heading">Startup Progress</div>
|
||||
<div class="panel-body">
|
||||
<p>Elapsed Time: {elapsedTime|fmt_time}, Percent Complete: {percentComplete|fmt_percentage}</p>
|
||||
<table class="table">
|
||||
<thead>
|
||||
<tr>
|
||||
<th>Phase</th>
|
||||
<th>Completion</th>
|
||||
<th>Elapsed Time</th>
|
||||
</tr>
|
||||
</thead>
|
||||
<tbody>
|
||||
{#phases}
|
||||
<tr class="phase">
|
||||
<td class="startupdesc">{desc} {file} {size|fmt_bytes}</td>
|
||||
<td>{percentComplete|fmt_percentage}</td>
|
||||
<td>{elapsedTime|fmt_time}</td>
|
||||
</tr>
|
||||
{#steps root_file=file}
|
||||
<tr class="step">
|
||||
<td class="startupdesc">{stepDesc} {stepFile} {stepSize|fmt_bytes} ({count}/{total})</td>
|
||||
<td>{percentComplete|fmt_percentage}</td>
|
||||
<td></td>
|
||||
</tr>
|
||||
{/steps}
|
||||
{/phases}
|
||||
</table>
|
||||
</div>
|
||||
</div>
|
||||
{/startup}
|
||||
|
||||
<hr/>
|
||||
<div class="panel panel-primary">
|
||||
<div class="panel-heading">Datanode Information</div>
|
||||
<div class="panel-body">
|
||||
<div class="panel panel-default" id="nodelist-operation">
|
||||
<div class="panel-heading">Nodes in operation</div>
|
||||
<div class="panel-body">
|
||||
<table class="table">
|
||||
<thead>
|
||||
<tr>
|
||||
<th>Node</th>
|
||||
<th>Last contact</th>
|
||||
<th>Admin State</th>
|
||||
<th>Capacity</th>
|
||||
<th>Used</th>
|
||||
<th>Non DFS Used</th>
|
||||
<th>Remaining</th>
|
||||
<th>Blocks</th>
|
||||
<th>Block pool used</th>
|
||||
<th>Failed Volumes</th>
|
||||
</tr>
|
||||
</thead>
|
||||
{#nn.LiveNodes}
|
||||
<tr>
|
||||
<td>{name} ({xferaddr})</td>
|
||||
<td>{lastContact}</td>
|
||||
<td>{adminState}</td>
|
||||
<td>{capacity|fmt_bytes}</td>
|
||||
<td>{used|fmt_bytes}</td>
|
||||
<td>{nonDfsUsedSpace|fmt_bytes}</td>
|
||||
<td>{remaining|fmt_bytes}</td>
|
||||
<td>{numBlocks}</td>
|
||||
<td>{blockPoolUsed|fmt_bytes} ({blockPoolUsedPercent|fmt_percentage})</td>
|
||||
<td>{volfails}</td>
|
||||
</tr>
|
||||
{/nn.LiveNodes}
|
||||
{#nn.DeadNodes}
|
||||
<tr class="danger">
|
||||
<td>{name} ({xferaddr})</td>
|
||||
<td>{lastContact}</td>
|
||||
<td>Dead{?decomissioned}, Decomissioned{/decomissioned}</td>
|
||||
<td>-</td>
|
||||
<td>-</td>
|
||||
<td>-</td>
|
||||
<td>-</td>
|
||||
<td>-</td>
|
||||
<td>-</td>
|
||||
<td>-</td>
|
||||
</tr>
|
||||
{/nn.DeadNodes}
|
||||
</table>
|
||||
</div>
|
||||
</div>
|
||||
<div class="panel panel-default" id="nodelist-decom">
|
||||
<div class="panel-heading">Nodes being decomissioned</div>
|
||||
<div class="panel-body">
|
||||
<table class="table">
|
||||
<thead>
|
||||
<tr>
|
||||
<th>Node</th>
|
||||
<th>Last contact</th>
|
||||
<th>Under replicated blocks</th>
|
||||
<th>Blocks with no live replicas</th>
|
||||
<th>Under Replicated Blocks <br/>In files under construction</th>
|
||||
</tr>
|
||||
</thead>
|
||||
{#nn.DecomNodes}
|
||||
<tr>
|
||||
<td>{name} ({xferaddr})</td>
|
||||
<td>{lastContact}</td>
|
||||
<td>{underReplicatedBlocks}</td>
|
||||
<td>{decommissionOnlyReplicas}</td>
|
||||
<td>{underReplicateInOpenFiles}</td>
|
||||
</tr>
|
||||
{/nn.DecomNodes}
|
||||
</table>
|
||||
</div>
|
||||
</div>
|
||||
</div>
|
||||
</div>
|
|
@ -0,0 +1,43 @@
|
|||
<!--
|
||||
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.
|
||||
-->
|
||||
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
|
||||
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
|
||||
<html xmlns="http://www.w3.org/1999/xhtml">
|
||||
<head>
|
||||
<link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
|
||||
<link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
|
||||
<title>Namenode information</title>
|
||||
</head>
|
||||
<body>
|
||||
<div class="container">
|
||||
<div class="alert alert-danger" id="alert-panel" style="display:none">
|
||||
<button type="button" class="close" onclick="$('#alert-panel').hide();">×</button>
|
||||
<div class="alert-body" id="alert-panel-body"></div>
|
||||
</div>
|
||||
<div id="panel"></div>
|
||||
</div>
|
||||
<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="/static/dust-full-2.0.0.min.js">
|
||||
</script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
|
||||
</script><script type="text/javascript" src="dfs-dust.js">
|
||||
</script><script type="text/javascript" src="dfshealth.js">
|
||||
</script>
|
||||
<hr />
|
||||
<p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
|
||||
</body>
|
||||
</html>
|
|
@ -0,0 +1,126 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
(function () {
|
||||
"use strict";
|
||||
|
||||
var data = {};
|
||||
|
||||
function render() {
|
||||
var helpers = {
|
||||
'helper_fs_max_objects': function (chunk, ctx, bodies, params) {
|
||||
var o = ctx.current();
|
||||
if (o.MaxObjects > 0) {
|
||||
chunk.write('(' + Math.round((o.FilesTotal + o.BlockTotal) / o.MaxObjects * 100) * 100 + ')%');
|
||||
}
|
||||
},
|
||||
|
||||
'helper_dir_status': function (chunk, ctx, bodies, params) {
|
||||
var j = ctx.current();
|
||||
for (var i in j) {
|
||||
chunk.write('<tr><td>' + i + '</td><td>' + j[i] + '</td><td>' + params.type + '</td></tr>');
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
var base = dust.makeBase(helpers);
|
||||
|
||||
var TEMPLATES = [ { 'name': 'dfshealth', 'url': 'dfshealth.dust.html' } ];
|
||||
|
||||
load_templates(dust, TEMPLATES, function() {
|
||||
dust.render('dfshealth', base.push(data), function(err, out) {
|
||||
$('#panel').html(out);
|
||||
});
|
||||
}, function () {
|
||||
show_err_msg('Failed to load the page.');
|
||||
});
|
||||
}
|
||||
|
||||
var BEANS = [
|
||||
{"name": "nn", "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeInfo"},
|
||||
{"name": "nnstat", "url": "/jmx?qry=Hadoop:service=NameNode,name=NameNodeStatus"},
|
||||
{"name": "fs", "url": "/jmx?qry=Hadoop:service=NameNode,name=FSNamesystemState"},
|
||||
{"name": "mem", "url": "/jmx?qry=java.lang:type=Memory"},
|
||||
{"name": "startup", "url": "/startupProgress"}
|
||||
];
|
||||
|
||||
// Workarounds for the fact that JMXJsonServlet returns non-standard JSON strings
|
||||
function data_workaround(d) {
|
||||
function node_map_to_array(nodes) {
|
||||
var res = [];
|
||||
for (var n in nodes) {
|
||||
var p = nodes[n];
|
||||
p.name = n;
|
||||
res.push(p);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
function startup_progress_workaround(r) {
|
||||
function rename_property(o, s, d) {
|
||||
if (o[s] !== undefined) {
|
||||
o[d] = o[s];
|
||||
delete o[s];
|
||||
}
|
||||
}
|
||||
r.percentComplete *= 100;
|
||||
$.each(r.phases, function (idx, p) {
|
||||
p.percentComplete *= 100;
|
||||
$.each(p.steps, function (idx2, s) {
|
||||
s.percentComplete *= 100;
|
||||
// dust.js is confused by these optional keys in nested
|
||||
// structure, rename them
|
||||
rename_property(s, "desc", "stepDesc");
|
||||
rename_property(s, "file", "stepFile");
|
||||
rename_property(s, "size", "stepSize");
|
||||
});
|
||||
});
|
||||
return r;
|
||||
}
|
||||
|
||||
d.nn.JournalTransactionInfo = JSON.parse(d.nn.JournalTransactionInfo);
|
||||
d.nn.NameJournalStatus = JSON.parse(d.nn.NameJournalStatus);
|
||||
d.nn.NameDirStatuses = JSON.parse(d.nn.NameDirStatuses);
|
||||
d.nn.NodeUsage = JSON.parse(d.nn.NodeUsage);
|
||||
d.nn.LiveNodes = node_map_to_array(JSON.parse(d.nn.LiveNodes));
|
||||
d.nn.DeadNodes = node_map_to_array(JSON.parse(d.nn.DeadNodes));
|
||||
d.nn.DecomNodes = node_map_to_array(JSON.parse(d.nn.DecomNodes));
|
||||
d.nn.CorruptFiles = JSON.parse(d.nn.CorruptFiles);
|
||||
|
||||
d.fs.SnapshotStats = JSON.parse(d.fs.SnapshotStats);
|
||||
d.startup = startup_progress_workaround(d.startup);
|
||||
return d;
|
||||
}
|
||||
|
||||
function show_err_msg(msg) {
|
||||
$('#alert-panel-body').html(msg);
|
||||
$('#alert-panel').show();
|
||||
}
|
||||
|
||||
load_json(
|
||||
BEANS,
|
||||
function(d) {
|
||||
for (var k in d) {
|
||||
data[k] = k === "startup" ? d[k] : d[k].beans[0];
|
||||
}
|
||||
data = data_workaround(data);
|
||||
render();
|
||||
},
|
||||
function (url, jqxhr, text, err) {
|
||||
show_err_msg('<p>Failed to retrieve data from ' + url + ', cause: ' + err + '</p>');
|
||||
});
|
||||
})();
|
|
@ -0,0 +1,13 @@
|
|||
{#block}
|
||||
<p>Block ID: {blockId}</p>
|
||||
<p>Block Pool ID: {blockPoolId}</p>
|
||||
<p>Generation Stamp: {generationStamp}</p>
|
||||
<p>Size: {numBytes}</p>
|
||||
{/block}
|
||||
<p>Availability:
|
||||
<ul>
|
||||
{#locations}
|
||||
<li>{hostName}</li>
|
||||
{/locations}
|
||||
</ul>
|
||||
</p>
|
|
@ -0,0 +1,26 @@
|
|||
<table class="table">
|
||||
<thead>
|
||||
<tr>
|
||||
<th>Permission</th>
|
||||
<th>Owner</th>
|
||||
<th>Group</th>
|
||||
<th>Size</th>
|
||||
<th>Replication</th>
|
||||
<th>Block Size</th>
|
||||
<th>Name</th>
|
||||
</tr>
|
||||
</thead>
|
||||
<tbody>
|
||||
{#FileStatus}
|
||||
<tr>
|
||||
<td>{#helper_to_permission/}</td>
|
||||
<td>{owner}</td>
|
||||
<td>{group}</td>
|
||||
<td>{length|fmt_bytes}</td>
|
||||
<td>{replication}</td>
|
||||
<td>{blockSize|fmt_bytes}</td>
|
||||
<td><a style="cursor:pointer" inode-type="{type}" class="explorer-browse-links" inode-path="{pathSuffix}">{pathSuffix}</a></td>
|
||||
</tr>
|
||||
{/FileStatus}
|
||||
</tbody>
|
||||
</table>
|
|
@ -0,0 +1,86 @@
|
|||
<!--
|
||||
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.
|
||||
-->
|
||||
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
|
||||
"http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
|
||||
<html xmlns="http://www.w3.org/1999/xhtml">
|
||||
<head>
|
||||
<link rel="stylesheet" type="text/css" href="//netdna.bootstrapcdn.com/bootstrap/3.0.0/css/bootstrap.min.css" />
|
||||
<link rel="stylesheet" type="text/css" href="/static/hadoop.css" />
|
||||
<title>Browsing HDFS</title>
|
||||
</head>
|
||||
<body>
|
||||
<div class="modal" id="file-info" tabindex="-1" role="dialog" aria-hidden="true">
|
||||
<div class="modal-dialog">
|
||||
<div class="modal-content">
|
||||
<div class="modal-header"><button type="button" class="close" data-dismiss="modal" aria-hidden="true">×</button>
|
||||
<h4 class="modal-title" id="file-info-title">File information</h4>
|
||||
</div>
|
||||
<div class="modal-body" id="file-info-body">
|
||||
<a id="file-info-download">Download</a>
|
||||
<a id="file-info-preview" style="cursor:pointer">Tail the file (last 32K)</a>
|
||||
<hr />
|
||||
<div class="panel panel-info" id="file-info-blockinfo-panel">
|
||||
<div class="panel-heading">
|
||||
Block information --
|
||||
<select class="btn btn-default" id="file-info-blockinfo-list">
|
||||
</select>
|
||||
</div>
|
||||
<div class="panel-body" id="file-info-blockinfo-body"></div>
|
||||
</div>
|
||||
<div class="panel panel-info" id="file-info-tail" style="display:none">
|
||||
<div class="panel-heading">File contents</div>
|
||||
<div class="panel-body">
|
||||
<div class="input-group-sm">
|
||||
<textarea class="form-control" style="height: 150px" id="file-info-preview-body"></textarea>
|
||||
</div>
|
||||
</div>
|
||||
</div>
|
||||
</div>
|
||||
<div class="modal-footer"><button type="button" class="btn btn-primary"
|
||||
data-dismiss="modal">Close</button></div>
|
||||
</div>
|
||||
</div>
|
||||
</div>
|
||||
<div class="container">
|
||||
<div class="page-header">
|
||||
<h1>Browse Directory</h1>
|
||||
</div>
|
||||
<div class="alert alert-danger" id="alert-panel" style="display:none">
|
||||
<button type="button" class="close" onclick="$('#alert-panel').hide();">×</button>
|
||||
<div class="alert-body" id="alert-panel-body"></div>
|
||||
</div>
|
||||
<div class="row">
|
||||
<form onsubmit="return false;">
|
||||
<div class="input-group"><input type="text" class="form-control" id=
|
||||
"directory" /> <span class="input-group-btn"><button class="btn btn-default"
|
||||
type="submit" id="btn-nav-directory"><span class="input-group-btn">Go!</span></button></span></div>
|
||||
</form>
|
||||
</div>
|
||||
<br />
|
||||
<div id="panel"></div>
|
||||
</div>
|
||||
<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="/static/dust-full-2.0.0.min.js">
|
||||
</script><script type="text/javascript" src="/static/dust-helpers-1.1.1.min.js">
|
||||
</script><script type="text/javascript" src="dfs-dust.js">
|
||||
</script><script type="text/javascript" src="explorer.js">
|
||||
</script>
|
||||
<hr />
|
||||
<p><a href="http://hadoop.apache.org/core">Hadoop</a>, 2013.</p>
|
||||
</body>
|
||||
</html>
|
|
@ -0,0 +1,182 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
(function() {
|
||||
"use strict";
|
||||
|
||||
// The chunk size of tailing the files, i.e., how many bytes will be shown
|
||||
// in the preview.
|
||||
var TAIL_CHUNK_SIZE = 32768;
|
||||
var helpers = {
|
||||
'helper_to_permission': function(chunk, ctx, bodies, params) {
|
||||
var p = ctx.current().permission;
|
||||
var dir = ctx.current().type == 'DIRECTORY' ? 'd' : '-';
|
||||
var symbols = [ '---', '--x', '-w-', '-wx', 'r--', 'r-x', 'rw-', 'rwx' ];
|
||||
var sticky = p > 1000;
|
||||
|
||||
var res = "";
|
||||
for (var i = 0; i < 3; ++i) {
|
||||
res = symbols[(p % 10)] + res;
|
||||
p = Math.floor(p / 10);
|
||||
}
|
||||
|
||||
if (sticky) {
|
||||
var exec = ((parms.perm % 10) & 1) == 1;
|
||||
res[res.length - 1] = exec ? 't' : 'T';
|
||||
}
|
||||
|
||||
chunk.write(dir + res);
|
||||
return chunk;
|
||||
}
|
||||
};
|
||||
|
||||
var base = dust.makeBase(helpers);
|
||||
var current_directory = "";
|
||||
|
||||
function show_err_msg(msg) {
|
||||
$('#alert-panel-body').html(msg);
|
||||
$('#alert-panel').show();
|
||||
}
|
||||
|
||||
function network_error_handler(url) {
|
||||
return function (jqxhr, text, err) {
|
||||
var msg = '<p>Failed to retreive data from ' + url + ', cause: ' + err + '</p>';
|
||||
if (url.indexOf('/webhdfs/v1') === 0) {
|
||||
msg += '<p>WebHDFS might be disabled. WebHDFS is required to browse the filesystem.</p>';
|
||||
}
|
||||
show_err_msg(msg);
|
||||
};
|
||||
}
|
||||
|
||||
function append_path(prefix, s) {
|
||||
var l = prefix.length;
|
||||
var p = l > 0 && prefix[l - 1] == '/' ? prefix.substring(0, l - 1) : prefix;
|
||||
return p + '/' + s;
|
||||
}
|
||||
|
||||
function get_response(data, type) {
|
||||
return data[type] !== undefined ? data[type] : null;
|
||||
}
|
||||
|
||||
function get_response_err_msg(data) {
|
||||
var msg = data.RemoteException !== undefined ? data.RemoteException.message : "";
|
||||
return msg;
|
||||
}
|
||||
|
||||
function view_file_details(path, abs_path) {
|
||||
function show_block_info(blocks) {
|
||||
var menus = $('#file-info-blockinfo-list');
|
||||
menus.empty();
|
||||
|
||||
menus.data("blocks", blocks);
|
||||
menus.change(function() {
|
||||
var d = $(this).data('blocks')[$(this).val()];
|
||||
if (d === undefined) {
|
||||
return;
|
||||
}
|
||||
|
||||
dust.render('block-info', d, function(err, out) {
|
||||
$('#file-info-blockinfo-body').html(out);
|
||||
});
|
||||
|
||||
});
|
||||
for (var i = 0; i < blocks.length; ++i) {
|
||||
var item = $('<option value="' + i + '">Block ' + i + '</option>');
|
||||
menus.append(item);
|
||||
}
|
||||
menus.change();
|
||||
}
|
||||
|
||||
var url = '/webhdfs/v1' + abs_path + '?op=GET_BLOCK_LOCATIONS';
|
||||
$.ajax({"url": url, "crossDomain": true}).done(function(data) {
|
||||
var d = get_response(data, "LocatedBlocks");
|
||||
if (d === null) {
|
||||
show_err_msg(get_response_err_msg(data));
|
||||
return;
|
||||
}
|
||||
|
||||
$('#file-info-tail').hide();
|
||||
$('#file-info-title').text("File information - " + path);
|
||||
|
||||
var download_url = '/webhdfs/v1' + abs_path + '/?op=OPEN';
|
||||
|
||||
$('#file-info-download').attr('href', download_url);
|
||||
$('#file-info-preview').click(function() {
|
||||
var offset = d.fileLength - TAIL_CHUNK_SIZE;
|
||||
var url = offset > 0 ? download_url + '&offset=' + offset : download_url;
|
||||
$.get(url, function(t) {
|
||||
$('#file-info-preview-body').val(t);
|
||||
$('#file-info-tail').show();
|
||||
}, "text").error(network_error_handler(url));
|
||||
});
|
||||
|
||||
if (d.fileLength > 0) {
|
||||
show_block_info(d.locatedBlocks);
|
||||
$('#file-info-blockinfo-panel').show();
|
||||
} else {
|
||||
$('#file-info-blockinfo-panel').hide();
|
||||
}
|
||||
$('#file-info').modal();
|
||||
}).error(network_error_handler(url));
|
||||
}
|
||||
|
||||
function browse_directory(dir) {
|
||||
var url = '/webhdfs/v1' + dir + '?op=LISTSTATUS';
|
||||
$.get(url, function(data) {
|
||||
var d = get_response(data, "FileStatuses");
|
||||
if (d === null) {
|
||||
show_err_msg(get_response_err_msg(data));
|
||||
return;
|
||||
}
|
||||
|
||||
current_directory = dir;
|
||||
$('#directory').val(dir);
|
||||
dust.render('explorer', base.push(d), function(err, out) {
|
||||
$('#panel').html(out);
|
||||
|
||||
$('.explorer-browse-links').click(function() {
|
||||
var type = $(this).attr('inode-type');
|
||||
var path = $(this).attr('inode-path');
|
||||
var abs_path = append_path(current_directory, path);
|
||||
if (type == 'DIRECTORY') {
|
||||
browse_directory(abs_path);
|
||||
} else {
|
||||
view_file_details(path, abs_path);
|
||||
}
|
||||
});
|
||||
});
|
||||
}).error(network_error_handler(url));
|
||||
}
|
||||
|
||||
|
||||
function init() {
|
||||
var templates = [
|
||||
{ 'name': 'explorer', 'url': 'explorer.dust.html'},
|
||||
{ 'name': 'block-info', 'url': 'explorer-block-info.dust.html'}
|
||||
];
|
||||
|
||||
load_templates(dust, templates, function () {
|
||||
var b = function() { browse_directory($('#directory').val()); };
|
||||
$('#btn-nav-directory').click(b);
|
||||
browse_directory('/');
|
||||
}, function (url, jqxhr, text, err) {
|
||||
network_error_handler(url)(jqxhr, text, err);
|
||||
});
|
||||
}
|
||||
|
||||
init();
|
||||
})();
|
1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js
vendored
Normal file
1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js
vendored
Normal file
File diff suppressed because one or more lines are too long
8
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js
vendored
Normal file
8
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js
vendored
Normal file
|
@ -0,0 +1,8 @@
|
|||
(function(k){function n(b){b=b.f();return"object"===typeof b&&!0===b.h}function p(b,c){return"function"===typeof c?c.toString():c}function l(b,c,d,a,e){a=a||{};var m=d.a,g,f,l=a.d||"";if("undefined"!==typeof a.key)g=k.b.c(a.key,b,c);else if(n(c))g=c.f().i,c.f().g&&(e=function(){return!1});else return h.log("No key specified for filter in:"+l+" helper "),b;f=k.b.c(a.value,b,c);if(e(q(f,a.type,c),q(g,a.type,c))){n(c)&&(c.f().g=!0);if(m)return b.e(m,c);h.log("Missing body block in the "+l+" helper ")}else if(d["else"])return b.e(d["else"],
|
||||
c);return b}function q(b,c,d){if(b)switch(c||typeof b){case "number":return+b;case "string":return String(b);case "boolean":return Boolean("false"===b?!1:b);case "date":return new Date(b);case "context":return d.get(b)}return b}var h="undefined"!==typeof console?console:{log:function(){}};k.b={tap:function(b,c,d){var a=b;"function"===typeof b&&(!0===b.l?a=b():(a="",c.c(function(b){a+=b;return""}).e(b,d).p(),""===a&&(a=!1)));return a},sep:function(b,c,d){return c.stack.index===c.stack.m-1?b:d.a?d.a(b,
|
||||
c):b},idx:function(b,c,d){return d.a?d.a(b,c.push(c.stack.index)):b},contextDump:function(b,c,d,a){a=a||{};d=a.o||"output";a=a.key||"current";d=k.b.c(d,b,c);a=k.b.c(a,b,c);c="full"===a?JSON.stringify(c.stack,p,2):JSON.stringify(c.stack.head,p,2);return"console"===d?(h.log(c),b):b.write(c)},"if":function(b,c,d,a){var e=d.a,m=d["else"];if(a&&a.j){a=a.j;a=k.b.c(a,b,c);if(eval(a)){if(e)return b.e(d.a,c);h.log("Missing body block in the if helper!");return b}if(m)return b.e(d["else"],c)}else h.log("No condition given in the if helper!");
|
||||
return b},math:function(b,c,d,a){if(a&&"undefined"!==typeof a.key&&a.method){var e=a.key,m=a.method,g=a.n;a=a.round;var f=null,e=k.b.c(e,b,c),g=k.b.c(g,b,c);switch(m){case "mod":0!==g&&-0!==g||h.log("operand for divide operation is 0/-0: expect Nan!");f=parseFloat(e)%parseFloat(g);break;case "add":f=parseFloat(e)+parseFloat(g);break;case "subtract":f=parseFloat(e)-parseFloat(g);break;case "multiply":f=parseFloat(e)*parseFloat(g);break;case "divide":0!==g&&-0!==g||h.log("operand for divide operation is 0/-0: expect Nan/Infinity!");
|
||||
f=parseFloat(e)/parseFloat(g);break;case "ceil":f=Math.ceil(parseFloat(e));break;case "floor":f=Math.floor(parseFloat(e));break;case "round":f=Math.round(parseFloat(e));break;case "abs":f=Math.abs(parseFloat(e));break;default:h.log("method passed is not supported")}if(null!==f)return a&&(f=Math.round(f)),d&&d.a?b.e(d.a,c.push({h:!0,g:!1,i:f})):b.write(f)}else h.log("Key is a required parameter for math helper along with method/operand!");return b},select:function(b,c,d,a){var e=d.a;if(a&&"undefined"!==
|
||||
typeof a.key){a=k.b.c(a.key,b,c);if(e)return b.e(d.a,c.push({h:!0,g:!1,i:a}));h.log("Missing body block in the select helper ")}else h.log("No key given in the select helper!");return b},eq:function(b,c,d,a){a&&(a.d="eq");return l(b,c,d,a,function(a,b){return b===a})},ne:function(b,c,d,a){return a?(a.d="ne",l(b,c,d,a,function(a,b){return b!==a})):b},lt:function(b,c,d,a){if(a)return a.d="lt",l(b,c,d,a,function(a,b){return b<a})},lte:function(b,c,d,a){return a?(a.d="lte",l(b,c,d,a,function(a,b){return b<=
|
||||
a})):b},gt:function(b,c,d,a){return a?(a.d="gt",l(b,c,d,a,function(a,b){return b>a})):b},gte:function(b,c,d,a){return a?(a.d="gte",l(b,c,d,a,function(a,b){return b>=a})):b},"default":function(b,c,d,a){a&&(a.d="default");return l(b,c,d,a,function(){return!0})},size:function(b,c,d,a){c=0;var e;a=a||{};if((a=a.key)&&!0!==a)if(k.isArray(a))c=a.length;else if(!isNaN(parseFloat(a))&&isFinite(a))c=a;else if("object"===typeof a)for(e in c=0,a)Object.hasOwnProperty.call(a,e)&&c++;else c=(a+"").length;else c=
|
||||
0;return b.write(c)}}})("undefined"!==typeof exports?module.k=require("dustjs-linkedin"):dust);
|
|
@ -0,0 +1,258 @@
|
|||
|
||||
~~ Licensed 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. See accompanying LICENSE file.
|
||||
|
||||
---
|
||||
Hadoop Distributed File System-${project.version} - HDFS NFS Gateway
|
||||
---
|
||||
---
|
||||
${maven.build.timestamp}
|
||||
|
||||
HDFS NFS Gateway
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
%{toc|section=1|fromDepth=0}
|
||||
|
||||
* {Overview}
|
||||
|
||||
The NFS Gateway supports NFSv3 and allows HDFS to be mounted as part of the client's local file system.
|
||||
Currently NFS Gateway supports and enables the following usage patterns:
|
||||
|
||||
* Users can browse the HDFS file system through their local file system
|
||||
on NFSv3 client compatible operating systems.
|
||||
|
||||
* Users can download files from the the HDFS file system on to their
|
||||
local file system.
|
||||
|
||||
* Users can upload files from their local file system directly to the
|
||||
HDFS file system.
|
||||
|
||||
* Users can stream data directly to HDFS through the mount point. File
|
||||
append is supported but random write is not supported.
|
||||
|
||||
The NFS gateway machine needs the same thing to run an HDFS client like Hadoop JAR files, HADOOP_CONF directory.
|
||||
The NFS gateway can be on the same host as DataNode, NameNode, or any HDFS client.
|
||||
|
||||
|
||||
* {Configuration}
|
||||
|
||||
NFS gateway can work with its default settings in most cases. However, it's
|
||||
strongly recommended for the users to update a few configuration properties based on their use
|
||||
cases. All the related configuration properties can be added or updated in hdfs-site.xml.
|
||||
|
||||
* If the client mounts the export with access time update allowed, make sure the following
|
||||
property is not disabled in the configuration file. Only NameNode needs to restart after
|
||||
this property is changed. On some Unix systems, the user can disable access time update
|
||||
by mounting the export with "noatime".
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.access.time.precision</name>
|
||||
<value>3600000</value>
|
||||
<description>The access time for HDFS file is precise upto this value.
|
||||
The default value is 1 hour. Setting a value of 0 disables
|
||||
access times for HDFS.
|
||||
</description>
|
||||
</property>
|
||||
----
|
||||
|
||||
* Users are expected to update the file dump directory. NFS client often
|
||||
reorders writes. Sequential writes can arrive at the NFS gateway at random
|
||||
order. This directory is used to temporarily save out-of-order writes
|
||||
before writing to HDFS. For each file, the out-of-order writes are dumped after
|
||||
they are accumulated to exceed certain threshold (e.g., 1MB) in memory.
|
||||
One needs to make sure the directory has enough
|
||||
space. For example, if the application uploads 10 files with each having
|
||||
100MB, it is recommended for this directory to have roughly 1GB space in case if a
|
||||
worst-case write reorder happens to every file. Only NFS gateway needs to restart after
|
||||
this property is updated.
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs3.dump.dir</name>
|
||||
<value>/tmp/.hdfs-nfs</value>
|
||||
</property>
|
||||
----
|
||||
|
||||
* By default, the export can be mounted by any client. To better control the access,
|
||||
users can update the following property. The value string contains machine name and
|
||||
access privilege, separated by whitespace
|
||||
characters. Machine name format can be single host, wildcards, and IPv4 networks.The
|
||||
access privilege uses rw or ro to specify readwrite or readonly access of the machines to exports. If the access
|
||||
privilege is not provided, the default is read-only. Entries are separated by ";".
|
||||
For example: "192.168.0.0/22 rw ; host*.example.com ; host1.test.org ro;". Only NFS gateway needs to restart after
|
||||
this property is updated.
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs.exports.allowed.hosts</name>
|
||||
<value>* rw</value>
|
||||
</property>
|
||||
----
|
||||
|
||||
* Customize log settings. To get NFS debug trace, users can edit the log4j.property file
|
||||
to add the following. Note, debug trace, especially for ONCRPC, can be very verbose.
|
||||
|
||||
To change logging level:
|
||||
|
||||
-----------------------------------------------
|
||||
log4j.logger.org.apache.hadoop.hdfs.nfs=DEBUG
|
||||
-----------------------------------------------
|
||||
|
||||
To get more details of ONCRPC requests:
|
||||
|
||||
-----------------------------------------------
|
||||
log4j.logger.org.apache.hadoop.oncrpc=DEBUG
|
||||
-----------------------------------------------
|
||||
|
||||
|
||||
* {Start and stop NFS gateway service}
|
||||
|
||||
Three daemons are required to provide NFS service: rpcbind (or portmap), mountd and nfsd.
|
||||
The NFS gateway process has both nfsd and mountd. It shares the HDFS root "/" as the
|
||||
only export. It is recommended to use the portmap included in NFS gateway package. Even
|
||||
though NFS gateway works with portmap/rpcbind provide by most Linux distributions, the
|
||||
package included portmap is needed on some Linux systems such as REHL6.2 due to an
|
||||
{{{https://bugzilla.redhat.com/show_bug.cgi?id=731542}rpcbind bug}}. More detailed discussions can
|
||||
be found in {{{https://issues.apache.org/jira/browse/HDFS-4763}HDFS-4763}}.
|
||||
|
||||
[[1]] Stop nfs/rpcbind/portmap services provided by the platform (commands can be different on various Unix platforms):
|
||||
|
||||
-------------------------
|
||||
service nfs stop
|
||||
|
||||
service rpcbind stop
|
||||
-------------------------
|
||||
|
||||
|
||||
[[2]] Start package included portmap (needs root privileges):
|
||||
|
||||
-------------------------
|
||||
hadoop portmap
|
||||
|
||||
OR
|
||||
|
||||
hadoop-daemon.sh start portmap
|
||||
-------------------------
|
||||
|
||||
[[3]] Start mountd and nfsd.
|
||||
|
||||
No root privileges are required for this command. However, ensure that the user starting
|
||||
the Hadoop cluster and the user starting the NFS gateway are same.
|
||||
|
||||
-------------------------
|
||||
hadoop nfs3
|
||||
|
||||
OR
|
||||
|
||||
hadoop-daemon.sh start nfs3
|
||||
-------------------------
|
||||
|
||||
Note, if the hadoop-daemon.sh script starts the NFS gateway, its log can be found in the hadoop log folder.
|
||||
|
||||
|
||||
[[4]] Stop NFS gateway services.
|
||||
|
||||
-------------------------
|
||||
hadoop-daemon.sh stop nfs3
|
||||
|
||||
hadoop-daemon.sh stop portmap
|
||||
-------------------------
|
||||
|
||||
|
||||
* {Verify validity of NFS related services}
|
||||
|
||||
[[1]] Execute the following command to verify if all the services are up and running:
|
||||
|
||||
-------------------------
|
||||
rpcinfo -p $nfs_server_ip
|
||||
-------------------------
|
||||
|
||||
You should see output similar to the following:
|
||||
|
||||
-------------------------
|
||||
program vers proto port
|
||||
|
||||
100005 1 tcp 4242 mountd
|
||||
|
||||
100005 2 udp 4242 mountd
|
||||
|
||||
100005 2 tcp 4242 mountd
|
||||
|
||||
100000 2 tcp 111 portmapper
|
||||
|
||||
100000 2 udp 111 portmapper
|
||||
|
||||
100005 3 udp 4242 mountd
|
||||
|
||||
100005 1 udp 4242 mountd
|
||||
|
||||
100003 3 tcp 2049 nfs
|
||||
|
||||
100005 3 tcp 4242 mountd
|
||||
-------------------------
|
||||
|
||||
[[2]] Verify if the HDFS namespace is exported and can be mounted.
|
||||
|
||||
-------------------------
|
||||
showmount -e $nfs_server_ip
|
||||
-------------------------
|
||||
|
||||
You should see output similar to the following:
|
||||
|
||||
-------------------------
|
||||
Exports list on $nfs_server_ip :
|
||||
|
||||
/ (everyone)
|
||||
-------------------------
|
||||
|
||||
|
||||
* {Mount the export “/”}
|
||||
|
||||
Currently NFS v3 only uses TCP as the transportation protocol.
|
||||
NLM is not supported so mount option "nolock" is needed. It's recommended to use
|
||||
hard mount. This is because, even after the client sends all data to
|
||||
NFS gateway, it may take NFS gateway some extra time to transfer data to HDFS
|
||||
when writes were reorderd by NFS client Kernel.
|
||||
|
||||
If soft mount has to be used, the user should give it a relatively
|
||||
long timeout (at least no less than the default timeout on the host) .
|
||||
|
||||
The users can mount the HDFS namespace as shown below:
|
||||
|
||||
-------------------------------------------------------------------
|
||||
mount -t nfs -o vers=3,proto=tcp,nolock $server:/ $mount_point
|
||||
-------------------------------------------------------------------
|
||||
|
||||
Then the users can access HDFS as part of the local file system except that,
|
||||
hard link and random write are not supported yet.
|
||||
|
||||
* {User authentication and mapping}
|
||||
|
||||
NFS gateway in this release uses AUTH_UNIX style authentication. When the user on NFS client
|
||||
accesses the mount point, NFS client passes the UID to NFS gateway.
|
||||
NFS gateway does a lookup to find user name from the UID, and then passes the
|
||||
username to the HDFS along with the HDFS requests.
|
||||
For example, if the NFS client has current user as "admin", when the user accesses
|
||||
the mounted directory, NFS gateway will access HDFS as user "admin". To access HDFS
|
||||
as the user "hdfs", one needs to switch the current user to "hdfs" on the client system
|
||||
when accessing the mounted directory.
|
||||
|
||||
The system administrator must ensure that the user on NFS client host has the same
|
||||
name and UID as that on the NFS gateway host. This is usually not a problem if
|
||||
the same user management system (e.g., LDAP/NIS) is used to create and deploy users on
|
||||
HDFS nodes and NFS client node. In case the user account is created manually in different hosts, one might need to
|
||||
modify UID (e.g., do "usermod -u 123 myusername") on either NFS client or NFS gateway host
|
||||
in order to make it the same on both sides. More technical details of RPC AUTH_UNIX can be found
|
||||
in {{{http://tools.ietf.org/html/rfc1057}RPC specification}}.
|
||||
|
|
@ -83,7 +83,6 @@ import org.apache.log4j.RollingFileAppender;
|
|||
import org.junit.Test;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import org.mockito.Mockito;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
/**
|
||||
|
@ -892,6 +891,80 @@ public class TestFsck {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that the # of misreplaced replicas is correct
|
||||
* @throws IOException
|
||||
*/
|
||||
@Test
|
||||
public void testFsckMisPlacedReplicas() throws IOException {
|
||||
// Desired replication factor
|
||||
final short REPL_FACTOR = 2;
|
||||
// Number of replicas to actually start
|
||||
short NUM_DN = 2;
|
||||
// Number of blocks to write
|
||||
final short NUM_BLOCKS = 3;
|
||||
// Set a small-ish blocksize
|
||||
final long blockSize = 512;
|
||||
|
||||
String [] racks = {"/rack1", "/rack1"};
|
||||
String [] hosts = {"host1", "host2"};
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
||||
|
||||
MiniDFSCluster cluster = null;
|
||||
DistributedFileSystem dfs = null;
|
||||
|
||||
try {
|
||||
// Startup a minicluster
|
||||
cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DN).hosts(hosts)
|
||||
.racks(racks).build();
|
||||
assertNotNull("Failed Cluster Creation", cluster);
|
||||
cluster.waitClusterUp();
|
||||
dfs = (DistributedFileSystem) cluster.getFileSystem();
|
||||
assertNotNull("Failed to get FileSystem", dfs);
|
||||
|
||||
// Create a file that will be intentionally under-replicated
|
||||
final String pathString = new String("/testfile");
|
||||
final Path path = new Path(pathString);
|
||||
long fileLen = blockSize * NUM_BLOCKS;
|
||||
DFSTestUtil.createFile(dfs, path, fileLen, REPL_FACTOR, 1);
|
||||
|
||||
// Create an under-replicated file
|
||||
NameNode namenode = cluster.getNameNode();
|
||||
NetworkTopology nettop = cluster.getNamesystem().getBlockManager()
|
||||
.getDatanodeManager().getNetworkTopology();
|
||||
// Add a new node on different rack, so previous blocks' replicas
|
||||
// are considered to be misplaced
|
||||
nettop.add(DFSTestUtil.getDatanodeDescriptor("/rack2", "/host3"));
|
||||
NUM_DN++;
|
||||
|
||||
Map<String,String[]> pmap = new HashMap<String, String[]>();
|
||||
Writer result = new StringWriter();
|
||||
PrintWriter out = new PrintWriter(result, true);
|
||||
InetAddress remoteAddress = InetAddress.getLocalHost();
|
||||
NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out,
|
||||
NUM_DN, (short)REPL_FACTOR, remoteAddress);
|
||||
|
||||
// Run the fsck and check the Result
|
||||
final HdfsFileStatus file =
|
||||
namenode.getRpcServer().getFileInfo(pathString);
|
||||
assertNotNull(file);
|
||||
Result res = new Result(conf);
|
||||
fsck.check(pathString, file, res);
|
||||
// check misReplicatedBlock number.
|
||||
assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS);
|
||||
} finally {
|
||||
if(dfs != null) {
|
||||
dfs.close();
|
||||
}
|
||||
if(cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Test fsck with FileNotFound */
|
||||
@Test
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
@ -58,4 +59,27 @@ public class TestJsonUtil {
|
|||
System.out.println("fs2 = " + fs2);
|
||||
Assert.assertEquals(fstatus, fs2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testToDatanodeInfoWithoutSecurePort() {
|
||||
Map<String, Object> response = new HashMap<String, Object>();
|
||||
|
||||
response.put("ipAddr", "127.0.0.1");
|
||||
response.put("hostName", "localhost");
|
||||
response.put("storageID", "fake-id");
|
||||
response.put("xferPort", 1337l);
|
||||
response.put("infoPort", 1338l);
|
||||
// deliberately don't include an entry for "infoSecurePort"
|
||||
response.put("ipcPort", 1339l);
|
||||
response.put("capacity", 1024l);
|
||||
response.put("dfsUsed", 512l);
|
||||
response.put("remaining", 512l);
|
||||
response.put("blockPoolUsed", 512l);
|
||||
response.put("lastUpdate", 0l);
|
||||
response.put("xceiverCount", 4096l);
|
||||
response.put("networkLocation", "foo.bar.baz");
|
||||
response.put("adminState", "NORMAL");
|
||||
|
||||
JsonUtil.toDatanodeInfo(response);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.web;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URL;
|
||||
import java.util.List;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
public final class TestURLConnectionFactory {
|
||||
|
||||
@Test
|
||||
public void testConnConfiguratior() throws IOException {
|
||||
final URL u = new URL("http://localhost");
|
||||
final List<HttpURLConnection> conns = Lists.newArrayList();
|
||||
URLConnectionFactory fc = new URLConnectionFactory(
|
||||
URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT);
|
||||
|
||||
fc.setConnConfigurator(new ConnectionConfigurator() {
|
||||
@Override
|
||||
public HttpURLConnection configure(HttpURLConnection conn)
|
||||
throws IOException {
|
||||
Assert.assertEquals(u, conn.getURL());
|
||||
conns.add(conn);
|
||||
return conn;
|
||||
}
|
||||
});
|
||||
|
||||
fc.openConnection(u);
|
||||
Assert.assertEquals(1, conns.size());
|
||||
}
|
||||
}
|
|
@ -25,9 +25,11 @@ import java.io.IOException;
|
|||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.OutputStream;
|
||||
import java.net.InetAddress;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.ServerSocket;
|
||||
import java.net.Socket;
|
||||
import java.net.SocketAddress;
|
||||
import java.net.SocketTimeoutException;
|
||||
import java.nio.channels.SocketChannel;
|
||||
import java.util.ArrayList;
|
||||
|
@ -41,6 +43,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
|
@ -71,8 +74,9 @@ public class TestWebHdfsTimeouts {
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
Configuration conf = WebHdfsTestUtil.createConf();
|
||||
nnHttpAddress = NameNode.getHttpAddress(conf);
|
||||
serverSocket = new ServerSocket(nnHttpAddress.getPort(), CONNECTION_BACKLOG);
|
||||
serverSocket = new ServerSocket(0, CONNECTION_BACKLOG);
|
||||
nnHttpAddress = new InetSocketAddress("localhost", serverSocket.getLocalPort());
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "localhost:" + serverSocket.getLocalPort());
|
||||
fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf);
|
||||
fs.connectionFactory = connectionFactory;
|
||||
clients = new ArrayList<SocketChannel>();
|
||||
|
|
|
@ -0,0 +1,358 @@
|
|||
/**
|
||||
* 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.tools;
|
||||
|
||||
import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
|
||||
import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
|
||||
import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.HftpFileSystem;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.jboss.netty.bootstrap.ServerBootstrap;
|
||||
import org.jboss.netty.buffer.ChannelBuffer;
|
||||
import org.jboss.netty.buffer.ChannelBuffers;
|
||||
import org.jboss.netty.channel.Channel;
|
||||
import org.jboss.netty.channel.ChannelFutureListener;
|
||||
import org.jboss.netty.channel.ChannelHandlerContext;
|
||||
import org.jboss.netty.channel.ChannelPipeline;
|
||||
import org.jboss.netty.channel.ChannelPipelineFactory;
|
||||
import org.jboss.netty.channel.Channels;
|
||||
import org.jboss.netty.channel.ExceptionEvent;
|
||||
import org.jboss.netty.channel.MessageEvent;
|
||||
import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
|
||||
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
|
||||
import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
|
||||
import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
|
||||
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
||||
import org.jboss.netty.handler.codec.http.HttpRequest;
|
||||
import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponse;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
|
||||
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.UnmodifiableIterator;
|
||||
|
||||
public class TestDelegationTokenRemoteFetcher {
|
||||
private static final Logger LOG = Logger
|
||||
.getLogger(TestDelegationTokenRemoteFetcher.class);
|
||||
|
||||
private static final String EXP_DATE = "124123512361236";
|
||||
private static final String tokenFile = "http.file.dta";
|
||||
|
||||
private int httpPort;
|
||||
private String serviceUrl;
|
||||
private FileSystem fileSys;
|
||||
private Configuration conf;
|
||||
private ServerBootstrap bootstrap;
|
||||
private Token<DelegationTokenIdentifier> testToken;
|
||||
private volatile AssertionError assertionError;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
conf = new Configuration();
|
||||
fileSys = FileSystem.getLocal(conf);
|
||||
httpPort = NetUtils.getFreeSocketPort();
|
||||
serviceUrl = "http://localhost:" + httpPort;
|
||||
testToken = createToken(serviceUrl);
|
||||
}
|
||||
|
||||
@After
|
||||
public void clean() throws IOException {
|
||||
if (fileSys != null)
|
||||
fileSys.delete(new Path(tokenFile), true);
|
||||
if (bootstrap != null)
|
||||
bootstrap.releaseExternalResources();
|
||||
}
|
||||
|
||||
/**
|
||||
* try to fetch token without http server with IOException
|
||||
*/
|
||||
@Test
|
||||
public void testTokenFetchFail() throws Exception {
|
||||
try {
|
||||
DelegationTokenFetcher.main(new String[] { "-webservice=" + serviceUrl,
|
||||
tokenFile });
|
||||
fail("Token fetcher shouldn't start in absense of NN");
|
||||
} catch (IOException ex) {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* try to fetch token without http server with IOException
|
||||
*/
|
||||
@Test
|
||||
public void testTokenRenewFail() {
|
||||
try {
|
||||
DelegationTokenFetcher.renewDelegationToken(serviceUrl, testToken);
|
||||
fail("Token fetcher shouldn't be able to renew tokens in absense of NN");
|
||||
} catch (IOException ex) {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* try cancel token without http server with IOException
|
||||
*/
|
||||
@Test
|
||||
public void expectedTokenCancelFail() {
|
||||
try {
|
||||
DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
|
||||
fail("Token fetcher shouldn't be able to cancel tokens in absense of NN");
|
||||
} catch (IOException ex) {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* try fetch token and get http response with error
|
||||
*/
|
||||
@Test
|
||||
public void expectedTokenRenewErrorHttpResponse() {
|
||||
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
||||
try {
|
||||
DelegationTokenFetcher.renewDelegationToken(serviceUrl + "/exception",
|
||||
createToken(serviceUrl));
|
||||
fail("Token fetcher shouldn't be able to renew tokens using an invalid"
|
||||
+ " NN URL");
|
||||
} catch (IOException ex) {
|
||||
}
|
||||
if (assertionError != null)
|
||||
throw assertionError;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*
|
||||
*/
|
||||
@Test
|
||||
public void testCancelTokenFromHttp() throws IOException {
|
||||
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
||||
DelegationTokenFetcher.cancelDelegationToken(serviceUrl, testToken);
|
||||
if (assertionError != null)
|
||||
throw assertionError;
|
||||
}
|
||||
|
||||
/**
|
||||
* Call renew token using http server return new expiration time
|
||||
*/
|
||||
@Test
|
||||
public void testRenewTokenFromHttp() throws IOException {
|
||||
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
||||
assertTrue("testRenewTokenFromHttp error",
|
||||
Long.valueOf(EXP_DATE) == DelegationTokenFetcher.renewDelegationToken(
|
||||
serviceUrl, testToken));
|
||||
if (assertionError != null)
|
||||
throw assertionError;
|
||||
}
|
||||
|
||||
/**
|
||||
* Call fetch token using http server
|
||||
*/
|
||||
@Test
|
||||
public void expectedTokenIsRetrievedFromHttp() throws Exception {
|
||||
bootstrap = startHttpServer(httpPort, testToken, serviceUrl);
|
||||
DelegationTokenFetcher.main(new String[] { "-webservice=" + serviceUrl,
|
||||
tokenFile });
|
||||
Path p = new Path(fileSys.getWorkingDirectory(), tokenFile);
|
||||
Credentials creds = Credentials.readTokenStorageFile(p, conf);
|
||||
Iterator<Token<?>> itr = creds.getAllTokens().iterator();
|
||||
assertTrue("token not exist error", itr.hasNext());
|
||||
Token<?> fetchedToken = itr.next();
|
||||
Assert.assertArrayEquals("token wrong identifier error",
|
||||
testToken.getIdentifier(), fetchedToken.getIdentifier());
|
||||
Assert.assertArrayEquals("token wrong password error",
|
||||
testToken.getPassword(), fetchedToken.getPassword());
|
||||
if (assertionError != null)
|
||||
throw assertionError;
|
||||
}
|
||||
|
||||
private static Token<DelegationTokenIdentifier> createToken(String serviceUri) {
|
||||
byte[] pw = "hadoop".getBytes();
|
||||
byte[] ident = new DelegationTokenIdentifier(new Text("owner"), new Text(
|
||||
"renewer"), new Text("realuser")).getBytes();
|
||||
Text service = new Text(serviceUri);
|
||||
return new Token<DelegationTokenIdentifier>(ident, pw,
|
||||
HftpFileSystem.TOKEN_KIND, service);
|
||||
}
|
||||
|
||||
private interface Handler {
|
||||
void handle(Channel channel, Token<DelegationTokenIdentifier> token,
|
||||
String serviceUrl) throws IOException;
|
||||
}
|
||||
|
||||
private class FetchHandler implements Handler {
|
||||
|
||||
@Override
|
||||
public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
|
||||
String serviceUrl) throws IOException {
|
||||
Assert.assertEquals(testToken, token);
|
||||
|
||||
Credentials creds = new Credentials();
|
||||
creds.addToken(new Text(serviceUrl), token);
|
||||
DataOutputBuffer out = new DataOutputBuffer();
|
||||
creds.write(out);
|
||||
int fileLength = out.getData().length;
|
||||
ChannelBuffer cbuffer = ChannelBuffers.buffer(fileLength);
|
||||
cbuffer.writeBytes(out.getData());
|
||||
HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
|
||||
response.setHeader(HttpHeaders.Names.CONTENT_LENGTH,
|
||||
String.valueOf(fileLength));
|
||||
response.setContent(cbuffer);
|
||||
channel.write(response).addListener(ChannelFutureListener.CLOSE);
|
||||
}
|
||||
}
|
||||
|
||||
private class RenewHandler implements Handler {
|
||||
|
||||
@Override
|
||||
public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
|
||||
String serviceUrl) throws IOException {
|
||||
Assert.assertEquals(testToken, token);
|
||||
byte[] bytes = EXP_DATE.getBytes();
|
||||
ChannelBuffer cbuffer = ChannelBuffers.buffer(bytes.length);
|
||||
cbuffer.writeBytes(bytes);
|
||||
HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
|
||||
response.setHeader(HttpHeaders.Names.CONTENT_LENGTH,
|
||||
String.valueOf(bytes.length));
|
||||
response.setContent(cbuffer);
|
||||
channel.write(response).addListener(ChannelFutureListener.CLOSE);
|
||||
}
|
||||
}
|
||||
|
||||
private class ExceptionHandler implements Handler {
|
||||
|
||||
@Override
|
||||
public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
|
||||
String serviceUrl) throws IOException {
|
||||
Assert.assertEquals(testToken, token);
|
||||
HttpResponse response = new DefaultHttpResponse(HTTP_1_1,
|
||||
HttpResponseStatus.METHOD_NOT_ALLOWED);
|
||||
channel.write(response).addListener(ChannelFutureListener.CLOSE);
|
||||
}
|
||||
}
|
||||
|
||||
private class CancelHandler implements Handler {
|
||||
|
||||
@Override
|
||||
public void handle(Channel channel, Token<DelegationTokenIdentifier> token,
|
||||
String serviceUrl) throws IOException {
|
||||
Assert.assertEquals(testToken, token);
|
||||
HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
|
||||
channel.write(response).addListener(ChannelFutureListener.CLOSE);
|
||||
}
|
||||
}
|
||||
|
||||
private final class CredentialsLogicHandler extends
|
||||
SimpleChannelUpstreamHandler {
|
||||
|
||||
private final Token<DelegationTokenIdentifier> token;
|
||||
private final String serviceUrl;
|
||||
private ImmutableMap<String, Handler> routes = ImmutableMap.of(
|
||||
"/exception", new ExceptionHandler(),
|
||||
"/cancelDelegationToken", new CancelHandler(),
|
||||
"/getDelegationToken", new FetchHandler() ,
|
||||
"/renewDelegationToken", new RenewHandler());
|
||||
|
||||
public CredentialsLogicHandler(Token<DelegationTokenIdentifier> token,
|
||||
String serviceUrl) {
|
||||
this.token = token;
|
||||
this.serviceUrl = serviceUrl;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void messageReceived(ChannelHandlerContext ctx, final MessageEvent e)
|
||||
throws Exception {
|
||||
HttpRequest request = (HttpRequest) e.getMessage();
|
||||
if (request.getMethod() != GET) {
|
||||
return;
|
||||
}
|
||||
UnmodifiableIterator<Map.Entry<String, Handler>> iter = routes.entrySet()
|
||||
.iterator();
|
||||
while (iter.hasNext()) {
|
||||
Map.Entry<String, Handler> entry = iter.next();
|
||||
if (request.getUri().contains(entry.getKey())) {
|
||||
Handler handler = entry.getValue();
|
||||
try {
|
||||
handler.handle(e.getChannel(), token, serviceUrl);
|
||||
} catch (AssertionError ee) {
|
||||
TestDelegationTokenRemoteFetcher.this.assertionError = ee;
|
||||
HttpResponse response = new DefaultHttpResponse(HTTP_1_1,
|
||||
HttpResponseStatus.BAD_REQUEST);
|
||||
response.setContent(ChannelBuffers.copiedBuffer(ee.getMessage(),
|
||||
Charset.defaultCharset()));
|
||||
e.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
|
||||
throws Exception {
|
||||
Channel ch = e.getChannel();
|
||||
Throwable cause = e.getCause();
|
||||
|
||||
if (LOG.isDebugEnabled())
|
||||
LOG.debug(cause.getMessage());
|
||||
ch.close().addListener(ChannelFutureListener.CLOSE);
|
||||
}
|
||||
}
|
||||
|
||||
private ServerBootstrap startHttpServer(int port,
|
||||
final Token<DelegationTokenIdentifier> token, final String url) {
|
||||
ServerBootstrap bootstrap = new ServerBootstrap(
|
||||
new NioServerSocketChannelFactory(Executors.newCachedThreadPool(),
|
||||
Executors.newCachedThreadPool()));
|
||||
|
||||
bootstrap.setPipelineFactory(new ChannelPipelineFactory() {
|
||||
@Override
|
||||
public ChannelPipeline getPipeline() throws Exception {
|
||||
return Channels.pipeline(new HttpRequestDecoder(),
|
||||
new HttpChunkAggregator(65536), new HttpResponseEncoder(),
|
||||
new CredentialsLogicHandler(token, url));
|
||||
}
|
||||
});
|
||||
bootstrap.bind(new InetSocketAddress("localhost", port));
|
||||
return bootstrap;
|
||||
}
|
||||
|
||||
}
|
|
@ -21,9 +21,13 @@ package org.apache.hadoop.tools;
|
|||
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
|
||||
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.PipedInputStream;
|
||||
import java.io.PipedOutputStream;
|
||||
import java.io.PrintStream;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -92,6 +96,7 @@ public class TestJMXGet {
|
|||
//jmx.init();
|
||||
//jmx = new JMXGet();
|
||||
jmx.init(); // default lists namenode mbeans only
|
||||
assertTrue("error printAllValues", checkPrintAllValues(jmx));
|
||||
|
||||
//get some data from different source
|
||||
assertEquals(numDatanodes, Integer.parseInt(
|
||||
|
@ -103,7 +108,24 @@ public class TestJMXGet {
|
|||
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
|
||||
private static boolean checkPrintAllValues(JMXGet jmx) throws Exception {
|
||||
int size = 0;
|
||||
byte[] bytes = null;
|
||||
String pattern = "List of all the available keys:";
|
||||
PipedOutputStream pipeOut = new PipedOutputStream();
|
||||
PipedInputStream pipeIn = new PipedInputStream(pipeOut);
|
||||
System.setErr(new PrintStream(pipeOut));
|
||||
jmx.printAllValues();
|
||||
if ((size = pipeIn.available()) != 0) {
|
||||
bytes = new byte[size];
|
||||
pipeIn.read(bytes, 0, bytes.length);
|
||||
}
|
||||
pipeOut.close();
|
||||
pipeIn.close();
|
||||
return bytes != null ? new String(bytes).contains(pattern) : false;
|
||||
}
|
||||
|
||||
/**
|
||||
* test JMX connection to DataNode..
|
||||
* @throws Exception
|
||||
|
|
|
@ -0,0 +1,160 @@
|
|||
package org.apache.hadoop.tools;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.PipedInputStream;
|
||||
import java.io.PipedOutputStream;
|
||||
import java.io.PrintStream;
|
||||
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
||||
import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
|
||||
import org.apache.hadoop.hdfs.tools.JMXGet;
|
||||
import org.apache.hadoop.util.ExitUtil;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.apache.hadoop.util.ExitUtil.ExitException;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.io.ByteStreams;
|
||||
|
||||
public class TestTools {
|
||||
|
||||
private static final int PIPE_BUFFER_SIZE = 1024 * 5;
|
||||
private final static String INVALID_OPTION = "-invalidOption";
|
||||
private static final String[] OPTIONS = new String[2];
|
||||
|
||||
@BeforeClass
|
||||
public static void before() {
|
||||
ExitUtil.disableSystemExit();
|
||||
OPTIONS[1] = INVALID_OPTION;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDelegationTokenFetcherPrintUsage() {
|
||||
String pattern = "Options:";
|
||||
checkOutput(new String[] { "-help" }, pattern, System.out,
|
||||
DelegationTokenFetcher.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDelegationTokenFetcherErrorOption() {
|
||||
String pattern = "ERROR: Only specify cancel, renew or print.";
|
||||
checkOutput(new String[] { "-cancel", "-renew" }, pattern, System.err,
|
||||
DelegationTokenFetcher.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJMXToolHelp() {
|
||||
String pattern = "usage: jmxget options are:";
|
||||
checkOutput(new String[] { "-help" }, pattern, System.out, JMXGet.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testJMXToolAdditionParameter() {
|
||||
String pattern = "key = -addition";
|
||||
checkOutput(new String[] { "-service=NameNode", "-server=localhost",
|
||||
"-addition" }, pattern, System.err, JMXGet.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDFSAdminInvalidUsageHelp() {
|
||||
ImmutableSet<String> args = ImmutableSet.of("-report", "-saveNamespace",
|
||||
"-rollEdits", "-restoreFailedStorage", "-refreshNodes",
|
||||
"-finalizeUpgrade", "-metasave", "-refreshUserToGroupsMappings",
|
||||
"-printTopology", "-refreshNamenodes", "-deleteBlockPool",
|
||||
"-setBalancerBandwidth", "-fetchImage");
|
||||
try {
|
||||
for (String arg : args)
|
||||
assertTrue(ToolRunner.run(new DFSAdmin(), fillArgs(arg)) == -1);
|
||||
|
||||
assertTrue(ToolRunner.run(new DFSAdmin(),
|
||||
new String[] { "-help", "-some" }) == 0);
|
||||
} catch (Exception e) {
|
||||
fail("testDFSAdminHelp error" + e);
|
||||
}
|
||||
|
||||
String pattern = "Usage: java DFSAdmin";
|
||||
checkOutput(new String[] { "-cancel", "-renew" }, pattern, System.err,
|
||||
DFSAdmin.class);
|
||||
}
|
||||
|
||||
private static String[] fillArgs(String arg) {
|
||||
OPTIONS[0] = arg;
|
||||
return OPTIONS;
|
||||
}
|
||||
|
||||
private void checkOutput(String[] args, String pattern, PrintStream out,
|
||||
Class<?> clazz) {
|
||||
ByteArrayOutputStream outBytes = new ByteArrayOutputStream();
|
||||
try {
|
||||
PipedOutputStream pipeOut = new PipedOutputStream();
|
||||
PipedInputStream pipeIn = new PipedInputStream(pipeOut, PIPE_BUFFER_SIZE);
|
||||
if (out == System.out) {
|
||||
System.setOut(new PrintStream(pipeOut));
|
||||
} else if (out == System.err) {
|
||||
System.setErr(new PrintStream(pipeOut));
|
||||
}
|
||||
|
||||
if (clazz == DelegationTokenFetcher.class) {
|
||||
expectDelegationTokenFetcherExit(args);
|
||||
} else if (clazz == JMXGet.class) {
|
||||
expectJMXGetExit(args);
|
||||
} else if (clazz == DFSAdmin.class) {
|
||||
expectDfsAdminPrint(args);
|
||||
}
|
||||
pipeOut.close();
|
||||
ByteStreams.copy(pipeIn, outBytes);
|
||||
pipeIn.close();
|
||||
assertTrue(new String(outBytes.toByteArray()).contains(pattern));
|
||||
} catch (Exception ex) {
|
||||
fail("checkOutput error " + ex);
|
||||
}
|
||||
}
|
||||
|
||||
private void expectDfsAdminPrint(String[] args) {
|
||||
try {
|
||||
ToolRunner.run(new DFSAdmin(), args);
|
||||
} catch (Exception ex) {
|
||||
fail("expectDelegationTokenFetcherExit ex error " + ex);
|
||||
}
|
||||
}
|
||||
|
||||
private static void expectDelegationTokenFetcherExit(String[] args) {
|
||||
try {
|
||||
DelegationTokenFetcher.main(args);
|
||||
fail("should call exit");
|
||||
} catch (ExitException e) {
|
||||
ExitUtil.resetFirstExitException();
|
||||
} catch (Exception ex) {
|
||||
fail("expectDelegationTokenFetcherExit ex error " + ex);
|
||||
}
|
||||
}
|
||||
|
||||
private static void expectJMXGetExit(String[] args) {
|
||||
try {
|
||||
JMXGet.main(args);
|
||||
fail("should call exit");
|
||||
} catch (ExitException e) {
|
||||
ExitUtil.resetFirstExitException();
|
||||
} catch (Exception ex) {
|
||||
fail("expectJMXGetExit ex error " + ex);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -209,6 +209,9 @@ Release 2.2.1 - UNRELEASED
|
|||
MAPREDUCE-5463. Deprecate SLOTS_MILLIS counters (Tzuyoshi Ozawa via Sandy
|
||||
Ryza)
|
||||
|
||||
MAPREDUCE-5457. Add a KeyOnlyTextOutputReader to enable streaming to write
|
||||
out text files without separators (Sandy Ryza)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -222,6 +225,9 @@ Release 2.2.1 - UNRELEASED
|
|||
MAPREDUCE-5518. Fixed typo "can't read paritions file". (Albert Chu
|
||||
via devaraj)
|
||||
|
||||
MAPREDUCE-5561. org.apache.hadoop.mapreduce.v2.app.job.impl.TestJobImpl
|
||||
testcase failing on trunk (Karthik Kambatla via jlowe)
|
||||
|
||||
Release 2.2.0 - 2013-10-13
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -1497,6 +1503,8 @@ Release 0.23.10 - UNRELEASED
|
|||
MAPREDUCE-5586. TestCopyMapper#testCopyFailOnBlockSizeDifference fails when
|
||||
run from hadoop-tools/hadoop-distcp directory (jeagles)
|
||||
|
||||
MAPREDUCE-5587. TestTextOutputFormat fails on JDK7 (jeagles)
|
||||
|
||||
Release 0.23.9 - 2013-07-08
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -415,7 +415,6 @@ public class TestJobImpl {
|
|||
TaskEventType.T_ATTEMPT_FAILED));
|
||||
}
|
||||
}
|
||||
assertJobState(job, JobStateInternal.FAIL_ABORT);
|
||||
|
||||
dispatcher.await();
|
||||
//Verify abortJob is called once and the job failed
|
||||
|
|
|
@ -18,13 +18,24 @@
|
|||
|
||||
package org.apache.hadoop.mapred;
|
||||
|
||||
import java.io.*;
|
||||
import junit.framework.TestCase;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.hadoop.io.*;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestTextOutputFormat extends TestCase {
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.compress.DefaultCodec;
|
||||
import org.apache.hadoop.io.compress.CompressionInputStream;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.util.LineReader;
|
||||
|
||||
public class TestTextOutputFormat {
|
||||
private static JobConf defaultConf = new JobConf();
|
||||
|
||||
private static FileSystem localFs = null;
|
||||
|
@ -38,12 +49,13 @@ public class TestTextOutputFormat extends TestCase {
|
|||
// A random task attempt id for testing.
|
||||
private static String attempt = "attempt_200707121733_0001_m_000000_0";
|
||||
|
||||
private static Path workDir =
|
||||
private static Path workDir =
|
||||
new Path(new Path(
|
||||
new Path(System.getProperty("test.build.data", "."),
|
||||
"data"),
|
||||
new Path(System.getProperty("test.build.data", "."),
|
||||
"data"),
|
||||
FileOutputCommitter.TEMP_DIR_NAME), "_" + attempt);
|
||||
|
||||
@Test
|
||||
public void testFormat() throws Exception {
|
||||
JobConf job = new JobConf();
|
||||
job.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
||||
|
@ -53,7 +65,7 @@ public class TestTextOutputFormat extends TestCase {
|
|||
if (!fs.mkdirs(workDir)) {
|
||||
fail("Failed to create output directory");
|
||||
}
|
||||
String file = "test.txt";
|
||||
String file = "test_format.txt";
|
||||
|
||||
// A reporter that does nothing
|
||||
Reporter reporter = Reporter.NULL;
|
||||
|
@ -90,10 +102,11 @@ public class TestTextOutputFormat extends TestCase {
|
|||
expectedOutput.append(key1).append("\n");
|
||||
expectedOutput.append(key2).append('\t').append(val2).append("\n");
|
||||
String output = UtilsForTests.slurp(expectedFile);
|
||||
assertEquals(output, expectedOutput.toString());
|
||||
assertEquals(expectedOutput.toString(), output);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFormatWithCustomSeparator() throws Exception {
|
||||
JobConf job = new JobConf();
|
||||
String separator = "\u0001";
|
||||
|
@ -105,7 +118,7 @@ public class TestTextOutputFormat extends TestCase {
|
|||
if (!fs.mkdirs(workDir)) {
|
||||
fail("Failed to create output directory");
|
||||
}
|
||||
String file = "test.txt";
|
||||
String file = "test_custom.txt";
|
||||
|
||||
// A reporter that does nothing
|
||||
Reporter reporter = Reporter.NULL;
|
||||
|
@ -142,27 +155,27 @@ public class TestTextOutputFormat extends TestCase {
|
|||
expectedOutput.append(key1).append("\n");
|
||||
expectedOutput.append(key2).append(separator).append(val2).append("\n");
|
||||
String output = UtilsForTests.slurp(expectedFile);
|
||||
assertEquals(output, expectedOutput.toString());
|
||||
assertEquals(expectedOutput.toString(), output);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* test compressed file
|
||||
* @throws IOException
|
||||
*/
|
||||
public void testCompress() throws IOException{
|
||||
@Test
|
||||
public void testCompress() throws IOException {
|
||||
JobConf job = new JobConf();
|
||||
String separator = "\u0001";
|
||||
job.set("mapreduce.output.textoutputformat.separator", separator);
|
||||
job.set(JobContext.TASK_ATTEMPT_ID, attempt);
|
||||
job.set(org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.COMPRESS,"true");
|
||||
|
||||
|
||||
FileOutputFormat.setOutputPath(job, workDir.getParent().getParent());
|
||||
FileOutputFormat.setWorkOutputPath(job, workDir);
|
||||
FileSystem fs = workDir.getFileSystem(job);
|
||||
if (!fs.mkdirs(workDir)) {
|
||||
fail("Failed to create output directory");
|
||||
}
|
||||
String file = "test.txt";
|
||||
String file = "test_compress.txt";
|
||||
|
||||
// A reporter that does nothing
|
||||
Reporter reporter = Reporter.NULL;
|
||||
|
@ -189,16 +202,30 @@ public class TestTextOutputFormat extends TestCase {
|
|||
} finally {
|
||||
theRecordWriter.close(reporter);
|
||||
}
|
||||
File expectedFile = new File(new Path(workDir, file).toString());
|
||||
StringBuffer expectedOutput = new StringBuffer();
|
||||
expectedOutput.append(key1).append(separator).append(val1).append("\n");
|
||||
expectedOutput.append(key1).append("\t").append(val1).append("\n");
|
||||
expectedOutput.append(val1).append("\n");
|
||||
expectedOutput.append(val2).append("\n");
|
||||
expectedOutput.append(key2).append("\n");
|
||||
expectedOutput.append(key1).append("\n");
|
||||
expectedOutput.append(key2).append(separator).append(val2).append("\n");
|
||||
String output = UtilsForTests.slurp(expectedFile);
|
||||
assertEquals(output, expectedOutput.toString());
|
||||
expectedOutput.append(key2).append("\t").append(val2).append("\n");
|
||||
|
||||
DefaultCodec codec = new DefaultCodec();
|
||||
codec.setConf(job);
|
||||
Path expectedFile = new Path(workDir, file + codec.getDefaultExtension());
|
||||
final FileInputStream istream = new FileInputStream(expectedFile.toString());
|
||||
CompressionInputStream cistream = codec.createInputStream(istream);
|
||||
LineReader reader = new LineReader(cistream);
|
||||
|
||||
String output = "";
|
||||
Text out = new Text();
|
||||
while (reader.readLine(out) > 0) {
|
||||
output += out;
|
||||
output += "\n";
|
||||
}
|
||||
reader.close();
|
||||
|
||||
assertEquals(expectedOutput.toString(), output);
|
||||
}
|
||||
public static void main(String[] args) throws Exception {
|
||||
new TestTextOutputFormat().testFormat();
|
||||
|
|
|
@ -80,6 +80,7 @@
|
|||
<item name="HttpFS Gateway" href="hadoop-hdfs-httpfs/index.html"/>
|
||||
<item name="Short Circuit Local Reads"
|
||||
href="hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html"/>
|
||||
<item name="HDFS NFS Gateway" href="hadoop-project-dist/hadoop-hdfs/HdfsNfsGateway.html"/>
|
||||
</menu>
|
||||
|
||||
<menu name="MapReduce" inherit="top">
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.tools;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.FilterInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.net.URI;
|
||||
|
@ -30,9 +31,13 @@ import java.util.StringTokenizer;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.commons.logging.impl.Log4JLogger;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.FsShell;
|
||||
import org.apache.hadoop.fs.HarFileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
|
@ -42,6 +47,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
|
|||
import org.apache.log4j.Level;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import static org.junit.Assert.*;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -62,19 +68,36 @@ public class TestHadoopArchives {
|
|||
private static final String inputDir = "input";
|
||||
|
||||
private Path inputPath;
|
||||
private Path archivePath;
|
||||
private final List<String> fileList = new ArrayList<String>();
|
||||
private MiniDFSCluster dfscluster;
|
||||
|
||||
private Configuration conf;
|
||||
private FileSystem fs;
|
||||
private Path archivePath;
|
||||
|
||||
static private Path createFile(Path dir, String filename, FileSystem fs)
|
||||
throws IOException {
|
||||
final Path f = new Path(dir, filename);
|
||||
private static String createFile(Path root, FileSystem fs, String... dirsAndFile
|
||||
) throws IOException {
|
||||
String fileBaseName = dirsAndFile[dirsAndFile.length - 1];
|
||||
return createFile(root, fs, fileBaseName.getBytes("UTF-8"), dirsAndFile);
|
||||
}
|
||||
|
||||
private static String createFile(Path root, FileSystem fs, byte[] fileContent, String... dirsAndFile
|
||||
) throws IOException {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
for (String segment: dirsAndFile) {
|
||||
if (sb.length() > 0) {
|
||||
sb.append(Path.SEPARATOR);
|
||||
}
|
||||
sb.append(segment);
|
||||
}
|
||||
final Path f = new Path(root, sb.toString());
|
||||
final FSDataOutputStream out = fs.create(f);
|
||||
out.write(filename.getBytes());
|
||||
out.close();
|
||||
return f;
|
||||
try {
|
||||
out.write(fileContent);
|
||||
} finally {
|
||||
out.close();
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
@Before
|
||||
|
@ -86,102 +109,80 @@ public class TestHadoopArchives {
|
|||
conf.set(CapacitySchedulerConfiguration.PREFIX
|
||||
+ CapacitySchedulerConfiguration.ROOT + ".default."
|
||||
+ CapacitySchedulerConfiguration.CAPACITY, "100");
|
||||
dfscluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true)
|
||||
.build();
|
||||
dfscluster = new MiniDFSCluster
|
||||
.Builder(conf)
|
||||
.checkExitOnShutdown(true)
|
||||
.numDataNodes(2)
|
||||
.format(true)
|
||||
.racks(null)
|
||||
.build();
|
||||
|
||||
fs = dfscluster.getFileSystem();
|
||||
inputPath = new Path(fs.getHomeDirectory(), inputDir);
|
||||
|
||||
// prepare archive path:
|
||||
archivePath = new Path(fs.getHomeDirectory(), "archive");
|
||||
fs.delete(archivePath, true);
|
||||
|
||||
// prepare input path:
|
||||
inputPath = new Path(fs.getHomeDirectory(), inputDir);
|
||||
fs.delete(inputPath, true);
|
||||
fs.mkdirs(inputPath);
|
||||
createFile(inputPath, "a", fs);
|
||||
createFile(inputPath, "b", fs);
|
||||
createFile(inputPath, "c", fs);
|
||||
// create basic input files:
|
||||
fileList.add(createFile(inputPath, fs, "a"));
|
||||
fileList.add(createFile(inputPath, fs, "b"));
|
||||
fileList.add(createFile(inputPath, fs, "c"));
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
try {
|
||||
if (dfscluster != null) {
|
||||
dfscluster.shutdown();
|
||||
}
|
||||
if (dfscluster != null) {
|
||||
dfscluster.shutdown();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
System.err.println(e);
|
||||
if (dfscluster != null) {
|
||||
dfscluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRelativePath() throws Exception {
|
||||
fs.delete(archivePath, true);
|
||||
|
||||
final Path sub1 = new Path(inputPath, "dir1");
|
||||
fs.mkdirs(sub1);
|
||||
createFile(sub1, "a", fs);
|
||||
createFile(inputPath, fs, sub1.getName(), "a");
|
||||
final FsShell shell = new FsShell(conf);
|
||||
|
||||
final List<String> originalPaths = lsr(shell, "input");
|
||||
System.out.println("originalPath: " + originalPaths);
|
||||
final URI uri = fs.getUri();
|
||||
final String prefix = "har://hdfs-" + uri.getHost() + ":" + uri.getPort()
|
||||
+ archivePath.toUri().getPath() + Path.SEPARATOR;
|
||||
System.out.println("originalPaths: " + originalPaths);
|
||||
|
||||
{
|
||||
final String harName = "foo.har";
|
||||
final String[] args = { "-archiveName", harName, "-p", "input", "*",
|
||||
"archive" };
|
||||
System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
|
||||
HADOOP_ARCHIVES_JAR);
|
||||
final HadoopArchives har = new HadoopArchives(conf);
|
||||
Assert.assertEquals(0, ToolRunner.run(har, args));
|
||||
// make the archive:
|
||||
final String fullHarPathStr = makeArchive();
|
||||
|
||||
// compare results
|
||||
final List<String> harPaths = lsr(shell, prefix + harName);
|
||||
Assert.assertEquals(originalPaths, harPaths);
|
||||
}
|
||||
// compare results:
|
||||
final List<String> harPaths = lsr(shell, fullHarPathStr);
|
||||
Assert.assertEquals(originalPaths, harPaths);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPathWithSpaces() throws Exception {
|
||||
fs.delete(archivePath, true);
|
||||
|
||||
// create files/directories with spaces
|
||||
createFile(inputPath, "c c", fs);
|
||||
createFile(inputPath, fs, "c c");
|
||||
final Path sub1 = new Path(inputPath, "sub 1");
|
||||
fs.mkdirs(sub1);
|
||||
createFile(sub1, "file x y z", fs);
|
||||
createFile(sub1, "file", fs);
|
||||
createFile(sub1, "x", fs);
|
||||
createFile(sub1, "y", fs);
|
||||
createFile(sub1, "z", fs);
|
||||
createFile(sub1, fs, "file x y z");
|
||||
createFile(sub1, fs, "file");
|
||||
createFile(sub1, fs, "x");
|
||||
createFile(sub1, fs, "y");
|
||||
createFile(sub1, fs, "z");
|
||||
final Path sub2 = new Path(inputPath, "sub 1 with suffix");
|
||||
fs.mkdirs(sub2);
|
||||
createFile(sub2, "z", fs);
|
||||
createFile(sub2, fs, "z");
|
||||
|
||||
final FsShell shell = new FsShell(conf);
|
||||
|
||||
final String inputPathStr = inputPath.toUri().getPath();
|
||||
|
||||
final List<String> originalPaths = lsr(shell, inputPathStr);
|
||||
final URI uri = fs.getUri();
|
||||
final String prefix = "har://hdfs-" + uri.getHost() + ":" + uri.getPort()
|
||||
+ archivePath.toUri().getPath() + Path.SEPARATOR;
|
||||
|
||||
{// Enable space replacement
|
||||
final String harName = "foo.har";
|
||||
final String[] args = { "-archiveName", harName, "-p", inputPathStr, "*",
|
||||
archivePath.toString() };
|
||||
System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
|
||||
HADOOP_ARCHIVES_JAR);
|
||||
final HadoopArchives har = new HadoopArchives(conf);
|
||||
Assert.assertEquals(0, ToolRunner.run(har, args));
|
||||
|
||||
// compare results
|
||||
final List<String> harPaths = lsr(shell, prefix + harName);
|
||||
Assert.assertEquals(originalPaths, harPaths);
|
||||
}
|
||||
// make the archive:
|
||||
final String fullHarPathStr = makeArchive();
|
||||
|
||||
// compare results
|
||||
final List<String> harPaths = lsr(shell, fullHarPathStr);
|
||||
Assert.assertEquals(originalPaths, harPaths);
|
||||
}
|
||||
|
||||
private static List<String> lsr(final FsShell shell, String dir)
|
||||
|
@ -222,4 +223,442 @@ public class TestHadoopArchives {
|
|||
.println("lsr paths = " + paths.toString().replace(", ", ",\n "));
|
||||
return paths;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadFileContent() throws Exception {
|
||||
fileList.add(createFile(inputPath, fs, "c c"));
|
||||
final Path sub1 = new Path(inputPath, "sub 1");
|
||||
fs.mkdirs(sub1);
|
||||
fileList.add(createFile(inputPath, fs, sub1.getName(), "file x y z"));
|
||||
fileList.add(createFile(inputPath, fs, sub1.getName(), "file"));
|
||||
fileList.add(createFile(inputPath, fs, sub1.getName(), "x"));
|
||||
fileList.add(createFile(inputPath, fs, sub1.getName(), "y"));
|
||||
fileList.add(createFile(inputPath, fs, sub1.getName(), "z"));
|
||||
final Path sub2 = new Path(inputPath, "sub 1 with suffix");
|
||||
fs.mkdirs(sub2);
|
||||
fileList.add(createFile(inputPath, fs, sub2.getName(), "z"));
|
||||
// Generate a big binary file content:
|
||||
final byte[] binContent = prepareBin();
|
||||
fileList.add(createFile(inputPath, fs, binContent, sub2.getName(), "bin"));
|
||||
fileList.add(createFile(inputPath, fs, new byte[0], sub2.getName(), "zero-length"));
|
||||
|
||||
final String fullHarPathStr = makeArchive();
|
||||
|
||||
// Create fresh HarFs:
|
||||
final HarFileSystem harFileSystem = new HarFileSystem(fs);
|
||||
try {
|
||||
final URI harUri = new URI(fullHarPathStr);
|
||||
harFileSystem.initialize(harUri, fs.getConf());
|
||||
// now read the file content and compare it against the expected:
|
||||
int readFileCount = 0;
|
||||
for (final String pathStr0 : fileList) {
|
||||
final Path path = new Path(fullHarPathStr + Path.SEPARATOR + pathStr0);
|
||||
final String baseName = path.getName();
|
||||
final FileStatus status = harFileSystem.getFileStatus(path);
|
||||
if (status.isFile()) {
|
||||
// read the file:
|
||||
final byte[] actualContentSimple = readAllSimple(
|
||||
harFileSystem.open(path), true);
|
||||
|
||||
final byte[] actualContentBuffer = readAllWithBuffer(
|
||||
harFileSystem.open(path), true);
|
||||
assertArrayEquals(actualContentSimple, actualContentBuffer);
|
||||
|
||||
final byte[] actualContentFully = readAllWithReadFully(
|
||||
actualContentSimple.length,
|
||||
harFileSystem.open(path), true);
|
||||
assertArrayEquals(actualContentSimple, actualContentFully);
|
||||
|
||||
final byte[] actualContentSeek = readAllWithSeek(
|
||||
actualContentSimple.length,
|
||||
harFileSystem.open(path), true);
|
||||
assertArrayEquals(actualContentSimple, actualContentSeek);
|
||||
|
||||
final byte[] actualContentRead4
|
||||
= readAllWithRead4(harFileSystem.open(path), true);
|
||||
assertArrayEquals(actualContentSimple, actualContentRead4);
|
||||
|
||||
final byte[] actualContentSkip = readAllWithSkip(
|
||||
actualContentSimple.length,
|
||||
harFileSystem.open(path),
|
||||
harFileSystem.open(path),
|
||||
true);
|
||||
assertArrayEquals(actualContentSimple, actualContentSkip);
|
||||
|
||||
if ("bin".equals(baseName)) {
|
||||
assertArrayEquals(binContent, actualContentSimple);
|
||||
} else if ("zero-length".equals(baseName)) {
|
||||
assertEquals(0, actualContentSimple.length);
|
||||
} else {
|
||||
String actual = new String(actualContentSimple, "UTF-8");
|
||||
assertEquals(baseName, actual);
|
||||
}
|
||||
readFileCount++;
|
||||
}
|
||||
}
|
||||
assertEquals(fileList.size(), readFileCount);
|
||||
} finally {
|
||||
harFileSystem.close();
|
||||
}
|
||||
}
|
||||
|
||||
private static byte[] readAllSimple(FSDataInputStream fsdis, boolean close) throws IOException {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
try {
|
||||
int b;
|
||||
while (true) {
|
||||
b = fsdis.read();
|
||||
if (b < 0) {
|
||||
break;
|
||||
} else {
|
||||
baos.write(b);
|
||||
}
|
||||
}
|
||||
baos.close();
|
||||
return baos.toByteArray();
|
||||
} finally {
|
||||
if (close) {
|
||||
fsdis.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static byte[] readAllWithBuffer(FSDataInputStream fsdis, boolean close)
|
||||
throws IOException {
|
||||
try {
|
||||
final int available = fsdis.available();
|
||||
final byte[] buffer;
|
||||
final ByteArrayOutputStream baos;
|
||||
if (available < 0) {
|
||||
buffer = new byte[1024];
|
||||
baos = new ByteArrayOutputStream(buffer.length * 2);
|
||||
} else {
|
||||
buffer = new byte[available];
|
||||
baos = new ByteArrayOutputStream(available);
|
||||
}
|
||||
int readIntoBuffer = 0;
|
||||
int read;
|
||||
while (true) {
|
||||
read = fsdis.read(buffer, readIntoBuffer, buffer.length - readIntoBuffer);
|
||||
if (read < 0) {
|
||||
// end of stream:
|
||||
if (readIntoBuffer > 0) {
|
||||
baos.write(buffer, 0, readIntoBuffer);
|
||||
}
|
||||
return baos.toByteArray();
|
||||
} else {
|
||||
readIntoBuffer += read;
|
||||
if (readIntoBuffer == buffer.length) {
|
||||
// buffer is full, need to clean the buffer.
|
||||
// drop the buffered data to baos:
|
||||
baos.write(buffer);
|
||||
// reset the counter to start reading to the buffer beginning:
|
||||
readIntoBuffer = 0;
|
||||
} else if (readIntoBuffer > buffer.length) {
|
||||
throw new IOException("Read more than the buffer length: "
|
||||
+ readIntoBuffer + ", buffer length = " + buffer.length);
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (close) {
|
||||
fsdis.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static byte[] readAllWithReadFully(int totalLength, FSDataInputStream fsdis, boolean close)
|
||||
throws IOException {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
// Simulate reading of some data structures of known length:
|
||||
final byte[] buffer = new byte[17];
|
||||
final int times = totalLength / buffer.length;
|
||||
final int remainder = totalLength % buffer.length;
|
||||
// it would be simpler to leave the position tracking to the
|
||||
// InputStream, but we need to check the methods #readFully(2)
|
||||
// and #readFully(4) that receive the position as a parameter:
|
||||
int position = 0;
|
||||
try {
|
||||
// read "data structures":
|
||||
for (int i=0; i<times; i++) {
|
||||
fsdis.readFully(position, buffer);
|
||||
position += buffer.length;
|
||||
baos.write(buffer);
|
||||
}
|
||||
if (remainder > 0) {
|
||||
// read the remainder:
|
||||
fsdis.readFully(position, buffer, 0, remainder);
|
||||
position += remainder;
|
||||
baos.write(buffer, 0, remainder);
|
||||
}
|
||||
try {
|
||||
fsdis.readFully(position, buffer, 0, 1);
|
||||
assertTrue(false);
|
||||
} catch (IOException ioe) {
|
||||
// okay
|
||||
}
|
||||
assertEquals(totalLength, position);
|
||||
final byte[] result = baos.toByteArray();
|
||||
assertEquals(totalLength, result.length);
|
||||
return result;
|
||||
} finally {
|
||||
if (close) {
|
||||
fsdis.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static byte[] readAllWithRead4(FSDataInputStream fsdis, boolean close)
|
||||
throws IOException {
|
||||
try {
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
final byte[] buffer = new byte[17];
|
||||
int totalRead = 0;
|
||||
int read;
|
||||
while (true) {
|
||||
read = fsdis.read(totalRead, buffer, 0, buffer.length);
|
||||
if (read > 0) {
|
||||
totalRead += read;
|
||||
baos.write(buffer, 0, read);
|
||||
} else if (read < 0) {
|
||||
break; // EOF
|
||||
} else {
|
||||
// read == 0:
|
||||
// zero result may be returned *only* in case if the 4th
|
||||
// parameter is 0. Since in our case this is 'buffer.length',
|
||||
// zero return value clearly indicates a bug:
|
||||
throw new AssertionError("FSDataInputStream#read(4) returned 0, while " +
|
||||
" the 4th method parameter is " + buffer.length + ".");
|
||||
}
|
||||
}
|
||||
final byte[] result = baos.toByteArray();
|
||||
return result;
|
||||
} finally {
|
||||
if (close) {
|
||||
fsdis.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static byte[] readAllWithSeek(final int totalLength,
|
||||
final FSDataInputStream fsdis, final boolean close)
|
||||
throws IOException {
|
||||
final byte[] result = new byte[totalLength];
|
||||
long pos;
|
||||
try {
|
||||
// read the data in the reverse order, from
|
||||
// the tail to the head by pieces of 'buffer' length:
|
||||
final byte[] buffer = new byte[17];
|
||||
final int times = totalLength / buffer.length;
|
||||
int read;
|
||||
int expectedRead;
|
||||
for (int i=times; i>=0; i--) {
|
||||
pos = i * buffer.length;
|
||||
fsdis.seek(pos);
|
||||
// check that seek is successful:
|
||||
assertEquals(pos, fsdis.getPos());
|
||||
read = fsdis.read(buffer);
|
||||
// check we read right number of bytes:
|
||||
if (i == times) {
|
||||
expectedRead = totalLength % buffer.length; // remainder
|
||||
if (expectedRead == 0) {
|
||||
// zero remainder corresponds to the EOS, so
|
||||
// by the contract of DataInpitStream#read(byte[]) -1 should be
|
||||
// returned:
|
||||
expectedRead = -1;
|
||||
}
|
||||
} else {
|
||||
expectedRead = buffer.length;
|
||||
}
|
||||
assertEquals(expectedRead, read);
|
||||
if (read > 0) {
|
||||
System.arraycopy(buffer, 0, result, (int)pos, read);
|
||||
}
|
||||
}
|
||||
|
||||
// finally, check that #seek() to not existing position leads to IOE:
|
||||
expectSeekIOE(fsdis, Long.MAX_VALUE, "Seek to Long.MAX_VALUE should lead to IOE.");
|
||||
expectSeekIOE(fsdis, Long.MIN_VALUE, "Seek to Long.MIN_VALUE should lead to IOE.");
|
||||
long pp = -1L;
|
||||
expectSeekIOE(fsdis, pp, "Seek to "+pp+" should lead to IOE.");
|
||||
|
||||
// NB: is is *possible* to #seek(length), but *impossible* to #seek(length + 1):
|
||||
fsdis.seek(totalLength);
|
||||
assertEquals(totalLength, fsdis.getPos());
|
||||
pp = totalLength + 1;
|
||||
expectSeekIOE(fsdis, pp, "Seek to the length position + 1 ("+pp+") should lead to IOE.");
|
||||
|
||||
return result;
|
||||
} finally {
|
||||
if (close) {
|
||||
fsdis.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void expectSeekIOE(FSDataInputStream fsdis, long seekPos, String message) {
|
||||
try {
|
||||
fsdis.seek(seekPos);
|
||||
assertTrue(message + " (Position = " + fsdis.getPos() + ")", false);
|
||||
} catch (IOException ioe) {
|
||||
// okay
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Reads data by chunks from 2 input streams:
|
||||
* reads chunk from stream 1, and skips this chunk in the stream 2;
|
||||
* Then reads next chunk from stream 2, and skips this chunk in stream 1.
|
||||
*/
|
||||
private static byte[] readAllWithSkip(
|
||||
final int totalLength,
|
||||
final FSDataInputStream fsdis1,
|
||||
final FSDataInputStream fsdis2,
|
||||
final boolean close)
|
||||
throws IOException {
|
||||
// test negative skip arg:
|
||||
assertEquals(0, fsdis1.skip(-1));
|
||||
// test zero skip arg:
|
||||
assertEquals(0, fsdis1.skip(0));
|
||||
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream(totalLength);
|
||||
try {
|
||||
// read the data in the reverse order, from
|
||||
// the tail to the head by pieces of 'buffer' length:
|
||||
final byte[] buffer = new byte[17];
|
||||
final int times = totalLength / buffer.length;
|
||||
final int remainder = totalLength % buffer.length;
|
||||
long skipped;
|
||||
long expectedPosition;
|
||||
int toGo;
|
||||
for (int i=0; i<=times; i++) {
|
||||
toGo = (i < times) ? buffer.length : remainder;
|
||||
if (i % 2 == 0) {
|
||||
fsdis1.readFully(buffer, 0, toGo);
|
||||
skipped = skipUntilZero(fsdis2, toGo);
|
||||
} else {
|
||||
fsdis2.readFully(buffer, 0, toGo);
|
||||
skipped = skipUntilZero(fsdis1, toGo);
|
||||
}
|
||||
if (i < times) {
|
||||
assertEquals(buffer.length, skipped);
|
||||
expectedPosition = (i + 1) * buffer.length;
|
||||
} else {
|
||||
// remainder:
|
||||
if (remainder > 0) {
|
||||
assertEquals(remainder, skipped);
|
||||
} else {
|
||||
assertEquals(0, skipped);
|
||||
}
|
||||
expectedPosition = totalLength;
|
||||
}
|
||||
// check if the 2 streams have equal and correct positions:
|
||||
assertEquals(expectedPosition, fsdis1.getPos());
|
||||
assertEquals(expectedPosition, fsdis2.getPos());
|
||||
// save the read data:
|
||||
if (toGo > 0) {
|
||||
baos.write(buffer, 0, toGo);
|
||||
}
|
||||
}
|
||||
|
||||
// finally, check up if ended stream cannot skip:
|
||||
assertEquals(0, fsdis1.skip(-1));
|
||||
assertEquals(0, fsdis1.skip(0));
|
||||
assertEquals(0, fsdis1.skip(1));
|
||||
assertEquals(0, fsdis1.skip(Long.MAX_VALUE));
|
||||
|
||||
return baos.toByteArray();
|
||||
} finally {
|
||||
if (close) {
|
||||
fsdis1.close();
|
||||
fsdis2.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static long skipUntilZero(final FilterInputStream fis,
|
||||
final long toSkip) throws IOException {
|
||||
long skipped = 0;
|
||||
long remainsToSkip = toSkip;
|
||||
long s;
|
||||
while (skipped < toSkip) {
|
||||
s = fis.skip(remainsToSkip); // actually skippped
|
||||
if (s == 0) {
|
||||
return skipped; // EOF or impossible to skip.
|
||||
}
|
||||
skipped += s;
|
||||
remainsToSkip -= s;
|
||||
}
|
||||
return skipped;
|
||||
}
|
||||
|
||||
private static byte[] prepareBin() {
|
||||
byte[] bb = new byte[77777];
|
||||
for (int i=0; i<bb.length; i++) {
|
||||
// Generate unique values, as possible:
|
||||
double d = Math.log(i + 2);
|
||||
long bits = Double.doubleToLongBits(d);
|
||||
bb[i] = (byte)bits;
|
||||
}
|
||||
return bb;
|
||||
}
|
||||
|
||||
/*
|
||||
* Run the HadoopArchives tool to create an archive on the
|
||||
* given file system.
|
||||
*/
|
||||
private String makeArchive() throws Exception {
|
||||
final String inputPathStr = inputPath.toUri().getPath();
|
||||
System.out.println("inputPathStr = " + inputPathStr);
|
||||
|
||||
final URI uri = fs.getUri();
|
||||
final String prefix = "har://hdfs-" + uri.getHost() + ":" + uri.getPort()
|
||||
+ archivePath.toUri().getPath() + Path.SEPARATOR;
|
||||
|
||||
final String harName = "foo.har";
|
||||
final String fullHarPathStr = prefix + harName;
|
||||
final String[] args = { "-archiveName", harName, "-p", inputPathStr, "*",
|
||||
archivePath.toString() };
|
||||
System.setProperty(HadoopArchives.TEST_HADOOP_ARCHIVES_JAR_PATH,
|
||||
HADOOP_ARCHIVES_JAR);
|
||||
final HadoopArchives har = new HadoopArchives(conf);
|
||||
assertEquals(0, ToolRunner.run(har, args));
|
||||
return fullHarPathStr;
|
||||
}
|
||||
|
||||
@Test
|
||||
/*
|
||||
* Tests copying from archive file system to a local file system
|
||||
*/
|
||||
public void testCopyToLocal() throws Exception {
|
||||
final String fullHarPathStr = makeArchive();
|
||||
|
||||
// make path to copy the file to:
|
||||
final String tmpDir
|
||||
= System.getProperty("test.build.data","build/test/data") + "/work-dir/har-fs-tmp";
|
||||
final Path tmpPath = new Path(tmpDir);
|
||||
final LocalFileSystem localFs = FileSystem.getLocal(new Configuration());
|
||||
localFs.delete(tmpPath, true);
|
||||
localFs.mkdirs(tmpPath);
|
||||
assertTrue(localFs.exists(tmpPath));
|
||||
|
||||
// Create fresh HarFs:
|
||||
final HarFileSystem harFileSystem = new HarFileSystem(fs);
|
||||
try {
|
||||
final URI harUri = new URI(fullHarPathStr);
|
||||
harFileSystem.initialize(harUri, fs.getConf());
|
||||
|
||||
final Path sourcePath = new Path(fullHarPathStr + Path.SEPARATOR + "a");
|
||||
final Path targetPath = new Path(tmpPath, "straus");
|
||||
// copy the Har file to a local file system:
|
||||
harFileSystem.copyToLocalFile(false, sourcePath, targetPath);
|
||||
FileStatus straus = localFs.getFileStatus(targetPath);
|
||||
// the file should contain just 1 character:
|
||||
assertEquals(1, straus.getLen());
|
||||
} finally {
|
||||
harFileSystem.close();
|
||||
localFs.delete(tmpPath, true);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.streaming.io;
|
||||
|
||||
import org.apache.hadoop.io.BytesWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.typedbytes.TypedBytesWritable;
|
||||
|
||||
|
@ -34,6 +35,7 @@ public class IdentifierResolver {
|
|||
public static final String TEXT_ID = "text";
|
||||
public static final String RAW_BYTES_ID = "rawbytes";
|
||||
public static final String TYPED_BYTES_ID = "typedbytes";
|
||||
public static final String KEY_ONLY_TEXT_ID = "keyonlytext";
|
||||
|
||||
private Class<? extends InputWriter> inputWriterClass = null;
|
||||
private Class<? extends OutputReader> outputReaderClass = null;
|
||||
|
@ -55,6 +57,11 @@ public class IdentifierResolver {
|
|||
setOutputReaderClass(TypedBytesOutputReader.class);
|
||||
setOutputKeyClass(TypedBytesWritable.class);
|
||||
setOutputValueClass(TypedBytesWritable.class);
|
||||
} else if (identifier.equalsIgnoreCase(KEY_ONLY_TEXT_ID)) {
|
||||
setInputWriterClass(KeyOnlyTextInputWriter.class);
|
||||
setOutputReaderClass(KeyOnlyTextOutputReader.class);
|
||||
setOutputKeyClass(Text.class);
|
||||
setOutputValueClass(NullWritable.class);
|
||||
} else { // assume TEXT_ID
|
||||
setInputWriterClass(TextInputWriter.class);
|
||||
setOutputReaderClass(TextOutputReader.class);
|
||||
|
|
|
@ -0,0 +1,35 @@
|
|||
/**
|
||||
* 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.streaming.io;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
public class KeyOnlyTextInputWriter extends TextInputWriter {
|
||||
|
||||
@Override
|
||||
public void writeKey(Object key) throws IOException {
|
||||
writeUTF8(key);
|
||||
clientOut.write('\n');
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeValue(Object value) throws IOException {}
|
||||
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.streaming.io;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.streaming.PipeMapRed;
|
||||
import org.apache.hadoop.util.LineReader;
|
||||
|
||||
/**
|
||||
* OutputReader that reads the client's output as text, interpreting each line
|
||||
* as a key and outputting NullWritables for values.
|
||||
*/
|
||||
public class KeyOnlyTextOutputReader extends OutputReader<Text, NullWritable> {
|
||||
|
||||
private LineReader lineReader;
|
||||
private byte[] bytes;
|
||||
private DataInput clientIn;
|
||||
private Configuration conf;
|
||||
private Text key;
|
||||
private Text line;
|
||||
|
||||
@Override
|
||||
public void initialize(PipeMapRed pipeMapRed) throws IOException {
|
||||
super.initialize(pipeMapRed);
|
||||
clientIn = pipeMapRed.getClientInput();
|
||||
conf = pipeMapRed.getConfiguration();
|
||||
lineReader = new LineReader((InputStream)clientIn, conf);
|
||||
key = new Text();
|
||||
line = new Text();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean readKeyValue() throws IOException {
|
||||
if (lineReader.readLine(line) <= 0) {
|
||||
return false;
|
||||
}
|
||||
bytes = line.getBytes();
|
||||
key.set(bytes, 0, line.getLength());
|
||||
|
||||
line.clear();
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Text getCurrentKey() throws IOException {
|
||||
return key;
|
||||
}
|
||||
|
||||
@Override
|
||||
public NullWritable getCurrentValue() throws IOException {
|
||||
return NullWritable.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLastOutput() {
|
||||
if (bytes != null) {
|
||||
try {
|
||||
return new String(bytes, "UTF-8");
|
||||
} catch (UnsupportedEncodingException e) {
|
||||
return "<undecodable>";
|
||||
}
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.streaming.PipeMapRed;
|
|||
*/
|
||||
public class TextInputWriter extends InputWriter<Object, Object> {
|
||||
|
||||
private DataOutput clientOut;
|
||||
protected DataOutput clientOut;
|
||||
private byte[] inputSeparator;
|
||||
|
||||
@Override
|
||||
|
@ -53,7 +53,7 @@ public class TextInputWriter extends InputWriter<Object, Object> {
|
|||
}
|
||||
|
||||
// Write an object to the output stream using UTF-8 encoding
|
||||
private void writeUTF8(Object object) throws IOException {
|
||||
protected void writeUTF8(Object object) throws IOException {
|
||||
byte[] bval;
|
||||
int valSize;
|
||||
if (object instanceof BytesWritable) {
|
||||
|
|
|
@ -0,0 +1,51 @@
|
|||
/**
|
||||
* 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.streaming;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestStreamingOutputOnlyKeys extends TestStreaming {
|
||||
|
||||
public TestStreamingOutputOnlyKeys() throws IOException {
|
||||
super();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOutputOnlyKeys() throws Exception {
|
||||
args.add("-jobconf"); args.add("stream.reduce.input" +
|
||||
"=keyonlytext");
|
||||
args.add("-jobconf"); args.add("stream.reduce.output" +
|
||||
"=keyonlytext");
|
||||
super.testCommandLine();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getExpectedOutput() {
|
||||
return outputExpect.replaceAll("\t", "");
|
||||
}
|
||||
|
||||
@Override
|
||||
@Test
|
||||
public void testCommandLine() {
|
||||
// Do nothing
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,68 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.streaming.io;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.streaming.PipeMapRed;
|
||||
import org.apache.hadoop.streaming.PipeMapper;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestKeyOnlyTextOutputReader {
|
||||
@Test
|
||||
public void testKeyOnlyTextOutputReader() throws IOException {
|
||||
String text = "key,value\nkey2,value2\nnocomma\n";
|
||||
PipeMapRed pipeMapRed = new MyPipeMapRed(text);
|
||||
KeyOnlyTextOutputReader outputReader = new KeyOnlyTextOutputReader();
|
||||
outputReader.initialize(pipeMapRed);
|
||||
outputReader.readKeyValue();
|
||||
Assert.assertEquals(new Text("key,value"), outputReader.getCurrentKey());
|
||||
outputReader.readKeyValue();
|
||||
Assert.assertEquals(new Text("key2,value2"), outputReader.getCurrentKey());
|
||||
outputReader.readKeyValue();
|
||||
Assert.assertEquals(new Text("nocomma"), outputReader.getCurrentKey());
|
||||
Assert.assertEquals(false, outputReader.readKeyValue());
|
||||
}
|
||||
|
||||
private class MyPipeMapRed extends PipeMapper {
|
||||
private DataInput clientIn;
|
||||
private Configuration conf = new Configuration();
|
||||
|
||||
public MyPipeMapRed(String text) {
|
||||
clientIn = new DataInputStream(new ByteArrayInputStream(text.getBytes()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataInput getClientInput() {
|
||||
return clientIn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConfiguration() {
|
||||
return conf;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -61,11 +61,12 @@ Release 2.3.0 - UNRELEASED
|
|||
|
||||
YARN-976. Document the meaning of a virtual core. (Sandy Ryza)
|
||||
|
||||
YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
|
||||
|
||||
YARN-1182. MiniYARNCluster creates and inits the RM/NM only on start()
|
||||
(Karthik Kambatla via Sandy Ryza)
|
||||
|
||||
HADOOP-9598. Improve code coverage of RMAdminCLI (Aleksey Gorshkov and
|
||||
Andrey Klochkov via jeagles)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -84,6 +85,12 @@ Release 2.3.0 - UNRELEASED
|
|||
YARN-1300. SLS tests fail because conf puts YARN properties in
|
||||
fair-scheduler.xml (Ted Yu via Sandy Ryza)
|
||||
|
||||
YARN-1183. MiniYARNCluster shutdown takes several minutes intermittently
|
||||
(Andrey Klochkov via jeagles)
|
||||
|
||||
YARN-1305. RMHAProtocolService#serviceInit should handle HAUtil's
|
||||
IllegalArgumentException (Tsuyoshi Ozawa via bikas)
|
||||
|
||||
Release 2.2.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -95,6 +102,12 @@ Release 2.2.1 - UNRELEASED
|
|||
YARN-305. Fair scheduler logs too many "Node offered to app" messages.
|
||||
(Lohit Vijayarenu via Sandy Ryza)
|
||||
|
||||
YARN-1258. Allow configuring the Fair Scheduler root queue (Sandy Ryza)
|
||||
|
||||
YARN-1288. Make Fair Scheduler ACLs more user friendly (Sandy Ryza)
|
||||
|
||||
YARN-1315. TestQueueACLs should also test FairScheduler (Sandy Ryza)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -120,6 +133,15 @@ Release 2.2.1 - UNRELEASED
|
|||
YARN-1295. In UnixLocalWrapperScriptBuilder, using bash -c can cause Text
|
||||
file busy errors (Sandy Ryza)
|
||||
|
||||
YARN-1185. Fixed FileSystemRMStateStore to not leave partial files that
|
||||
prevent subsequent ResourceManager recovery. (Omkar Vinit Joshi via vinodkv)
|
||||
|
||||
YARN-1331. yarn.cmd exits with NoClassDefFoundError trying to run rmadmin or
|
||||
logs. (cnauroth)
|
||||
|
||||
YARN-1330. Fair Scheduler: defaultQueueSchedulingPolicy does not take effect
|
||||
(Sandy Ryza)
|
||||
|
||||
Release 2.2.0 - 2013-10-13
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -149,7 +149,7 @@ goto :eof
|
|||
goto :eof
|
||||
|
||||
:rmadmin
|
||||
set CLASS=org.apache.hadoop.yarn.server.resourcemanager.tools.RMAdmin
|
||||
set CLASS=org.apache.hadoop.yarn.client.cli.RMAdminCLI
|
||||
set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
|
||||
goto :eof
|
||||
|
||||
|
@ -200,7 +200,7 @@ goto :eof
|
|||
goto :eof
|
||||
|
||||
:logs
|
||||
set CLASS=org.apache.hadoop.yarn.logaggregation.LogDumper
|
||||
set CLASS=org.apache.hadoop.yarn.client.cli.LogsCLI
|
||||
set YARN_OPTS=%YARN_OPTS% %YARN_CLIENT_OPTS%
|
||||
goto :eof
|
||||
|
||||
|
@ -237,7 +237,6 @@ goto :eof
|
|||
@echo where COMMAND is one of:
|
||||
@echo resourcemanager run the ResourceManager
|
||||
@echo nodemanager run a nodemanager on each slave
|
||||
@echo historyserver run job history servers as a standalone daemon
|
||||
@echo rmadmin admin tools
|
||||
@echo version print the version
|
||||
@echo jar ^<jar^> run a jar file
|
||||
|
|
|
@ -23,6 +23,7 @@ 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.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
@ -42,10 +43,13 @@ public class HAUtil {
|
|||
YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
|
||||
YarnConfiguration.RM_WEBAPP_ADDRESS));
|
||||
|
||||
public static final String BAD_CONFIG_MESSAGE_PREFIX =
|
||||
"Invalid configuration! ";
|
||||
|
||||
private HAUtil() { /* Hidden constructor */ }
|
||||
|
||||
private static void throwBadConfigurationException(String msg) {
|
||||
throw new YarnRuntimeException("Invalid configuration! " + msg);
|
||||
throw new YarnRuntimeException(BAD_CONFIG_MESSAGE_PREFIX + msg);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -59,29 +63,137 @@ public class HAUtil {
|
|||
YarnConfiguration.DEFAULT_RM_HA_ENABLED);
|
||||
}
|
||||
|
||||
public static Collection<String> getRMHAIds(Configuration conf) {
|
||||
return conf.getTrimmedStringCollection(YarnConfiguration.RM_HA_IDS);
|
||||
/**
|
||||
* Verify configuration for Resource Manager HA.
|
||||
* @param conf Configuration
|
||||
* @throws YarnRuntimeException
|
||||
*/
|
||||
public static void verifyAndSetConfiguration(Configuration conf)
|
||||
throws YarnRuntimeException {
|
||||
verifyAndSetRMHAIds(conf);
|
||||
verifyAndSetRMHAId(conf);
|
||||
verifyAndSetAllRpcAddresses(conf);
|
||||
}
|
||||
|
||||
|
||||
private static void verifyAndSetRMHAIds(Configuration conf) {
|
||||
Collection<String> ids =
|
||||
conf.getTrimmedStringCollection(YarnConfiguration.RM_HA_IDS);
|
||||
if (ids.size() <= 0) {
|
||||
throwBadConfigurationException(
|
||||
getInvalidValueMessage(YarnConfiguration.RM_HA_IDS,
|
||||
conf.get(YarnConfiguration.RM_HA_IDS)));
|
||||
} else if (ids.size() == 1) {
|
||||
LOG.warn(getRMHAIdsWarningMessage(ids.toString()));
|
||||
}
|
||||
|
||||
StringBuilder setValue = new StringBuilder();
|
||||
for (String id: ids) {
|
||||
setValue.append(id);
|
||||
setValue.append(",");
|
||||
}
|
||||
conf.set(YarnConfiguration.RM_HA_IDS,
|
||||
setValue.substring(0, setValue.length() - 1));
|
||||
}
|
||||
|
||||
private static void verifyAndSetRMHAId(Configuration conf) {
|
||||
String rmId = conf.getTrimmed(YarnConfiguration.RM_HA_ID);
|
||||
if (rmId == null) {
|
||||
throwBadConfigurationException(
|
||||
getNeedToSetValueMessage(YarnConfiguration.RM_HA_ID));
|
||||
} else {
|
||||
Collection<String> ids = getRMHAIds(conf);
|
||||
if (!ids.contains(rmId)) {
|
||||
throwBadConfigurationException(
|
||||
getRMHAIdNeedToBeIncludedMessage(ids.toString(), rmId));
|
||||
}
|
||||
}
|
||||
conf.set(YarnConfiguration.RM_HA_ID, rmId);
|
||||
}
|
||||
|
||||
private static void verifyAndSetConfValue(String prefix, Configuration conf) {
|
||||
String confKey = null;
|
||||
String confValue = null;
|
||||
try {
|
||||
confKey = getConfKeyForRMInstance(prefix, conf);
|
||||
confValue = getConfValueForRMInstance(prefix, conf);
|
||||
conf.set(prefix, confValue);
|
||||
} catch (YarnRuntimeException yre) {
|
||||
// Error at getRMHAId()
|
||||
throw yre;
|
||||
} catch (IllegalArgumentException iae) {
|
||||
String errmsg;
|
||||
if (confKey == null) {
|
||||
// Error at addSuffix
|
||||
errmsg = getInvalidValueMessage(YarnConfiguration.RM_HA_ID,
|
||||
getRMHAId(conf));
|
||||
} else {
|
||||
// Error at Configuration#set.
|
||||
errmsg = getNeedToSetValueMessage(confKey);
|
||||
}
|
||||
throwBadConfigurationException(errmsg);
|
||||
}
|
||||
}
|
||||
|
||||
public static void verifyAndSetAllRpcAddresses(Configuration conf) {
|
||||
for (String confKey : RPC_ADDRESS_CONF_KEYS) {
|
||||
verifyAndSetConfValue(confKey, conf);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param conf Configuration
|
||||
* @param conf Configuration. Please use getRMHAIds to check.
|
||||
* @return RM Ids on success
|
||||
*/
|
||||
public static Collection<String> getRMHAIds(Configuration conf) {
|
||||
return conf.getStringCollection(YarnConfiguration.RM_HA_IDS);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param conf Configuration. Please use verifyAndSetRMHAId to check.
|
||||
* @return RM Id on success
|
||||
* @throws YarnRuntimeException for configurations without a node id
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public static String getRMHAId(Configuration conf) {
|
||||
String rmId = conf.get(YarnConfiguration.RM_HA_ID);
|
||||
if (rmId == null) {
|
||||
throwBadConfigurationException(YarnConfiguration.RM_HA_ID +
|
||||
" needs to be set in a HA configuration");
|
||||
}
|
||||
return rmId;
|
||||
static String getRMHAId(Configuration conf) {
|
||||
return conf.get(YarnConfiguration.RM_HA_ID);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static String getNeedToSetValueMessage(String confKey) {
|
||||
return confKey + " needs to be set in a HA configuration.";
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static String getInvalidValueMessage(String confKey,
|
||||
String invalidValue){
|
||||
return "Invalid value of " + confKey +". "
|
||||
+ "Current value is " + invalidValue;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static String getRMHAIdNeedToBeIncludedMessage(String ids,
|
||||
String rmId) {
|
||||
return YarnConfiguration.RM_HA_IDS + "("
|
||||
+ ids + ") need to contain " + YarnConfiguration.RM_HA_ID + "("
|
||||
+ rmId + ") in a HA configuration.";
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static String getRMHAIdsWarningMessage(String ids) {
|
||||
return "Resource Manager HA is enabled, but " +
|
||||
YarnConfiguration.RM_HA_IDS + " has only one id(" +
|
||||
ids.toString() + ")";
|
||||
}
|
||||
|
||||
private static String getConfKeyForRMInstance(String prefix,
|
||||
Configuration conf) {
|
||||
return addSuffix(prefix, getRMHAId(conf));
|
||||
}
|
||||
|
||||
private static String getConfValueForRMInstance(String prefix,
|
||||
Configuration conf) {
|
||||
String confKey = addSuffix(prefix, getRMHAId(conf));
|
||||
String retVal = conf.get(confKey);
|
||||
String confKey = getConfKeyForRMInstance(prefix, conf);
|
||||
String retVal = conf.getTrimmed(confKey);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("getConfValueForRMInstance: prefix = " + prefix +
|
||||
"; confKey being looked up = " + confKey +
|
||||
|
@ -96,16 +208,6 @@ public class HAUtil {
|
|||
return (value == null) ? defaultValue : value;
|
||||
}
|
||||
|
||||
private static void setConfValue(String prefix, Configuration conf) {
|
||||
conf.set(prefix, getConfValueForRMInstance(prefix, conf));
|
||||
}
|
||||
|
||||
public static void setAllRpcAddresses(Configuration conf) {
|
||||
for (String confKey : RPC_ADDRESS_CONF_KEYS) {
|
||||
setConfValue(confKey, conf);
|
||||
}
|
||||
}
|
||||
|
||||
/** Add non empty and non null suffix to a key */
|
||||
@VisibleForTesting
|
||||
public static String addSuffix(String key, String suffix) {
|
||||
|
|
|
@ -162,7 +162,7 @@ public class RMAdminCLI extends Configured implements Tool {
|
|||
}
|
||||
}
|
||||
|
||||
private ResourceManagerAdministrationProtocol createAdminProtocol() throws IOException {
|
||||
protected ResourceManagerAdministrationProtocol createAdminProtocol() throws IOException {
|
||||
// Get the current configuration
|
||||
final YarnConfiguration conf = new YarnConfiguration(getConf());
|
||||
return ClientRMProxy.createRMProxy(conf, ResourceManagerAdministrationProtocol.class);
|
||||
|
|
|
@ -0,0 +1,243 @@
|
|||
/**
|
||||
* 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.client;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.argThat;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
||||
import org.apache.hadoop.yarn.client.cli.RMAdminCLI;
|
||||
import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshAdminAclsRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshNodesRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshQueuesRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshServiceAclsRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsConfigurationRequest;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsRequest;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.mockito.ArgumentMatcher;
|
||||
|
||||
public class TestRMAdminCLI {
|
||||
|
||||
private ResourceManagerAdministrationProtocol admin;
|
||||
private RMAdminCLI rmAdminCLI;
|
||||
|
||||
@Before
|
||||
public void configure() {
|
||||
admin = mock(ResourceManagerAdministrationProtocol.class);
|
||||
rmAdminCLI = new RMAdminCLI() {
|
||||
|
||||
@Override
|
||||
protected ResourceManagerAdministrationProtocol createAdminProtocol()
|
||||
throws IOException {
|
||||
return admin;
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testRefreshQueues() throws Exception {
|
||||
String[] args = { "-refreshQueues" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
verify(admin).refreshQueues(any(RefreshQueuesRequest.class));
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testRefreshUserToGroupsMappings() throws Exception {
|
||||
String[] args = { "-refreshUserToGroupsMappings" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
verify(admin).refreshUserToGroupsMappings(
|
||||
any(RefreshUserToGroupsMappingsRequest.class));
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testRefreshSuperUserGroupsConfiguration() throws Exception {
|
||||
String[] args = { "-refreshSuperUserGroupsConfiguration" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
verify(admin).refreshSuperUserGroupsConfiguration(
|
||||
any(RefreshSuperUserGroupsConfigurationRequest.class));
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testRefreshAdminAcls() throws Exception {
|
||||
String[] args = { "-refreshAdminAcls" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
verify(admin).refreshAdminAcls(any(RefreshAdminAclsRequest.class));
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testRefreshServiceAcl() throws Exception {
|
||||
String[] args = { "-refreshServiceAcl" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
verify(admin).refreshServiceAcls(any(RefreshServiceAclsRequest.class));
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testRefreshNodes() throws Exception {
|
||||
String[] args = { "-refreshNodes" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
verify(admin).refreshNodes(any(RefreshNodesRequest.class));
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testGetGroups() throws Exception {
|
||||
when(admin.getGroupsForUser(eq("admin"))).thenReturn(
|
||||
new String[] {"group1", "group2"});
|
||||
PrintStream origOut = System.out;
|
||||
PrintStream out = mock(PrintStream.class);
|
||||
System.setOut(out);
|
||||
try {
|
||||
String[] args = { "-getGroups", "admin" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
verify(admin).getGroupsForUser(eq("admin"));
|
||||
verify(out).println(argThat(new ArgumentMatcher<StringBuilder>() {
|
||||
@Override
|
||||
public boolean matches(Object argument) {
|
||||
return ("" + argument).equals("admin : group1 group2");
|
||||
}
|
||||
}));
|
||||
} finally {
|
||||
System.setOut(origOut);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test printing of help messages
|
||||
*/
|
||||
@Test(timeout=500)
|
||||
public void testHelp() throws Exception {
|
||||
PrintStream oldOutPrintStream = System.out;
|
||||
PrintStream oldErrPrintStream = System.err;
|
||||
ByteArrayOutputStream dataOut = new ByteArrayOutputStream();
|
||||
ByteArrayOutputStream dataErr = new ByteArrayOutputStream();
|
||||
System.setOut(new PrintStream(dataOut));
|
||||
System.setErr(new PrintStream(dataErr));
|
||||
try {
|
||||
String[] args = { "-help" };
|
||||
assertEquals(0, rmAdminCLI.run(args));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"rmadmin is the command to execute Map-Reduce" +
|
||||
" administrative commands."));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"hadoop rmadmin [-refreshQueues] [-refreshNodes] [-refreshSuper" +
|
||||
"UserGroupsConfiguration] [-refreshUserToGroupsMappings] " +
|
||||
"[-refreshAdminAcls] [-refreshServiceAcl] [-getGroup" +
|
||||
" [username]] [-help [cmd]]"));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"-refreshQueues: Reload the queues' acls, states and scheduler " +
|
||||
"specific properties."));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"-refreshNodes: Refresh the hosts information at the " +
|
||||
"ResourceManager."));
|
||||
assertTrue(dataOut.toString().contains(
|
||||
"-refreshUserToGroupsMappings: Refresh user-to-groups mappings"));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"-refreshSuperUserGroupsConfiguration: Refresh superuser proxy" +
|
||||
" groups mappings"));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"-refreshAdminAcls: Refresh acls for administration of " +
|
||||
"ResourceManager"));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"-refreshServiceAcl: Reload the service-level authorization" +
|
||||
" policy file"));
|
||||
assertTrue(dataOut
|
||||
.toString()
|
||||
.contains(
|
||||
"-help [cmd]: \tDisplays help for the given command or all " +
|
||||
"commands if none"));
|
||||
|
||||
testError(new String[] { "-help", "-refreshQueues" },
|
||||
"Usage: java RMAdmin [-refreshQueues]", dataErr, 0);
|
||||
testError(new String[] { "-help", "-refreshNodes" },
|
||||
"Usage: java RMAdmin [-refreshNodes]", dataErr, 0);
|
||||
testError(new String[] { "-help", "-refreshUserToGroupsMappings" },
|
||||
"Usage: java RMAdmin [-refreshUserToGroupsMappings]", dataErr, 0);
|
||||
testError(
|
||||
new String[] { "-help", "-refreshSuperUserGroupsConfiguration" },
|
||||
"Usage: java RMAdmin [-refreshSuperUserGroupsConfiguration]",
|
||||
dataErr, 0);
|
||||
testError(new String[] { "-help", "-refreshAdminAcls" },
|
||||
"Usage: java RMAdmin [-refreshAdminAcls]", dataErr, 0);
|
||||
testError(new String[] { "-help", "-refreshService" },
|
||||
"Usage: java RMAdmin [-refreshServiceAcl]", dataErr, 0);
|
||||
testError(new String[] { "-help", "-getGroups" },
|
||||
"Usage: java RMAdmin [-getGroups [username]]", dataErr, 0);
|
||||
|
||||
|
||||
testError(new String[] { "-help", "-badParameter" },
|
||||
"Usage: java RMAdmin", dataErr, 0);
|
||||
testError(new String[] { "-badParameter" },
|
||||
"badParameter: Unknown command", dataErr, -1);
|
||||
} finally {
|
||||
System.setOut(oldOutPrintStream);
|
||||
System.setErr(oldErrPrintStream);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=500)
|
||||
public void testException() throws Exception {
|
||||
PrintStream oldErrPrintStream = System.err;
|
||||
ByteArrayOutputStream dataErr = new ByteArrayOutputStream();
|
||||
System.setErr(new PrintStream(dataErr));
|
||||
try {
|
||||
when(admin.refreshQueues(any(RefreshQueuesRequest.class)))
|
||||
.thenThrow(new IOException("test exception"));
|
||||
String[] args = { "-refreshQueues" };
|
||||
|
||||
assertEquals(-1, rmAdminCLI.run(args));
|
||||
verify(admin).refreshQueues(any(RefreshQueuesRequest.class));
|
||||
assertTrue(dataErr.toString().contains("refreshQueues: test exception"));
|
||||
} finally {
|
||||
System.setErr(oldErrPrintStream);
|
||||
}
|
||||
}
|
||||
|
||||
private void testError(String[] args, String template,
|
||||
ByteArrayOutputStream data, int resultCode) throws Exception {
|
||||
assertEquals(resultCode, rmAdminCLI.run(args));
|
||||
assertTrue(data.toString().contains(template));
|
||||
data.reset();
|
||||
}
|
||||
|
||||
}
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.conf;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -27,53 +28,134 @@ import org.junit.Test;
|
|||
import java.util.Collection;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class TestHAUtil {
|
||||
private Configuration conf;
|
||||
|
||||
private static final String RM1_ADDRESS = "1.2.3.4:8021";
|
||||
private static final String RM1_ADDRESS_UNTRIMMED = " \t\t\n 1.2.3.4:8021 \n\t ";
|
||||
private static final String RM1_ADDRESS = RM1_ADDRESS_UNTRIMMED.trim();
|
||||
private static final String RM2_ADDRESS = "localhost:8022";
|
||||
private static final String RM1_NODE_ID = "rm1";
|
||||
private static final String RM1_NODE_ID_UNTRIMMED = "rm1 ";
|
||||
private static final String RM1_NODE_ID = RM1_NODE_ID_UNTRIMMED.trim();
|
||||
private static final String RM2_NODE_ID = "rm2";
|
||||
private static final String RM3_NODE_ID = "rm3";
|
||||
private static final String RM_INVALID_NODE_ID = ".rm";
|
||||
private static final String RM_NODE_IDS_UNTRIMMED = RM1_NODE_ID_UNTRIMMED + "," + RM2_NODE_ID;
|
||||
private static final String RM_NODE_IDS = RM1_NODE_ID + "," + RM2_NODE_ID;
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
conf = new Configuration();
|
||||
conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
|
||||
conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
|
||||
conf.set(YarnConfiguration.RM_HA_IDS, RM_NODE_IDS_UNTRIMMED);
|
||||
conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID_UNTRIMMED);
|
||||
|
||||
for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
|
||||
conf.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS);
|
||||
// configuration key itself cannot contains space/tab/return chars.
|
||||
conf.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS_UNTRIMMED);
|
||||
conf.set(HAUtil.addSuffix(confKey, RM2_NODE_ID), RM2_ADDRESS);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRMServiceId() throws Exception {
|
||||
conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID + "," + RM2_NODE_ID);
|
||||
Collection<String> rmhaIds = HAUtil.getRMHAIds(conf);
|
||||
assertEquals(2, rmhaIds.size());
|
||||
|
||||
String[] ids = rmhaIds.toArray(new String[0]);
|
||||
assertEquals(RM1_NODE_ID, ids[0]);
|
||||
assertEquals(RM2_NODE_ID, ids[1]);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetRMId() throws Exception {
|
||||
conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
|
||||
assertEquals("Does not honor " + YarnConfiguration.RM_HA_ID,
|
||||
RM1_NODE_ID, HAUtil.getRMHAId(conf));
|
||||
conf = new YarnConfiguration();
|
||||
try {
|
||||
HAUtil.getRMHAId(conf);
|
||||
fail("getRMHAId() fails to throw an exception when RM_HA_ID is not set");
|
||||
} catch (YarnRuntimeException yre) {
|
||||
// do nothing
|
||||
}
|
||||
RM1_NODE_ID, HAUtil.getRMHAId(conf));
|
||||
|
||||
conf.clear();
|
||||
assertNull("Return null when " + YarnConfiguration.RM_HA_ID
|
||||
+ " is not set", HAUtil.getRMHAId(conf));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSetGetRpcAddresses() throws Exception {
|
||||
HAUtil.setAllRpcAddresses(conf);
|
||||
public void testVerifyAndSetConfiguration() throws Exception {
|
||||
try {
|
||||
HAUtil.verifyAndSetConfiguration(conf);
|
||||
} catch (YarnRuntimeException e) {
|
||||
fail("Should not throw any exceptions.");
|
||||
}
|
||||
|
||||
assertEquals("Should be saved as Trimmed collection",
|
||||
StringUtils.getStringCollection(RM_NODE_IDS), HAUtil.getRMHAIds(conf));
|
||||
assertEquals("Should be saved as Trimmed string",
|
||||
RM1_NODE_ID, HAUtil.getRMHAId(conf));
|
||||
for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
|
||||
assertEquals("RPC address not set for " + confKey,
|
||||
RM1_ADDRESS, conf.get(confKey));
|
||||
RM1_ADDRESS, conf.get(confKey));
|
||||
}
|
||||
|
||||
conf.clear();
|
||||
conf.set(YarnConfiguration.RM_HA_IDS, RM_INVALID_NODE_ID);
|
||||
try {
|
||||
HAUtil.verifyAndSetConfiguration(conf);
|
||||
} catch (YarnRuntimeException e) {
|
||||
assertEquals("YarnRuntimeException by getRMId()",
|
||||
HAUtil.BAD_CONFIG_MESSAGE_PREFIX +
|
||||
HAUtil.getNeedToSetValueMessage(YarnConfiguration.RM_HA_ID),
|
||||
e.getMessage());
|
||||
}
|
||||
|
||||
conf.clear();
|
||||
conf.set(YarnConfiguration.RM_HA_ID, RM_INVALID_NODE_ID);
|
||||
conf.set(YarnConfiguration.RM_HA_IDS, RM_INVALID_NODE_ID);
|
||||
for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
|
||||
// simulate xml with invalid node id
|
||||
conf.set(confKey + RM_INVALID_NODE_ID, RM_INVALID_NODE_ID);
|
||||
}
|
||||
try {
|
||||
HAUtil.verifyAndSetConfiguration(conf);
|
||||
} catch (YarnRuntimeException e) {
|
||||
assertEquals("YarnRuntimeException by addSuffix()",
|
||||
HAUtil.BAD_CONFIG_MESSAGE_PREFIX +
|
||||
HAUtil.getInvalidValueMessage(YarnConfiguration.RM_HA_ID,
|
||||
RM_INVALID_NODE_ID),
|
||||
e.getMessage());
|
||||
}
|
||||
|
||||
conf.clear();
|
||||
// simulate the case HAUtil.RPC_ADDRESS_CONF_KEYS are not set
|
||||
conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID);
|
||||
conf.set(YarnConfiguration.RM_HA_IDS, RM1_NODE_ID);
|
||||
try {
|
||||
HAUtil.verifyAndSetConfiguration(conf);
|
||||
fail("Should throw YarnRuntimeException. by Configuration#set()");
|
||||
} catch (YarnRuntimeException e) {
|
||||
String confKey =
|
||||
HAUtil.addSuffix(YarnConfiguration.RM_ADDRESS, RM1_NODE_ID);
|
||||
assertEquals("YarnRuntimeException by Configuration#set()",
|
||||
HAUtil.BAD_CONFIG_MESSAGE_PREFIX + HAUtil.getNeedToSetValueMessage(confKey),
|
||||
e.getMessage());
|
||||
}
|
||||
|
||||
// simulate the case YarnConfiguration.RM_HA_IDS doesn't contain
|
||||
// the value of YarnConfiguration.RM_HA_ID
|
||||
conf.clear();
|
||||
conf.set(YarnConfiguration.RM_HA_IDS, RM2_NODE_ID + "," + RM3_NODE_ID);
|
||||
conf.set(YarnConfiguration.RM_HA_ID, RM1_NODE_ID_UNTRIMMED);
|
||||
for (String confKey : HAUtil.RPC_ADDRESS_CONF_KEYS) {
|
||||
conf.set(HAUtil.addSuffix(confKey, RM1_NODE_ID), RM1_ADDRESS_UNTRIMMED);
|
||||
conf.set(HAUtil.addSuffix(confKey, RM2_NODE_ID), RM2_ADDRESS);
|
||||
}
|
||||
try {
|
||||
HAUtil.verifyAndSetConfiguration(conf);
|
||||
} catch (YarnRuntimeException e) {
|
||||
assertEquals("YarnRuntimeException by getRMId()'s validation",
|
||||
HAUtil.BAD_CONFIG_MESSAGE_PREFIX +
|
||||
HAUtil.getRMHAIdNeedToBeIncludedMessage("[rm2, rm3]", RM1_NODE_ID),
|
||||
e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,7 +57,7 @@ public class RMHAProtocolService extends AbstractService implements
|
|||
this.conf = conf;
|
||||
haEnabled = HAUtil.isHAEnabled(this.conf);
|
||||
if (haEnabled) {
|
||||
HAUtil.setAllRpcAddresses(this.conf);
|
||||
HAUtil.verifyAndSetConfiguration(conf);
|
||||
rm.setConf(this.conf);
|
||||
}
|
||||
rm.createAndInitActiveServices();
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream;
|
|||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -118,6 +119,9 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|||
for (FileStatus childNodeStatus : fs.listStatus(appDir.getPath())) {
|
||||
assert childNodeStatus.isFile();
|
||||
String childNodeName = childNodeStatus.getPath().getName();
|
||||
if (checkAndRemovePartialRecord(childNodeStatus.getPath())) {
|
||||
continue;
|
||||
}
|
||||
byte[] childData =
|
||||
readFile(childNodeStatus.getPath(), childNodeStatus.getLen());
|
||||
if (childNodeName.startsWith(ApplicationId.appIdStrPrefix)) {
|
||||
|
@ -178,12 +182,28 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|||
}
|
||||
}
|
||||
|
||||
private boolean checkAndRemovePartialRecord(Path record) throws IOException {
|
||||
// If the file ends with .tmp then it shows that it failed
|
||||
// during saving state into state store. The file will be deleted as a
|
||||
// part of this call
|
||||
if (record.getName().endsWith(".tmp")) {
|
||||
LOG.error("incomplete rm state store entry found :"
|
||||
+ record);
|
||||
fs.delete(record, false);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void loadRMDTSecretManagerState(RMState rmState) throws Exception {
|
||||
FileStatus[] childNodes = fs.listStatus(rmDTSecretManagerRoot);
|
||||
|
||||
for(FileStatus childNodeStatus : childNodes) {
|
||||
assert childNodeStatus.isFile();
|
||||
String childNodeName = childNodeStatus.getPath().getName();
|
||||
if (checkAndRemovePartialRecord(childNodeStatus.getPath())) {
|
||||
continue;
|
||||
}
|
||||
if(childNodeName.startsWith(DELEGATION_TOKEN_SEQUENCE_NUMBER_PREFIX)) {
|
||||
rmState.rmSecretManagerState.dtSequenceNumber =
|
||||
Integer.parseInt(childNodeName.split("_")[1]);
|
||||
|
@ -344,10 +364,19 @@ public class FileSystemRMStateStore extends RMStateStore {
|
|||
return data;
|
||||
}
|
||||
|
||||
/*
|
||||
* In order to make this write atomic as a part of write we will first write
|
||||
* data to .tmp file and then rename it. Here we are assuming that rename is
|
||||
* atomic for underlying file system.
|
||||
*/
|
||||
private void writeFile(Path outputPath, byte[] data) throws Exception {
|
||||
FSDataOutputStream fsOut = fs.create(outputPath, false);
|
||||
Path tempPath =
|
||||
new Path(outputPath.getParent(), outputPath.getName() + ".tmp");
|
||||
FSDataOutputStream fsOut = null;
|
||||
fsOut = fs.create(tempPath, false);
|
||||
fsOut.write(data);
|
||||
fsOut.close();
|
||||
fs.rename(tempPath, outputPath);
|
||||
}
|
||||
|
||||
private boolean renameFile(Path src, Path dst) throws Exception {
|
||||
|
|
|
@ -19,12 +19,10 @@
|
|||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Evolving;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.apache.hadoop.yarn.api.records.QueueACL;
|
||||
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
||||
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
||||
|
@ -44,12 +42,6 @@ public interface Queue {
|
|||
*/
|
||||
QueueMetrics getMetrics();
|
||||
|
||||
/**
|
||||
* Get ACLs for the queue.
|
||||
* @return ACLs for the queue
|
||||
*/
|
||||
public Map<QueueACL, AccessControlList> getQueueAcls();
|
||||
|
||||
/**
|
||||
* Get queue information
|
||||
* @param includeChildQueues include child queues?
|
||||
|
|
|
@ -526,11 +526,6 @@ public class LeafQueue implements CSQueue {
|
|||
return userLimitFactor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
|
||||
return new HashMap<QueueACL, AccessControlList>(acls);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized QueueInfo getQueueInfo(
|
||||
boolean includeChildQueues, boolean recursive) {
|
||||
|
|
|
@ -299,11 +299,6 @@ public class ParentQueue implements CSQueue {
|
|||
return state;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Map<QueueACL, AccessControlList> getQueueAcls() {
|
||||
return new HashMap<QueueACL, AccessControlList>(acls);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized QueueInfo getQueueInfo(
|
||||
boolean includeChildQueues, boolean recursive) {
|
||||
|
|
|
@ -24,14 +24,12 @@ import java.util.Collections;
|
|||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.apache.hadoop.yarn.api.records.QueueACL;
|
||||
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
|
@ -177,8 +175,7 @@ public class FSLeafQueue extends FSQueue {
|
|||
recordFactory.newRecordInstance(QueueUserACLInfo.class);
|
||||
List<QueueACL> operations = new ArrayList<QueueACL>();
|
||||
for (QueueACL operation : QueueACL.values()) {
|
||||
Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
|
||||
if (acls.get(operation).isUserAllowed(user)) {
|
||||
if (hasAccess(operation, user)) {
|
||||
operations.add(operation);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,13 +20,10 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.api.records.QueueACL;
|
||||
import org.apache.hadoop.yarn.api.records.QueueInfo;
|
||||
|
@ -135,12 +132,6 @@ public abstract class FSQueue extends Schedulable implements Queue {
|
|||
return queueInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<QueueACL, AccessControlList> getQueueAcls() {
|
||||
Map<QueueACL, AccessControlList> acls = queueMgr.getQueueAcls(getName());
|
||||
return new HashMap<QueueACL, AccessControlList>(acls);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSQueueMetrics getMetrics() {
|
||||
return metrics;
|
||||
|
@ -154,7 +145,7 @@ public abstract class FSQueue extends Schedulable implements Queue {
|
|||
|
||||
public boolean hasAccess(QueueACL acl, UserGroupInformation user) {
|
||||
// Check if the leaf-queue allows access
|
||||
if (queueMgr.getQueueAcls(getName()).get(acl).isUserAllowed(user)) {
|
||||
if (queueMgr.getQueueAcl(getName(), acl).isUserAllowed(user)) {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -50,7 +50,7 @@ public class FairSchedulerConfiguration extends Configuration {
|
|||
|
||||
private static final String CONF_PREFIX = "yarn.scheduler.fair.";
|
||||
|
||||
protected static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
|
||||
public static final String ALLOCATION_FILE = CONF_PREFIX + "allocation.file";
|
||||
protected static final String DEFAULT_ALLOCATION_FILE = "fair-scheduler.xml";
|
||||
protected static final String EVENT_LOG_DIR = "eventlog.dir";
|
||||
|
||||
|
@ -113,6 +113,10 @@ public class FairSchedulerConfiguration extends Configuration {
|
|||
protected static final String MAX_ASSIGN = CONF_PREFIX + "max.assign";
|
||||
protected static final int DEFAULT_MAX_ASSIGN = -1;
|
||||
|
||||
public FairSchedulerConfiguration() {
|
||||
super();
|
||||
}
|
||||
|
||||
public FairSchedulerConfiguration(Configuration conf) {
|
||||
super(conf);
|
||||
}
|
||||
|
|
|
@ -72,6 +72,9 @@ public class QueueManager {
|
|||
* (this is done to prevent loading a file that hasn't been fully written).
|
||||
*/
|
||||
public static final long ALLOC_RELOAD_WAIT = 5 * 1000;
|
||||
|
||||
private static final AccessControlList EVERYBODY_ACL = new AccessControlList("*");
|
||||
private static final AccessControlList NOBODY_ACL = new AccessControlList(" ");
|
||||
|
||||
private final FairScheduler scheduler;
|
||||
|
||||
|
@ -375,31 +378,24 @@ public class QueueManager {
|
|||
queueMaxAppsDefault, defaultSchedPolicy, minSharePreemptionTimeouts,
|
||||
queueAcls, fairSharePreemptionTimeout, defaultMinSharePreemptionTimeout);
|
||||
|
||||
// Update metrics
|
||||
for (FSQueue queue : queues.values()) {
|
||||
FSQueueMetrics queueMetrics = queue.getMetrics();
|
||||
queueMetrics.setMinShare(queue.getMinShare());
|
||||
queueMetrics.setMaxShare(queue.getMaxShare());
|
||||
}
|
||||
|
||||
// Root queue should have empty ACLs. As a queue's ACL is the union of
|
||||
// its ACL and all its parents' ACLs, setting the roots' to empty will
|
||||
// neither allow nor prohibit more access to its children.
|
||||
Map<QueueACL, AccessControlList> rootAcls =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
rootAcls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(" "));
|
||||
rootAcls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(" "));
|
||||
queueAcls.put(ROOT_QUEUE, rootAcls);
|
||||
|
||||
// Create all queus
|
||||
// Make sure all queues exist
|
||||
for (String name: queueNamesInAllocFile) {
|
||||
getLeafQueue(name, true);
|
||||
}
|
||||
|
||||
// Set custom policies as specified
|
||||
for (Map.Entry<String, SchedulingPolicy> entry : queuePolicies.entrySet()) {
|
||||
queues.get(entry.getKey()).setPolicy(entry.getValue());
|
||||
for (FSQueue queue : queues.values()) {
|
||||
// Update queue metrics
|
||||
FSQueueMetrics queueMetrics = queue.getMetrics();
|
||||
queueMetrics.setMinShare(queue.getMinShare());
|
||||
queueMetrics.setMaxShare(queue.getMaxShare());
|
||||
// Set scheduling policies
|
||||
if (queuePolicies.containsKey(queue.getName())) {
|
||||
queue.setPolicy(queuePolicies.get(queue.getName()));
|
||||
} else {
|
||||
queue.setPolicy(SchedulingPolicy.getDefault());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -454,10 +450,10 @@ public class QueueManager {
|
|||
policy.initialize(scheduler.getClusterCapacity());
|
||||
queuePolicies.put(queueName, policy);
|
||||
} else if ("aclSubmitApps".equals(field.getTagName())) {
|
||||
String text = ((Text)field.getFirstChild()).getData().trim();
|
||||
String text = ((Text)field.getFirstChild()).getData();
|
||||
acls.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList(text));
|
||||
} else if ("aclAdministerApps".equals(field.getTagName())) {
|
||||
String text = ((Text)field.getFirstChild()).getData().trim();
|
||||
String text = ((Text)field.getFirstChild()).getData();
|
||||
acls.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList(text));
|
||||
} else if ("queue".endsWith(field.getTagName()) ||
|
||||
"pool".equals(field.getTagName())) {
|
||||
|
@ -577,21 +573,16 @@ public class QueueManager {
|
|||
|
||||
/**
|
||||
* Get the ACLs associated with this queue. If a given ACL is not explicitly
|
||||
* configured, include the default value for that ACL.
|
||||
* configured, include the default value for that ACL. The default for the
|
||||
* root queue is everybody ("*") and the default for all other queues is
|
||||
* nobody ("")
|
||||
*/
|
||||
public Map<QueueACL, AccessControlList> getQueueAcls(String queue) {
|
||||
HashMap<QueueACL, AccessControlList> out = new HashMap<QueueACL, AccessControlList>();
|
||||
Map<QueueACL, AccessControlList> queueAcl = info.queueAcls.get(queue);
|
||||
if (queueAcl != null) {
|
||||
out.putAll(queueAcl);
|
||||
public AccessControlList getQueueAcl(String queue, QueueACL operation) {
|
||||
Map<QueueACL, AccessControlList> queueAcls = info.queueAcls.get(queue);
|
||||
if (queueAcls == null || !queueAcls.containsKey(operation)) {
|
||||
return (queue.equals(ROOT_QUEUE)) ? EVERYBODY_ACL : NOBODY_ACL;
|
||||
}
|
||||
if (!out.containsKey(QueueACL.ADMINISTER_QUEUE)) {
|
||||
out.put(QueueACL.ADMINISTER_QUEUE, new AccessControlList("*"));
|
||||
}
|
||||
if (!out.containsKey(QueueACL.SUBMIT_APPLICATIONS)) {
|
||||
out.put(QueueACL.SUBMIT_APPLICATIONS, new AccessControlList("*"));
|
||||
}
|
||||
return out;
|
||||
return queueAcls.get(operation);
|
||||
}
|
||||
|
||||
static class QueueManagerInfo {
|
||||
|
|
|
@ -156,7 +156,6 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
|
|||
return queueInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<QueueACL, AccessControlList> getQueueAcls() {
|
||||
Map<QueueACL, AccessControlList> acls =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
|
|
|
@ -42,41 +42,43 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
|
|||
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.QueueACL;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.ipc.YarnRPC;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestQueueACLs {
|
||||
public abstract class QueueACLsTestBase {
|
||||
|
||||
private static final String COMMON_USER = "common_user";
|
||||
private static final String QUEUE_A_USER = "queueA_user";
|
||||
private static final String QUEUE_B_USER = "queueB_user";
|
||||
private static final String ROOT_ADMIN = "root_admin";
|
||||
private static final String QUEUE_A_ADMIN = "queueA_admin";
|
||||
private static final String QUEUE_B_ADMIN = "queueB_admin";
|
||||
protected static final String COMMON_USER = "common_user";
|
||||
protected static final String QUEUE_A_USER = "queueA_user";
|
||||
protected static final String QUEUE_B_USER = "queueB_user";
|
||||
protected static final String ROOT_ADMIN = "root_admin";
|
||||
protected static final String QUEUE_A_ADMIN = "queueA_admin";
|
||||
protected static final String QUEUE_B_ADMIN = "queueB_admin";
|
||||
|
||||
private static final String QUEUEA = "queueA";
|
||||
private static final String QUEUEB = "queueB";
|
||||
protected static final String QUEUEA = "queueA";
|
||||
protected static final String QUEUEB = "queueB";
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestApplicationACLs.class);
|
||||
|
||||
static MockRM resourceManager;
|
||||
static Configuration conf = createConfiguration();
|
||||
final static YarnRPC rpc = YarnRPC.create(conf);
|
||||
final static InetSocketAddress rmAddress = conf.getSocketAddr(
|
||||
YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_RM_PORT);
|
||||
MockRM resourceManager;
|
||||
Configuration conf;
|
||||
YarnRPC rpc;
|
||||
InetSocketAddress rmAddress;
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() throws InterruptedException, IOException {
|
||||
@Before
|
||||
public void setup() throws InterruptedException, IOException {
|
||||
conf = createConfiguration();
|
||||
rpc = YarnRPC.create(conf);
|
||||
rmAddress = conf.getSocketAddr(
|
||||
YarnConfiguration.RM_ADDRESS, YarnConfiguration.DEFAULT_RM_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_RM_PORT);
|
||||
|
||||
AccessControlList adminACL = new AccessControlList("");
|
||||
conf.set(YarnConfiguration.YARN_ADMIN_ACL, adminACL.getAclString());
|
||||
|
||||
|
@ -109,8 +111,8 @@ public class TestQueueACLs {
|
|||
}
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
@After
|
||||
public void tearDown() {
|
||||
if (resourceManager != null) {
|
||||
resourceManager.stop();
|
||||
}
|
||||
|
@ -262,45 +264,5 @@ public class TestQueueACLs {
|
|||
return userClient;
|
||||
}
|
||||
|
||||
private static YarnConfiguration createConfiguration() {
|
||||
CapacitySchedulerConfiguration csConf =
|
||||
new CapacitySchedulerConfiguration();
|
||||
csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
|
||||
QUEUEA, QUEUEB });
|
||||
|
||||
csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
|
||||
csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
|
||||
|
||||
Map<QueueACL, AccessControlList> aclsOnQueueA =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
|
||||
submitACLonQueueA.addUser(COMMON_USER);
|
||||
AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
|
||||
aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
|
||||
aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
|
||||
csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
|
||||
aclsOnQueueA);
|
||||
|
||||
Map<QueueACL, AccessControlList> aclsOnQueueB =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
|
||||
submitACLonQueueB.addUser(COMMON_USER);
|
||||
AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
|
||||
aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
|
||||
aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
|
||||
csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
|
||||
aclsOnQueueB);
|
||||
|
||||
Map<QueueACL, AccessControlList> aclsOnRootQueue =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
AccessControlList submitACLonRoot = new AccessControlList("");
|
||||
AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
|
||||
aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
|
||||
aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
|
||||
csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
|
||||
|
||||
YarnConfiguration conf = new YarnConfiguration(csConf);
|
||||
conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
|
||||
return conf;
|
||||
}
|
||||
protected abstract Configuration createConfiguration() throws IOException;
|
||||
}
|
|
@ -39,6 +39,7 @@ import junit.framework.Assert;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -75,9 +76,9 @@ import org.apache.zookeeper.ZooKeeper;
|
|||
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRMStateStore extends ClientBaseWithFixes{
|
||||
public class RMStateStoreTestBase extends ClientBaseWithFixes{
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(TestRMStateStore.class);
|
||||
public static final Log LOG = LogFactory.getLog(RMStateStoreTestBase.class);
|
||||
|
||||
static class TestDispatcher implements
|
||||
Dispatcher, EventHandler<RMAppAttemptStoredEvent> {
|
||||
|
@ -116,104 +117,6 @@ public class TestRMStateStore extends ClientBaseWithFixes{
|
|||
boolean isFinalStateValid() throws Exception;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZKRMStateStoreRealZK() throws Exception {
|
||||
TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
|
||||
testRMAppStateStore(zkTester);
|
||||
testRMDTSecretManagerStateStore(zkTester);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFSRMStateStore() throws Exception {
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
try {
|
||||
TestFSRMStateStoreTester fsTester = new TestFSRMStateStoreTester(cluster);
|
||||
testRMAppStateStore(fsTester);
|
||||
testRMDTSecretManagerStateStore(fsTester);
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
class TestZKRMStateStoreTester implements RMStateStoreHelper {
|
||||
ZooKeeper client;
|
||||
ZKRMStateStore store;
|
||||
|
||||
class TestZKRMStateStore extends ZKRMStateStore {
|
||||
public TestZKRMStateStore(Configuration conf, String workingZnode)
|
||||
throws Exception {
|
||||
init(conf);
|
||||
start();
|
||||
assertTrue(znodeWorkingPath.equals(workingZnode));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeper getNewZooKeeper() throws IOException {
|
||||
return client;
|
||||
}
|
||||
}
|
||||
|
||||
public RMStateStore getRMStateStore() throws Exception {
|
||||
String workingZnode = "/Test";
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
|
||||
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
|
||||
this.client = createClient();
|
||||
this.store = new TestZKRMStateStore(conf, workingZnode);
|
||||
return this.store;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFinalStateValid() throws Exception {
|
||||
List<String> nodes = client.getChildren(store.znodeWorkingPath, false);
|
||||
return nodes.size() == 1;
|
||||
}
|
||||
}
|
||||
|
||||
class TestFSRMStateStoreTester implements RMStateStoreHelper {
|
||||
Path workingDirPathURI;
|
||||
FileSystemRMStateStore store;
|
||||
MiniDFSCluster cluster;
|
||||
|
||||
class TestFileSystemRMStore extends FileSystemRMStateStore {
|
||||
TestFileSystemRMStore(Configuration conf) throws Exception {
|
||||
init(conf);
|
||||
Assert.assertNull(fs);
|
||||
assertTrue(workingDirPathURI.equals(fsWorkingPath));
|
||||
start();
|
||||
Assert.assertNotNull(fs);
|
||||
}
|
||||
}
|
||||
|
||||
public TestFSRMStateStoreTester(MiniDFSCluster cluster) throws Exception {
|
||||
Path workingDirPath = new Path("/Test");
|
||||
this.cluster = cluster;
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
fs.mkdirs(workingDirPath);
|
||||
Path clusterURI = new Path(cluster.getURI());
|
||||
workingDirPathURI = new Path(clusterURI, workingDirPath);
|
||||
fs.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RMStateStore getRMStateStore() throws Exception {
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI,
|
||||
workingDirPathURI.toString());
|
||||
this.store = new TestFileSystemRMStore(conf);
|
||||
return store;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFinalStateValid() throws Exception {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
FileStatus[] files = fs.listStatus(workingDirPathURI);
|
||||
return files.length == 1;
|
||||
}
|
||||
}
|
||||
|
||||
void waitNotify(TestDispatcher dispatcher) {
|
||||
long startTime = System.currentTimeMillis();
|
||||
while(!dispatcher.notified) {
|
|
@ -0,0 +1,120 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertTrue;
|
||||
import junit.framework.Assert;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestFSRMStateStore extends RMStateStoreTestBase {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(TestFSRMStateStore.class);
|
||||
|
||||
class TestFSRMStateStoreTester implements RMStateStoreHelper {
|
||||
|
||||
Path workingDirPathURI;
|
||||
FileSystemRMStateStore store;
|
||||
MiniDFSCluster cluster;
|
||||
|
||||
class TestFileSystemRMStore extends FileSystemRMStateStore {
|
||||
|
||||
TestFileSystemRMStore(Configuration conf) throws Exception {
|
||||
init(conf);
|
||||
Assert.assertNull(fs);
|
||||
assertTrue(workingDirPathURI.equals(fsWorkingPath));
|
||||
start();
|
||||
Assert.assertNotNull(fs);
|
||||
}
|
||||
}
|
||||
|
||||
public TestFSRMStateStoreTester(MiniDFSCluster cluster) throws Exception {
|
||||
Path workingDirPath = new Path("/Test");
|
||||
this.cluster = cluster;
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
fs.mkdirs(workingDirPath);
|
||||
Path clusterURI = new Path(cluster.getURI());
|
||||
workingDirPathURI = new Path(clusterURI, workingDirPath);
|
||||
fs.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RMStateStore getRMStateStore() throws Exception {
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
conf.set(YarnConfiguration.FS_RM_STATE_STORE_URI,
|
||||
workingDirPathURI.toString());
|
||||
this.store = new TestFileSystemRMStore(conf);
|
||||
return store;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFinalStateValid() throws Exception {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
FileStatus[] files = fs.listStatus(workingDirPathURI);
|
||||
return files.length == 1;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFSRMStateStore() throws Exception {
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
try {
|
||||
TestFSRMStateStoreTester fsTester = new TestFSRMStateStoreTester(cluster);
|
||||
// If the state store is FileSystemRMStateStore then add corrupted entry.
|
||||
// It should discard the entry and remove it from file system.
|
||||
FSDataOutputStream fsOut = null;
|
||||
FileSystemRMStateStore fileSystemRMStateStore =
|
||||
(FileSystemRMStateStore) fsTester.getRMStateStore();
|
||||
String appAttemptIdStr3 = "appattempt_1352994193343_0001_000003";
|
||||
ApplicationAttemptId attemptId3 =
|
||||
ConverterUtils.toApplicationAttemptId(appAttemptIdStr3);
|
||||
Path rootDir =
|
||||
new Path(fileSystemRMStateStore.fsWorkingPath, "FSRMStateRoot");
|
||||
Path appRootDir = new Path(rootDir, "RMAppRoot");
|
||||
Path appDir =
|
||||
new Path(appRootDir, attemptId3.getApplicationId().toString());
|
||||
Path tempAppAttemptFile =
|
||||
new Path(appDir, attemptId3.toString() + ".tmp");
|
||||
fsOut = fileSystemRMStateStore.fs.create(tempAppAttemptFile, false);
|
||||
fsOut.write("Some random data ".getBytes());
|
||||
fsOut.close();
|
||||
|
||||
testRMAppStateStore(fsTester);
|
||||
Assert.assertFalse(fileSystemRMStateStore.fsWorkingPath
|
||||
.getFileSystem(conf).exists(tempAppAttemptFile));
|
||||
testRMDTSecretManagerStateStore(fsTester);
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestZKRMStateStore extends RMStateStoreTestBase {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(TestZKRMStateStore.class);
|
||||
|
||||
class TestZKRMStateStoreTester implements RMStateStoreHelper {
|
||||
|
||||
ZooKeeper client;
|
||||
ZKRMStateStore store;
|
||||
|
||||
class TestZKRMStateStoreInternal extends ZKRMStateStore {
|
||||
|
||||
public TestZKRMStateStoreInternal(Configuration conf, String workingZnode)
|
||||
throws Exception {
|
||||
init(conf);
|
||||
start();
|
||||
assertTrue(znodeWorkingPath.equals(workingZnode));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ZooKeeper getNewZooKeeper() throws IOException {
|
||||
return client;
|
||||
}
|
||||
}
|
||||
|
||||
public RMStateStore getRMStateStore() throws Exception {
|
||||
String workingZnode = "/Test";
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_ADDRESS, hostPort);
|
||||
conf.set(YarnConfiguration.ZK_RM_STATE_STORE_PARENT_PATH, workingZnode);
|
||||
this.client = createClient();
|
||||
this.store = new TestZKRMStateStoreInternal(conf, workingZnode);
|
||||
return this.store;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isFinalStateValid() throws Exception {
|
||||
List<String> nodes = client.getChildren(store.znodeWorkingPath, false);
|
||||
return nodes.size() == 1;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testZKRMStateStoreRealZK() throws Exception {
|
||||
TestZKRMStateStoreTester zkTester = new TestZKRMStateStoreTester();
|
||||
testRMAppStateStore(zkTester);
|
||||
testRMDTSecretManagerStateStore(zkTester);
|
||||
}
|
||||
}
|
|
@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.ha.ClientBaseWithFixes;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.TestRMStateStore.TestDispatcher;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreTestBase.TestDispatcher;
|
||||
import org.apache.hadoop.util.ZKUtil;
|
||||
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
|
@ -43,17 +43,20 @@ import static org.junit.Assert.fail;
|
|||
|
||||
public class TestZKRMStateStoreZKClientConnections extends
|
||||
ClientBaseWithFixes {
|
||||
|
||||
private static final int ZK_OP_WAIT_TIME = 3000;
|
||||
private Log LOG =
|
||||
LogFactory.getLog(TestZKRMStateStoreZKClientConnections.class);
|
||||
|
||||
class TestZKClient {
|
||||
|
||||
ZKRMStateStore store;
|
||||
boolean forExpire = false;
|
||||
TestForwardingWatcher watcher;
|
||||
CyclicBarrier syncBarrier = new CyclicBarrier(2);
|
||||
|
||||
protected class TestZKRMStateStore extends ZKRMStateStore {
|
||||
|
||||
public TestZKRMStateStore(Configuration conf, String workingZnode)
|
||||
throws Exception {
|
||||
init(conf);
|
||||
|
@ -87,6 +90,7 @@ public class TestZKRMStateStoreZKClientConnections extends
|
|||
|
||||
private class TestForwardingWatcher extends
|
||||
ClientBaseWithFixes.CountdownWatcher {
|
||||
|
||||
public void process(WatchedEvent event) {
|
||||
super.process(event);
|
||||
try {
|
||||
|
@ -187,7 +191,7 @@ public class TestZKRMStateStoreZKClientConnections extends
|
|||
}
|
||||
}
|
||||
|
||||
@Test (timeout = 20000)
|
||||
@Test(timeout = 20000)
|
||||
public void testSetZKAcl() {
|
||||
TestZKClient zkClientTester = new TestZKClient();
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
|
@ -196,10 +200,11 @@ public class TestZKRMStateStoreZKClientConnections extends
|
|||
zkClientTester.store.zkClient.delete(zkClientTester.store
|
||||
.znodeWorkingPath, -1);
|
||||
fail("Shouldn't be able to delete path");
|
||||
} catch (Exception e) {/* expected behavior */}
|
||||
} catch (Exception e) {/* expected behavior */
|
||||
}
|
||||
}
|
||||
|
||||
@Test (timeout = 20000)
|
||||
@Test(timeout = 20000)
|
||||
public void testInvalidZKAclConfiguration() {
|
||||
TestZKClient zkClientTester = new TestZKClient();
|
||||
YarnConfiguration conf = new YarnConfiguration();
|
||||
|
|
|
@ -0,0 +1,73 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.apache.hadoop.yarn.api.records.QueueACL;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
|
||||
|
||||
public class TestCapacitySchedulerQueueACLs extends QueueACLsTestBase {
|
||||
@Override
|
||||
protected Configuration createConfiguration() {
|
||||
CapacitySchedulerConfiguration csConf =
|
||||
new CapacitySchedulerConfiguration();
|
||||
csConf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {
|
||||
QUEUEA, QUEUEB });
|
||||
|
||||
csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA, 50f);
|
||||
csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB, 50f);
|
||||
|
||||
Map<QueueACL, AccessControlList> aclsOnQueueA =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
AccessControlList submitACLonQueueA = new AccessControlList(QUEUE_A_USER);
|
||||
submitACLonQueueA.addUser(COMMON_USER);
|
||||
AccessControlList adminACLonQueueA = new AccessControlList(QUEUE_A_ADMIN);
|
||||
aclsOnQueueA.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueA);
|
||||
aclsOnQueueA.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueA);
|
||||
csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEA,
|
||||
aclsOnQueueA);
|
||||
|
||||
Map<QueueACL, AccessControlList> aclsOnQueueB =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
AccessControlList submitACLonQueueB = new AccessControlList(QUEUE_B_USER);
|
||||
submitACLonQueueB.addUser(COMMON_USER);
|
||||
AccessControlList adminACLonQueueB = new AccessControlList(QUEUE_B_ADMIN);
|
||||
aclsOnQueueB.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonQueueB);
|
||||
aclsOnQueueB.put(QueueACL.ADMINISTER_QUEUE, adminACLonQueueB);
|
||||
csConf.setAcls(CapacitySchedulerConfiguration.ROOT + "." + QUEUEB,
|
||||
aclsOnQueueB);
|
||||
|
||||
Map<QueueACL, AccessControlList> aclsOnRootQueue =
|
||||
new HashMap<QueueACL, AccessControlList>();
|
||||
AccessControlList submitACLonRoot = new AccessControlList("");
|
||||
AccessControlList adminACLonRoot = new AccessControlList(ROOT_ADMIN);
|
||||
aclsOnRootQueue.put(QueueACL.SUBMIT_APPLICATIONS, submitACLonRoot);
|
||||
aclsOnRootQueue.put(QueueACL.ADMINISTER_QUEUE, adminACLonRoot);
|
||||
csConf.setAcls(CapacitySchedulerConfiguration.ROOT, aclsOnRootQueue);
|
||||
|
||||
csConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
|
||||
csConf.set("yarn.resourcemanager.scheduler.class", CapacityScheduler.class.getName());
|
||||
|
||||
return csConf;
|
||||
}
|
||||
}
|
|
@ -88,6 +88,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSc
|
|||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.DominantResourceFairnessPolicy;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FairSharePolicy;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.policies.FifoPolicy;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
|
||||
|
@ -807,6 +808,7 @@ public class TestFairScheduler {
|
|||
out.println("<queue name=\"queueB\">");
|
||||
out.println("<minResources>2048mb,0vcores</minResources>");
|
||||
out.println("<aclAdministerApps>alice,bob admins</aclAdministerApps>");
|
||||
out.println("<schedulingPolicy>fair</schedulingPolicy>");
|
||||
out.println("</queue>");
|
||||
// Give queue C no minimum
|
||||
out.println("<queue name=\"queueC\">");
|
||||
|
@ -833,6 +835,8 @@ public class TestFairScheduler {
|
|||
+ "</defaultMinSharePreemptionTimeout>");
|
||||
// Set fair share preemption timeout to 5 minutes
|
||||
out.println("<fairSharePreemptionTimeout>300</fairSharePreemptionTimeout>");
|
||||
// Set default scheduling policy to DRF
|
||||
out.println("<defaultQueueSchedulingPolicy>drf</defaultQueueSchedulingPolicy>");
|
||||
out.println("</allocations>");
|
||||
out.close();
|
||||
|
||||
|
@ -865,22 +869,25 @@ public class TestFairScheduler {
|
|||
assertEquals(10, queueManager.getUserMaxApps("user1"));
|
||||
assertEquals(5, queueManager.getUserMaxApps("user2"));
|
||||
|
||||
// Root should get * ACL
|
||||
assertEquals("*",queueManager.getQueueAcl("root",
|
||||
QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
assertEquals("*", queueManager.getQueueAcl("root",
|
||||
QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
|
||||
// Unspecified queues should get default ACL
|
||||
Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("root.queueA");
|
||||
assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
|
||||
assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
|
||||
assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
assertEquals(" ",queueManager.getQueueAcl("root.queueA",
|
||||
QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
assertEquals(" ", queueManager.getQueueAcl("root.queueA",
|
||||
QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
|
||||
// Queue B ACL
|
||||
Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
|
||||
assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
|
||||
assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueB",
|
||||
QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
|
||||
// Queue c ACL
|
||||
Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
|
||||
assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
|
||||
assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
// Queue C ACL
|
||||
assertEquals("alice,bob admins",queueManager.getQueueAcl("root.queueC",
|
||||
QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
|
||||
assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
|
||||
YarnConfiguration.DEFAULT_QUEUE_NAME));
|
||||
|
@ -891,6 +898,18 @@ public class TestFairScheduler {
|
|||
assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root.queueA"));
|
||||
assertEquals(60000, queueManager.getMinSharePreemptionTimeout("root.queueE"));
|
||||
assertEquals(300000, queueManager.getFairSharePreemptionTimeout());
|
||||
|
||||
// Verify existing queues have default scheduling policy
|
||||
assertEquals(DominantResourceFairnessPolicy.NAME,
|
||||
queueManager.getQueue("root").getPolicy().getName());
|
||||
assertEquals(DominantResourceFairnessPolicy.NAME,
|
||||
queueManager.getQueue("root.queueA").getPolicy().getName());
|
||||
// Verify default is overriden if specified explicitly
|
||||
assertEquals(FairSharePolicy.NAME,
|
||||
queueManager.getQueue("root.queueB").getPolicy().getName());
|
||||
// Verify new queue gets default scheduling policy
|
||||
assertEquals(DominantResourceFairnessPolicy.NAME,
|
||||
queueManager.getLeafQueue("root.newqueue", true).getPolicy().getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1063,21 +1082,19 @@ public class TestFairScheduler {
|
|||
assertEquals(5, queueManager.getUserMaxApps("user2"));
|
||||
|
||||
// Unspecified queues should get default ACL
|
||||
Map<QueueACL, AccessControlList> aclsA = queueManager.getQueueAcls("queueA");
|
||||
assertTrue(aclsA.containsKey(QueueACL.ADMINISTER_QUEUE));
|
||||
assertEquals("*", aclsA.get(QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
assertTrue(aclsA.containsKey(QueueACL.SUBMIT_APPLICATIONS));
|
||||
assertEquals("*", aclsA.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
assertEquals(" ", queueManager.getQueueAcl("root.queueA",
|
||||
QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
assertEquals(" ", queueManager.getQueueAcl("root.queueA",
|
||||
QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
|
||||
// Queue B ACL
|
||||
Map<QueueACL, AccessControlList> aclsB = queueManager.getQueueAcls("root.queueB");
|
||||
assertTrue(aclsB.containsKey(QueueACL.ADMINISTER_QUEUE));
|
||||
assertEquals("alice,bob admins", aclsB.get(QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueB",
|
||||
QueueACL.ADMINISTER_QUEUE).getAclString());
|
||||
|
||||
// Queue C ACL
|
||||
assertEquals("alice,bob admins", queueManager.getQueueAcl("root.queueC",
|
||||
QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
|
||||
// Queue c ACL
|
||||
Map<QueueACL, AccessControlList> aclsC = queueManager.getQueueAcls("root.queueC");
|
||||
assertTrue(aclsC.containsKey(QueueACL.SUBMIT_APPLICATIONS));
|
||||
assertEquals("alice,bob admins", aclsC.get(QueueACL.SUBMIT_APPLICATIONS).getAclString());
|
||||
|
||||
assertEquals(120000, queueManager.getMinSharePreemptionTimeout("root." +
|
||||
YarnConfiguration.DEFAULT_QUEUE_NAME));
|
||||
|
@ -1664,9 +1681,13 @@ public class TestFairScheduler {
|
|||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
out.println("<?xml version=\"1.0\"?>");
|
||||
out.println("<allocations>");
|
||||
out.println("<queue name=\"queue1\">");
|
||||
out.println("<aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
|
||||
out.println("<aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
|
||||
out.println("<queue name=\"root\">");
|
||||
out.println(" <aclSubmitApps> </aclSubmitApps>");
|
||||
out.println(" <aclAdministerApps> </aclAdministerApps>");
|
||||
out.println(" <queue name=\"queue1\">");
|
||||
out.println(" <aclSubmitApps>norealuserhasthisname</aclSubmitApps>");
|
||||
out.println(" <aclAdministerApps>norealuserhasthisname</aclAdministerApps>");
|
||||
out.println(" </queue>");
|
||||
out.println("</queue>");
|
||||
out.println("</allocations>");
|
||||
out.close();
|
||||
|
@ -1893,9 +1914,13 @@ public class TestFairScheduler {
|
|||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
out.println("<?xml version=\"1.0\"?>");
|
||||
out.println("<allocations>");
|
||||
out.println("<queue name=\"queue1\">");
|
||||
out.println("<aclSubmitApps>userallow</aclSubmitApps>");
|
||||
out.println("<aclAdministerApps>userallow</aclAdministerApps>");
|
||||
out.println("<queue name=\"root\">");
|
||||
out.println(" <aclSubmitApps> </aclSubmitApps>");
|
||||
out.println(" <aclAdministerApps> </aclAdministerApps>");
|
||||
out.println(" <queue name=\"queue1\">");
|
||||
out.println(" <aclSubmitApps>userallow</aclSubmitApps>");
|
||||
out.println(" <aclAdministerApps>userallow</aclAdministerApps>");
|
||||
out.println(" </queue>");
|
||||
out.println("</queue>");
|
||||
out.println("</allocations>");
|
||||
out.close();
|
||||
|
|
|
@ -0,0 +1,62 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.QueueACLsTestBase;
|
||||
|
||||
public class TestFairSchedulerQueueACLs extends QueueACLsTestBase {
|
||||
@Override
|
||||
protected Configuration createConfiguration() throws IOException {
|
||||
FairSchedulerConfiguration fsConf = new FairSchedulerConfiguration();
|
||||
|
||||
final String TEST_DIR = new File(System.getProperty("test.build.data",
|
||||
"/tmp")).getAbsolutePath();
|
||||
final String ALLOC_FILE = new File(TEST_DIR, "test-queues.xml")
|
||||
.getAbsolutePath();
|
||||
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
||||
out.println("<?xml version=\"1.0\"?>");
|
||||
out.println("<allocations>");
|
||||
out.println("<queue name=\"root\">");
|
||||
out.println(" <aclSubmitApps> </aclSubmitApps>");
|
||||
out.println(" <aclAdministerApps>root_admin </aclAdministerApps>");
|
||||
out.println(" <queue name=\"queueA\">");
|
||||
out.println(" <aclSubmitApps>queueA_user,common_user </aclSubmitApps>");
|
||||
out.println(" <aclAdministerApps>queueA_admin </aclAdministerApps>");
|
||||
out.println(" </queue>");
|
||||
out.println(" <queue name=\"queueB\">");
|
||||
out.println(" <aclSubmitApps>queueB_user,common_user </aclSubmitApps>");
|
||||
out.println(" <aclAdministerApps>queueB_admin </aclAdministerApps>");
|
||||
out.println(" </queue>");
|
||||
out.println("</queue>");
|
||||
out.println("</allocations>");
|
||||
out.close();
|
||||
fsConf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
||||
|
||||
fsConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
|
||||
fsConf.set("yarn.resourcemanager.scheduler.class", FairScheduler.class.getName());
|
||||
|
||||
return fsConf;
|
||||
}
|
||||
}
|
|
@ -22,6 +22,8 @@ import java.io.File;
|
|||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -34,8 +36,10 @@ import org.apache.hadoop.service.AbstractService;
|
|||
import org.apache.hadoop.service.CompositeService;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||
import org.apache.hadoop.yarn.event.EventHandler;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||
|
@ -52,6 +56,10 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
|
|||
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
|
||||
import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
|
||||
|
||||
/**
|
||||
|
@ -83,6 +91,9 @@ public class MiniYARNCluster extends CompositeService {
|
|||
|
||||
private ResourceManagerWrapper resourceManagerWrapper;
|
||||
|
||||
private ConcurrentMap<ApplicationAttemptId, Long> appMasters =
|
||||
new ConcurrentHashMap<ApplicationAttemptId, Long>(16, 0.75f, 2);
|
||||
|
||||
private File testWorkDir;
|
||||
|
||||
// Number of nm-local-dirs per nodemanager
|
||||
|
@ -210,6 +221,16 @@ public class MiniYARNCluster extends CompositeService {
|
|||
};
|
||||
};
|
||||
resourceManager.init(conf);
|
||||
resourceManager.getRMContext().getDispatcher().register(RMAppAttemptEventType.class,
|
||||
new EventHandler<RMAppAttemptEvent>() {
|
||||
public void handle(RMAppAttemptEvent event) {
|
||||
if (event instanceof RMAppAttemptRegistrationEvent) {
|
||||
appMasters.put(event.getApplicationAttemptId(), event.getTimestamp());
|
||||
} else if (event instanceof RMAppAttemptUnregistrationEvent) {
|
||||
appMasters.remove(event.getApplicationAttemptId());
|
||||
}
|
||||
}
|
||||
});
|
||||
super.serviceInit(conf);
|
||||
}
|
||||
|
||||
|
@ -243,9 +264,22 @@ public class MiniYARNCluster extends CompositeService {
|
|||
WebAppUtils.getRMWebAppURLWithoutScheme(getConfig()));
|
||||
}
|
||||
|
||||
private void waitForAppMastersToFinish(long timeoutMillis) throws InterruptedException {
|
||||
long started = System.currentTimeMillis();
|
||||
synchronized (appMasters) {
|
||||
while (!appMasters.isEmpty() && System.currentTimeMillis() - started < timeoutMillis) {
|
||||
appMasters.wait(1000);
|
||||
}
|
||||
}
|
||||
if (!appMasters.isEmpty()) {
|
||||
LOG.warn("Stopping RM while some app masters are still alive");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected synchronized void serviceStop() throws Exception {
|
||||
if (resourceManager != null) {
|
||||
waitForAppMastersToFinish(5000);
|
||||
resourceManager.stop();
|
||||
}
|
||||
super.serviceStop();
|
||||
|
|
|
@ -221,10 +221,14 @@ Allocation file format
|
|||
for containers, but apps submitted later may run concurrently if there is
|
||||
leftover space on the cluster after satisfying the earlier app's requests.
|
||||
|
||||
* aclSubmitApps: a list of users that can submit apps to the queue. A (default)
|
||||
value of "*" means that any users can submit apps. A queue inherits the ACL of
|
||||
its parent, so if a queue2 descends from queue1, and user1 is in queue1's ACL,
|
||||
and user2 is in queue2's ACL, then both users may submit to queue2.
|
||||
* aclSubmitApps: a list of users and/or groups that can submit apps to the
|
||||
queue. Refer to the ACLs section below for more info on the format of this
|
||||
list and how queue ACLs work.
|
||||
|
||||
* aclAdministerApps: a list of users and/or groups that can administer a
|
||||
queue. Currently the only administrative action is killing an application.
|
||||
Refer to the ACLs section below for more info on the format of this list
|
||||
and how queue ACLs work.
|
||||
|
||||
* minSharePreemptionTimeout: number of seconds the queue is under its minimum share
|
||||
before it will try to preempt containers to take resources from other queues.
|
||||
|
@ -246,6 +250,24 @@ Allocation file format
|
|||
|
||||
An example allocation file is given here:
|
||||
|
||||
Queue Access Control Lists (ACLs)
|
||||
|
||||
Queue Access Control Lists (ACLs) allow administrators to control who may
|
||||
take actions on particular queues. They are configured with the aclSubmitApps
|
||||
and aclAdministerApps properties, which can be set per queue. Currently the
|
||||
only supported administrative action is killing an application. Anybody who
|
||||
may administer a queue may also submit applications to it. These properties
|
||||
take values in a format like "user1,user2 group1,group2" or " group1,group2".
|
||||
An action on a queue will be permitted if its user or group is in the ACL of
|
||||
that queue or in the ACL of any of that queue's ancestors. So if queue2
|
||||
is inside queue1, and user1 is in queue1's ACL, and user2 is in queue2's
|
||||
ACL, then both users may submit to queue2.
|
||||
|
||||
The root queue's ACLs are "*" by default which, because ACLs are passed down,
|
||||
means that everybody may submit to and kill applications from every queue.
|
||||
To start restricting access, change the root queue's ACLs to something other
|
||||
than "*".
|
||||
|
||||
---
|
||||
<?xml version="1.0"?>
|
||||
<allocations>
|
||||
|
@ -256,6 +278,7 @@ Allocation file format
|
|||
<weight>2.0</weight>
|
||||
<schedulingPolicy>fair</schedulingPolicy>
|
||||
<queue name="sample_sub_queue">
|
||||
<aclSubmitApps>charlie</aclSubmitApps>
|
||||
<minResources>5000 mb,0vcores</minResources>
|
||||
</queue>
|
||||
</queue>
|
||||
|
|
4
pom.xml
4
pom.xml
|
@ -143,7 +143,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
|
|||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-antrun-plugin</artifactId>
|
||||
<version>1.6</version>
|
||||
<version>1.7</version>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
|
@ -175,7 +175,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
|
|||
<pluginExecutionFilter>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-antrun-plugin</artifactId>
|
||||
<versionRange>[1.6,)</versionRange>
|
||||
<versionRange>[1.7,)</versionRange>
|
||||
<goals>
|
||||
<goal>run</goal>
|
||||
</goals>
|
||||
|
|
Loading…
Reference in New Issue