Merge HDFS-4949 branch back into trunk
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1536572 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
1b48f44feb
|
@ -0,0 +1,118 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.fs;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
* A RemoteIterator that fetches elements in batches.
|
||||
*/
|
||||
public abstract class BatchedRemoteIterator<K, E> implements RemoteIterator<E> {
|
||||
public interface BatchedEntries<E> {
|
||||
public E get(int i);
|
||||
public int size();
|
||||
public boolean hasMore();
|
||||
}
|
||||
|
||||
public static class BatchedListEntries<E> implements BatchedEntries<E> {
|
||||
private final List<E> entries;
|
||||
private final boolean hasMore;
|
||||
|
||||
public BatchedListEntries(List<E> entries, boolean hasMore) {
|
||||
this.entries = entries;
|
||||
this.hasMore = hasMore;
|
||||
}
|
||||
|
||||
public E get(int i) {
|
||||
return entries.get(i);
|
||||
}
|
||||
|
||||
public int size() {
|
||||
return entries.size();
|
||||
}
|
||||
|
||||
public boolean hasMore() {
|
||||
return hasMore;
|
||||
}
|
||||
}
|
||||
|
||||
private K prevKey;
|
||||
private BatchedEntries<E> entries;
|
||||
private int idx;
|
||||
|
||||
public BatchedRemoteIterator(K prevKey) {
|
||||
this.prevKey = prevKey;
|
||||
this.entries = null;
|
||||
this.idx = -1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual remote request.
|
||||
*
|
||||
* @param prevKey The key to send.
|
||||
* @return A list of replies.
|
||||
*/
|
||||
public abstract BatchedEntries<E> makeRequest(K prevKey) throws IOException;
|
||||
|
||||
private void makeRequest() throws IOException {
|
||||
idx = 0;
|
||||
entries = null;
|
||||
entries = makeRequest(prevKey);
|
||||
if (entries.size() == 0) {
|
||||
entries = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void makeRequestIfNeeded() throws IOException {
|
||||
if (idx == -1) {
|
||||
makeRequest();
|
||||
} else if ((entries != null) && (idx >= entries.size())) {
|
||||
if (!entries.hasMore()) {
|
||||
// Last time, we got fewer entries than requested.
|
||||
// So we should be at the end.
|
||||
entries = null;
|
||||
} else {
|
||||
makeRequest();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() throws IOException {
|
||||
makeRequestIfNeeded();
|
||||
return (entries != null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the next list key associated with an element.
|
||||
*/
|
||||
public abstract K elementToPrevKey(E element);
|
||||
|
||||
@Override
|
||||
public E next() throws IOException {
|
||||
makeRequestIfNeeded();
|
||||
if (entries == null) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
E entry = entries.get(idx++);
|
||||
prevKey = elementToPrevKey(entry);
|
||||
return entry;
|
||||
}
|
||||
}
|
|
@ -303,6 +303,13 @@ public class FsPermission implements Writable {
|
|||
return new FsPermission((short)00666);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the default permission for cache pools.
|
||||
*/
|
||||
public static FsPermission getCachePoolDefault() {
|
||||
return new FsPermission((short)00755);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a FsPermission from a Unix symbolic permission string
|
||||
* @param unixSymbolicPermission e.g. "-rw-rw-rw-"
|
||||
|
|
|
@ -454,10 +454,7 @@ public class Text extends BinaryComparable
|
|||
/** Read a UTF8 encoded string from in
|
||||
*/
|
||||
public static String readString(DataInput in) throws IOException {
|
||||
int length = WritableUtils.readVInt(in);
|
||||
byte [] bytes = new byte[length];
|
||||
in.readFully(bytes, 0, length);
|
||||
return decode(bytes);
|
||||
return readString(in, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
/** Read a UTF8 encoded string with a maximum size
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.io.FileInputStream;
|
|||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
|
@ -145,6 +146,12 @@ public class NativeIO {
|
|||
return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded;
|
||||
}
|
||||
|
||||
private static void assertCodeLoaded() throws IOException {
|
||||
if (!isAvailable()) {
|
||||
throw new IOException("NativeIO was not loaded");
|
||||
}
|
||||
}
|
||||
|
||||
/** Wrapper around open(2) */
|
||||
public static native FileDescriptor open(String path, int flags, int mode) throws IOException;
|
||||
/** Wrapper around fstat(2) */
|
||||
|
@ -225,6 +232,46 @@ public class NativeIO {
|
|||
}
|
||||
}
|
||||
|
||||
static native void mlock_native(
|
||||
ByteBuffer buffer, long len) throws NativeIOException;
|
||||
static native void munlock_native(
|
||||
ByteBuffer buffer, long len) throws NativeIOException;
|
||||
|
||||
/**
|
||||
* Locks the provided direct ByteBuffer into memory, preventing it from
|
||||
* swapping out. After a buffer is locked, future accesses will not incur
|
||||
* a page fault.
|
||||
*
|
||||
* See the mlock(2) man page for more information.
|
||||
*
|
||||
* @throws NativeIOException
|
||||
*/
|
||||
public static void mlock(ByteBuffer buffer, long len)
|
||||
throws IOException {
|
||||
assertCodeLoaded();
|
||||
if (!buffer.isDirect()) {
|
||||
throw new IOException("Cannot mlock a non-direct ByteBuffer");
|
||||
}
|
||||
mlock_native(buffer, len);
|
||||
}
|
||||
|
||||
/**
|
||||
* Unlocks a locked direct ByteBuffer, allowing it to swap out of memory.
|
||||
* This is a no-op if the ByteBuffer was not previously locked.
|
||||
*
|
||||
* See the munlock(2) man page for more information.
|
||||
*
|
||||
* @throws NativeIOException
|
||||
*/
|
||||
public static void munlock(ByteBuffer buffer, long len)
|
||||
throws IOException {
|
||||
assertCodeLoaded();
|
||||
if (!buffer.isDirect()) {
|
||||
throw new IOException("Cannot munlock a non-direct ByteBuffer");
|
||||
}
|
||||
munlock_native(buffer, len);
|
||||
}
|
||||
|
||||
/** Linux only methods used for getOwner() implementation */
|
||||
private static native long getUIDforFDOwnerforOwner(FileDescriptor fd) throws IOException;
|
||||
private static native String getUserName(long uid) throws IOException;
|
||||
|
@ -478,6 +525,20 @@ public class NativeIO {
|
|||
/** Initialize the JNI method ID and class ID cache */
|
||||
private static native void initNative();
|
||||
|
||||
/**
|
||||
* Get the maximum number of bytes that can be locked into memory at any
|
||||
* given point.
|
||||
*
|
||||
* @return 0 if no bytes can be locked into memory;
|
||||
* Long.MAX_VALUE if there is no limit;
|
||||
* The number of bytes that can be locked into memory otherwise.
|
||||
*/
|
||||
public static long getMemlockLimit() {
|
||||
return isAvailable() ? getMemlockLimit0() : 0;
|
||||
}
|
||||
|
||||
private static native long getMemlockLimit0();
|
||||
|
||||
private static class CachedUid {
|
||||
final long timestamp;
|
||||
final String username;
|
||||
|
|
|
@ -1254,6 +1254,14 @@ public class UserGroupInformation {
|
|||
return null;
|
||||
}
|
||||
|
||||
public String getPrimaryGroupName() throws IOException {
|
||||
String[] groups = getGroupNames();
|
||||
if (groups.length == 0) {
|
||||
throw new IOException("There is no primary group for UGI " + this);
|
||||
}
|
||||
return groups[0];
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the user's full principal name.
|
||||
* @return the user's full principal name.
|
||||
|
|
|
@ -0,0 +1,373 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.util;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Implements an intrusive doubly-linked list.
|
||||
*
|
||||
* An intrusive linked list is one in which the elements themselves are
|
||||
* responsible for storing the pointers to previous and next elements.
|
||||
* This can save a lot of memory if there are many elements in the list or
|
||||
* many lists.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class IntrusiveCollection<E extends IntrusiveCollection.Element>
|
||||
implements Collection<E> {
|
||||
/**
|
||||
* An element contained in this list.
|
||||
*
|
||||
* We pass the list itself as a parameter so that elements can belong to
|
||||
* multiple lists. (The element will need to store separate prev and next
|
||||
* pointers for each.)
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface Element {
|
||||
/**
|
||||
* Insert this element into the list. This is the first thing that will
|
||||
* be called on the element.
|
||||
*/
|
||||
void insertInternal(IntrusiveCollection<? extends Element> list,
|
||||
Element prev, Element next);
|
||||
|
||||
/**
|
||||
* Set the prev pointer of an element already in the list.
|
||||
*/
|
||||
void setPrev(IntrusiveCollection<? extends Element> list, Element prev);
|
||||
|
||||
/**
|
||||
* Set the next pointer of an element already in the list.
|
||||
*/
|
||||
void setNext(IntrusiveCollection<? extends Element> list, Element next);
|
||||
|
||||
/**
|
||||
* Remove an element from the list. This is the last thing that will be
|
||||
* called on an element.
|
||||
*/
|
||||
void removeInternal(IntrusiveCollection<? extends Element> list);
|
||||
|
||||
/**
|
||||
* Get the prev pointer of an element.
|
||||
*/
|
||||
Element getPrev(IntrusiveCollection<? extends Element> list);
|
||||
|
||||
/**
|
||||
* Get the next pointer of an element.
|
||||
*/
|
||||
Element getNext(IntrusiveCollection<? extends Element> list);
|
||||
|
||||
/**
|
||||
* Returns true if this element is in the provided list.
|
||||
*/
|
||||
boolean isInList(IntrusiveCollection<? extends Element> list);
|
||||
}
|
||||
|
||||
private Element root = new Element() {
|
||||
// We keep references to the first and last elements for easy access.
|
||||
Element first = this;
|
||||
Element last = this;
|
||||
|
||||
@Override
|
||||
public void insertInternal(IntrusiveCollection<? extends Element> list,
|
||||
Element prev, Element next) {
|
||||
throw new RuntimeException("Can't insert root element");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPrev(IntrusiveCollection<? extends Element> list,
|
||||
Element prev) {
|
||||
Preconditions.checkState(list == IntrusiveCollection.this);
|
||||
last = prev;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNext(IntrusiveCollection<? extends Element> list,
|
||||
Element next) {
|
||||
Preconditions.checkState(list == IntrusiveCollection.this);
|
||||
first = next;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeInternal(IntrusiveCollection<? extends Element> list) {
|
||||
throw new RuntimeException("Can't remove root element");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Element getNext(
|
||||
IntrusiveCollection<? extends Element> list) {
|
||||
Preconditions.checkState(list == IntrusiveCollection.this);
|
||||
return first;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Element getPrev(
|
||||
IntrusiveCollection<? extends Element> list) {
|
||||
Preconditions.checkState(list == IntrusiveCollection.this);
|
||||
return last;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInList(IntrusiveCollection<? extends Element> list) {
|
||||
return list == IntrusiveCollection.this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "root"; // + IntrusiveCollection.this + "]";
|
||||
}
|
||||
};
|
||||
|
||||
private int size = 0;
|
||||
|
||||
/**
|
||||
* An iterator over the intrusive collection.
|
||||
*
|
||||
* Currently, you can remove elements from the list using
|
||||
* #{IntrusiveIterator#remove()}, but modifying the collection in other
|
||||
* ways during the iteration is not supported.
|
||||
*/
|
||||
public class IntrusiveIterator implements Iterator<E> {
|
||||
Element cur;
|
||||
Element next;
|
||||
|
||||
IntrusiveIterator() {
|
||||
this.cur = root;
|
||||
this.next = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
if (next == null) {
|
||||
next = cur.getNext(IntrusiveCollection.this);
|
||||
}
|
||||
return next != root;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public E next() {
|
||||
if (next == null) {
|
||||
next = cur.getNext(IntrusiveCollection.this);
|
||||
}
|
||||
if (next == root) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
cur = next;
|
||||
next = null;
|
||||
return (E)cur;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
if (cur == null) {
|
||||
throw new IllegalStateException("Already called remove " +
|
||||
"once on this element.");
|
||||
}
|
||||
next = removeElement(cur);
|
||||
cur = null;
|
||||
}
|
||||
}
|
||||
|
||||
private Element removeElement(Element elem) {
|
||||
Element prev = elem.getPrev(IntrusiveCollection.this);
|
||||
Element next = elem.getNext(IntrusiveCollection.this);
|
||||
elem.removeInternal(IntrusiveCollection.this);
|
||||
prev.setNext(IntrusiveCollection.this, next);
|
||||
next.setPrev(IntrusiveCollection.this, prev);
|
||||
size--;
|
||||
return next;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get an iterator over the list. This can be used to remove elements.
|
||||
* It is not safe to do concurrent modifications from other threads while
|
||||
* using this iterator.
|
||||
*
|
||||
* @return The iterator.
|
||||
*/
|
||||
public Iterator<E> iterator() {
|
||||
return new IntrusiveIterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return size;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
return size == 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(Object o) {
|
||||
try {
|
||||
Element element = (Element)o;
|
||||
return element.isInList(this);
|
||||
} catch (ClassCastException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] toArray() {
|
||||
Object ret[] = new Object[size];
|
||||
int i = 0;
|
||||
for (Iterator<E> iter = iterator(); iter.hasNext(); ) {
|
||||
ret[i++] = iter.next();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public <T> T[] toArray(T[] array) {
|
||||
if (array.length < size) {
|
||||
return (T[])toArray();
|
||||
} else {
|
||||
int i = 0;
|
||||
for (Iterator<E> iter = iterator(); iter.hasNext(); ) {
|
||||
array[i++] = (T)iter.next();
|
||||
}
|
||||
}
|
||||
return array;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add an element to the end of the list.
|
||||
*
|
||||
* @param elem The new element to add.
|
||||
*/
|
||||
@Override
|
||||
public boolean add(E elem) {
|
||||
if (elem == null) {
|
||||
return false;
|
||||
}
|
||||
if (elem.isInList(this)) {
|
||||
return false;
|
||||
}
|
||||
Element prev = root.getPrev(IntrusiveCollection.this);
|
||||
prev.setNext(IntrusiveCollection.this, elem);
|
||||
root.setPrev(IntrusiveCollection.this, elem);
|
||||
elem.insertInternal(IntrusiveCollection.this, prev, root);
|
||||
size++;
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add an element to the front of the list.
|
||||
*
|
||||
* @param elem The new element to add.
|
||||
*/
|
||||
public boolean addFirst(Element elem) {
|
||||
if (elem == null) {
|
||||
return false;
|
||||
}
|
||||
if (elem.isInList(this)) {
|
||||
return false;
|
||||
}
|
||||
Element next = root.getNext(IntrusiveCollection.this);
|
||||
next.setPrev(IntrusiveCollection.this, elem);
|
||||
root.setNext(IntrusiveCollection.this, elem);
|
||||
elem.insertInternal(IntrusiveCollection.this, root, next);
|
||||
size++;
|
||||
return true;
|
||||
}
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(IntrusiveCollection.class);
|
||||
|
||||
@Override
|
||||
public boolean remove(Object o) {
|
||||
try {
|
||||
Element elem = (Element)o;
|
||||
if (!elem.isInList(this)) {
|
||||
return false;
|
||||
}
|
||||
removeElement(elem);
|
||||
return true;
|
||||
} catch (ClassCastException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsAll(Collection<?> collection) {
|
||||
for (Object o : collection) {
|
||||
if (!contains(o)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean addAll(Collection<? extends E> collection) {
|
||||
boolean changed = false;
|
||||
for (E elem : collection) {
|
||||
if (add(elem)) {
|
||||
changed = true;
|
||||
}
|
||||
}
|
||||
return changed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean removeAll(Collection<?> collection) {
|
||||
boolean changed = false;
|
||||
for (Object elem : collection) {
|
||||
if (remove(elem)) {
|
||||
changed = true;
|
||||
}
|
||||
}
|
||||
return changed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean retainAll(Collection<?> collection) {
|
||||
boolean changed = false;
|
||||
for (Iterator<E> iter = iterator();
|
||||
iter.hasNext(); ) {
|
||||
Element elem = iter.next();
|
||||
if (!collection.contains(elem)) {
|
||||
iter.remove();
|
||||
changed = true;
|
||||
}
|
||||
}
|
||||
return changed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove all elements.
|
||||
*/
|
||||
@Override
|
||||
public void clear() {
|
||||
for (Iterator<E> iter = iterator(); iter.hasNext(); ) {
|
||||
iter.next();
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.util;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.PriorityQueue;
|
||||
|
||||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
|
@ -235,4 +236,28 @@ public class LightWeightCache<K, E extends K> extends LightWeightGSet<K, E> {
|
|||
}
|
||||
return removed;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<E> iterator() {
|
||||
final Iterator<E> iter = super.iterator();
|
||||
return new Iterator<E>() {
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return iter.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public E next() {
|
||||
return iter.next();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
// It would be tricky to support this because LightWeightCache#remove
|
||||
// may evict multiple elements via evictExpiredEntries.
|
||||
throw new UnsupportedOperationException("Remove via iterator is " +
|
||||
"not supported for LightWeightCache");
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -246,10 +246,10 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
|
|||
|
||||
private class SetIterator implements Iterator<E> {
|
||||
/** The starting modification for fail-fast. */
|
||||
private final int startModification = modification;
|
||||
private int iterModification = modification;
|
||||
/** The current index of the entry array. */
|
||||
private int index = -1;
|
||||
/** The next element to return. */
|
||||
private LinkedElement cur = null;
|
||||
private LinkedElement next = nextNonemptyEntry();
|
||||
|
||||
/** Find the next nonempty entry starting at (index + 1). */
|
||||
|
@ -258,30 +258,51 @@ public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
|
|||
return index < entries.length? entries[index]: null;
|
||||
}
|
||||
|
||||
private void ensureNext() {
|
||||
if (modification != iterModification) {
|
||||
throw new ConcurrentModificationException("modification=" + modification
|
||||
+ " != iterModification = " + iterModification);
|
||||
}
|
||||
if (next != null) {
|
||||
return;
|
||||
}
|
||||
if (cur == null) {
|
||||
return;
|
||||
}
|
||||
next = cur.getNext();
|
||||
if (next == null) {
|
||||
next = nextNonemptyEntry();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
ensureNext();
|
||||
return next != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public E next() {
|
||||
if (modification != startModification) {
|
||||
throw new ConcurrentModificationException("modification=" + modification
|
||||
+ " != startModification = " + startModification);
|
||||
}
|
||||
|
||||
final E e = convert(next);
|
||||
|
||||
//find the next element
|
||||
final LinkedElement n = next.getNext();
|
||||
next = n != null? n: nextNonemptyEntry();
|
||||
|
||||
return e;
|
||||
ensureNext();
|
||||
if (next == null) {
|
||||
throw new IllegalStateException("There are no more elements");
|
||||
}
|
||||
cur = next;
|
||||
next = null;
|
||||
return convert(cur);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new UnsupportedOperationException("Remove is not supported.");
|
||||
ensureNext();
|
||||
if (cur == null) {
|
||||
throw new IllegalStateException("There is no current element " +
|
||||
"to remove");
|
||||
}
|
||||
LightWeightGSet.this.remove((K)cur);
|
||||
iterModification++;
|
||||
cur = null;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -905,4 +905,81 @@ public class StringUtils {
|
|||
}
|
||||
return str.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* From a list of command-line arguments, remove both an option and the
|
||||
* next argument.
|
||||
*
|
||||
* @param name Name of the option to remove. Example: -foo.
|
||||
* @param args List of arguments.
|
||||
* @return null if the option was not found; the value of the
|
||||
* option otherwise.
|
||||
*/
|
||||
public static String popOptionWithArgument(String name, List<String> args) {
|
||||
String val = null;
|
||||
for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
|
||||
String cur = iter.next();
|
||||
if (cur.equals("--")) {
|
||||
// stop parsing arguments when you see --
|
||||
break;
|
||||
} else if (cur.equals(name)) {
|
||||
iter.remove();
|
||||
if (!iter.hasNext()) {
|
||||
throw new RuntimeException("option " + name + " requires 1 " +
|
||||
"argument.");
|
||||
}
|
||||
val = iter.next();
|
||||
iter.remove();
|
||||
break;
|
||||
}
|
||||
}
|
||||
return val;
|
||||
}
|
||||
|
||||
/**
|
||||
* From a list of command-line arguments, remove an option.
|
||||
*
|
||||
* @param name Name of the option to remove. Example: -foo.
|
||||
* @param args List of arguments.
|
||||
* @return true if the option was found and removed; false otherwise.
|
||||
*/
|
||||
public static boolean popOption(String name, List<String> args) {
|
||||
for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
|
||||
String cur = iter.next();
|
||||
if (cur.equals("--")) {
|
||||
// stop parsing arguments when you see --
|
||||
break;
|
||||
} else if (cur.equals(name)) {
|
||||
iter.remove();
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* From a list of command-line arguments, return the first non-option
|
||||
* argument. Non-option arguments are those which either come after
|
||||
* a double dash (--) or do not start with a dash.
|
||||
*
|
||||
* @param args List of arguments.
|
||||
* @return The first non-option argument, or null if there were none.
|
||||
*/
|
||||
public static String popFirstNonOption(List<String> args) {
|
||||
for (Iterator<String> iter = args.iterator(); iter.hasNext(); ) {
|
||||
String cur = iter.next();
|
||||
if (cur.equals("--")) {
|
||||
if (!iter.hasNext()) {
|
||||
return null;
|
||||
}
|
||||
cur = iter.next();
|
||||
iter.remove();
|
||||
return cur;
|
||||
} else if (!cur.startsWith("-")) {
|
||||
iter.remove();
|
||||
return cur;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,8 +16,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
#define _GNU_SOURCE
|
||||
|
||||
#include "org_apache_hadoop.h"
|
||||
#include "org_apache_hadoop_io_nativeio_NativeIO.h"
|
||||
|
||||
|
@ -28,11 +26,15 @@
|
|||
#include <grp.h>
|
||||
#include <jni.h>
|
||||
#include <pwd.h>
|
||||
#include <stdint.h>
|
||||
#include <stdio.h>
|
||||
#include <stdlib.h>
|
||||
#include <string.h>
|
||||
#include <sys/mman.h>
|
||||
#include <sys/resource.h>
|
||||
#include <sys/stat.h>
|
||||
#include <sys/syscall.h>
|
||||
#include <sys/time.h>
|
||||
#include <sys/types.h>
|
||||
#include <unistd.h>
|
||||
#include "config.h"
|
||||
|
@ -360,6 +362,71 @@ Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_sync_1file_1range(
|
|||
#endif
|
||||
}
|
||||
|
||||
#define CHECK_DIRECT_BUFFER_ADDRESS(buf) \
|
||||
{ \
|
||||
if (!buf) { \
|
||||
THROW(env, "java/lang/UnsupportedOperationException", \
|
||||
"JNI access to direct buffers not available"); \
|
||||
return; \
|
||||
} \
|
||||
}
|
||||
|
||||
/**
|
||||
* public static native void mlock_native(
|
||||
* ByteBuffer buffer, long offset);
|
||||
*
|
||||
* The "00024" in the function name is an artifact of how JNI encodes
|
||||
* special characters. U+0024 is '$'.
|
||||
*/
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_mlock_1native(
|
||||
JNIEnv *env, jclass clazz,
|
||||
jobject buffer, jlong len)
|
||||
{
|
||||
#ifdef UNIX
|
||||
void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
|
||||
PASS_EXCEPTIONS(env);
|
||||
|
||||
if (mlock(buf, len)) {
|
||||
CHECK_DIRECT_BUFFER_ADDRESS(buf);
|
||||
throw_ioe(env, errno);
|
||||
}
|
||||
#endif
|
||||
|
||||
#ifdef WINDOWS
|
||||
THROW(env, "java/io/IOException",
|
||||
"The function POSIX.mlock_native() is not supported on Windows");
|
||||
#endif
|
||||
}
|
||||
|
||||
/**
|
||||
* public static native void munlock_native(
|
||||
* ByteBuffer buffer, long offset);
|
||||
*
|
||||
* The "00024" in the function name is an artifact of how JNI encodes
|
||||
* special characters. U+0024 is '$'.
|
||||
*/
|
||||
JNIEXPORT void JNICALL
|
||||
Java_org_apache_hadoop_io_nativeio_NativeIO_00024POSIX_munlock_1native(
|
||||
JNIEnv *env, jclass clazz,
|
||||
jobject buffer, jlong len)
|
||||
{
|
||||
#ifdef UNIX
|
||||
void* buf = (void*)(*env)->GetDirectBufferAddress(env, buffer);
|
||||
PASS_EXCEPTIONS(env);
|
||||
|
||||
if (munlock(buf, len)) {
|
||||
CHECK_DIRECT_BUFFER_ADDRESS(buf);
|
||||
throw_ioe(env, errno);
|
||||
}
|
||||
#endif
|
||||
|
||||
#ifdef WINDOWS
|
||||
THROW(env, "java/io/IOException",
|
||||
"The function POSIX.munlock_native() is not supported on Windows");
|
||||
#endif
|
||||
}
|
||||
|
||||
#ifdef __FreeBSD__
|
||||
static int toFreeBSDFlags(int flags)
|
||||
{
|
||||
|
@ -924,6 +991,24 @@ done:
|
|||
#endif
|
||||
}
|
||||
|
||||
JNIEXPORT jlong JNICALL
|
||||
Java_org_apache_hadoop_io_nativeio_NativeIO_getMemlockLimit0(
|
||||
JNIEnv *env, jclass clazz)
|
||||
{
|
||||
#ifdef WINDOWS
|
||||
return 0;
|
||||
#else
|
||||
struct rlimit rlim;
|
||||
int rc = getrlimit(RLIMIT_MEMLOCK, &rlim);
|
||||
if (rc != 0) {
|
||||
throw_ioe(env, errno);
|
||||
return 0;
|
||||
}
|
||||
return (rlim.rlim_cur == RLIM_INFINITY) ?
|
||||
INT64_MAX : rlim.rlim_cur;
|
||||
#endif
|
||||
}
|
||||
|
||||
/**
|
||||
* vim: sw=2: ts=2: et:
|
||||
*/
|
||||
|
|
|
@ -24,6 +24,9 @@ import java.io.FileOutputStream;
|
|||
import java.io.FileReader;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.nio.MappedByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.channels.FileChannel.MapMode;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -32,6 +35,7 @@ import java.util.List;
|
|||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assume.*;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
|
@ -45,6 +49,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.NativeCodeLoader;
|
||||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
public class TestNativeIO {
|
||||
|
@ -563,4 +568,60 @@ public class TestNativeIO {
|
|||
|
||||
FileUtils.deleteQuietly(TEST_DIR);
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testMlock() throws Exception {
|
||||
assumeTrue(NativeIO.isAvailable());
|
||||
assumeTrue(Shell.LINUX);
|
||||
final File TEST_FILE = new File(new File(
|
||||
System.getProperty("test.build.data","build/test/data")),
|
||||
"testMlockFile");
|
||||
final int BUF_LEN = 12289;
|
||||
byte buf[] = new byte[BUF_LEN];
|
||||
int bufSum = 0;
|
||||
for (int i = 0; i < buf.length; i++) {
|
||||
buf[i] = (byte)(i % 60);
|
||||
bufSum += buf[i];
|
||||
}
|
||||
FileOutputStream fos = new FileOutputStream(TEST_FILE);
|
||||
try {
|
||||
fos.write(buf);
|
||||
fos.getChannel().force(true);
|
||||
} finally {
|
||||
fos.close();
|
||||
}
|
||||
|
||||
FileInputStream fis = null;
|
||||
FileChannel channel = null;
|
||||
try {
|
||||
// Map file into memory
|
||||
fis = new FileInputStream(TEST_FILE);
|
||||
channel = fis.getChannel();
|
||||
long fileSize = channel.size();
|
||||
MappedByteBuffer mapbuf = channel.map(MapMode.READ_ONLY, 0, fileSize);
|
||||
// mlock the buffer
|
||||
NativeIO.POSIX.mlock(mapbuf, fileSize);
|
||||
// Read the buffer
|
||||
int sum = 0;
|
||||
for (int i=0; i<fileSize; i++) {
|
||||
sum += mapbuf.get(i);
|
||||
}
|
||||
assertEquals("Expected sums to be equal", bufSum, sum);
|
||||
// munlock the buffer
|
||||
NativeIO.POSIX.munlock(mapbuf, fileSize);
|
||||
} finally {
|
||||
if (channel != null) {
|
||||
channel.close();
|
||||
}
|
||||
if (fis != null) {
|
||||
fis.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testGetMemlockLimit() throws Exception {
|
||||
assumeTrue(NativeIO.isAvailable());
|
||||
NativeIO.getMemlockLimit();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,110 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.util;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
/** Testing {@link LightWeightGSet} */
|
||||
public class TestLightWeightGSet {
|
||||
public static final Log LOG = LogFactory.getLog(TestLightWeightGSet.class);
|
||||
|
||||
private static ArrayList<Integer> getRandomList(int length, int randomSeed) {
|
||||
Random random = new Random(randomSeed);
|
||||
ArrayList<Integer> list = new ArrayList<Integer>(length);
|
||||
for (int i = 0; i < length; i++) {
|
||||
list.add(random.nextInt());
|
||||
}
|
||||
return list;
|
||||
}
|
||||
|
||||
private static class TestElement implements LightWeightGSet.LinkedElement {
|
||||
private final int val;
|
||||
private LinkedElement next;
|
||||
|
||||
TestElement(int val) {
|
||||
this.val = val;
|
||||
this.next = null;
|
||||
}
|
||||
|
||||
public int getVal() {
|
||||
return val;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNext(LinkedElement next) {
|
||||
this.next = next;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LinkedElement getNext() {
|
||||
return next;
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testRemoveAllViaIterator() {
|
||||
ArrayList<Integer> list = getRandomList(100, 123);
|
||||
LightWeightGSet<TestElement, TestElement> set =
|
||||
new LightWeightGSet<TestElement, TestElement>(16);
|
||||
for (Integer i : list) {
|
||||
set.put(new TestElement(i));
|
||||
}
|
||||
for (Iterator<TestElement> iter = set.iterator();
|
||||
iter.hasNext(); ) {
|
||||
iter.next();
|
||||
iter.remove();
|
||||
}
|
||||
Assert.assertEquals(0, set.size());
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testRemoveSomeViaIterator() {
|
||||
ArrayList<Integer> list = getRandomList(100, 123);
|
||||
LightWeightGSet<TestElement, TestElement> set =
|
||||
new LightWeightGSet<TestElement, TestElement>(16);
|
||||
for (Integer i : list) {
|
||||
set.put(new TestElement(i));
|
||||
}
|
||||
long sum = 0;
|
||||
for (Iterator<TestElement> iter = set.iterator();
|
||||
iter.hasNext(); ) {
|
||||
sum += iter.next().getVal();
|
||||
}
|
||||
long mode = sum / set.size();
|
||||
LOG.info("Removing all elements above " + mode);
|
||||
for (Iterator<TestElement> iter = set.iterator();
|
||||
iter.hasNext(); ) {
|
||||
int item = iter.next().getVal();
|
||||
if (item > mode) {
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
for (Iterator<TestElement> iter = set.iterator();
|
||||
iter.hasNext(); ) {
|
||||
Assert.assertTrue(iter.next().getVal() <= mode);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -123,7 +123,73 @@ Trunk (Unreleased)
|
|||
HDFS-5041. Add the time of last heartbeat to dead server Web UI (Shinichi
|
||||
Yamashita via brandonli)
|
||||
|
||||
HDFS-5049. Add JNI mlock support. (Andrew Wang via Colin Patrick McCabe)
|
||||
|
||||
HDFS-5051. Propagate cache status information from the DataNode to the
|
||||
NameNode (Andrew Wang via Colin Patrick McCabe)
|
||||
|
||||
HDFS-5052. Add cacheRequest/uncacheRequest support to NameNode.
|
||||
(contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5050. Add DataNode support for mlock and munlock
|
||||
(Andrew Wang via Colin Patrick McCabe)
|
||||
|
||||
HDFS-5141. Add cache status information to datanode heartbeat.
|
||||
(Contributed by Andrew Wang)
|
||||
|
||||
HDFS-5121. Add RPCs for creating and manipulating cache pools.
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5163. Miscellaneous cache pool RPC fixes. (Contributed by Colin
|
||||
Patrick McCabe)
|
||||
|
||||
HDFS-5120. Add command-line support for manipulating cache pools.
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5158. Add command-line support for manipulating cache directives.
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5053. NameNode should invoke DataNode APIs to coordinate caching.
|
||||
(Andrew Wang)
|
||||
|
||||
HDFS-5197. Document dfs.cachereport.intervalMsec in hdfs-default.xml.
|
||||
(cnauroth)
|
||||
|
||||
HDFS-5213. Separate PathBasedCacheEntry and PathBasedCacheDirectiveWithId.
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5236. Change PathBasedCacheDirective APIs to be a single value
|
||||
rather than batch. (Contributed by Andrew Wang)
|
||||
|
||||
HDFS-5191. Revisit zero-copy API in FSDataInputStream to make it more
|
||||
intuitive. (Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5119. Persist CacheManager state in the edit log.
|
||||
(Contributed by Andrew Wang)
|
||||
|
||||
HDFS-5190. Move cache pool related CLI commands to CacheAdmin.
|
||||
(Contributed by Andrew Wang)
|
||||
|
||||
HDFS-5304. Expose if a block replica is cached in getFileBlockLocations.
|
||||
(Contributed by Andrew Wang)
|
||||
|
||||
HDFS-5224. Refactor PathBasedCache* methods to use a Path rather than a
|
||||
String. (cnauroth)
|
||||
|
||||
HDFS-5358. Add replication field to PathBasedCacheDirective.
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5359. Allow LightWeightGSet#Iterator to remove elements.
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5096. Automatically cache new data added to a cached path.
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5378. In CacheReport, don't send genstamp and length on the wire
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
OPTIMIZATIONS
|
||||
HDFS-5349. DNA_CACHE and DNA_UNCACHE should be by blockId only. (cmccabe)
|
||||
|
||||
BUG FIXES
|
||||
HADOOP-9635 Fix potential Stack Overflow in DomainSocket.c (V. Karthik Kumar
|
||||
|
@ -241,6 +307,53 @@ Trunk (Unreleased)
|
|||
HDFS-4366. Block Replication Policy Implementation May Skip Higher-Priority
|
||||
Blocks for Lower-Priority Blocks (Derek Dagit via kihwal)
|
||||
|
||||
HDFS-5169. hdfs.c: translateZCRException: null pointer deref when
|
||||
translating some exceptions. (Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5198. NameNodeRpcServer must not send back DNA_FINALIZE in reply to a
|
||||
cache report. (Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5195. Prevent passing null pointer to mlock and munlock. (cnauroth)
|
||||
|
||||
HDFS-5201. NativeIO: consolidate getrlimit into NativeIO#getMemlockLimit
|
||||
(Contributed by Colin Patrick McCabe)
|
||||
|
||||
HDFS-5210. Fix some failing unit tests on HDFS-4949 branch.
|
||||
(Contributed by Andrew Wang)
|
||||
|
||||
HDFS-5266. ElasticByteBufferPool#Key does not implement equals. (cnauroth)
|
||||
|
||||
HDFS-5309. Fix failing caching unit tests. (Andrew Wang)
|
||||
|
||||
HDFS-5314. Do not expose CachePool type in AddCachePoolOp (Colin Patrick
|
||||
McCabe)
|
||||
|
||||
HDFS-5348. Fix error message when dfs.datanode.max.locked.memory is
|
||||
improperly configured. (Colin Patrick McCabe)
|
||||
|
||||
HDFS-5373. hdfs cacheadmin -addDirective short usage does not mention
|
||||
-replication parameter. (cnauroth)
|
||||
|
||||
HDFS-5383. fix broken caching unit tests. (Andrew Wang)
|
||||
|
||||
HDFS-5388. Loading fsimage fails to find cache pools during namenode
|
||||
startup. (Chris Nauroth via Colin Patrick McCabe)
|
||||
|
||||
HDFS-5203. Concurrent clients that add a cache directive on the same path
|
||||
may prematurely uncache from each other. (Chris Nauroth via Colin Patrick
|
||||
McCabe)
|
||||
|
||||
HDFS-5385. Caching RPCs are AtMostOnce, but do not persist client ID and
|
||||
call ID to edit log. (Chris Nauroth via Colin Patrick McCabe)
|
||||
|
||||
HDFS-5404. Resolve regressions in Windows compatibility on HDFS-4949
|
||||
branch. (Chris Nauroth via Andrew Wang)
|
||||
|
||||
HDFS-5405. Fix possible RetryCache hang for caching RPC handlers in
|
||||
FSNamesystem. (wang)
|
||||
|
||||
HDFS-5419. Fixup test-patch.sh warnings on HDFS-4949 branch. (wang)
|
||||
|
||||
Release 2.3.0 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -346,4 +346,22 @@
|
|||
<Method name="create" />
|
||||
<Bug pattern="UL_UNRELEASED_LOCK" />
|
||||
</Match>
|
||||
<!-- Manually verified to be okay, we want to throw away the top bit here -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.hdfs.server.namenode.CachedBlock" />
|
||||
<Method name="getReplication" />
|
||||
<Bug pattern="ICAST_QUESTIONABLE_UNSIGNED_RIGHT_SHIFT" />
|
||||
</Match>
|
||||
<!-- These two are used for shutting down and kicking the CRMon, do not need strong sync -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
|
||||
<Field name="shutdown" />
|
||||
<Bug pattern="IS2_INCONSISTENT_SYNC" />
|
||||
</Match>
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor" />
|
||||
<Field name="rescanImmediately" />
|
||||
<Bug pattern="IS2_INCONSISTENT_SYNC" />
|
||||
</Match>
|
||||
|
||||
</FindBugsFilter>
|
||||
|
|
|
@ -59,6 +59,7 @@ function print_usage(){
|
|||
echo " Use -help to see options"
|
||||
echo " portmap run a portmap service"
|
||||
echo " nfs3 run an NFS version 3 gateway"
|
||||
echo " cacheadmin configure the HDFS cache"
|
||||
echo ""
|
||||
echo "Most commands print help when invoked w/o parameters."
|
||||
}
|
||||
|
@ -155,6 +156,8 @@ elif [ "$COMMAND" = "portmap" ] ; then
|
|||
CLASS=org.apache.hadoop.portmap.Portmap
|
||||
elif [ "$COMMAND" = "nfs3" ] ; then
|
||||
CLASS=org.apache.hadoop.hdfs.nfs.nfs3.Nfs3
|
||||
elif [ "$COMMAND" = "cacheadmin" ] ; then
|
||||
CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin
|
||||
else
|
||||
CLASS="$COMMAND"
|
||||
fi
|
||||
|
|
|
@ -98,6 +98,7 @@ import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
|
|||
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
|
||||
import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
|
||||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.Options.ChecksumOpt;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -107,6 +108,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.hdfs.client.ClientMmapManager;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
|
||||
|
@ -115,6 +117,8 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
|
@ -2287,6 +2291,72 @@ public class DFSClient implements java.io.Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
public PathBasedCacheDescriptor addPathBasedCacheDirective(
|
||||
PathBasedCacheDirective directive) throws IOException {
|
||||
checkOpen();
|
||||
try {
|
||||
return namenode.addPathBasedCacheDirective(directive);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException();
|
||||
}
|
||||
}
|
||||
|
||||
public void removePathBasedCacheDescriptor(long id)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
try {
|
||||
namenode.removePathBasedCacheDescriptor(id);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException();
|
||||
}
|
||||
}
|
||||
|
||||
public RemoteIterator<PathBasedCacheDescriptor> listPathBasedCacheDescriptors(
|
||||
String pool, String path) throws IOException {
|
||||
checkOpen();
|
||||
try {
|
||||
return namenode.listPathBasedCacheDescriptors(0, pool, path);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException();
|
||||
}
|
||||
}
|
||||
|
||||
public void addCachePool(CachePoolInfo info) throws IOException {
|
||||
checkOpen();
|
||||
try {
|
||||
namenode.addCachePool(info);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException();
|
||||
}
|
||||
}
|
||||
|
||||
public void modifyCachePool(CachePoolInfo info) throws IOException {
|
||||
checkOpen();
|
||||
try {
|
||||
namenode.modifyCachePool(info);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException();
|
||||
}
|
||||
}
|
||||
|
||||
public void removeCachePool(String poolName) throws IOException {
|
||||
checkOpen();
|
||||
try {
|
||||
namenode.removeCachePool(poolName);
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException();
|
||||
}
|
||||
}
|
||||
|
||||
public RemoteIterator<CachePoolInfo> listCachePools() throws IOException {
|
||||
checkOpen();
|
||||
try {
|
||||
return namenode.listCachePools("");
|
||||
} catch (RemoteException re) {
|
||||
throw re.unwrapRemoteException();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Save namespace image.
|
||||
*
|
||||
|
|
|
@ -99,6 +99,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT = false;
|
||||
public static final String DFS_DATANODE_USE_DN_HOSTNAME = "dfs.datanode.use.datanode.hostname";
|
||||
public static final boolean DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT = false;
|
||||
public static final String DFS_DATANODE_MAX_LOCKED_MEMORY_KEY = "dfs.datanode.max.locked.memory";
|
||||
public static final long DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT = 0;
|
||||
public static final String DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY = "dfs.datanode.fsdatasetcache.max.threads.per.volume";
|
||||
public static final int DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT = 4;
|
||||
public static final String DFS_NAMENODE_CACHING_ENABLED_KEY = "dfs.namenode.caching.enabled";
|
||||
public static final boolean DFS_NAMENODE_CACHING_ENABLED_DEFAULT = false;
|
||||
|
||||
public static final String DFS_NAMENODE_HTTP_PORT_KEY = "dfs.http.port";
|
||||
public static final int DFS_NAMENODE_HTTP_PORT_DEFAULT = 50070;
|
||||
|
@ -197,6 +203,16 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final String DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_KEY = "dfs.namenode.datanode.registration.ip-hostname-check";
|
||||
public static final boolean DFS_NAMENODE_DATANODE_REGISTRATION_IP_HOSTNAME_CHECK_DEFAULT = true;
|
||||
|
||||
public static final String DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES =
|
||||
"dfs.namenode.list.cache.pools.num.responses";
|
||||
public static final int DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT = 100;
|
||||
public static final String DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES =
|
||||
"dfs.namenode.list.cache.descriptors.num.responses";
|
||||
public static final int DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES_DEFAULT = 100;
|
||||
public static final String DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS =
|
||||
"dfs.namenode.path.based.cache.refresh.interval.ms";
|
||||
public static final long DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT = 300000L;
|
||||
|
||||
// Whether to enable datanode's stale state detection and usage for reads
|
||||
public static final String DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_KEY = "dfs.namenode.avoid.read.stale.datanode";
|
||||
public static final boolean DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_READ_DEFAULT = false;
|
||||
|
@ -362,6 +378,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
public static final long DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT = 60 * 60 * 1000;
|
||||
public static final String DFS_BLOCKREPORT_INITIAL_DELAY_KEY = "dfs.blockreport.initialDelay";
|
||||
public static final int DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT = 0;
|
||||
public static final String DFS_CACHEREPORT_INTERVAL_MSEC_KEY = "dfs.cachereport.intervalMsec";
|
||||
public static final long DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT = 10 * 1000;
|
||||
public static final String DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";
|
||||
public static final int DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT = 1000;
|
||||
public static final String DFS_DEFAULT_MAX_CORRUPT_FILES_RETURNED_KEY = "dfs.corruptfilesreturned.max";
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.hdfs.client.HdfsAdmin;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
@ -66,6 +67,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
||||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
||||
|
@ -1580,4 +1583,112 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}.resolve(this, absF);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new PathBasedCacheDirective.
|
||||
*
|
||||
* @param directive A PathBasedCacheDirectives to add
|
||||
* @return PathBasedCacheDescriptor associated with the added directive
|
||||
* @throws IOException if the directive could not be added
|
||||
*/
|
||||
public PathBasedCacheDescriptor addPathBasedCacheDirective(
|
||||
PathBasedCacheDirective directive) throws IOException {
|
||||
Path path = new Path(getPathName(fixRelativePart(directive.getPath()))).
|
||||
makeQualified(getUri(), getWorkingDirectory());
|
||||
return dfs.addPathBasedCacheDirective(new PathBasedCacheDirective.Builder().
|
||||
setPath(path).
|
||||
setReplication(directive.getReplication()).
|
||||
setPool(directive.getPool()).
|
||||
build());
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a PathBasedCacheDescriptor.
|
||||
*
|
||||
* @param descriptor PathBasedCacheDescriptor to remove
|
||||
* @throws IOException if the descriptor could not be removed
|
||||
*/
|
||||
public void removePathBasedCacheDescriptor(PathBasedCacheDescriptor descriptor)
|
||||
throws IOException {
|
||||
dfs.removePathBasedCacheDescriptor(descriptor.getEntryId());
|
||||
}
|
||||
|
||||
/**
|
||||
* List the set of cached paths of a cache pool. Incrementally fetches results
|
||||
* from the server.
|
||||
*
|
||||
* @param pool The cache pool to list, or null to list all pools.
|
||||
* @param path The path name to list, or null to list all paths.
|
||||
* @return A RemoteIterator which returns PathBasedCacheDescriptor objects.
|
||||
*/
|
||||
public RemoteIterator<PathBasedCacheDescriptor> listPathBasedCacheDescriptors(
|
||||
String pool, final Path path) throws IOException {
|
||||
String pathName = path != null ? getPathName(fixRelativePart(path)) : null;
|
||||
final RemoteIterator<PathBasedCacheDescriptor> iter =
|
||||
dfs.listPathBasedCacheDescriptors(pool, pathName);
|
||||
return new RemoteIterator<PathBasedCacheDescriptor>() {
|
||||
@Override
|
||||
public boolean hasNext() throws IOException {
|
||||
return iter.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PathBasedCacheDescriptor next() throws IOException {
|
||||
PathBasedCacheDescriptor desc = iter.next();
|
||||
Path qualPath = desc.getPath().makeQualified(getUri(), path);
|
||||
return new PathBasedCacheDescriptor(desc.getEntryId(), qualPath,
|
||||
desc.getReplication(), desc.getPool());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a cache pool.
|
||||
*
|
||||
* @param info
|
||||
* The request to add a cache pool.
|
||||
* @throws IOException
|
||||
* If the request could not be completed.
|
||||
*/
|
||||
public void addCachePool(CachePoolInfo info) throws IOException {
|
||||
CachePoolInfo.validate(info);
|
||||
dfs.addCachePool(info);
|
||||
}
|
||||
|
||||
/**
|
||||
* Modify an existing cache pool.
|
||||
*
|
||||
* @param info
|
||||
* The request to modify a cache pool.
|
||||
* @throws IOException
|
||||
* If the request could not be completed.
|
||||
*/
|
||||
public void modifyCachePool(CachePoolInfo info) throws IOException {
|
||||
CachePoolInfo.validate(info);
|
||||
dfs.modifyCachePool(info);
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a cache pool.
|
||||
*
|
||||
* @param poolName
|
||||
* Name of the cache pool to remove.
|
||||
* @throws IOException
|
||||
* if the cache pool did not exist, or could not be removed.
|
||||
*/
|
||||
public void removeCachePool(String poolName) throws IOException {
|
||||
CachePoolInfo.validateName(poolName);
|
||||
dfs.removeCachePool(poolName);
|
||||
}
|
||||
|
||||
/**
|
||||
* List all cache pools.
|
||||
*
|
||||
* @return A remote iterator from which you can get CachePoolInfo objects.
|
||||
* Requests will be made as needed.
|
||||
* @throws IOException
|
||||
* If there was an error listing cache pools.
|
||||
*/
|
||||
public RemoteIterator<CachePoolInfo> listCachePools() throws IOException {
|
||||
return dfs.listCachePools();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,108 @@
|
|||
/**
|
||||
* 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.protocol;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An exception which occurred when trying to add a PathBasedCache directive.
|
||||
*/
|
||||
public abstract class AddPathBasedCacheDirectiveException extends IOException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public AddPathBasedCacheDirectiveException(String description) {
|
||||
super(description);
|
||||
}
|
||||
|
||||
public static final class EmptyPathError
|
||||
extends AddPathBasedCacheDirectiveException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public EmptyPathError() {
|
||||
super("empty path in directive");
|
||||
}
|
||||
}
|
||||
|
||||
public static class InvalidPathNameError
|
||||
extends AddPathBasedCacheDirectiveException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public InvalidPathNameError(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public InvalidPathNameError(PathBasedCacheDirective directive) {
|
||||
this("can't handle invalid path name " + directive.getPath());
|
||||
}
|
||||
}
|
||||
|
||||
public static class InvalidPoolNameError
|
||||
extends AddPathBasedCacheDirectiveException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public InvalidPoolNameError(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public InvalidPoolNameError(PathBasedCacheDirective directive) {
|
||||
this("invalid pool name '" + directive.getPool() + "'");
|
||||
}
|
||||
}
|
||||
|
||||
public static class PoolWritePermissionDeniedError
|
||||
extends AddPathBasedCacheDirectiveException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public PoolWritePermissionDeniedError(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public PoolWritePermissionDeniedError(PathBasedCacheDirective directive) {
|
||||
this("write permission denied for pool '" + directive.getPool() + "'");
|
||||
}
|
||||
}
|
||||
|
||||
public static class PathAlreadyExistsInPoolError
|
||||
extends AddPathBasedCacheDirectiveException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public PathAlreadyExistsInPoolError(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public PathAlreadyExistsInPoolError(PathBasedCacheDirective directive) {
|
||||
this("path " + directive.getPath() + " already exists in pool " +
|
||||
directive.getPool());
|
||||
}
|
||||
}
|
||||
|
||||
public static class UnexpectedAddPathBasedCacheDirectiveException
|
||||
extends AddPathBasedCacheDirectiveException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public UnexpectedAddPathBasedCacheDirectiveException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public UnexpectedAddPathBasedCacheDirectiveException(
|
||||
PathBasedCacheDirective directive) {
|
||||
this("encountered an unexpected error when trying to " +
|
||||
"add PathBasedCache directive " + directive);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,228 @@
|
|||
/**
|
||||
* 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.protocol;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import org.apache.commons.lang.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang.builder.HashCodeBuilder;
|
||||
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.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.fs.permission.PermissionStatus;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
|
||||
import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.xml.sax.ContentHandler;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
/**
|
||||
* CachePoolInfo describes a cache pool.
|
||||
*
|
||||
* This class is used in RPCs to create and modify cache pools.
|
||||
* It is serializable and can be stored in the edit log.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class CachePoolInfo {
|
||||
public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
|
||||
|
||||
final String poolName;
|
||||
|
||||
@Nullable
|
||||
String ownerName;
|
||||
|
||||
@Nullable
|
||||
String groupName;
|
||||
|
||||
@Nullable
|
||||
FsPermission mode;
|
||||
|
||||
@Nullable
|
||||
Integer weight;
|
||||
|
||||
public CachePoolInfo(String poolName) {
|
||||
this.poolName = poolName;
|
||||
}
|
||||
|
||||
public String getPoolName() {
|
||||
return poolName;
|
||||
}
|
||||
|
||||
public String getOwnerName() {
|
||||
return ownerName;
|
||||
}
|
||||
|
||||
public CachePoolInfo setOwnerName(String ownerName) {
|
||||
this.ownerName = ownerName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String getGroupName() {
|
||||
return groupName;
|
||||
}
|
||||
|
||||
public CachePoolInfo setGroupName(String groupName) {
|
||||
this.groupName = groupName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public FsPermission getMode() {
|
||||
return mode;
|
||||
}
|
||||
|
||||
public CachePoolInfo setMode(FsPermission mode) {
|
||||
this.mode = mode;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Integer getWeight() {
|
||||
return weight;
|
||||
}
|
||||
|
||||
public CachePoolInfo setWeight(Integer weight) {
|
||||
this.weight = weight;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return new StringBuilder().append("{").
|
||||
append("poolName:").append(poolName).
|
||||
append(", ownerName:").append(ownerName).
|
||||
append(", groupName:").append(groupName).
|
||||
append(", mode:").append((mode == null) ? "null" :
|
||||
String.format("0%03o", mode.toShort())).
|
||||
append(", weight:").append(weight).
|
||||
append("}").toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null) { return false; }
|
||||
if (o == this) { return true; }
|
||||
if (o.getClass() != getClass()) {
|
||||
return false;
|
||||
}
|
||||
CachePoolInfo other = (CachePoolInfo)o;
|
||||
return new EqualsBuilder().
|
||||
append(poolName, other.poolName).
|
||||
append(ownerName, other.ownerName).
|
||||
append(groupName, other.groupName).
|
||||
append(mode, other.mode).
|
||||
append(weight, other.weight).
|
||||
isEquals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().
|
||||
append(poolName).
|
||||
append(ownerName).
|
||||
append(groupName).
|
||||
append(mode).
|
||||
append(weight).
|
||||
hashCode();
|
||||
}
|
||||
|
||||
public static void validate(CachePoolInfo info) throws IOException {
|
||||
if (info == null) {
|
||||
throw new IOException("CachePoolInfo is null");
|
||||
}
|
||||
validateName(info.poolName);
|
||||
}
|
||||
|
||||
public static void validateName(String poolName) throws IOException {
|
||||
if (poolName == null || poolName.isEmpty()) {
|
||||
// Empty pool names are not allowed because they would be highly
|
||||
// confusing. They would also break the ability to list all pools
|
||||
// by starting with prevKey = ""
|
||||
throw new IOException("invalid empty cache pool name");
|
||||
}
|
||||
}
|
||||
|
||||
public static CachePoolInfo readFrom(DataInput in) throws IOException {
|
||||
String poolName = Text.readString(in);
|
||||
CachePoolInfo info = new CachePoolInfo(poolName);
|
||||
if (in.readBoolean()) {
|
||||
info.setOwnerName(Text.readString(in));
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
info.setGroupName(Text.readString(in));
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
info.setMode(FsPermission.read(in));
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
info.setWeight(in.readInt());
|
||||
}
|
||||
return info;
|
||||
}
|
||||
|
||||
public void writeTo(DataOutput out) throws IOException {
|
||||
Text.writeString(out, poolName);
|
||||
boolean hasOwner, hasGroup, hasMode, hasWeight;
|
||||
hasOwner = ownerName != null;
|
||||
hasGroup = groupName != null;
|
||||
hasMode = mode != null;
|
||||
hasWeight = weight != null;
|
||||
out.writeBoolean(hasOwner);
|
||||
if (hasOwner) {
|
||||
Text.writeString(out, ownerName);
|
||||
}
|
||||
out.writeBoolean(hasGroup);
|
||||
if (hasGroup) {
|
||||
Text.writeString(out, groupName);
|
||||
}
|
||||
out.writeBoolean(hasMode);
|
||||
if (hasMode) {
|
||||
mode.write(out);
|
||||
}
|
||||
out.writeBoolean(hasWeight);
|
||||
if (hasWeight) {
|
||||
out.writeInt(weight);
|
||||
}
|
||||
}
|
||||
|
||||
public void writeXmlTo(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "POOLNAME", poolName);
|
||||
PermissionStatus perm = new PermissionStatus(ownerName,
|
||||
groupName, mode);
|
||||
FSEditLogOp.permissionStatusToXml(contentHandler, perm);
|
||||
XMLUtils.addSaxString(contentHandler, "WEIGHT", Integer.toString(weight));
|
||||
}
|
||||
|
||||
public static CachePoolInfo readXmlFrom(Stanza st) throws InvalidXmlException {
|
||||
String poolName = st.getValue("POOLNAME");
|
||||
PermissionStatus perm = FSEditLogOp.permissionStatusFromXml(st);
|
||||
int weight = Integer.parseInt(st.getValue("WEIGHT"));
|
||||
return new CachePoolInfo(poolName).
|
||||
setOwnerName(perm.getUserName()).
|
||||
setGroupName(perm.getGroupName()).
|
||||
setMode(perm.getPermission()).
|
||||
setWeight(weight);
|
||||
}
|
||||
}
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.fs.InvalidPathException;
|
|||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.Options.Rename;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
|
@ -1093,5 +1094,79 @@ public interface ClientProtocol {
|
|||
@Idempotent
|
||||
public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
|
||||
String fromSnapshot, String toSnapshot) throws IOException;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a PathBasedCache entry to the CacheManager.
|
||||
*
|
||||
* @param directive A PathBasedCacheDirective to be added
|
||||
* @return A PathBasedCacheDescriptor associated with the added directive
|
||||
* @throws IOException if the directive could not be added
|
||||
*/
|
||||
@AtMostOnce
|
||||
public PathBasedCacheDescriptor addPathBasedCacheDirective(
|
||||
PathBasedCacheDirective directive) throws IOException;
|
||||
|
||||
/**
|
||||
* Remove a PathBasedCacheDescriptor from the CacheManager.
|
||||
*
|
||||
* @param id of a PathBasedCacheDescriptor
|
||||
* @throws IOException if the cache descriptor could not be removed
|
||||
*/
|
||||
@AtMostOnce
|
||||
public void removePathBasedCacheDescriptor(Long id) throws IOException;
|
||||
|
||||
/**
|
||||
* List the set of cached paths of a cache pool. Incrementally fetches results
|
||||
* from the server.
|
||||
*
|
||||
* @param prevId The last listed entry ID, or -1 if this is the first call to
|
||||
* listPathBasedCacheDescriptors.
|
||||
* @param pool The cache pool to list, or null to list all pools.
|
||||
* @param path The path name to list, or null to list all paths.
|
||||
* @return A RemoteIterator which returns PathBasedCacheDescriptor objects.
|
||||
*/
|
||||
@Idempotent
|
||||
public RemoteIterator<PathBasedCacheDescriptor> listPathBasedCacheDescriptors(
|
||||
long prevId, String pool, String path) throws IOException;
|
||||
|
||||
/**
|
||||
* Add a new cache pool.
|
||||
*
|
||||
* @param info Description of the new cache pool
|
||||
* @throws IOException If the request could not be completed.
|
||||
*/
|
||||
@AtMostOnce
|
||||
public void addCachePool(CachePoolInfo info) throws IOException;
|
||||
|
||||
/**
|
||||
* Modify an existing cache pool.
|
||||
*
|
||||
* @param req
|
||||
* The request to modify a cache pool.
|
||||
* @throws IOException
|
||||
* If the request could not be completed.
|
||||
*/
|
||||
@AtMostOnce
|
||||
public void modifyCachePool(CachePoolInfo req) throws IOException;
|
||||
|
||||
/**
|
||||
* Remove a cache pool.
|
||||
*
|
||||
* @param pool name of the cache pool to remove.
|
||||
* @throws IOException if the cache pool did not exist, or could not be
|
||||
* removed.
|
||||
*/
|
||||
@AtMostOnce
|
||||
public void removeCachePool(String pool) throws IOException;
|
||||
|
||||
/**
|
||||
* List the set of cache pools. Incrementally fetches results from the server.
|
||||
*
|
||||
* @param prevPool name of the last pool listed, or the empty string if this is
|
||||
* the first invocation of listCachePools
|
||||
* @return A RemoteIterator which returns CachePool objects.
|
||||
*/
|
||||
@Idempotent
|
||||
public RemoteIterator<CachePoolInfo> listCachePools(String prevPool)
|
||||
throws IOException;
|
||||
}
|
||||
|
|
|
@ -44,6 +44,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
private long dfsUsed;
|
||||
private long remaining;
|
||||
private long blockPoolUsed;
|
||||
private long cacheCapacity;
|
||||
private long cacheUsed;
|
||||
private long lastUpdate;
|
||||
private int xceiverCount;
|
||||
private String location = NetworkTopology.DEFAULT_RACK;
|
||||
|
@ -82,6 +84,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
this.dfsUsed = from.getDfsUsed();
|
||||
this.remaining = from.getRemaining();
|
||||
this.blockPoolUsed = from.getBlockPoolUsed();
|
||||
this.cacheCapacity = from.getCacheCapacity();
|
||||
this.cacheUsed = from.getCacheUsed();
|
||||
this.lastUpdate = from.getLastUpdate();
|
||||
this.xceiverCount = from.getXceiverCount();
|
||||
this.location = from.getNetworkLocation();
|
||||
|
@ -94,6 +98,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
this.dfsUsed = 0L;
|
||||
this.remaining = 0L;
|
||||
this.blockPoolUsed = 0L;
|
||||
this.cacheCapacity = 0L;
|
||||
this.cacheUsed = 0L;
|
||||
this.lastUpdate = 0L;
|
||||
this.xceiverCount = 0;
|
||||
this.adminState = null;
|
||||
|
@ -106,12 +112,14 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
|
||||
public DatanodeInfo(DatanodeID nodeID, String location,
|
||||
final long capacity, final long dfsUsed, final long remaining,
|
||||
final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
|
||||
final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
|
||||
final long lastUpdate, final int xceiverCount,
|
||||
final AdminStates adminState) {
|
||||
this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getStorageID(),
|
||||
nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
|
||||
nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
|
||||
lastUpdate, xceiverCount, location, adminState);
|
||||
cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location,
|
||||
adminState);
|
||||
}
|
||||
|
||||
/** Constructor */
|
||||
|
@ -119,7 +127,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
final String storageID, final int xferPort, final int infoPort,
|
||||
final int infoSecurePort, final int ipcPort,
|
||||
final long capacity, final long dfsUsed, final long remaining,
|
||||
final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
|
||||
final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
|
||||
final long lastUpdate, final int xceiverCount,
|
||||
final String networkLocation, final AdminStates adminState) {
|
||||
super(ipAddr, hostName, storageID, xferPort, infoPort,
|
||||
infoSecurePort, ipcPort);
|
||||
|
@ -127,6 +136,8 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
this.dfsUsed = dfsUsed;
|
||||
this.remaining = remaining;
|
||||
this.blockPoolUsed = blockPoolUsed;
|
||||
this.cacheCapacity = cacheCapacity;
|
||||
this.cacheUsed = cacheUsed;
|
||||
this.lastUpdate = lastUpdate;
|
||||
this.xceiverCount = xceiverCount;
|
||||
this.location = networkLocation;
|
||||
|
@ -172,6 +183,42 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
return DFSUtil.getPercentRemaining(remaining, capacity);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Amount of cache capacity in bytes
|
||||
*/
|
||||
public long getCacheCapacity() {
|
||||
return cacheCapacity;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Amount of cache used in bytes
|
||||
*/
|
||||
public long getCacheUsed() {
|
||||
return cacheUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Cache used as a percentage of the datanode's total cache capacity
|
||||
*/
|
||||
public float getCacheUsedPercent() {
|
||||
return DFSUtil.getPercentUsed(cacheUsed, cacheCapacity);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Amount of cache remaining in bytes
|
||||
*/
|
||||
public long getCacheRemaining() {
|
||||
return cacheCapacity - cacheUsed;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Cache remaining as a percentage of the datanode's total cache
|
||||
* capacity
|
||||
*/
|
||||
public float getCacheRemainingPercent() {
|
||||
return DFSUtil.getPercentRemaining(getCacheRemaining(), cacheCapacity);
|
||||
}
|
||||
|
||||
/** The time when this information was accurate. */
|
||||
public long getLastUpdate() { return lastUpdate; }
|
||||
|
||||
|
@ -198,6 +245,16 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
this.blockPoolUsed = bpUsed;
|
||||
}
|
||||
|
||||
/** Sets cache capacity. */
|
||||
public void setCacheCapacity(long cacheCapacity) {
|
||||
this.cacheCapacity = cacheCapacity;
|
||||
}
|
||||
|
||||
/** Sets cache used. */
|
||||
public void setCacheUsed(long cacheUsed) {
|
||||
this.cacheUsed = cacheUsed;
|
||||
}
|
||||
|
||||
/** Sets time when this information was accurate. */
|
||||
public void setLastUpdate(long lastUpdate) {
|
||||
this.lastUpdate = lastUpdate;
|
||||
|
@ -227,6 +284,11 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
long nonDFSUsed = getNonDfsUsed();
|
||||
float usedPercent = getDfsUsedPercent();
|
||||
float remainingPercent = getRemainingPercent();
|
||||
long cc = getCacheCapacity();
|
||||
long cr = getCacheRemaining();
|
||||
long cu = getCacheUsed();
|
||||
float cacheUsedPercent = getCacheUsedPercent();
|
||||
float cacheRemainingPercent = getCacheRemainingPercent();
|
||||
String lookupName = NetUtils.getHostNameOfIP(getName());
|
||||
|
||||
buffer.append("Name: "+ getName());
|
||||
|
@ -253,6 +315,12 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
|
||||
buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
|
||||
buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
|
||||
buffer.append("Configured Cache Capacity: "+c+" ("+StringUtils.byteDesc(cc)+")"+"\n");
|
||||
buffer.append("Cache Used: "+cu+" ("+StringUtils.byteDesc(u)+")"+"\n");
|
||||
buffer.append("Cache Remaining: " +cr+ " ("+StringUtils.byteDesc(r)+")"+"\n");
|
||||
buffer.append("Cache Used%: "+percent2String(cacheUsedPercent) + "\n");
|
||||
buffer.append("Cache Remaining%: "+percent2String(cacheRemainingPercent) + "\n");
|
||||
|
||||
buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
|
||||
return buffer.toString();
|
||||
}
|
||||
|
@ -263,6 +331,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
long c = getCapacity();
|
||||
long r = getRemaining();
|
||||
long u = getDfsUsed();
|
||||
long cc = getCacheCapacity();
|
||||
long cr = getCacheRemaining();
|
||||
long cu = getCacheUsed();
|
||||
buffer.append(getName());
|
||||
if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
|
||||
buffer.append(" "+location);
|
||||
|
@ -278,6 +349,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|||
buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
|
||||
buffer.append(" " + percent2String(u/(double)c));
|
||||
buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
|
||||
buffer.append(" " + cc + "(" + StringUtils.byteDesc(cc)+")");
|
||||
buffer.append(" " + cu + "(" + StringUtils.byteDesc(cu)+")");
|
||||
buffer.append(" " + percent2String(cu/(double)cc));
|
||||
buffer.append(" " + cr + "(" + StringUtils.byteDesc(cr)+")");
|
||||
buffer.append(" " + new Date(lastUpdate));
|
||||
return buffer.toString();
|
||||
}
|
||||
|
|
|
@ -106,7 +106,8 @@ public class LayoutVersion {
|
|||
SEQUENTIAL_BLOCK_ID(-46, "Allocate block IDs sequentially and store " +
|
||||
"block IDs in the edits log and image files"),
|
||||
EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to "
|
||||
+ "enable rebuilding retry cache in case of HA failover");
|
||||
+ "enable rebuilding retry cache in case of HA failover"),
|
||||
CACHING(-48, "Support for cache pools and path-based caching");
|
||||
|
||||
final int lv;
|
||||
final int ancestorLV;
|
||||
|
|
|
@ -17,15 +17,21 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.protocol;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Associates a block with the Datanodes that contain its replicas
|
||||
* and other block metadata (E.g. the file offset associated with this
|
||||
* block, whether it is corrupt, security token, etc).
|
||||
* block, whether it is corrupt, a location is cached in memory,
|
||||
* security token, etc).
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
|
@ -39,9 +45,16 @@ public class LocatedBlock {
|
|||
// their locations are not part of this object
|
||||
private boolean corrupt;
|
||||
private Token<BlockTokenIdentifier> blockToken = new Token<BlockTokenIdentifier>();
|
||||
/**
|
||||
* List of cached datanode locations
|
||||
*/
|
||||
private DatanodeInfo[] cachedLocs;
|
||||
|
||||
// Used when there are no locations
|
||||
private static final DatanodeInfo[] EMPTY_LOCS = new DatanodeInfo[0];
|
||||
|
||||
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
|
||||
this(b, locs, -1, false); // startOffset is unknown
|
||||
this(b, locs, -1); // startOffset is unknown
|
||||
}
|
||||
|
||||
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset) {
|
||||
|
@ -50,14 +63,26 @@ public class LocatedBlock {
|
|||
|
||||
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
|
||||
boolean corrupt) {
|
||||
this(b, locs, startOffset, corrupt, EMPTY_LOCS);
|
||||
}
|
||||
|
||||
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
|
||||
boolean corrupt, DatanodeInfo[] cachedLocs) {
|
||||
this.b = b;
|
||||
this.offset = startOffset;
|
||||
this.corrupt = corrupt;
|
||||
if (locs==null) {
|
||||
this.locs = new DatanodeInfo[0];
|
||||
this.locs = EMPTY_LOCS;
|
||||
} else {
|
||||
this.locs = locs;
|
||||
}
|
||||
Preconditions.checkArgument(cachedLocs != null,
|
||||
"cachedLocs should not be null, use a different constructor");
|
||||
if (cachedLocs.length == 0) {
|
||||
this.cachedLocs = EMPTY_LOCS;
|
||||
} else {
|
||||
this.cachedLocs = cachedLocs;
|
||||
}
|
||||
}
|
||||
|
||||
public Token<BlockTokenIdentifier> getBlockToken() {
|
||||
|
@ -96,6 +121,36 @@ public class LocatedBlock {
|
|||
return this.corrupt;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a the location of a cached replica of the block.
|
||||
*
|
||||
* @param loc of datanode with the cached replica
|
||||
*/
|
||||
public void addCachedLoc(DatanodeInfo loc) {
|
||||
List<DatanodeInfo> cachedList = Lists.newArrayList(cachedLocs);
|
||||
if (cachedList.contains(loc)) {
|
||||
return;
|
||||
}
|
||||
// Try to re-use a DatanodeInfo already in loc
|
||||
for (int i=0; i<locs.length; i++) {
|
||||
if (locs[i].equals(loc)) {
|
||||
cachedList.add(locs[i]);
|
||||
cachedLocs = cachedList.toArray(cachedLocs);
|
||||
return;
|
||||
}
|
||||
}
|
||||
// Not present in loc, add it and go
|
||||
cachedList.add(loc);
|
||||
cachedLocs = cachedList.toArray(cachedLocs);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Datanodes with a cached block replica
|
||||
*/
|
||||
public DatanodeInfo[] getCachedLocations() {
|
||||
return cachedLocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return getClass().getSimpleName() + "{" + b
|
||||
|
|
|
@ -0,0 +1,82 @@
|
|||
/**
|
||||
* 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.protocol;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.commons.lang.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang.builder.HashCodeBuilder;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A directive in a cache pool that includes an identifying ID number.
|
||||
*/
|
||||
@InterfaceStability.Evolving
|
||||
@InterfaceAudience.Public
|
||||
public final class PathBasedCacheDescriptor extends PathBasedCacheDirective {
|
||||
private final long entryId;
|
||||
|
||||
public PathBasedCacheDescriptor(long entryId, Path path,
|
||||
short replication, String pool) {
|
||||
super(path, replication, pool);
|
||||
Preconditions.checkArgument(entryId > 0);
|
||||
this.entryId = entryId;
|
||||
}
|
||||
|
||||
public long getEntryId() {
|
||||
return entryId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
PathBasedCacheDescriptor other = (PathBasedCacheDescriptor)o;
|
||||
return new EqualsBuilder().append(entryId, other.entryId).
|
||||
append(getPath(), other.getPath()).
|
||||
append(getReplication(), other.getReplication()).
|
||||
append(getPool(), other.getPool()).
|
||||
isEquals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().append(entryId).
|
||||
append(getPath()).
|
||||
append(getReplication()).
|
||||
append(getPool()).
|
||||
hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("{ entryId:").append(entryId).
|
||||
append(", path:").append(getPath()).
|
||||
append(", replication:").append(getReplication()).
|
||||
append(", pool:").append(getPool()).
|
||||
append(" }");
|
||||
return builder.toString();
|
||||
}
|
||||
};
|
|
@ -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.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.protocol;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import org.apache.commons.lang.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang.builder.HashCodeBuilder;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPathNameError;
|
||||
|
||||
/**
|
||||
* A directive to add a path to a cache pool.
|
||||
*/
|
||||
@InterfaceStability.Evolving
|
||||
@InterfaceAudience.Public
|
||||
public class PathBasedCacheDirective {
|
||||
|
||||
/**
|
||||
* A builder for creating new PathBasedCacheDirective instances.
|
||||
*/
|
||||
public static class Builder {
|
||||
private Path path;
|
||||
private short replication = (short)1;
|
||||
private String pool;
|
||||
|
||||
/**
|
||||
* Builds a new PathBasedCacheDirective populated with the set properties.
|
||||
*
|
||||
* @return New PathBasedCacheDirective.
|
||||
*/
|
||||
public PathBasedCacheDirective build() {
|
||||
return new PathBasedCacheDirective(path, replication, pool);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the path used in this request.
|
||||
*
|
||||
* @param path The path used in this request.
|
||||
* @return This builder, for call chaining.
|
||||
*/
|
||||
public Builder setPath(Path path) {
|
||||
this.path = path;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the replication used in this request.
|
||||
*
|
||||
* @param replication The replication used in this request.
|
||||
* @return This builder, for call chaining.
|
||||
*/
|
||||
public Builder setReplication(short replication) {
|
||||
this.replication = replication;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the pool used in this request.
|
||||
*
|
||||
* @param pool The pool used in this request.
|
||||
* @return This builder, for call chaining.
|
||||
*/
|
||||
public Builder setPool(String pool) {
|
||||
this.pool = pool;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
private final Path path;
|
||||
private final short replication;
|
||||
private final String pool;
|
||||
|
||||
/**
|
||||
* @return The path used in this request.
|
||||
*/
|
||||
public Path getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The number of times the block should be cached.
|
||||
*/
|
||||
public short getReplication() {
|
||||
return replication;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The pool used in this request.
|
||||
*/
|
||||
public String getPool() {
|
||||
return pool;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if this PathBasedCacheDirective is valid.
|
||||
*
|
||||
* @throws IOException
|
||||
* If this PathBasedCacheDirective is not valid.
|
||||
*/
|
||||
public void validate() throws IOException {
|
||||
if (!DFSUtil.isValidName(path.toUri().getPath())) {
|
||||
throw new InvalidPathNameError(this);
|
||||
}
|
||||
if (replication <= 0) {
|
||||
throw new IOException("Tried to request a cache replication " +
|
||||
"factor of " + replication + ", but that is less than 1.");
|
||||
}
|
||||
if (pool.isEmpty()) {
|
||||
throw new InvalidPoolNameError(this);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null) {
|
||||
return false;
|
||||
}
|
||||
if (getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
PathBasedCacheDirective other = (PathBasedCacheDirective)o;
|
||||
return new EqualsBuilder().append(getPath(), other.getPath()).
|
||||
append(getReplication(), other.getReplication()).
|
||||
append(getPool(), other.getPool()).
|
||||
isEquals();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().append(getPath()).
|
||||
append(replication).
|
||||
append(getPool()).
|
||||
hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("{ path:").append(path).
|
||||
append(", replication:").append(replication).
|
||||
append(", pool:").append(pool).
|
||||
append(" }");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Protected constructor. Callers use Builder to create new instances.
|
||||
*
|
||||
* @param path The path used in this request.
|
||||
* @param replication The replication used in this request.
|
||||
* @param pool The pool used in this request.
|
||||
*/
|
||||
protected PathBasedCacheDirective(Path path, short replication, String pool) {
|
||||
Preconditions.checkNotNull(path);
|
||||
Preconditions.checkNotNull(pool);
|
||||
this.path = path;
|
||||
this.replication = replication;
|
||||
this.pool = pool;
|
||||
}
|
||||
};
|
|
@ -0,0 +1,98 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.protocol;
|
||||
|
||||
import org.apache.commons.lang.builder.HashCodeBuilder;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CachePool;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Represents an entry in the PathBasedCache on the NameNode.
|
||||
*
|
||||
* This is an implementation class, not part of the public API.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class PathBasedCacheEntry {
|
||||
private final long entryId;
|
||||
private final String path;
|
||||
private final short replication;
|
||||
private final CachePool pool;
|
||||
|
||||
public PathBasedCacheEntry(long entryId, String path,
|
||||
short replication, CachePool pool) {
|
||||
Preconditions.checkArgument(entryId > 0);
|
||||
this.entryId = entryId;
|
||||
Preconditions.checkArgument(replication > 0);
|
||||
this.path = path;
|
||||
Preconditions.checkNotNull(pool);
|
||||
this.replication = replication;
|
||||
Preconditions.checkNotNull(path);
|
||||
this.pool = pool;
|
||||
}
|
||||
|
||||
public long getEntryId() {
|
||||
return entryId;
|
||||
}
|
||||
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
public CachePool getPool() {
|
||||
return pool;
|
||||
}
|
||||
|
||||
public short getReplication() {
|
||||
return replication;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("{ entryId:").append(entryId).
|
||||
append(", path:").append(path).
|
||||
append(", replication:").append(replication).
|
||||
append(", pool:").append(pool).
|
||||
append(" }");
|
||||
return builder.toString();
|
||||
}
|
||||
|
||||
public PathBasedCacheDescriptor getDescriptor() {
|
||||
return new PathBasedCacheDescriptor(entryId, new Path(path), replication,
|
||||
pool.getPoolName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null) { return false; }
|
||||
if (o == this) { return true; }
|
||||
if (o.getClass() != this.getClass()) {
|
||||
return false;
|
||||
}
|
||||
PathBasedCacheEntry other = (PathBasedCacheEntry)o;
|
||||
return entryId == other.entryId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return new HashCodeBuilder().append(entryId).toHashCode();
|
||||
}
|
||||
};
|
|
@ -0,0 +1,85 @@
|
|||
/**
|
||||
* 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.protocol;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* An exception which occurred when trying to remove a PathBasedCache entry.
|
||||
*/
|
||||
public abstract class RemovePathBasedCacheDescriptorException extends IOException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public RemovePathBasedCacheDescriptorException(String description) {
|
||||
super(description);
|
||||
}
|
||||
|
||||
public final static class InvalidIdException
|
||||
extends RemovePathBasedCacheDescriptorException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public InvalidIdException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public InvalidIdException(long entryId) {
|
||||
this("invalid PathBasedCacheDescriptor id " + entryId);
|
||||
}
|
||||
}
|
||||
|
||||
public final static class RemovePermissionDeniedException
|
||||
extends RemovePathBasedCacheDescriptorException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public RemovePermissionDeniedException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public RemovePermissionDeniedException(long entryId) {
|
||||
this("permission denied when trying to remove " +
|
||||
"PathBasedCacheDescriptor id " + entryId);
|
||||
}
|
||||
}
|
||||
|
||||
public final static class NoSuchIdException
|
||||
extends RemovePathBasedCacheDescriptorException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public NoSuchIdException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public NoSuchIdException(long entryId) {
|
||||
this("there is no PathBasedCacheDescriptor with id " + entryId);
|
||||
}
|
||||
}
|
||||
|
||||
public final static class UnexpectedRemovePathBasedCacheDescriptorException
|
||||
extends RemovePathBasedCacheDescriptorException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public UnexpectedRemovePathBasedCacheDescriptorException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public UnexpectedRemovePathBasedCacheDescriptorException(long id) {
|
||||
this("encountered an unexpected error when trying to " +
|
||||
"remove PathBasedCacheDescriptor with id " + id);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/**
|
||||
* 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.protocol;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* An exception which occurred when trying to remove a PathBasedCache entry.
|
||||
*/
|
||||
public abstract class RemovePathBasedCacheEntryException extends IOException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final long entryId;
|
||||
|
||||
public RemovePathBasedCacheEntryException(String description, long entryId) {
|
||||
super(description);
|
||||
this.entryId = entryId;
|
||||
}
|
||||
|
||||
public long getEntryId() {
|
||||
return this.entryId;
|
||||
}
|
||||
|
||||
public final static class InvalidIdException
|
||||
extends RemovePathBasedCacheEntryException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public InvalidIdException(long entryId) {
|
||||
super("invalid cache path entry id " + entryId, entryId);
|
||||
}
|
||||
}
|
||||
|
||||
public final static class RemovePermissionDeniedException
|
||||
extends RemovePathBasedCacheEntryException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public RemovePermissionDeniedException(long entryId) {
|
||||
super("permission denied when trying to remove PathBasedCache entry id " +
|
||||
entryId, entryId);
|
||||
}
|
||||
}
|
||||
|
||||
public final static class NoSuchIdException
|
||||
extends RemovePathBasedCacheEntryException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public NoSuchIdException(long entryId) {
|
||||
super("there is no PathBasedCache entry with id " + entryId, entryId);
|
||||
}
|
||||
}
|
||||
|
||||
public final static class UnexpectedRemovePathBasedCacheEntryException
|
||||
extends RemovePathBasedCacheEntryException {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public UnexpectedRemovePathBasedCacheEntryException(long id) {
|
||||
super("encountered an unexpected error when trying to " +
|
||||
"remove PathBasedCache entry id " + id, id);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -25,18 +25,32 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.FsServerDefaults;
|
||||
import org.apache.hadoop.fs.Options.Rename;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.EmptyPathError;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
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.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.NoSuchIdException;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.RemovePermissionDeniedException;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
|
||||
|
@ -92,16 +106,29 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
|
|||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseElementProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDescriptorsElementProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDescriptorsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDescriptorsResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.PathBasedCacheDirectiveProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathBasedCacheDescriptorRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathBasedCacheDescriptorResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2ResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
|
||||
|
@ -151,6 +178,9 @@ import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenReque
|
|||
import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
|
||||
import com.google.common.primitives.Shorts;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -1004,4 +1034,182 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AddPathBasedCacheDirectiveResponseProto addPathBasedCacheDirective(
|
||||
RpcController controller, AddPathBasedCacheDirectiveRequestProto request)
|
||||
throws ServiceException {
|
||||
try {
|
||||
PathBasedCacheDirectiveProto proto = request.getDirective();
|
||||
if (StringUtils.isEmpty(proto.getPath())) {
|
||||
throw new EmptyPathError();
|
||||
}
|
||||
PathBasedCacheDirective directive = new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path(proto.getPath())).
|
||||
setReplication(Shorts.checkedCast(proto.getReplication())).
|
||||
setPool(proto.getPool()).
|
||||
build();
|
||||
PathBasedCacheDescriptor descriptor =
|
||||
server.addPathBasedCacheDirective(directive);
|
||||
AddPathBasedCacheDirectiveResponseProto.Builder builder =
|
||||
AddPathBasedCacheDirectiveResponseProto.newBuilder();
|
||||
builder.setDescriptorId(descriptor.getEntryId());
|
||||
return builder.build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemovePathBasedCacheDescriptorResponseProto removePathBasedCacheDescriptor(
|
||||
RpcController controller,
|
||||
RemovePathBasedCacheDescriptorRequestProto request)
|
||||
throws ServiceException {
|
||||
try {
|
||||
server.removePathBasedCacheDescriptor(request.getDescriptorId());
|
||||
RemovePathBasedCacheDescriptorResponseProto.Builder builder =
|
||||
RemovePathBasedCacheDescriptorResponseProto.newBuilder();
|
||||
return builder.build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListPathBasedCacheDescriptorsResponseProto listPathBasedCacheDescriptors(
|
||||
RpcController controller, ListPathBasedCacheDescriptorsRequestProto request)
|
||||
throws ServiceException {
|
||||
try {
|
||||
RemoteIterator<PathBasedCacheDescriptor> iter =
|
||||
server.listPathBasedCacheDescriptors(request.getPrevId(),
|
||||
request.hasPool() ? request.getPool() : null,
|
||||
request.hasPath() ? request.getPath() : null);
|
||||
ListPathBasedCacheDescriptorsResponseProto.Builder builder =
|
||||
ListPathBasedCacheDescriptorsResponseProto.newBuilder();
|
||||
long prevId = 0;
|
||||
while (iter.hasNext()) {
|
||||
PathBasedCacheDescriptor directive = iter.next();
|
||||
builder.addElements(
|
||||
ListPathBasedCacheDescriptorsElementProto.newBuilder().
|
||||
setId(directive.getEntryId()).
|
||||
setPath(directive.getPath().toUri().getPath()).
|
||||
setReplication(directive.getReplication()).
|
||||
setPool(directive.getPool()));
|
||||
prevId = directive.getEntryId();
|
||||
}
|
||||
if (prevId == 0) {
|
||||
builder.setHasMore(false);
|
||||
} else {
|
||||
iter = server.listPathBasedCacheDescriptors(prevId,
|
||||
request.hasPool() ? request.getPool() : null,
|
||||
request.hasPath() ? request.getPath() : null);
|
||||
builder.setHasMore(iter.hasNext());
|
||||
}
|
||||
return builder.build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public AddCachePoolResponseProto addCachePool(RpcController controller,
|
||||
AddCachePoolRequestProto request) throws ServiceException {
|
||||
try {
|
||||
CachePoolInfo info =
|
||||
new CachePoolInfo(request.getPoolName());
|
||||
if (request.hasOwnerName()) {
|
||||
info.setOwnerName(request.getOwnerName());
|
||||
}
|
||||
if (request.hasGroupName()) {
|
||||
info.setGroupName(request.getGroupName());
|
||||
}
|
||||
if (request.hasMode()) {
|
||||
info.setMode(new FsPermission((short)request.getMode()));
|
||||
}
|
||||
if (request.hasWeight()) {
|
||||
info.setWeight(request.getWeight());
|
||||
}
|
||||
server.addCachePool(info);
|
||||
return AddCachePoolResponseProto.newBuilder().build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ModifyCachePoolResponseProto modifyCachePool(RpcController controller,
|
||||
ModifyCachePoolRequestProto request) throws ServiceException {
|
||||
try {
|
||||
CachePoolInfo info =
|
||||
new CachePoolInfo(request.getPoolName());
|
||||
if (request.hasOwnerName()) {
|
||||
info.setOwnerName(request.getOwnerName());
|
||||
}
|
||||
if (request.hasGroupName()) {
|
||||
info.setGroupName(request.getGroupName());
|
||||
}
|
||||
if (request.hasMode()) {
|
||||
info.setMode(new FsPermission((short)request.getMode()));
|
||||
}
|
||||
if (request.hasWeight()) {
|
||||
info.setWeight(request.getWeight());
|
||||
}
|
||||
server.modifyCachePool(info);
|
||||
return ModifyCachePoolResponseProto.newBuilder().build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoveCachePoolResponseProto removeCachePool(RpcController controller,
|
||||
RemoveCachePoolRequestProto request) throws ServiceException {
|
||||
try {
|
||||
server.removeCachePool(request.getPoolName());
|
||||
return RemoveCachePoolResponseProto.newBuilder().build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListCachePoolsResponseProto listCachePools(RpcController controller,
|
||||
ListCachePoolsRequestProto request) throws ServiceException {
|
||||
try {
|
||||
RemoteIterator<CachePoolInfo> iter =
|
||||
server.listCachePools(request.getPrevPoolName());
|
||||
ListCachePoolsResponseProto.Builder responseBuilder =
|
||||
ListCachePoolsResponseProto.newBuilder();
|
||||
String prevPoolName = null;
|
||||
while (iter.hasNext()) {
|
||||
CachePoolInfo pool = iter.next();
|
||||
ListCachePoolsResponseElementProto.Builder elemBuilder =
|
||||
ListCachePoolsResponseElementProto.newBuilder();
|
||||
elemBuilder.setPoolName(pool.getPoolName());
|
||||
if (pool.getOwnerName() != null) {
|
||||
elemBuilder.setOwnerName(pool.getOwnerName());
|
||||
}
|
||||
if (pool.getGroupName() != null) {
|
||||
elemBuilder.setGroupName(pool.getGroupName());
|
||||
}
|
||||
if (pool.getMode() != null) {
|
||||
elemBuilder.setMode(pool.getMode().toShort());
|
||||
}
|
||||
if (pool.getWeight() != null) {
|
||||
elemBuilder.setWeight(pool.getWeight());
|
||||
}
|
||||
responseBuilder.addElements(elemBuilder.build());
|
||||
prevPoolName = pool.getPoolName();
|
||||
}
|
||||
// fill in hasNext
|
||||
if (prevPoolName == null) {
|
||||
responseBuilder.setHasMore(false);
|
||||
} else {
|
||||
iter = server.listCachePools(prevPoolName);
|
||||
responseBuilder.setHasMore(iter.hasNext());
|
||||
}
|
||||
return responseBuilder.build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,15 +24,20 @@ import java.util.Arrays;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.BatchedRemoteIterator;
|
||||
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
|
||||
import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FsServerDefaults;
|
||||
import org.apache.hadoop.fs.Options.Rename;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
|
||||
|
@ -46,10 +51,15 @@ 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.protocol.NSQuotaExceededException;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AbandonBlockRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddBlockRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddCachePoolRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AllowSnapshotRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AppendResponseProto;
|
||||
|
@ -87,11 +97,21 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
|
|||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSnapshottableDirListingResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseElementProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCachePoolsResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCorruptFileBlocksRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDescriptorsElementProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDescriptorsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListPathBasedCacheDescriptorsResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSaveRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.PathBasedCacheDirectiveProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemoveCachePoolRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RemovePathBasedCacheDescriptorRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rename2RequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RenameSnapshotRequestProto;
|
||||
|
@ -128,6 +148,7 @@ import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRespons
|
|||
import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
||||
import com.google.common.primitives.Shorts;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -982,4 +1003,231 @@ public class ClientNamenodeProtocolTranslatorPB implements
|
|||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public PathBasedCacheDescriptor addPathBasedCacheDirective(
|
||||
PathBasedCacheDirective directive) throws IOException {
|
||||
try {
|
||||
AddPathBasedCacheDirectiveRequestProto.Builder builder =
|
||||
AddPathBasedCacheDirectiveRequestProto.newBuilder();
|
||||
builder.setDirective(PathBasedCacheDirectiveProto.newBuilder()
|
||||
.setPath(directive.getPath().toUri().getPath())
|
||||
.setReplication(directive.getReplication())
|
||||
.setPool(directive.getPool())
|
||||
.build());
|
||||
AddPathBasedCacheDirectiveResponseProto result =
|
||||
rpcProxy.addPathBasedCacheDirective(null, builder.build());
|
||||
return new PathBasedCacheDescriptor(result.getDescriptorId(),
|
||||
directive.getPath(), directive.getReplication(),
|
||||
directive.getPool());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removePathBasedCacheDescriptor(Long id)
|
||||
throws IOException {
|
||||
try {
|
||||
RemovePathBasedCacheDescriptorRequestProto.Builder builder =
|
||||
RemovePathBasedCacheDescriptorRequestProto.newBuilder();
|
||||
builder.setDescriptorId(id);
|
||||
rpcProxy.removePathBasedCacheDescriptor(null, builder.build());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static class BatchedPathBasedCacheEntries
|
||||
implements BatchedEntries<PathBasedCacheDescriptor> {
|
||||
private ListPathBasedCacheDescriptorsResponseProto response;
|
||||
|
||||
BatchedPathBasedCacheEntries(ListPathBasedCacheDescriptorsResponseProto response) {
|
||||
this.response = response;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PathBasedCacheDescriptor get(int i) {
|
||||
ListPathBasedCacheDescriptorsElementProto elementProto =
|
||||
response.getElements(i);
|
||||
return new PathBasedCacheDescriptor(elementProto.getId(),
|
||||
new Path(elementProto.getPath()),
|
||||
Shorts.checkedCast(elementProto.getReplication()),
|
||||
elementProto.getPool());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return response.getElementsCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMore() {
|
||||
return response.getHasMore();
|
||||
}
|
||||
}
|
||||
|
||||
private class PathBasedCacheEntriesIterator
|
||||
extends BatchedRemoteIterator<Long, PathBasedCacheDescriptor> {
|
||||
private final String pool;
|
||||
private final String path;
|
||||
|
||||
public PathBasedCacheEntriesIterator(long prevKey, String pool, String path) {
|
||||
super(prevKey);
|
||||
this.pool = pool;
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BatchedEntries<PathBasedCacheDescriptor> makeRequest(
|
||||
Long nextKey) throws IOException {
|
||||
ListPathBasedCacheDescriptorsResponseProto response;
|
||||
try {
|
||||
ListPathBasedCacheDescriptorsRequestProto.Builder builder =
|
||||
ListPathBasedCacheDescriptorsRequestProto.newBuilder().setPrevId(nextKey);
|
||||
if (pool != null) {
|
||||
builder.setPool(pool);
|
||||
}
|
||||
if (path != null) {
|
||||
builder.setPath(path);
|
||||
}
|
||||
ListPathBasedCacheDescriptorsRequestProto req = builder.build();
|
||||
response = rpcProxy.listPathBasedCacheDescriptors(null, req);
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
return new BatchedPathBasedCacheEntries(response);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long elementToPrevKey(PathBasedCacheDescriptor element) {
|
||||
return element.getEntryId();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<PathBasedCacheDescriptor> listPathBasedCacheDescriptors(long prevId,
|
||||
String pool, String path) throws IOException {
|
||||
return new PathBasedCacheEntriesIterator(prevId, pool, path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCachePool(CachePoolInfo info) throws IOException {
|
||||
AddCachePoolRequestProto.Builder builder =
|
||||
AddCachePoolRequestProto.newBuilder();
|
||||
builder.setPoolName(info.getPoolName());
|
||||
if (info.getOwnerName() != null) {
|
||||
builder.setOwnerName(info.getOwnerName());
|
||||
}
|
||||
if (info.getGroupName() != null) {
|
||||
builder.setGroupName(info.getGroupName());
|
||||
}
|
||||
if (info.getMode() != null) {
|
||||
builder.setMode(info.getMode().toShort());
|
||||
}
|
||||
if (info.getWeight() != null) {
|
||||
builder.setWeight(info.getWeight());
|
||||
}
|
||||
try {
|
||||
rpcProxy.addCachePool(null, builder.build());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyCachePool(CachePoolInfo req) throws IOException {
|
||||
ModifyCachePoolRequestProto.Builder builder =
|
||||
ModifyCachePoolRequestProto.newBuilder();
|
||||
builder.setPoolName(req.getPoolName());
|
||||
if (req.getOwnerName() != null) {
|
||||
builder.setOwnerName(req.getOwnerName());
|
||||
}
|
||||
if (req.getGroupName() != null) {
|
||||
builder.setGroupName(req.getGroupName());
|
||||
}
|
||||
if (req.getMode() != null) {
|
||||
builder.setMode(req.getMode().toShort());
|
||||
}
|
||||
if (req.getWeight() != null) {
|
||||
builder.setWeight(req.getWeight());
|
||||
}
|
||||
try {
|
||||
rpcProxy.modifyCachePool(null, builder.build());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCachePool(String cachePoolName) throws IOException {
|
||||
try {
|
||||
rpcProxy.removeCachePool(null,
|
||||
RemoveCachePoolRequestProto.newBuilder().
|
||||
setPoolName(cachePoolName).build());
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static class BatchedPathDirectiveEntries
|
||||
implements BatchedEntries<CachePoolInfo> {
|
||||
private final ListCachePoolsResponseProto proto;
|
||||
|
||||
public BatchedPathDirectiveEntries(ListCachePoolsResponseProto proto) {
|
||||
this.proto = proto;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CachePoolInfo get(int i) {
|
||||
ListCachePoolsResponseElementProto elem = proto.getElements(i);
|
||||
return new CachePoolInfo(elem.getPoolName()).
|
||||
setOwnerName(elem.getOwnerName()).
|
||||
setGroupName(elem.getGroupName()).
|
||||
setMode(new FsPermission((short)elem.getMode())).
|
||||
setWeight(elem.getWeight());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return proto.getElementsCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMore() {
|
||||
return proto.getHasMore();
|
||||
}
|
||||
}
|
||||
|
||||
private class CachePoolIterator
|
||||
extends BatchedRemoteIterator<String, CachePoolInfo> {
|
||||
|
||||
public CachePoolIterator(String prevKey) {
|
||||
super(prevKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BatchedEntries<CachePoolInfo> makeRequest(String prevKey)
|
||||
throws IOException {
|
||||
try {
|
||||
return new BatchedPathDirectiveEntries(
|
||||
rpcProxy.listCachePools(null,
|
||||
ListCachePoolsRequestProto.newBuilder().
|
||||
setPrevPoolName(prevKey).build()));
|
||||
} catch (ServiceException e) {
|
||||
throw ProtobufHelper.getRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String elementToPrevKey(CachePoolInfo element) {
|
||||
return element.getPoolName();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<CachePoolInfo> listCachePools(String prevKey)
|
||||
throws IOException {
|
||||
return new CachePoolIterator(prevKey);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.Closeable;
|
|||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -36,6 +37,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
|
||||
|
@ -152,8 +155,9 @@ public class DatanodeProtocolClientSideTranslatorPB implements
|
|||
|
||||
@Override
|
||||
public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
|
||||
StorageReport[] reports, int xmitsInProgress, int xceiverCount,
|
||||
int failedVolumes) throws IOException {
|
||||
StorageReport[] reports, long dnCacheCapacity, long dnCacheUsed,
|
||||
int xmitsInProgress, int xceiverCount, int failedVolumes)
|
||||
throws IOException {
|
||||
HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
|
||||
.setRegistration(PBHelper.convert(registration))
|
||||
.setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
|
||||
|
@ -161,7 +165,12 @@ public class DatanodeProtocolClientSideTranslatorPB implements
|
|||
for (StorageReport r : reports) {
|
||||
builder.addReports(PBHelper.convert(r));
|
||||
}
|
||||
|
||||
if (dnCacheCapacity != 0) {
|
||||
builder.setDnCacheCapacity(dnCacheCapacity);
|
||||
}
|
||||
if (dnCacheUsed != 0) {
|
||||
builder.setDnCacheUsed(dnCacheUsed);
|
||||
}
|
||||
HeartbeatResponseProto resp;
|
||||
try {
|
||||
resp = rpcProxy.sendHeartbeat(NULL_CONTROLLER, builder.build());
|
||||
|
@ -202,6 +211,29 @@ public class DatanodeProtocolClientSideTranslatorPB implements
|
|||
return resp.hasCmd() ? PBHelper.convert(resp.getCmd()) : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatanodeCommand cacheReport(DatanodeRegistration registration,
|
||||
String poolId, List<Long> blockIds) throws IOException {
|
||||
CacheReportRequestProto.Builder builder =
|
||||
CacheReportRequestProto.newBuilder()
|
||||
.setRegistration(PBHelper.convert(registration))
|
||||
.setBlockPoolId(poolId);
|
||||
for (Long blockId : blockIds) {
|
||||
builder.addBlocks(blockId);
|
||||
}
|
||||
|
||||
CacheReportResponseProto resp;
|
||||
try {
|
||||
resp = rpcProxy.cacheReport(NULL_CONTROLLER, builder.build());
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufHelper.getRemoteException(se);
|
||||
}
|
||||
if (resp.hasCmd()) {
|
||||
return PBHelper.convert(resp.getCmd());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void blockReceivedAndDeleted(DatanodeRegistration registration,
|
||||
String poolId, StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks)
|
||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceive
|
|||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReceivedAndDeletedResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CacheReportResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.CommitBlockSynchronizationResponseProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ErrorReportRequestProto;
|
||||
|
@ -55,6 +57,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
|||
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
|
||||
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
||||
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
|
@ -109,8 +112,9 @@ public class DatanodeProtocolServerSideTranslatorPB implements
|
|||
p.getBlockPoolUsed());
|
||||
}
|
||||
response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
|
||||
report, request.getXmitsInProgress(), request.getXceiverCount(),
|
||||
request.getFailedVolumes());
|
||||
report, request.getDnCacheCapacity(), request.getDnCacheUsed(),
|
||||
request.getXmitsInProgress(),
|
||||
request.getXceiverCount(), request.getFailedVolumes());
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
|
@ -159,6 +163,27 @@ public class DatanodeProtocolServerSideTranslatorPB implements
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CacheReportResponseProto cacheReport(RpcController controller,
|
||||
CacheReportRequestProto request) throws ServiceException {
|
||||
DatanodeCommand cmd = null;
|
||||
try {
|
||||
cmd = impl.cacheReport(
|
||||
PBHelper.convert(request.getRegistration()),
|
||||
request.getBlockPoolId(),
|
||||
request.getBlocksList());
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
CacheReportResponseProto.Builder builder =
|
||||
CacheReportResponseProto.newBuilder();
|
||||
if (cmd != null) {
|
||||
builder.setCmd(PBHelper.convert(cmd));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
|
||||
RpcController controller, BlockReceivedAndDeletedRequestProto request)
|
||||
|
|
|
@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFsS
|
|||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeCommandProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
|
||||
|
@ -118,6 +119,7 @@ import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
|
|||
import org.apache.hadoop.hdfs.server.namenode.INodeId;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
|
||||
|
@ -148,6 +150,7 @@ import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
|
|||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.CodedInputStream;
|
||||
|
@ -472,7 +475,8 @@ public class PBHelper {
|
|||
PBHelper.convert(di.getId()),
|
||||
di.hasLocation() ? di.getLocation() : null ,
|
||||
di.getCapacity(), di.getDfsUsed(), di.getRemaining(),
|
||||
di.getBlockPoolUsed() , di.getLastUpdate() , di.getXceiverCount() ,
|
||||
di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
|
||||
di.getLastUpdate(), di.getXceiverCount(),
|
||||
PBHelper.convert(di.getAdminState()));
|
||||
}
|
||||
|
||||
|
@ -565,9 +569,21 @@ public class PBHelper {
|
|||
if (b == null) return null;
|
||||
Builder builder = LocatedBlockProto.newBuilder();
|
||||
DatanodeInfo[] locs = b.getLocations();
|
||||
List<DatanodeInfo> cachedLocs =
|
||||
Lists.newLinkedList(Arrays.asList(b.getCachedLocations()));
|
||||
for (int i = 0; i < locs.length; i++) {
|
||||
builder.addLocs(i, PBHelper.convert(locs[i]));
|
||||
DatanodeInfo loc = locs[i];
|
||||
builder.addLocs(i, PBHelper.convert(loc));
|
||||
boolean locIsCached = cachedLocs.contains(loc);
|
||||
builder.addIsCached(locIsCached);
|
||||
if (locIsCached) {
|
||||
cachedLocs.remove(loc);
|
||||
}
|
||||
}
|
||||
Preconditions.checkArgument(cachedLocs.size() == 0,
|
||||
"Found additional cached replica locations that are not in the set of"
|
||||
+ " storage-backed locations!");
|
||||
|
||||
return builder.setB(PBHelper.convert(b.getBlock()))
|
||||
.setBlockToken(PBHelper.convert(b.getBlockToken()))
|
||||
.setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
|
||||
|
@ -580,9 +596,20 @@ public class PBHelper {
|
|||
for (int i = 0; i < locs.size(); i++) {
|
||||
targets[i] = PBHelper.convert(locs.get(i));
|
||||
}
|
||||
// Set values from the isCached list, re-using references from loc
|
||||
List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
|
||||
List<Boolean> isCachedList = proto.getIsCachedList();
|
||||
for (int i=0; i<isCachedList.size(); i++) {
|
||||
if (isCachedList.get(i)) {
|
||||
cachedLocs.add(targets[i]);
|
||||
}
|
||||
}
|
||||
|
||||
LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
|
||||
proto.getOffset(), proto.getCorrupt());
|
||||
proto.getOffset(), proto.getCorrupt(),
|
||||
cachedLocs.toArray(new DatanodeInfo[0]));
|
||||
lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
|
||||
|
||||
return lb;
|
||||
}
|
||||
|
||||
|
@ -671,6 +698,8 @@ public class PBHelper {
|
|||
return PBHelper.convert(proto.getKeyUpdateCmd());
|
||||
case RegisterCommand:
|
||||
return REG_CMD;
|
||||
case BlockIdCommand:
|
||||
return PBHelper.convert(proto.getBlkIdCmd());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -724,6 +753,26 @@ public class PBHelper {
|
|||
return builder.build();
|
||||
}
|
||||
|
||||
public static BlockIdCommandProto convert(BlockIdCommand cmd) {
|
||||
BlockIdCommandProto.Builder builder = BlockIdCommandProto.newBuilder()
|
||||
.setBlockPoolId(cmd.getBlockPoolId());
|
||||
switch (cmd.getAction()) {
|
||||
case DatanodeProtocol.DNA_CACHE:
|
||||
builder.setAction(BlockIdCommandProto.Action.CACHE);
|
||||
break;
|
||||
case DatanodeProtocol.DNA_UNCACHE:
|
||||
builder.setAction(BlockIdCommandProto.Action.UNCACHE);
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Invalid action");
|
||||
}
|
||||
long[] blockIds = cmd.getBlockIds();
|
||||
for (int i = 0; i < blockIds.length; i++) {
|
||||
builder.addBlockIds(blockIds[i]);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
private static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
|
||||
DatanodeInfosProto[] ret = new DatanodeInfosProto[targets.length];
|
||||
for (int i = 0; i < targets.length; i++) {
|
||||
|
@ -766,8 +815,13 @@ public class PBHelper {
|
|||
case DatanodeProtocol.DNA_TRANSFER:
|
||||
case DatanodeProtocol.DNA_INVALIDATE:
|
||||
case DatanodeProtocol.DNA_SHUTDOWN:
|
||||
builder.setCmdType(DatanodeCommandProto.Type.BlockCommand).setBlkCmd(
|
||||
PBHelper.convert((BlockCommand) datanodeCommand));
|
||||
builder.setCmdType(DatanodeCommandProto.Type.BlockCommand).
|
||||
setBlkCmd(PBHelper.convert((BlockCommand) datanodeCommand));
|
||||
break;
|
||||
case DatanodeProtocol.DNA_CACHE:
|
||||
case DatanodeProtocol.DNA_UNCACHE:
|
||||
builder.setCmdType(DatanodeCommandProto.Type.BlockIdCommand).
|
||||
setBlkIdCmd(PBHelper.convert((BlockIdCommand) datanodeCommand));
|
||||
break;
|
||||
case DatanodeProtocol.DNA_UNKNOWN: //Not expected
|
||||
default:
|
||||
|
@ -818,10 +872,32 @@ public class PBHelper {
|
|||
case SHUTDOWN:
|
||||
action = DatanodeProtocol.DNA_SHUTDOWN;
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Unknown action type: " + blkCmd.getAction());
|
||||
}
|
||||
return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets);
|
||||
}
|
||||
|
||||
public static BlockIdCommand convert(BlockIdCommandProto blkIdCmd) {
|
||||
int numBlockIds = blkIdCmd.getBlockIdsCount();
|
||||
long blockIds[] = new long[numBlockIds];
|
||||
for (int i = 0; i < numBlockIds; i++) {
|
||||
blockIds[i] = blkIdCmd.getBlockIds(i);
|
||||
}
|
||||
int action = DatanodeProtocol.DNA_UNKNOWN;
|
||||
switch (blkIdCmd.getAction()) {
|
||||
case CACHE:
|
||||
action = DatanodeProtocol.DNA_CACHE;
|
||||
break;
|
||||
case UNCACHE:
|
||||
action = DatanodeProtocol.DNA_UNCACHE;
|
||||
break;
|
||||
default:
|
||||
throw new AssertionError("Unknown action type: " + blkIdCmd.getAction());
|
||||
}
|
||||
return new BlockIdCommand(action, blkIdCmd.getBlockPoolId(), blockIds);
|
||||
}
|
||||
|
||||
public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
|
||||
List<DatanodeInfoProto> proto = datanodeInfosProto.getDatanodesList();
|
||||
DatanodeInfo[] infos = new DatanodeInfo[proto.size()];
|
||||
|
@ -1358,10 +1434,11 @@ public class PBHelper {
|
|||
}
|
||||
|
||||
public static StorageReportProto convert(StorageReport r) {
|
||||
return StorageReportProto.newBuilder()
|
||||
StorageReportProto.Builder builder = StorageReportProto.newBuilder()
|
||||
.setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
|
||||
.setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
|
||||
.setStorageID(r.getStorageID()).build();
|
||||
.setStorageID(r.getStorageID());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
public static JournalInfo convert(JournalInfoProto info) {
|
||||
|
|
|
@ -85,7 +85,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
|
|||
this.bc = bc;
|
||||
}
|
||||
|
||||
DatanodeDescriptor getDatanode(int index) {
|
||||
public DatanodeDescriptor getDatanode(int index) {
|
||||
assert this.triplets != null : "BlockInfo is not initialized";
|
||||
assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
|
||||
return (DatanodeDescriptor)triplets[index*3];
|
||||
|
@ -153,7 +153,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
|
|||
return info;
|
||||
}
|
||||
|
||||
int getCapacity() {
|
||||
public int getCapacity() {
|
||||
assert this.triplets != null : "BlockInfo is not initialized";
|
||||
assert triplets.length % 3 == 0 : "Malformed BlockInfo";
|
||||
return triplets.length / 3;
|
||||
|
|
|
@ -3138,6 +3138,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
|
|||
UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the replicas which are corrupt for a given block.
|
||||
*/
|
||||
public Collection<DatanodeDescriptor> getCorruptReplicas(Block block) {
|
||||
return corruptReplicas.getNodes(block);
|
||||
}
|
||||
|
||||
/** @return the size of UnderReplicatedBlocks */
|
||||
public int numOfUnderReplicatedBlocks() {
|
||||
return neededReplications.size();
|
||||
|
|
|
@ -0,0 +1,453 @@
|
|||
/**
|
||||
* 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 static org.apache.hadoop.util.ExitUtil.terminate;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CacheManager;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INode;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
||||
import org.apache.hadoop.hdfs.util.ReadOnlyList;
|
||||
import org.apache.hadoop.util.GSet;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
/**
|
||||
* Scans the namesystem, scheduling blocks to be cached as appropriate.
|
||||
*
|
||||
* The CacheReplicationMonitor does a full scan when the NameNode first
|
||||
* starts up, and at configurable intervals afterwards.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||
public class CacheReplicationMonitor extends Thread implements Closeable {
|
||||
|
||||
private static final Log LOG =
|
||||
LogFactory.getLog(CacheReplicationMonitor.class);
|
||||
|
||||
private final FSNamesystem namesystem;
|
||||
|
||||
private final BlockManager blockManager;
|
||||
|
||||
private final CacheManager cacheManager;
|
||||
|
||||
private final GSet<CachedBlock, CachedBlock> cachedBlocks;
|
||||
|
||||
/**
|
||||
* Pseudorandom number source
|
||||
*/
|
||||
private final Random random = new Random();
|
||||
|
||||
/**
|
||||
* The interval at which we scan the namesystem for caching changes.
|
||||
*/
|
||||
private final long intervalMs;
|
||||
|
||||
/**
|
||||
* True if we should rescan immediately, regardless of how much time
|
||||
* elapsed since the previous scan.
|
||||
*/
|
||||
private boolean rescanImmediately;
|
||||
|
||||
/**
|
||||
* The monotonic time at which the current scan started.
|
||||
*/
|
||||
private long scanTimeMs;
|
||||
|
||||
/**
|
||||
* Mark status of the current scan.
|
||||
*/
|
||||
private boolean mark = false;
|
||||
|
||||
/**
|
||||
* True if this monitor should terminate.
|
||||
*/
|
||||
private boolean shutdown;
|
||||
|
||||
/**
|
||||
* Cache directives found in the previous scan.
|
||||
*/
|
||||
private int scannedDirectives;
|
||||
|
||||
/**
|
||||
* Blocks found in the previous scan.
|
||||
*/
|
||||
private long scannedBlocks;
|
||||
|
||||
public CacheReplicationMonitor(FSNamesystem namesystem,
|
||||
CacheManager cacheManager, long intervalMs) {
|
||||
this.namesystem = namesystem;
|
||||
this.blockManager = namesystem.getBlockManager();
|
||||
this.cacheManager = cacheManager;
|
||||
this.cachedBlocks = cacheManager.getCachedBlocks();
|
||||
this.intervalMs = intervalMs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
shutdown = false;
|
||||
rescanImmediately = true;
|
||||
scanTimeMs = 0;
|
||||
LOG.info("Starting CacheReplicationMonitor with interval " +
|
||||
intervalMs + " milliseconds");
|
||||
try {
|
||||
long curTimeMs = Time.monotonicNow();
|
||||
while (true) {
|
||||
synchronized(this) {
|
||||
while (true) {
|
||||
if (shutdown) {
|
||||
LOG.info("Shutting down CacheReplicationMonitor");
|
||||
return;
|
||||
}
|
||||
if (rescanImmediately) {
|
||||
LOG.info("Rescanning on request");
|
||||
rescanImmediately = false;
|
||||
break;
|
||||
}
|
||||
long delta = (scanTimeMs + intervalMs) - curTimeMs;
|
||||
if (delta <= 0) {
|
||||
LOG.info("Rescanning after " + (curTimeMs - scanTimeMs) +
|
||||
" milliseconds");
|
||||
break;
|
||||
}
|
||||
this.wait(delta);
|
||||
curTimeMs = Time.monotonicNow();
|
||||
}
|
||||
}
|
||||
scanTimeMs = curTimeMs;
|
||||
mark = !mark;
|
||||
rescan();
|
||||
curTimeMs = Time.monotonicNow();
|
||||
LOG.info("Scanned " + scannedDirectives + " directive(s) and " +
|
||||
scannedBlocks + " block(s) in " + (curTimeMs - scanTimeMs) + " " +
|
||||
"millisecond(s).");
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
LOG.fatal("Thread exiting", t);
|
||||
terminate(1, t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Kick the monitor thread.
|
||||
*
|
||||
* If it is sleeping, it will wake up and start scanning.
|
||||
* If it is currently scanning, it will finish the scan and immediately do
|
||||
* another one.
|
||||
*/
|
||||
public synchronized void kick() {
|
||||
rescanImmediately = true;
|
||||
this.notifyAll();
|
||||
}
|
||||
|
||||
/**
|
||||
* Shut down and join the monitor thread.
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
synchronized(this) {
|
||||
if (shutdown) return;
|
||||
shutdown = true;
|
||||
this.notifyAll();
|
||||
}
|
||||
try {
|
||||
if (this.isAlive()) {
|
||||
this.join(60000);
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
private void rescan() {
|
||||
scannedDirectives = 0;
|
||||
scannedBlocks = 0;
|
||||
namesystem.writeLock();
|
||||
try {
|
||||
rescanPathBasedCacheEntries();
|
||||
} finally {
|
||||
namesystem.writeUnlock();
|
||||
}
|
||||
namesystem.writeLock();
|
||||
try {
|
||||
rescanCachedBlockMap();
|
||||
} finally {
|
||||
namesystem.writeUnlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Scan all PathBasedCacheEntries. Use the information to figure out
|
||||
* what cache replication factor each block should have.
|
||||
*
|
||||
* @param mark Whether the current scan is setting or clearing the mark
|
||||
*/
|
||||
private void rescanPathBasedCacheEntries() {
|
||||
FSDirectory fsDir = namesystem.getFSDirectory();
|
||||
for (PathBasedCacheEntry pce : cacheManager.getEntriesById().values()) {
|
||||
scannedDirectives++;
|
||||
String path = pce.getPath();
|
||||
INode node;
|
||||
try {
|
||||
node = fsDir.getINode(path);
|
||||
} catch (UnresolvedLinkException e) {
|
||||
// We don't cache through symlinks
|
||||
continue;
|
||||
}
|
||||
if (node == null) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("No inode found at " + path);
|
||||
}
|
||||
} else if (node.isDirectory()) {
|
||||
INodeDirectory dir = node.asDirectory();
|
||||
ReadOnlyList<INode> children = dir.getChildrenList(null);
|
||||
for (INode child : children) {
|
||||
if (child.isFile()) {
|
||||
rescanFile(pce, child.asFile());
|
||||
}
|
||||
}
|
||||
} else if (node.isFile()) {
|
||||
rescanFile(pce, node.asFile());
|
||||
} else {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Ignoring non-directory, non-file inode " + node +
|
||||
" found at " + path);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply a PathBasedCacheEntry to a file.
|
||||
*
|
||||
* @param pce The PathBasedCacheEntry to apply.
|
||||
* @param file The file.
|
||||
*/
|
||||
private void rescanFile(PathBasedCacheEntry pce, INodeFile file) {
|
||||
BlockInfo[] blockInfos = file.getBlocks();
|
||||
for (BlockInfo blockInfo : blockInfos) {
|
||||
if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
|
||||
// We don't try to cache blocks that are under construction.
|
||||
continue;
|
||||
}
|
||||
Block block = new Block(blockInfo.getBlockId());
|
||||
CachedBlock ncblock = new CachedBlock(block.getBlockId(),
|
||||
pce.getReplication(), mark);
|
||||
CachedBlock ocblock = cachedBlocks.get(ncblock);
|
||||
if (ocblock == null) {
|
||||
cachedBlocks.put(ncblock);
|
||||
} else {
|
||||
if (mark != ocblock.getMark()) {
|
||||
// Mark hasn't been set in this scan, so update replication and mark.
|
||||
ocblock.setReplicationAndMark(pce.getReplication(), mark);
|
||||
} else {
|
||||
// Mark already set in this scan. Set replication to highest value in
|
||||
// any PathBasedCacheEntry that covers this file.
|
||||
ocblock.setReplicationAndMark((short)Math.max(
|
||||
pce.getReplication(), ocblock.getReplication()), mark);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Scan through the cached block map.
|
||||
* Any blocks which are under-replicated should be assigned new Datanodes.
|
||||
* Blocks that are over-replicated should be removed from Datanodes.
|
||||
*/
|
||||
private void rescanCachedBlockMap() {
|
||||
for (Iterator<CachedBlock> cbIter = cachedBlocks.iterator();
|
||||
cbIter.hasNext(); ) {
|
||||
scannedBlocks++;
|
||||
CachedBlock cblock = cbIter.next();
|
||||
List<DatanodeDescriptor> pendingCached =
|
||||
cblock.getDatanodes(Type.PENDING_CACHED);
|
||||
List<DatanodeDescriptor> cached =
|
||||
cblock.getDatanodes(Type.CACHED);
|
||||
List<DatanodeDescriptor> pendingUncached =
|
||||
cblock.getDatanodes(Type.PENDING_UNCACHED);
|
||||
// Remove nodes from PENDING_UNCACHED if they were actually uncached.
|
||||
for (Iterator<DatanodeDescriptor> iter = pendingUncached.iterator();
|
||||
iter.hasNext(); ) {
|
||||
DatanodeDescriptor datanode = iter.next();
|
||||
if (!cblock.isInList(datanode.getCached())) {
|
||||
datanode.getPendingUncached().remove(cblock);
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
// If the block's mark doesn't match with the mark of this scan, that
|
||||
// means that this block couldn't be reached during this scan. That means
|
||||
// it doesn't need to be cached any more.
|
||||
int neededCached = (cblock.getMark() != mark) ?
|
||||
0 : cblock.getReplication();
|
||||
int numCached = cached.size();
|
||||
if (numCached >= neededCached) {
|
||||
// If we have enough replicas, drop all pending cached.
|
||||
for (DatanodeDescriptor datanode : pendingCached) {
|
||||
datanode.getPendingCached().remove(cblock);
|
||||
}
|
||||
pendingCached.clear();
|
||||
}
|
||||
if (numCached < neededCached) {
|
||||
// If we don't have enough replicas, drop all pending uncached.
|
||||
for (DatanodeDescriptor datanode : pendingUncached) {
|
||||
datanode.getPendingUncached().remove(cblock);
|
||||
}
|
||||
pendingUncached.clear();
|
||||
}
|
||||
int neededUncached = numCached -
|
||||
(pendingUncached.size() + neededCached);
|
||||
if (neededUncached > 0) {
|
||||
addNewPendingUncached(neededUncached, cblock, cached,
|
||||
pendingUncached);
|
||||
} else {
|
||||
int additionalCachedNeeded = neededCached -
|
||||
(numCached + pendingCached.size());
|
||||
if (additionalCachedNeeded > 0) {
|
||||
addNewPendingCached(additionalCachedNeeded, cblock, cached,
|
||||
pendingCached);
|
||||
}
|
||||
}
|
||||
if ((neededCached == 0) &&
|
||||
pendingUncached.isEmpty() &&
|
||||
pendingCached.isEmpty()) {
|
||||
// we have nothing more to do with this block.
|
||||
cbIter.remove();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add new entries to the PendingUncached list.
|
||||
*
|
||||
* @param neededUncached The number of replicas that need to be uncached.
|
||||
* @param cachedBlock The block which needs to be uncached.
|
||||
* @param cached A list of DataNodes currently caching the block.
|
||||
* @param pendingUncached A list of DataNodes that will soon uncache the
|
||||
* block.
|
||||
*/
|
||||
private void addNewPendingUncached(int neededUncached,
|
||||
CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
|
||||
List<DatanodeDescriptor> pendingUncached) {
|
||||
if (!cacheManager.isActive()) {
|
||||
return;
|
||||
}
|
||||
// Figure out which replicas can be uncached.
|
||||
LinkedList<DatanodeDescriptor> possibilities =
|
||||
new LinkedList<DatanodeDescriptor>();
|
||||
for (DatanodeDescriptor datanode : cached) {
|
||||
if (!pendingUncached.contains(datanode)) {
|
||||
possibilities.add(datanode);
|
||||
}
|
||||
}
|
||||
while (neededUncached > 0) {
|
||||
if (possibilities.isEmpty()) {
|
||||
LOG.warn("Logic error: we're trying to uncache more replicas than " +
|
||||
"actually exist for " + cachedBlock);
|
||||
return;
|
||||
}
|
||||
DatanodeDescriptor datanode =
|
||||
possibilities.remove(random.nextInt(possibilities.size()));
|
||||
pendingUncached.add(datanode);
|
||||
boolean added = datanode.getPendingUncached().add(cachedBlock);
|
||||
assert added;
|
||||
neededUncached--;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add new entries to the PendingCached list.
|
||||
*
|
||||
* @param neededCached The number of replicas that need to be cached.
|
||||
* @param cachedBlock The block which needs to be cached.
|
||||
* @param cached A list of DataNodes currently caching the block.
|
||||
* @param pendingCached A list of DataNodes that will soon cache the
|
||||
* block.
|
||||
*/
|
||||
private void addNewPendingCached(int neededCached,
|
||||
CachedBlock cachedBlock, List<DatanodeDescriptor> cached,
|
||||
List<DatanodeDescriptor> pendingCached) {
|
||||
if (!cacheManager.isActive()) {
|
||||
return;
|
||||
}
|
||||
// To figure out which replicas can be cached, we consult the
|
||||
// blocksMap. We don't want to try to cache a corrupt replica, though.
|
||||
BlockInfo blockInfo = blockManager.
|
||||
getStoredBlock(new Block(cachedBlock.getBlockId()));
|
||||
if (blockInfo == null) {
|
||||
LOG.debug("Not caching block " + cachedBlock + " because it " +
|
||||
"was deleted from all DataNodes.");
|
||||
return;
|
||||
}
|
||||
if (!blockInfo.isComplete()) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Not caching block " + cachedBlock + " because it " +
|
||||
"is not yet complete.");
|
||||
}
|
||||
return;
|
||||
}
|
||||
List<DatanodeDescriptor> possibilities = new LinkedList<DatanodeDescriptor>();
|
||||
int numReplicas = blockInfo.getCapacity();
|
||||
Collection<DatanodeDescriptor> corrupt =
|
||||
blockManager.getCorruptReplicas(blockInfo);
|
||||
for (int i = 0; i < numReplicas; i++) {
|
||||
DatanodeDescriptor datanode = blockInfo.getDatanode(i);
|
||||
if ((datanode != null) &&
|
||||
((!pendingCached.contains(datanode)) &&
|
||||
((corrupt == null) || (!corrupt.contains(datanode))))) {
|
||||
possibilities.add(datanode);
|
||||
}
|
||||
}
|
||||
while (neededCached > 0) {
|
||||
if (possibilities.isEmpty()) {
|
||||
LOG.warn("We need " + neededCached + " more replica(s) than " +
|
||||
"actually exist to provide a cache replication of " +
|
||||
cachedBlock.getReplication() + " for " + cachedBlock);
|
||||
return;
|
||||
}
|
||||
DatanodeDescriptor datanode =
|
||||
possibilities.remove(random.nextInt(possibilities.size()));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("AddNewPendingCached: datanode " + datanode +
|
||||
" will now cache block " + cachedBlock);
|
||||
}
|
||||
pendingCached.add(datanode);
|
||||
boolean added = datanode.getPendingCached().add(cachedBlock);
|
||||
assert added;
|
||||
neededCached--;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -22,14 +22,20 @@ import java.util.Iterator;
|
|||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Queue;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
|
||||
import org.apache.hadoop.hdfs.util.LightWeightHashSet;
|
||||
import org.apache.hadoop.util.IntrusiveCollection;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* This class extends the DatanodeInfo class with ephemeral information (eg
|
||||
* health, capacity, what blocks are associated with the Datanode) that is
|
||||
|
@ -94,8 +100,74 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A list of CachedBlock objects on this datanode.
|
||||
*/
|
||||
public static class CachedBlocksList extends IntrusiveCollection<CachedBlock> {
|
||||
public enum Type {
|
||||
PENDING_CACHED,
|
||||
CACHED,
|
||||
PENDING_UNCACHED
|
||||
}
|
||||
|
||||
private final DatanodeDescriptor datanode;
|
||||
|
||||
private final Type type;
|
||||
|
||||
CachedBlocksList(DatanodeDescriptor datanode, Type type) {
|
||||
this.datanode = datanode;
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
public DatanodeDescriptor getDatanode() {
|
||||
return datanode;
|
||||
}
|
||||
|
||||
public Type getType() {
|
||||
return type;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The blocks which we want to cache on this DataNode.
|
||||
*/
|
||||
private final CachedBlocksList pendingCached =
|
||||
new CachedBlocksList(this, CachedBlocksList.Type.PENDING_CACHED);
|
||||
|
||||
/**
|
||||
* The blocks which we know are cached on this datanode.
|
||||
* This list is updated by periodic cache reports.
|
||||
*/
|
||||
private final CachedBlocksList cached =
|
||||
new CachedBlocksList(this, CachedBlocksList.Type.CACHED);
|
||||
|
||||
/**
|
||||
* The blocks which we want to uncache on this DataNode.
|
||||
*/
|
||||
private final CachedBlocksList pendingUncached =
|
||||
new CachedBlocksList(this, CachedBlocksList.Type.PENDING_UNCACHED);
|
||||
|
||||
public CachedBlocksList getPendingCached() {
|
||||
return pendingCached;
|
||||
}
|
||||
|
||||
public CachedBlocksList getCached() {
|
||||
return cached;
|
||||
}
|
||||
|
||||
public CachedBlocksList getPendingUncached() {
|
||||
return pendingUncached;
|
||||
}
|
||||
|
||||
/**
|
||||
* Head of the list of blocks on the datanode
|
||||
*/
|
||||
private volatile BlockInfo blockList = null;
|
||||
/**
|
||||
* Number of blocks on the datanode
|
||||
*/
|
||||
private int numBlocks = 0;
|
||||
|
||||
// isAlive == heartbeats.contains(this)
|
||||
// This is an optimization, because contains takes O(n) time on Arraylist
|
||||
public boolean isAlive = false;
|
||||
|
@ -160,7 +232,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
* @param nodeID id of the data node
|
||||
*/
|
||||
public DatanodeDescriptor(DatanodeID nodeID) {
|
||||
this(nodeID, 0L, 0L, 0L, 0L, 0, 0);
|
||||
this(nodeID, 0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -170,7 +242,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
*/
|
||||
public DatanodeDescriptor(DatanodeID nodeID,
|
||||
String networkLocation) {
|
||||
this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0, 0);
|
||||
this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -180,6 +252,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
* @param dfsUsed space used by the data node
|
||||
* @param remaining remaining capacity of the data node
|
||||
* @param bpused space used by the block pool corresponding to this namenode
|
||||
* @param cacheCapacity cache capacity of the data node
|
||||
* @param cacheUsed cache used on the data node
|
||||
* @param xceiverCount # of data transfers at the data node
|
||||
*/
|
||||
public DatanodeDescriptor(DatanodeID nodeID,
|
||||
|
@ -187,11 +261,13 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
long dfsUsed,
|
||||
long remaining,
|
||||
long bpused,
|
||||
long cacheCapacity,
|
||||
long cacheUsed,
|
||||
int xceiverCount,
|
||||
int failedVolumes) {
|
||||
super(nodeID);
|
||||
updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount,
|
||||
failedVolumes);
|
||||
updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
|
||||
cacheUsed, xceiverCount, failedVolumes);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -202,6 +278,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
* @param dfsUsed the used space by dfs datanode
|
||||
* @param remaining remaining capacity of the data node
|
||||
* @param bpused space used by the block pool corresponding to this namenode
|
||||
* @param cacheCapacity cache capacity of the data node
|
||||
* @param cacheUsed cache used on the data node
|
||||
* @param xceiverCount # of data transfers at the data node
|
||||
*/
|
||||
public DatanodeDescriptor(DatanodeID nodeID,
|
||||
|
@ -210,11 +288,13 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
long dfsUsed,
|
||||
long remaining,
|
||||
long bpused,
|
||||
long cacheCapacity,
|
||||
long cacheUsed,
|
||||
int xceiverCount,
|
||||
int failedVolumes) {
|
||||
super(nodeID, networkLocation);
|
||||
updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount,
|
||||
failedVolumes);
|
||||
updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
|
||||
cacheUsed, xceiverCount, failedVolumes);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -257,6 +337,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
* Used for testing only
|
||||
* @return the head of the blockList
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected BlockInfo getHead(){
|
||||
return blockList;
|
||||
}
|
||||
|
@ -285,6 +366,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
this.blockList = null;
|
||||
this.invalidateBlocks.clear();
|
||||
this.volumeFailures = 0;
|
||||
// pendingCached, cached, and pendingUncached are protected by the
|
||||
// FSN lock.
|
||||
this.pendingCached.clear();
|
||||
this.cached.clear();
|
||||
this.pendingUncached.clear();
|
||||
}
|
||||
|
||||
public void clearBlockQueues() {
|
||||
|
@ -293,6 +379,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
this.recoverBlocks.clear();
|
||||
this.replicateBlocks.clear();
|
||||
}
|
||||
// pendingCached, cached, and pendingUncached are protected by the
|
||||
// FSN lock.
|
||||
this.pendingCached.clear();
|
||||
this.cached.clear();
|
||||
this.pendingUncached.clear();
|
||||
}
|
||||
|
||||
public int numBlocks() {
|
||||
|
@ -303,11 +394,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
* Updates stats from datanode heartbeat.
|
||||
*/
|
||||
public void updateHeartbeat(long capacity, long dfsUsed, long remaining,
|
||||
long blockPoolUsed, int xceiverCount, int volFailures) {
|
||||
long blockPoolUsed, long cacheCapacity, long cacheUsed, int xceiverCount,
|
||||
int volFailures) {
|
||||
setCapacity(capacity);
|
||||
setRemaining(remaining);
|
||||
setBlockPoolUsed(blockPoolUsed);
|
||||
setDfsUsed(dfsUsed);
|
||||
setCacheCapacity(cacheCapacity);
|
||||
setCacheUsed(cacheUsed);
|
||||
setXceiverCount(xceiverCount);
|
||||
setLastUpdate(Time.now());
|
||||
this.volumeFailures = volFailures;
|
||||
|
|
|
@ -32,6 +32,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|||
import org.apache.hadoop.hdfs.protocol.*;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
|
||||
import org.apache.hadoop.hdfs.server.namenode.HostFileManager;
|
||||
import org.apache.hadoop.hdfs.server.namenode.HostFileManager.Entry;
|
||||
import org.apache.hadoop.hdfs.server.namenode.HostFileManager.EntrySet;
|
||||
|
@ -45,6 +47,7 @@ import org.apache.hadoop.ipc.Server;
|
|||
import org.apache.hadoop.net.*;
|
||||
import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
import org.apache.hadoop.util.IntrusiveCollection;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
|
@ -143,6 +146,12 @@ public class DatanodeManager {
|
|||
private boolean hasClusterEverBeenMultiRack = false;
|
||||
|
||||
private final boolean checkIpHostnameInRegistration;
|
||||
/**
|
||||
* Whether we should tell datanodes what to cache in replies to
|
||||
* heartbeat messages.
|
||||
*/
|
||||
private boolean sendCachingCommands = false;
|
||||
|
||||
/**
|
||||
* The number of datanodes for each software version. This list should change
|
||||
* during rolling upgrades.
|
||||
|
@ -1215,8 +1224,8 @@ public class DatanodeManager {
|
|||
public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
|
||||
final String blockPoolId,
|
||||
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
|
||||
int xceiverCount, int maxTransfers, int failedVolumes
|
||||
) throws IOException {
|
||||
long cacheCapacity, long cacheUsed, int xceiverCount, int maxTransfers,
|
||||
int failedVolumes) throws IOException {
|
||||
synchronized (heartbeatManager) {
|
||||
synchronized (datanodeMap) {
|
||||
DatanodeDescriptor nodeinfo = null;
|
||||
|
@ -1237,7 +1246,8 @@ public class DatanodeManager {
|
|||
}
|
||||
|
||||
heartbeatManager.updateHeartbeat(nodeinfo, capacity, dfsUsed,
|
||||
remaining, blockPoolUsed, xceiverCount, failedVolumes);
|
||||
remaining, blockPoolUsed, cacheCapacity, cacheUsed, xceiverCount,
|
||||
failedVolumes);
|
||||
|
||||
// If we are in safemode, do not send back any recovery / replication
|
||||
// requests. Don't even drain the existing queue of work.
|
||||
|
@ -1298,6 +1308,18 @@ public class DatanodeManager {
|
|||
cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
|
||||
blockPoolId, blks));
|
||||
}
|
||||
DatanodeCommand pendingCacheCommand =
|
||||
getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
|
||||
DatanodeProtocol.DNA_CACHE, blockPoolId);
|
||||
if (pendingCacheCommand != null) {
|
||||
cmds.add(pendingCacheCommand);
|
||||
}
|
||||
DatanodeCommand pendingUncacheCommand =
|
||||
getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
|
||||
DatanodeProtocol.DNA_UNCACHE, blockPoolId);
|
||||
if (pendingUncacheCommand != null) {
|
||||
cmds.add(pendingUncacheCommand);
|
||||
}
|
||||
|
||||
blockManager.addKeyUpdateCommand(cmds, nodeinfo);
|
||||
|
||||
|
@ -1317,6 +1339,40 @@ public class DatanodeManager {
|
|||
return new DatanodeCommand[0];
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a CachedBlockList into a DatanodeCommand with a list of blocks.
|
||||
*
|
||||
* @param list The {@link CachedBlocksList}. This function
|
||||
* clears the list.
|
||||
* @param datanode The datanode.
|
||||
* @param action The action to perform in the command.
|
||||
* @param poolId The block pool id.
|
||||
* @return A DatanodeCommand to be sent back to the DN, or null if
|
||||
* there is nothing to be done.
|
||||
*/
|
||||
private DatanodeCommand getCacheCommand(CachedBlocksList list,
|
||||
DatanodeDescriptor datanode, int action, String poolId) {
|
||||
int length = list.size();
|
||||
if (length == 0) {
|
||||
return null;
|
||||
}
|
||||
// Read and clear the existing cache commands.
|
||||
long[] blockIds = new long[length];
|
||||
int i = 0;
|
||||
for (Iterator<CachedBlock> iter = list.iterator();
|
||||
iter.hasNext(); ) {
|
||||
CachedBlock cachedBlock = iter.next();
|
||||
blockIds[i++] = cachedBlock.getBlockId();
|
||||
iter.remove();
|
||||
}
|
||||
if (!sendCachingCommands) {
|
||||
// Do not send caching commands unless the FSNamesystem told us we
|
||||
// should.
|
||||
return null;
|
||||
}
|
||||
return new BlockIdCommand(action, poolId, blockIds);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tell all datanodes to use a new, non-persistent bandwidth value for
|
||||
* dfs.balance.bandwidthPerSec.
|
||||
|
@ -1365,4 +1421,8 @@ public class DatanodeManager {
|
|||
public String toString() {
|
||||
return getClass().getSimpleName() + ": " + host2DatanodeMap;
|
||||
}
|
||||
|
||||
public void setSendCachingCommands(boolean sendCachingCommands) {
|
||||
this.sendCachingCommands = sendCachingCommands;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -170,7 +170,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
addDatanode(d);
|
||||
|
||||
//update its timestamp
|
||||
d.updateHeartbeat(0L, 0L, 0L, 0L, 0, 0);
|
||||
d.updateHeartbeat(0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -193,10 +193,10 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
|
||||
synchronized void updateHeartbeat(final DatanodeDescriptor node,
|
||||
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
|
||||
int xceiverCount, int failedVolumes) {
|
||||
long cacheCapacity, long cacheUsed, int xceiverCount, int failedVolumes) {
|
||||
stats.subtract(node);
|
||||
node.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed,
|
||||
xceiverCount, failedVolumes);
|
||||
cacheCapacity, cacheUsed, xceiverCount, failedVolumes);
|
||||
stats.add(node);
|
||||
}
|
||||
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|||
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
||||
|
@ -518,6 +519,8 @@ class BPOfferService {
|
|||
return true;
|
||||
final BlockCommand bcmd =
|
||||
cmd instanceof BlockCommand? (BlockCommand)cmd: null;
|
||||
final BlockIdCommand blockIdCmd =
|
||||
cmd instanceof BlockIdCommand ? (BlockIdCommand)cmd: null;
|
||||
|
||||
switch(cmd.getAction()) {
|
||||
case DatanodeProtocol.DNA_TRANSFER:
|
||||
|
@ -543,6 +546,16 @@ class BPOfferService {
|
|||
}
|
||||
dn.metrics.incrBlocksRemoved(toDelete.length);
|
||||
break;
|
||||
case DatanodeProtocol.DNA_CACHE:
|
||||
LOG.info("DatanodeCommand action: DNA_CACHE");
|
||||
dn.getFSDataset().cache(blockIdCmd.getBlockPoolId(), blockIdCmd.getBlockIds());
|
||||
dn.metrics.incrBlocksCached(blockIdCmd.getBlockIds().length);
|
||||
break;
|
||||
case DatanodeProtocol.DNA_UNCACHE:
|
||||
LOG.info("DatanodeCommand action: DNA_UNCACHE");
|
||||
dn.getFSDataset().uncache(blockIdCmd.getBlockPoolId(), blockIdCmd.getBlockIds());
|
||||
dn.metrics.incrBlocksUncached(blockIdCmd.getBlockIds().length);
|
||||
break;
|
||||
case DatanodeProtocol.DNA_SHUTDOWN:
|
||||
// TODO: DNA_SHUTDOWN appears to be unused - the NN never sends this command
|
||||
// See HDFS-2987.
|
||||
|
@ -615,6 +628,8 @@ class BPOfferService {
|
|||
case DatanodeProtocol.DNA_FINALIZE:
|
||||
case DatanodeProtocol.DNA_RECOVERBLOCK:
|
||||
case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE:
|
||||
case DatanodeProtocol.DNA_CACHE:
|
||||
case DatanodeProtocol.DNA_UNCACHE:
|
||||
LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction());
|
||||
break;
|
||||
default:
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.net.InetSocketAddress;
|
|||
import java.net.SocketTimeoutException;
|
||||
import java.net.URI;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -84,6 +85,8 @@ class BPServiceActor implements Runnable {
|
|||
|
||||
boolean resetBlockReportTime = true;
|
||||
|
||||
volatile long lastCacheReport = 0;
|
||||
|
||||
Thread bpThread;
|
||||
DatanodeProtocolClientSideTranslatorPB bpNamenode;
|
||||
private volatile long lastHeartbeat = 0;
|
||||
|
@ -430,6 +433,35 @@ class BPServiceActor implements Runnable {
|
|||
return cmd;
|
||||
}
|
||||
|
||||
DatanodeCommand cacheReport() throws IOException {
|
||||
// If caching is disabled, do not send a cache report
|
||||
if (dn.getFSDataset().getDnCacheCapacity() == 0) {
|
||||
return null;
|
||||
}
|
||||
// send cache report if timer has expired.
|
||||
DatanodeCommand cmd = null;
|
||||
long startTime = Time.monotonicNow();
|
||||
if (startTime - lastCacheReport > dnConf.cacheReportInterval) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Sending cacheReport from service actor: " + this);
|
||||
}
|
||||
lastCacheReport = startTime;
|
||||
|
||||
String bpid = bpos.getBlockPoolId();
|
||||
List<Long> blockIds = dn.getFSDataset().getCacheReport(bpid);
|
||||
long createTime = Time.monotonicNow();
|
||||
|
||||
cmd = bpNamenode.cacheReport(bpRegistration, bpid, blockIds);
|
||||
long sendTime = Time.monotonicNow();
|
||||
long createCost = createTime - startTime;
|
||||
long sendCost = sendTime - createTime;
|
||||
dn.getMetrics().addCacheReport(sendCost);
|
||||
LOG.info("CacheReport of " + blockIds.size()
|
||||
+ " blocks took " + createCost + " msec to generate and "
|
||||
+ sendCost + " msecs for RPC and NN processing");
|
||||
}
|
||||
return cmd;
|
||||
}
|
||||
|
||||
HeartbeatResponse sendHeartBeat() throws IOException {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -443,6 +475,8 @@ class BPServiceActor implements Runnable {
|
|||
dn.getFSDataset().getRemaining(),
|
||||
dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) };
|
||||
return bpNamenode.sendHeartbeat(bpRegistration, report,
|
||||
dn.getFSDataset().getDnCacheCapacity(),
|
||||
dn.getFSDataset().getDnCacheUsed(),
|
||||
dn.getXmitsInProgress(),
|
||||
dn.getXceiverCount(),
|
||||
dn.getFSDataset().getNumFailedVolumes());
|
||||
|
@ -496,11 +530,12 @@ class BPServiceActor implements Runnable {
|
|||
* forever calling remote NameNode functions.
|
||||
*/
|
||||
private void offerService() throws Exception {
|
||||
LOG.info("For namenode " + nnAddr + " using DELETEREPORT_INTERVAL of "
|
||||
+ dnConf.deleteReportInterval + " msec " + " BLOCKREPORT_INTERVAL of "
|
||||
+ dnConf.blockReportInterval + "msec" + " Initial delay: "
|
||||
+ dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
|
||||
+ dnConf.heartBeatInterval);
|
||||
LOG.info("For namenode " + nnAddr + " using"
|
||||
+ " DELETEREPORT_INTERVAL of " + dnConf.deleteReportInterval + " msec "
|
||||
+ " BLOCKREPORT_INTERVAL of " + dnConf.blockReportInterval + "msec"
|
||||
+ " CACHEREPORT_INTERVAL of " + dnConf.cacheReportInterval + "msec"
|
||||
+ " Initial delay: " + dnConf.initialBlockReportDelay + "msec"
|
||||
+ "; heartBeatInterval=" + dnConf.heartBeatInterval);
|
||||
|
||||
//
|
||||
// Now loop for a long time....
|
||||
|
@ -555,6 +590,9 @@ class BPServiceActor implements Runnable {
|
|||
DatanodeCommand cmd = blockReport();
|
||||
processCommand(new DatanodeCommand[]{ cmd });
|
||||
|
||||
cmd = cacheReport();
|
||||
processCommand(new DatanodeCommand[]{ cmd });
|
||||
|
||||
// Now safe to start scanning the block pool.
|
||||
// If it has already been started, this is a no-op.
|
||||
if (dn.blockScanner != null) {
|
||||
|
|
|
@ -18,13 +18,18 @@
|
|||
package org.apache.hadoop.hdfs.server.datanode;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
|
||||
|
@ -39,6 +44,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NA
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATA_ENCRYPTION_ALGORITHM_KEY;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
||||
|
@ -66,6 +72,7 @@ public class DNConf {
|
|||
final long blockReportInterval;
|
||||
final long deleteReportInterval;
|
||||
final long initialBlockReportDelay;
|
||||
final long cacheReportInterval;
|
||||
final int writePacketSize;
|
||||
|
||||
final String minimumNameNodeVersion;
|
||||
|
@ -73,6 +80,8 @@ public class DNConf {
|
|||
|
||||
final long xceiverStopTimeout;
|
||||
|
||||
final long maxLockedMemory;
|
||||
|
||||
public DNConf(Configuration conf) {
|
||||
socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
||||
HdfsServerConstants.READ_TIMEOUT);
|
||||
|
@ -108,6 +117,8 @@ public class DNConf {
|
|||
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
|
||||
this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
|
||||
DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
|
||||
this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
|
||||
DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT);
|
||||
|
||||
long initBRDelay = conf.getLong(
|
||||
DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
|
||||
|
@ -137,6 +148,10 @@ public class DNConf {
|
|||
this.xceiverStopTimeout = conf.getLong(
|
||||
DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY,
|
||||
DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT);
|
||||
|
||||
this.maxLockedMemory = conf.getLong(
|
||||
DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT);
|
||||
}
|
||||
|
||||
// We get minimumNameNodeVersion via a method so it can be mocked out in tests.
|
||||
|
@ -147,4 +162,8 @@ public class DNConf {
|
|||
public long getXceiverStopTimeout() {
|
||||
return xceiverStopTimeout;
|
||||
}
|
||||
|
||||
public long getMaxLockedMemory() {
|
||||
return maxLockedMemory;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -67,6 +67,7 @@ import org.apache.hadoop.hdfs.web.resources.Param;
|
|||
import org.apache.hadoop.http.HttpServer;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.ReadaheadPool;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
@ -656,6 +657,25 @@ public class DataNode extends Configured
|
|||
this.conf = conf;
|
||||
this.dnConf = new DNConf(conf);
|
||||
|
||||
if (dnConf.maxLockedMemory > 0) {
|
||||
if (!NativeIO.isAvailable()) {
|
||||
throw new RuntimeException(String.format(
|
||||
"Cannot start datanode because the configured max locked memory" +
|
||||
" size (%s) is greater than zero and native code is not available.",
|
||||
DFS_DATANODE_MAX_LOCKED_MEMORY_KEY));
|
||||
}
|
||||
long ulimit = NativeIO.getMemlockLimit();
|
||||
if (dnConf.maxLockedMemory > ulimit) {
|
||||
throw new RuntimeException(String.format(
|
||||
"Cannot start datanode because the configured max locked memory" +
|
||||
" size (%s) of %d bytes is more than the datanode's available" +
|
||||
" RLIMIT_MEMLOCK ulimit of %d bytes.",
|
||||
DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
dnConf.maxLockedMemory,
|
||||
ulimit));
|
||||
}
|
||||
}
|
||||
|
||||
storage = new DataStorage();
|
||||
|
||||
// global DN settings
|
||||
|
|
|
@ -269,6 +269,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
|
|||
*/
|
||||
public BlockListAsLongs getBlockReport(String bpid);
|
||||
|
||||
/**
|
||||
* Returns the cache report - the full list of cached block IDs of a
|
||||
* block pool.
|
||||
* @param bpid Block Pool Id
|
||||
* @return the cache report - the full list of cached block IDs.
|
||||
*/
|
||||
public List<Long> getCacheReport(String bpid);
|
||||
|
||||
/** Does the dataset contain the block? */
|
||||
public boolean contains(ExtendedBlock block);
|
||||
|
||||
|
@ -294,6 +302,20 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
|
|||
*/
|
||||
public void invalidate(String bpid, Block invalidBlks[]) throws IOException;
|
||||
|
||||
/**
|
||||
* Caches the specified blocks
|
||||
* @param bpid Block pool id
|
||||
* @param blockIds - block ids to cache
|
||||
*/
|
||||
public void cache(String bpid, long[] blockIds);
|
||||
|
||||
/**
|
||||
* Uncaches the specified blocks
|
||||
* @param bpid Block pool id
|
||||
* @param blockIds - blocks ids to uncache
|
||||
*/
|
||||
public void uncache(String bpid, long[] blockIds);
|
||||
|
||||
/**
|
||||
* Check if all the data directories are healthy
|
||||
* @throws DiskErrorException
|
||||
|
|
|
@ -0,0 +1,223 @@
|
|||
/**
|
||||
* 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.datanode.fsdataset.impl;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
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.fs.ChecksumException;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
|
||||
/**
|
||||
* Manages caching for an FsDatasetImpl by using the mmap(2) and mlock(2)
|
||||
* system calls to lock blocks into memory. Block checksums are verified upon
|
||||
* entry into the cache.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public class FsDatasetCache {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(FsDatasetCache.class);
|
||||
|
||||
/**
|
||||
* Map of cached blocks
|
||||
*/
|
||||
private final ConcurrentMap<Long, MappableBlock> cachedBlocks;
|
||||
|
||||
private final FsDatasetImpl dataset;
|
||||
/**
|
||||
* Number of cached bytes
|
||||
*/
|
||||
private AtomicLong usedBytes;
|
||||
/**
|
||||
* Total cache capacity in bytes
|
||||
*/
|
||||
private final long maxBytes;
|
||||
|
||||
public FsDatasetCache(FsDatasetImpl dataset) {
|
||||
this.dataset = dataset;
|
||||
this.cachedBlocks = new ConcurrentHashMap<Long, MappableBlock>();
|
||||
this.usedBytes = new AtomicLong(0);
|
||||
this.maxBytes = dataset.datanode.getDnConf().getMaxLockedMemory();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return if the block is cached
|
||||
*/
|
||||
boolean isCached(String bpid, long blockId) {
|
||||
MappableBlock mapBlock = cachedBlocks.get(blockId);
|
||||
if (mapBlock != null) {
|
||||
return mapBlock.getBlockPoolId().equals(bpid);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return List of cached blocks suitable for translation into a
|
||||
* {@link BlockListAsLongs} for a cache report.
|
||||
*/
|
||||
List<Long> getCachedBlocks(String bpid) {
|
||||
List<Long> blocks = new ArrayList<Long>();
|
||||
// ConcurrentHashMap iteration doesn't see latest updates, which is okay
|
||||
Iterator<MappableBlock> it = cachedBlocks.values().iterator();
|
||||
while (it.hasNext()) {
|
||||
MappableBlock mapBlock = it.next();
|
||||
if (mapBlock.getBlockPoolId().equals(bpid)) {
|
||||
blocks.add(mapBlock.getBlock().getBlockId());
|
||||
}
|
||||
}
|
||||
return blocks;
|
||||
}
|
||||
|
||||
/**
|
||||
* Asynchronously attempts to cache a block. This is subject to the
|
||||
* configured maximum locked memory limit.
|
||||
*
|
||||
* @param block block to cache
|
||||
* @param volume volume of the block
|
||||
* @param blockIn stream of the block's data file
|
||||
* @param metaIn stream of the block's meta file
|
||||
*/
|
||||
void cacheBlock(String bpid, Block block, FsVolumeImpl volume,
|
||||
FileInputStream blockIn, FileInputStream metaIn) {
|
||||
if (isCached(bpid, block.getBlockId())) {
|
||||
return;
|
||||
}
|
||||
MappableBlock mapBlock = null;
|
||||
try {
|
||||
mapBlock = new MappableBlock(bpid, block, volume, blockIn, metaIn);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to cache replica " + block + ": Could not instantiate"
|
||||
+ " MappableBlock", e);
|
||||
IOUtils.closeQuietly(blockIn);
|
||||
IOUtils.closeQuietly(metaIn);
|
||||
return;
|
||||
}
|
||||
// Check if there's sufficient cache capacity
|
||||
boolean success = false;
|
||||
long bytes = mapBlock.getNumBytes();
|
||||
long used = usedBytes.get();
|
||||
while (used+bytes < maxBytes) {
|
||||
if (usedBytes.compareAndSet(used, used+bytes)) {
|
||||
success = true;
|
||||
break;
|
||||
}
|
||||
used = usedBytes.get();
|
||||
}
|
||||
if (!success) {
|
||||
LOG.warn(String.format(
|
||||
"Failed to cache replica %s: %s exceeded (%d + %d > %d)",
|
||||
mapBlock.getBlock().toString(),
|
||||
DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
used, bytes, maxBytes));
|
||||
mapBlock.close();
|
||||
return;
|
||||
}
|
||||
// Submit it to the worker pool to be cached
|
||||
volume.getExecutor().execute(new WorkerTask(mapBlock));
|
||||
}
|
||||
|
||||
/**
|
||||
* Uncaches a block if it is cached.
|
||||
* @param blockId id to uncache
|
||||
*/
|
||||
void uncacheBlock(String bpid, long blockId) {
|
||||
MappableBlock mapBlock = cachedBlocks.get(blockId);
|
||||
if (mapBlock != null &&
|
||||
mapBlock.getBlockPoolId().equals(bpid) &&
|
||||
mapBlock.getBlock().getBlockId() == blockId) {
|
||||
mapBlock.close();
|
||||
cachedBlocks.remove(blockId);
|
||||
long bytes = mapBlock.getNumBytes();
|
||||
long used = usedBytes.get();
|
||||
while (!usedBytes.compareAndSet(used, used - bytes)) {
|
||||
used = usedBytes.get();
|
||||
}
|
||||
LOG.info("Successfully uncached block " + blockId);
|
||||
} else {
|
||||
LOG.info("Could not uncache block " + blockId + ": unknown block.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Background worker that mmaps, mlocks, and checksums a block
|
||||
*/
|
||||
private class WorkerTask implements Runnable {
|
||||
|
||||
private MappableBlock block;
|
||||
WorkerTask(MappableBlock block) {
|
||||
this.block = block;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
boolean success = false;
|
||||
try {
|
||||
block.map();
|
||||
block.lock();
|
||||
block.verifyChecksum();
|
||||
success = true;
|
||||
} catch (ChecksumException e) {
|
||||
// Exception message is bogus since this wasn't caused by a file read
|
||||
LOG.warn("Failed to cache block " + block.getBlock() + ": Checksum "
|
||||
+ "verification failed.");
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to cache block " + block.getBlock() + ": IOException",
|
||||
e);
|
||||
}
|
||||
// If we failed or the block became uncacheable in the meantime,
|
||||
// clean up and return the reserved cache allocation
|
||||
if (!success ||
|
||||
!dataset.validToCache(block.getBlockPoolId(),
|
||||
block.getBlock().getBlockId())) {
|
||||
block.close();
|
||||
long used = usedBytes.get();
|
||||
while (!usedBytes.compareAndSet(used, used-block.getNumBytes())) {
|
||||
used = usedBytes.get();
|
||||
}
|
||||
} else {
|
||||
LOG.info("Successfully cached block " + block.getBlock());
|
||||
cachedBlocks.put(block.getBlock().getBlockId(), block);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Stats related methods for FsDatasetMBean
|
||||
|
||||
public long getDnCacheUsed() {
|
||||
return usedBytes.get();
|
||||
}
|
||||
|
||||
public long getDnCacheCapacity() {
|
||||
return maxBytes;
|
||||
}
|
||||
}
|
|
@ -37,6 +37,7 @@ import javax.management.NotCompliantMBeanException;
|
|||
import javax.management.ObjectName;
|
||||
import javax.management.StandardMBean;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
@ -170,6 +171,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
final FsVolumeList volumes;
|
||||
final ReplicaMap volumeMap;
|
||||
final FsDatasetAsyncDiskService asyncDiskService;
|
||||
final FsDatasetCache cacheManager;
|
||||
private final int validVolsRequired;
|
||||
|
||||
// Used for synchronizing access to usage stats
|
||||
|
@ -228,6 +230,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
roots[idx] = storage.getStorageDir(idx).getCurrentDir();
|
||||
}
|
||||
asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
|
||||
cacheManager = new FsDatasetCache(this);
|
||||
registerMBean(storage.getStorageID());
|
||||
}
|
||||
|
||||
|
@ -287,6 +290,22 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
return volumes.numberOfFailedVolumes();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the total cache used by the datanode (in bytes).
|
||||
*/
|
||||
@Override // FSDatasetMBean
|
||||
public long getDnCacheUsed() {
|
||||
return cacheManager.getDnCacheUsed();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the total cache capacity of the datanode (in bytes).
|
||||
*/
|
||||
@Override // FSDatasetMBean
|
||||
public long getDnCacheCapacity() {
|
||||
return cacheManager.getDnCacheCapacity();
|
||||
}
|
||||
|
||||
/**
|
||||
* Find the block's on-disk length
|
||||
*/
|
||||
|
@ -534,6 +553,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
private synchronized ReplicaBeingWritten append(String bpid,
|
||||
FinalizedReplica replicaInfo, long newGS, long estimateBlockLen)
|
||||
throws IOException {
|
||||
// uncache the block
|
||||
cacheManager.uncacheBlock(bpid, replicaInfo.getBlockId());
|
||||
// unlink the finalized replica
|
||||
replicaInfo.unlinkBlock(1);
|
||||
|
||||
|
@ -1001,6 +1022,11 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
}
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public List<Long> getCacheReport(String bpid) {
|
||||
return cacheManager.getCachedBlocks(bpid);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of finalized blocks from in-memory blockmap for a block pool.
|
||||
*/
|
||||
|
@ -1143,6 +1169,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
volumeMap.remove(bpid, invalidBlks[i]);
|
||||
}
|
||||
|
||||
// Uncache the block synchronously
|
||||
cacheManager.uncacheBlock(bpid, invalidBlks[i].getBlockId());
|
||||
// Delete the block asynchronously to make sure we can do it fast enough
|
||||
asyncDiskService.deleteAsync(v, f,
|
||||
FsDatasetUtil.getMetaFile(f, invalidBlks[i].getGenerationStamp()),
|
||||
|
@ -1153,6 +1181,82 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
|
|||
}
|
||||
}
|
||||
|
||||
synchronized boolean validToCache(String bpid, long blockId) {
|
||||
ReplicaInfo info = volumeMap.get(bpid, blockId);
|
||||
if (info == null) {
|
||||
LOG.warn("Failed to cache replica in block pool " + bpid +
|
||||
" with block id " + blockId + ": ReplicaInfo not found.");
|
||||
return false;
|
||||
}
|
||||
FsVolumeImpl volume = (FsVolumeImpl)info.getVolume();
|
||||
if (volume == null) {
|
||||
LOG.warn("Failed to cache block with id " + blockId +
|
||||
": Volume not found.");
|
||||
return false;
|
||||
}
|
||||
if (info.getState() != ReplicaState.FINALIZED) {
|
||||
LOG.warn("Failed to block with id " + blockId +
|
||||
": Replica is not finalized.");
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Asynchronously attempts to cache a single block via {@link FsDatasetCache}.
|
||||
*/
|
||||
private void cacheBlock(String bpid, long blockId) {
|
||||
ReplicaInfo info;
|
||||
FsVolumeImpl volume;
|
||||
synchronized (this) {
|
||||
if (!validToCache(bpid, blockId)) {
|
||||
return;
|
||||
}
|
||||
info = volumeMap.get(bpid, blockId);
|
||||
volume = (FsVolumeImpl)info.getVolume();
|
||||
}
|
||||
// Try to open block and meta streams
|
||||
FileInputStream blockIn = null;
|
||||
FileInputStream metaIn = null;
|
||||
boolean success = false;
|
||||
ExtendedBlock extBlk =
|
||||
new ExtendedBlock(bpid, blockId,
|
||||
info.getBytesOnDisk(), info.getGenerationStamp());
|
||||
try {
|
||||
blockIn = (FileInputStream)getBlockInputStream(extBlk, 0);
|
||||
metaIn = (FileInputStream)getMetaDataInputStream(extBlk)
|
||||
.getWrappedStream();
|
||||
success = true;
|
||||
} catch (ClassCastException e) {
|
||||
LOG.warn("Failed to cache replica " + extBlk + ": Underlying blocks"
|
||||
+ " are not backed by files.", e);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to cache replica " + extBlk + ": IOException while"
|
||||
+ " trying to open block or meta files.", e);
|
||||
}
|
||||
if (!success) {
|
||||
IOUtils.closeQuietly(blockIn);
|
||||
IOUtils.closeQuietly(metaIn);
|
||||
return;
|
||||
}
|
||||
cacheManager.cacheBlock(bpid, extBlk.getLocalBlock(),
|
||||
volume, blockIn, metaIn);
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public void cache(String bpid, long[] blockIds) {
|
||||
for (int i=0; i < blockIds.length; i++) {
|
||||
cacheBlock(bpid, blockIds[i]);
|
||||
}
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public void uncache(String bpid, long[] blockIds) {
|
||||
for (int i=0; i < blockIds.length; i++) {
|
||||
cacheManager.uncacheBlock(bpid, blockIds[i]);
|
||||
}
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public synchronized boolean contains(final ExtendedBlock block) {
|
||||
final long blockId = block.getLocalBlock().getBlockId();
|
||||
|
|
|
@ -18,11 +18,17 @@
|
|||
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -34,6 +40,8 @@ import org.apache.hadoop.hdfs.server.datanode.DataStorage;
|
|||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* The underlying volume used to store replica.
|
||||
*
|
||||
|
@ -48,6 +56,13 @@ class FsVolumeImpl implements FsVolumeSpi {
|
|||
private final File currentDir; // <StorageDirectory>/current
|
||||
private final DF usage;
|
||||
private final long reserved;
|
||||
/**
|
||||
* Per-volume worker pool that processes new blocks to cache.
|
||||
* The maximum number of workers per volume is bounded (configurable via
|
||||
* dfs.datanode.fsdatasetcache.max.threads.per.volume) to limit resource
|
||||
* contention.
|
||||
*/
|
||||
private final ThreadPoolExecutor cacheExecutor;
|
||||
|
||||
FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
|
||||
Configuration conf) throws IOException {
|
||||
|
@ -59,6 +74,20 @@ class FsVolumeImpl implements FsVolumeSpi {
|
|||
this.currentDir = currentDir;
|
||||
File parent = currentDir.getParentFile();
|
||||
this.usage = new DF(parent, conf);
|
||||
final int maxNumThreads = dataset.datanode.getConf().getInt(
|
||||
DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY,
|
||||
DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT
|
||||
);
|
||||
ThreadFactory workerFactory = new ThreadFactoryBuilder()
|
||||
.setDaemon(true)
|
||||
.setNameFormat("FsVolumeImplWorker-" + parent.toString() + "-%d")
|
||||
.build();
|
||||
cacheExecutor = new ThreadPoolExecutor(
|
||||
1, maxNumThreads,
|
||||
60, TimeUnit.SECONDS,
|
||||
new LinkedBlockingQueue<Runnable>(),
|
||||
workerFactory);
|
||||
cacheExecutor.allowCoreThreadTimeOut(true);
|
||||
}
|
||||
|
||||
File getCurrentDir() {
|
||||
|
@ -167,6 +196,10 @@ class FsVolumeImpl implements FsVolumeSpi {
|
|||
return getBlockPoolSlice(bpid).addBlock(b, f);
|
||||
}
|
||||
|
||||
Executor getExecutor() {
|
||||
return cacheExecutor;
|
||||
}
|
||||
|
||||
void checkDirs() throws DiskErrorException {
|
||||
// TODO:FEDERATION valid synchronization
|
||||
for(BlockPoolSlice s : bpSlices.values()) {
|
||||
|
@ -210,6 +243,7 @@ class FsVolumeImpl implements FsVolumeSpi {
|
|||
}
|
||||
|
||||
void shutdown() {
|
||||
cacheExecutor.shutdown();
|
||||
Set<Entry<String, BlockPoolSlice>> set = bpSlices.entrySet();
|
||||
for (Entry<String, BlockPoolSlice> entry : set) {
|
||||
entry.getValue().shutdown();
|
||||
|
|
|
@ -0,0 +1,249 @@
|
|||
/**
|
||||
* 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.datanode.fsdataset.impl;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.Closeable;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.MappedByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.channels.FileChannel.MapMode;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.ChecksumException;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Low-level wrapper for a Block and its backing files that provides mmap,
|
||||
* mlock, and checksum verification operations.
|
||||
*
|
||||
* This could be a private class of FsDatasetCache, not meant for other users.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
class MappableBlock implements Closeable {
|
||||
|
||||
private final String bpid;
|
||||
private final Block block;
|
||||
private final FsVolumeImpl volume;
|
||||
|
||||
private final FileInputStream blockIn;
|
||||
private final FileInputStream metaIn;
|
||||
private final FileChannel blockChannel;
|
||||
private final FileChannel metaChannel;
|
||||
private final long blockSize;
|
||||
|
||||
private boolean isMapped;
|
||||
private boolean isLocked;
|
||||
private boolean isChecksummed;
|
||||
|
||||
private MappedByteBuffer blockMapped = null;
|
||||
|
||||
public MappableBlock(String bpid, Block blk, FsVolumeImpl volume,
|
||||
FileInputStream blockIn, FileInputStream metaIn) throws IOException {
|
||||
this.bpid = bpid;
|
||||
this.block = blk;
|
||||
this.volume = volume;
|
||||
|
||||
this.blockIn = blockIn;
|
||||
this.metaIn = metaIn;
|
||||
this.blockChannel = blockIn.getChannel();
|
||||
this.metaChannel = metaIn.getChannel();
|
||||
this.blockSize = blockChannel.size();
|
||||
|
||||
this.isMapped = false;
|
||||
this.isLocked = false;
|
||||
this.isChecksummed = false;
|
||||
}
|
||||
|
||||
public String getBlockPoolId() {
|
||||
return bpid;
|
||||
}
|
||||
|
||||
public Block getBlock() {
|
||||
return block;
|
||||
}
|
||||
|
||||
public FsVolumeImpl getVolume() {
|
||||
return volume;
|
||||
}
|
||||
|
||||
public boolean isMapped() {
|
||||
return isMapped;
|
||||
}
|
||||
|
||||
public boolean isLocked() {
|
||||
return isLocked;
|
||||
}
|
||||
|
||||
public boolean isChecksummed() {
|
||||
return isChecksummed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of bytes on disk for the block file
|
||||
*/
|
||||
public long getNumBytes() {
|
||||
return blockSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Maps the block into memory. See mmap(2).
|
||||
*/
|
||||
public void map() throws IOException {
|
||||
if (isMapped) {
|
||||
return;
|
||||
}
|
||||
blockMapped = blockChannel.map(MapMode.READ_ONLY, 0, blockSize);
|
||||
isMapped = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Unmaps the block from memory. See munmap(2).
|
||||
*/
|
||||
public void unmap() {
|
||||
if (!isMapped) {
|
||||
return;
|
||||
}
|
||||
if (blockMapped instanceof sun.nio.ch.DirectBuffer) {
|
||||
sun.misc.Cleaner cleaner =
|
||||
((sun.nio.ch.DirectBuffer)blockMapped).cleaner();
|
||||
cleaner.clean();
|
||||
}
|
||||
isMapped = false;
|
||||
isLocked = false;
|
||||
isChecksummed = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Locks the block into memory. This prevents the block from being paged out.
|
||||
* See mlock(2).
|
||||
*/
|
||||
public void lock() throws IOException {
|
||||
Preconditions.checkArgument(isMapped,
|
||||
"Block must be mapped before it can be locked!");
|
||||
if (isLocked) {
|
||||
return;
|
||||
}
|
||||
NativeIO.POSIX.mlock(blockMapped, blockSize);
|
||||
isLocked = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Unlocks the block from memory, allowing it to be paged out. See munlock(2).
|
||||
*/
|
||||
public void unlock() throws IOException {
|
||||
if (!isLocked || !isMapped) {
|
||||
return;
|
||||
}
|
||||
NativeIO.POSIX.munlock(blockMapped, blockSize);
|
||||
isLocked = false;
|
||||
isChecksummed = false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads bytes into a buffer until EOF or the buffer's limit is reached
|
||||
*/
|
||||
private int fillBuffer(FileChannel channel, ByteBuffer buf)
|
||||
throws IOException {
|
||||
int bytesRead = channel.read(buf);
|
||||
if (bytesRead < 0) {
|
||||
//EOF
|
||||
return bytesRead;
|
||||
}
|
||||
while (buf.remaining() > 0) {
|
||||
int n = channel.read(buf);
|
||||
if (n < 0) {
|
||||
//EOF
|
||||
return bytesRead;
|
||||
}
|
||||
bytesRead += n;
|
||||
}
|
||||
return bytesRead;
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies the block's checksum. This is an I/O intensive operation.
|
||||
* @return if the block was successfully checksummed.
|
||||
*/
|
||||
public void verifyChecksum() throws IOException, ChecksumException {
|
||||
Preconditions.checkArgument(isLocked && isMapped,
|
||||
"Block must be mapped and locked before checksum verification!");
|
||||
// skip if checksum has already been successfully verified
|
||||
if (isChecksummed) {
|
||||
return;
|
||||
}
|
||||
// Verify the checksum from the block's meta file
|
||||
// Get the DataChecksum from the meta file header
|
||||
metaChannel.position(0);
|
||||
BlockMetadataHeader header =
|
||||
BlockMetadataHeader.readHeader(new DataInputStream(
|
||||
new BufferedInputStream(metaIn, BlockMetadataHeader
|
||||
.getHeaderSize())));
|
||||
DataChecksum checksum = header.getChecksum();
|
||||
final int bytesPerChecksum = checksum.getBytesPerChecksum();
|
||||
final int checksumSize = checksum.getChecksumSize();
|
||||
final int numChunks = (8*1024*1024) / bytesPerChecksum;
|
||||
ByteBuffer blockBuf = ByteBuffer.allocate(numChunks*bytesPerChecksum);
|
||||
ByteBuffer checksumBuf = ByteBuffer.allocate(numChunks*checksumSize);
|
||||
// Verify the checksum
|
||||
int bytesVerified = 0;
|
||||
while (bytesVerified < blockChannel.size()) {
|
||||
Preconditions.checkState(bytesVerified % bytesPerChecksum == 0,
|
||||
"Unexpected partial chunk before EOF");
|
||||
assert bytesVerified % bytesPerChecksum == 0;
|
||||
int bytesRead = fillBuffer(blockChannel, blockBuf);
|
||||
if (bytesRead == -1) {
|
||||
throw new IOException("Premature EOF");
|
||||
}
|
||||
blockBuf.flip();
|
||||
// Number of read chunks, including partial chunk at end
|
||||
int chunks = (bytesRead+bytesPerChecksum-1) / bytesPerChecksum;
|
||||
checksumBuf.limit(chunks*checksumSize);
|
||||
fillBuffer(metaChannel, checksumBuf);
|
||||
checksumBuf.flip();
|
||||
checksum.verifyChunkedSums(blockBuf, checksumBuf, block.getBlockName(),
|
||||
bytesVerified);
|
||||
// Success
|
||||
bytesVerified += bytesRead;
|
||||
blockBuf.clear();
|
||||
checksumBuf.clear();
|
||||
}
|
||||
isChecksummed = true;
|
||||
// Can close the backing file since everything is safely in memory
|
||||
blockChannel.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
unmap();
|
||||
IOUtils.closeQuietly(blockIn);
|
||||
IOUtils.closeQuietly(metaIn);
|
||||
}
|
||||
}
|
|
@ -57,6 +57,8 @@ public class DataNodeMetrics {
|
|||
@Metric MutableCounterLong blocksRemoved;
|
||||
@Metric MutableCounterLong blocksVerified;
|
||||
@Metric MutableCounterLong blockVerificationFailures;
|
||||
@Metric MutableCounterLong blocksCached;
|
||||
@Metric MutableCounterLong blocksUncached;
|
||||
@Metric MutableCounterLong readsFromLocalClient;
|
||||
@Metric MutableCounterLong readsFromRemoteClient;
|
||||
@Metric MutableCounterLong writesFromLocalClient;
|
||||
|
@ -74,6 +76,7 @@ public class DataNodeMetrics {
|
|||
@Metric MutableRate replaceBlockOp;
|
||||
@Metric MutableRate heartbeats;
|
||||
@Metric MutableRate blockReports;
|
||||
@Metric MutableRate cacheReports;
|
||||
@Metric MutableRate packetAckRoundTripTimeNanos;
|
||||
MutableQuantiles[] packetAckRoundTripTimeNanosQuantiles;
|
||||
|
||||
|
@ -151,6 +154,10 @@ public class DataNodeMetrics {
|
|||
blockReports.add(latency);
|
||||
}
|
||||
|
||||
public void addCacheReport(long latency) {
|
||||
cacheReports.add(latency);
|
||||
}
|
||||
|
||||
public void incrBlocksReplicated(int delta) {
|
||||
blocksReplicated.incr(delta);
|
||||
}
|
||||
|
@ -175,6 +182,15 @@ public class DataNodeMetrics {
|
|||
blocksVerified.incr();
|
||||
}
|
||||
|
||||
|
||||
public void incrBlocksCached(int delta) {
|
||||
blocksCached.incr(delta);
|
||||
}
|
||||
|
||||
public void incrBlocksUncached(int delta) {
|
||||
blocksUncached.incr(delta);
|
||||
}
|
||||
|
||||
public void addReadBlockOp(long latency) {
|
||||
readBlockOp.add(latency);
|
||||
}
|
||||
|
|
|
@ -77,4 +77,14 @@ public interface FSDatasetMBean {
|
|||
* @return The number of failed volumes in the datanode.
|
||||
*/
|
||||
public int getNumFailedVolumes();
|
||||
|
||||
/**
|
||||
* Returns the total cache used by the datanode (in bytes).
|
||||
*/
|
||||
public long getDnCacheUsed();
|
||||
|
||||
/**
|
||||
* Returns the total cache capacity of the datanode (in bytes).
|
||||
*/
|
||||
public long getDnCacheCapacity();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,743 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES_DEFAULT;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.UnexpectedAddPathBasedCacheDirectiveException;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.PoolWritePermissionDeniedError;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.NoSuchIdException;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.UnexpectedRemovePathBasedCacheDescriptorException;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.RemovePermissionDeniedException;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.CacheReplicationMonitor;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
||||
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
||||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
|
||||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
|
||||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
|
||||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
|
||||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.util.GSet;
|
||||
import org.apache.hadoop.util.LightWeightGSet;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* The Cache Manager handles caching on DataNodes.
|
||||
*
|
||||
* This class is instantiated by the FSNamesystem when caching is enabled.
|
||||
* It maintains the mapping of cached blocks to datanodes via processing
|
||||
* datanode cache reports. Based on these reports and addition and removal of
|
||||
* caching directives, we will schedule caching and uncaching work.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||
public final class CacheManager {
|
||||
public static final Log LOG = LogFactory.getLog(CacheManager.class);
|
||||
|
||||
// TODO: add pending / underCached / schedule cached blocks stats.
|
||||
|
||||
/**
|
||||
* The FSNamesystem that contains this CacheManager.
|
||||
*/
|
||||
private final FSNamesystem namesystem;
|
||||
|
||||
/**
|
||||
* The BlockManager associated with the FSN that owns this CacheManager.
|
||||
*/
|
||||
private final BlockManager blockManager;
|
||||
|
||||
/**
|
||||
* Cache entries, sorted by ID.
|
||||
*
|
||||
* listPathBasedCacheDescriptors relies on the ordering of elements in this map
|
||||
* to track what has already been listed by the client.
|
||||
*/
|
||||
private final TreeMap<Long, PathBasedCacheEntry> entriesById =
|
||||
new TreeMap<Long, PathBasedCacheEntry>();
|
||||
|
||||
/**
|
||||
* The entry ID to use for a new entry. Entry IDs always increase, and are
|
||||
* never reused.
|
||||
*/
|
||||
private long nextEntryId;
|
||||
|
||||
/**
|
||||
* Cache entries, sorted by path
|
||||
*/
|
||||
private final TreeMap<String, List<PathBasedCacheEntry>> entriesByPath =
|
||||
new TreeMap<String, List<PathBasedCacheEntry>>();
|
||||
|
||||
/**
|
||||
* Cache pools, sorted by name.
|
||||
*/
|
||||
private final TreeMap<String, CachePool> cachePools =
|
||||
new TreeMap<String, CachePool>();
|
||||
|
||||
/**
|
||||
* Maximum number of cache pools to list in one operation.
|
||||
*/
|
||||
private final int maxListCachePoolsResponses;
|
||||
|
||||
/**
|
||||
* Maximum number of cache pool directives to list in one operation.
|
||||
*/
|
||||
private final int maxListCacheDescriptorsResponses;
|
||||
|
||||
/**
|
||||
* Interval between scans in milliseconds.
|
||||
*/
|
||||
private final long scanIntervalMs;
|
||||
|
||||
/**
|
||||
* Whether caching is enabled.
|
||||
*
|
||||
* If caching is disabled, we will not process cache reports or store
|
||||
* information about what is cached where. We also do not start the
|
||||
* CacheReplicationMonitor thread. This will save resources, but provide
|
||||
* less functionality.
|
||||
*
|
||||
* Even when caching is disabled, we still store path-based cache
|
||||
* information. This information is stored in the edit log and fsimage. We
|
||||
* don't want to lose it just because a configuration setting was turned off.
|
||||
* However, we will not act on this information if caching is disabled.
|
||||
*/
|
||||
private final boolean enabled;
|
||||
|
||||
/**
|
||||
* Whether the CacheManager is active.
|
||||
*
|
||||
* When the CacheManager is active, it tells the DataNodes what to cache
|
||||
* and uncache. The CacheManager cannot become active if enabled = false.
|
||||
*/
|
||||
private boolean active = false;
|
||||
|
||||
/**
|
||||
* All cached blocks.
|
||||
*/
|
||||
private final GSet<CachedBlock, CachedBlock> cachedBlocks;
|
||||
|
||||
/**
|
||||
* The CacheReplicationMonitor.
|
||||
*/
|
||||
private CacheReplicationMonitor monitor;
|
||||
|
||||
CacheManager(FSNamesystem namesystem, Configuration conf,
|
||||
BlockManager blockManager) {
|
||||
this.namesystem = namesystem;
|
||||
this.blockManager = blockManager;
|
||||
this.nextEntryId = 1;
|
||||
this.maxListCachePoolsResponses = conf.getInt(
|
||||
DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES,
|
||||
DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES_DEFAULT);
|
||||
this.maxListCacheDescriptorsResponses = conf.getInt(
|
||||
DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES,
|
||||
DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES_DEFAULT);
|
||||
scanIntervalMs = conf.getLong(
|
||||
DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS,
|
||||
DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS_DEFAULT);
|
||||
this.enabled = conf.getBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY,
|
||||
DFS_NAMENODE_CACHING_ENABLED_DEFAULT);
|
||||
this.cachedBlocks = !enabled ? null :
|
||||
new LightWeightGSet<CachedBlock, CachedBlock>(
|
||||
LightWeightGSet.computeCapacity(0.25, "cachedBlocks"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Activate the cache manager.
|
||||
*
|
||||
* When the cache manager is active, tell the datanodes where to cache files.
|
||||
*/
|
||||
public void activate() {
|
||||
assert namesystem.hasWriteLock();
|
||||
if (enabled && (!active)) {
|
||||
LOG.info("Activating CacheManager. " +
|
||||
"Starting replication monitor thread...");
|
||||
active = true;
|
||||
monitor = new CacheReplicationMonitor(namesystem, this,
|
||||
scanIntervalMs);
|
||||
monitor.start();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Deactivate the cache manager.
|
||||
*
|
||||
* When the cache manager is inactive, it does not tell the datanodes where to
|
||||
* cache files.
|
||||
*/
|
||||
public void deactivate() {
|
||||
assert namesystem.hasWriteLock();
|
||||
if (active) {
|
||||
LOG.info("Deactivating CacheManager. " +
|
||||
"stopping CacheReplicationMonitor thread...");
|
||||
active = false;
|
||||
IOUtils.closeQuietly(monitor);
|
||||
monitor = null;
|
||||
LOG.info("CacheReplicationMonitor thread stopped and deactivated.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true only if the cache manager is active.
|
||||
* Must be called under the FSN read or write lock.
|
||||
*/
|
||||
public boolean isActive() {
|
||||
return active;
|
||||
}
|
||||
|
||||
public TreeMap<Long, PathBasedCacheEntry> getEntriesById() {
|
||||
assert namesystem.hasReadOrWriteLock();
|
||||
return entriesById;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public GSet<CachedBlock, CachedBlock> getCachedBlocks() {
|
||||
assert namesystem.hasReadOrWriteLock();
|
||||
return cachedBlocks;
|
||||
}
|
||||
|
||||
private long getNextEntryId() throws IOException {
|
||||
assert namesystem.hasWriteLock();
|
||||
if (nextEntryId == Long.MAX_VALUE) {
|
||||
throw new IOException("No more available IDs");
|
||||
}
|
||||
return nextEntryId++;
|
||||
}
|
||||
|
||||
public PathBasedCacheDescriptor addDirective(
|
||||
PathBasedCacheDirective directive, FSPermissionChecker pc)
|
||||
throws IOException {
|
||||
assert namesystem.hasWriteLock();
|
||||
CachePool pool = cachePools.get(directive.getPool());
|
||||
if (pool == null) {
|
||||
LOG.info("addDirective " + directive + ": pool not found.");
|
||||
throw new InvalidPoolNameError(directive);
|
||||
}
|
||||
if ((pc != null) && (!pc.checkPermission(pool, FsAction.WRITE))) {
|
||||
LOG.info("addDirective " + directive + ": write permission denied.");
|
||||
throw new PoolWritePermissionDeniedError(directive);
|
||||
}
|
||||
try {
|
||||
directive.validate();
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("addDirective " + directive + ": validation failed: "
|
||||
+ ioe.getClass().getName() + ": " + ioe.getMessage());
|
||||
throw ioe;
|
||||
}
|
||||
|
||||
// Add a new entry with the next available ID.
|
||||
PathBasedCacheEntry entry;
|
||||
try {
|
||||
entry = new PathBasedCacheEntry(getNextEntryId(),
|
||||
directive.getPath().toUri().getPath(),
|
||||
directive.getReplication(), pool);
|
||||
} catch (IOException ioe) {
|
||||
throw new UnexpectedAddPathBasedCacheDirectiveException(directive);
|
||||
}
|
||||
LOG.info("addDirective " + directive + ": added cache directive "
|
||||
+ directive);
|
||||
|
||||
// Success!
|
||||
// First, add it to the various maps
|
||||
entriesById.put(entry.getEntryId(), entry);
|
||||
String path = directive.getPath().toUri().getPath();
|
||||
List<PathBasedCacheEntry> entryList = entriesByPath.get(path);
|
||||
if (entryList == null) {
|
||||
entryList = new ArrayList<PathBasedCacheEntry>(1);
|
||||
entriesByPath.put(path, entryList);
|
||||
}
|
||||
entryList.add(entry);
|
||||
if (monitor != null) {
|
||||
monitor.kick();
|
||||
}
|
||||
return entry.getDescriptor();
|
||||
}
|
||||
|
||||
public void removeDescriptor(long id, FSPermissionChecker pc)
|
||||
throws IOException {
|
||||
assert namesystem.hasWriteLock();
|
||||
// Check for invalid IDs.
|
||||
if (id <= 0) {
|
||||
LOG.info("removeDescriptor " + id + ": invalid non-positive " +
|
||||
"descriptor ID.");
|
||||
throw new InvalidIdException(id);
|
||||
}
|
||||
// Find the entry.
|
||||
PathBasedCacheEntry existing = entriesById.get(id);
|
||||
if (existing == null) {
|
||||
LOG.info("removeDescriptor " + id + ": entry not found.");
|
||||
throw new NoSuchIdException(id);
|
||||
}
|
||||
CachePool pool = cachePools.get(existing.getDescriptor().getPool());
|
||||
if (pool == null) {
|
||||
LOG.info("removeDescriptor " + id + ": pool not found for directive " +
|
||||
existing.getDescriptor());
|
||||
throw new UnexpectedRemovePathBasedCacheDescriptorException(id);
|
||||
}
|
||||
if ((pc != null) && (!pc.checkPermission(pool, FsAction.WRITE))) {
|
||||
LOG.info("removeDescriptor " + id + ": write permission denied to " +
|
||||
"pool " + pool + " for entry " + existing);
|
||||
throw new RemovePermissionDeniedException(id);
|
||||
}
|
||||
|
||||
// Remove the corresponding entry in entriesByPath.
|
||||
String path = existing.getDescriptor().getPath().toUri().getPath();
|
||||
List<PathBasedCacheEntry> entries = entriesByPath.get(path);
|
||||
if (entries == null || !entries.remove(existing)) {
|
||||
throw new UnexpectedRemovePathBasedCacheDescriptorException(id);
|
||||
}
|
||||
if (entries.size() == 0) {
|
||||
entriesByPath.remove(path);
|
||||
}
|
||||
entriesById.remove(id);
|
||||
if (monitor != null) {
|
||||
monitor.kick();
|
||||
}
|
||||
LOG.info("removeDescriptor successful for PathCacheEntry id " + id);
|
||||
}
|
||||
|
||||
public BatchedListEntries<PathBasedCacheDescriptor>
|
||||
listPathBasedCacheDescriptors(long prevId, String filterPool,
|
||||
String filterPath, FSPermissionChecker pc) throws IOException {
|
||||
assert namesystem.hasReadOrWriteLock();
|
||||
final int NUM_PRE_ALLOCATED_ENTRIES = 16;
|
||||
if (filterPath != null) {
|
||||
if (!DFSUtil.isValidName(filterPath)) {
|
||||
throw new IOException("invalid path name '" + filterPath + "'");
|
||||
}
|
||||
}
|
||||
ArrayList<PathBasedCacheDescriptor> replies =
|
||||
new ArrayList<PathBasedCacheDescriptor>(NUM_PRE_ALLOCATED_ENTRIES);
|
||||
int numReplies = 0;
|
||||
SortedMap<Long, PathBasedCacheEntry> tailMap = entriesById.tailMap(prevId + 1);
|
||||
for (Entry<Long, PathBasedCacheEntry> cur : tailMap.entrySet()) {
|
||||
if (numReplies >= maxListCacheDescriptorsResponses) {
|
||||
return new BatchedListEntries<PathBasedCacheDescriptor>(replies, true);
|
||||
}
|
||||
PathBasedCacheEntry curEntry = cur.getValue();
|
||||
PathBasedCacheDirective directive = cur.getValue().getDescriptor();
|
||||
if (filterPool != null &&
|
||||
!directive.getPool().equals(filterPool)) {
|
||||
continue;
|
||||
}
|
||||
if (filterPath != null &&
|
||||
!directive.getPath().toUri().getPath().equals(filterPath)) {
|
||||
continue;
|
||||
}
|
||||
if (pc.checkPermission(curEntry.getPool(), FsAction.READ)) {
|
||||
replies.add(cur.getValue().getDescriptor());
|
||||
numReplies++;
|
||||
}
|
||||
}
|
||||
return new BatchedListEntries<PathBasedCacheDescriptor>(replies, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a cache pool.
|
||||
*
|
||||
* Only the superuser should be able to call this function.
|
||||
*
|
||||
* @param info The info for the cache pool to create.
|
||||
* @return Information about the cache pool we created.
|
||||
*/
|
||||
public CachePoolInfo addCachePool(CachePoolInfo info)
|
||||
throws IOException {
|
||||
assert namesystem.hasWriteLock();
|
||||
CachePoolInfo.validate(info);
|
||||
String poolName = info.getPoolName();
|
||||
CachePool pool = cachePools.get(poolName);
|
||||
if (pool != null) {
|
||||
throw new IOException("cache pool " + poolName + " already exists.");
|
||||
}
|
||||
pool = CachePool.createFromInfoAndDefaults(info);
|
||||
cachePools.put(pool.getPoolName(), pool);
|
||||
LOG.info("created new cache pool " + pool);
|
||||
return pool.getInfo(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Modify a cache pool.
|
||||
*
|
||||
* Only the superuser should be able to call this function.
|
||||
*
|
||||
* @param info
|
||||
* The info for the cache pool to modify.
|
||||
*/
|
||||
public void modifyCachePool(CachePoolInfo info)
|
||||
throws IOException {
|
||||
assert namesystem.hasWriteLock();
|
||||
CachePoolInfo.validate(info);
|
||||
String poolName = info.getPoolName();
|
||||
CachePool pool = cachePools.get(poolName);
|
||||
if (pool == null) {
|
||||
throw new IOException("cache pool " + poolName + " does not exist.");
|
||||
}
|
||||
StringBuilder bld = new StringBuilder();
|
||||
String prefix = "";
|
||||
if (info.getOwnerName() != null) {
|
||||
pool.setOwnerName(info.getOwnerName());
|
||||
bld.append(prefix).
|
||||
append("set owner to ").append(info.getOwnerName());
|
||||
prefix = "; ";
|
||||
}
|
||||
if (info.getGroupName() != null) {
|
||||
pool.setGroupName(info.getGroupName());
|
||||
bld.append(prefix).
|
||||
append("set group to ").append(info.getGroupName());
|
||||
prefix = "; ";
|
||||
}
|
||||
if (info.getMode() != null) {
|
||||
pool.setMode(info.getMode());
|
||||
bld.append(prefix).append("set mode to " + info.getMode());
|
||||
prefix = "; ";
|
||||
}
|
||||
if (info.getWeight() != null) {
|
||||
pool.setWeight(info.getWeight());
|
||||
bld.append(prefix).
|
||||
append("set weight to ").append(info.getWeight());
|
||||
prefix = "; ";
|
||||
}
|
||||
if (prefix.isEmpty()) {
|
||||
bld.append("no changes.");
|
||||
}
|
||||
LOG.info("modified " + poolName + "; " + bld.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a cache pool.
|
||||
*
|
||||
* Only the superuser should be able to call this function.
|
||||
*
|
||||
* @param poolName
|
||||
* The name for the cache pool to remove.
|
||||
*/
|
||||
public void removeCachePool(String poolName)
|
||||
throws IOException {
|
||||
assert namesystem.hasWriteLock();
|
||||
CachePoolInfo.validateName(poolName);
|
||||
CachePool pool = cachePools.remove(poolName);
|
||||
if (pool == null) {
|
||||
throw new IOException("can't remove non-existent cache pool " + poolName);
|
||||
}
|
||||
|
||||
// Remove entries using this pool
|
||||
// TODO: could optimize this somewhat to avoid the need to iterate
|
||||
// over all entries in entriesById
|
||||
Iterator<Entry<Long, PathBasedCacheEntry>> iter =
|
||||
entriesById.entrySet().iterator();
|
||||
while (iter.hasNext()) {
|
||||
Entry<Long, PathBasedCacheEntry> entry = iter.next();
|
||||
if (entry.getValue().getPool() == pool) {
|
||||
entriesByPath.remove(entry.getValue().getPath());
|
||||
iter.remove();
|
||||
}
|
||||
}
|
||||
if (monitor != null) {
|
||||
monitor.kick();
|
||||
}
|
||||
}
|
||||
|
||||
public BatchedListEntries<CachePoolInfo>
|
||||
listCachePools(FSPermissionChecker pc, String prevKey) {
|
||||
assert namesystem.hasReadOrWriteLock();
|
||||
final int NUM_PRE_ALLOCATED_ENTRIES = 16;
|
||||
ArrayList<CachePoolInfo> results =
|
||||
new ArrayList<CachePoolInfo>(NUM_PRE_ALLOCATED_ENTRIES);
|
||||
SortedMap<String, CachePool> tailMap = cachePools.tailMap(prevKey, false);
|
||||
int numListed = 0;
|
||||
for (Entry<String, CachePool> cur : tailMap.entrySet()) {
|
||||
if (numListed++ >= maxListCachePoolsResponses) {
|
||||
return new BatchedListEntries<CachePoolInfo>(results, true);
|
||||
}
|
||||
if (pc == null) {
|
||||
results.add(cur.getValue().getInfo(true));
|
||||
} else {
|
||||
results.add(cur.getValue().getInfo(pc));
|
||||
}
|
||||
}
|
||||
return new BatchedListEntries<CachePoolInfo>(results, false);
|
||||
}
|
||||
|
||||
public void setCachedLocations(LocatedBlock block) {
|
||||
if (!enabled) {
|
||||
return;
|
||||
}
|
||||
CachedBlock cachedBlock =
|
||||
new CachedBlock(block.getBlock().getBlockId(),
|
||||
(short)0, false);
|
||||
cachedBlock = cachedBlocks.get(cachedBlock);
|
||||
if (cachedBlock == null) {
|
||||
return;
|
||||
}
|
||||
List<DatanodeDescriptor> datanodes = cachedBlock.getDatanodes(Type.CACHED);
|
||||
for (DatanodeDescriptor datanode : datanodes) {
|
||||
block.addCachedLoc(datanode);
|
||||
}
|
||||
}
|
||||
|
||||
public final void processCacheReport(final DatanodeID datanodeID,
|
||||
final List<Long> blockIds) throws IOException {
|
||||
if (!enabled) {
|
||||
LOG.info("Ignoring cache report from " + datanodeID +
|
||||
" because " + DFS_NAMENODE_CACHING_ENABLED_KEY + " = false. " +
|
||||
"number of blocks: " + blockIds.size());
|
||||
return;
|
||||
}
|
||||
namesystem.writeLock();
|
||||
final long startTime = Time.monotonicNow();
|
||||
final long endTime;
|
||||
try {
|
||||
final DatanodeDescriptor datanode =
|
||||
blockManager.getDatanodeManager().getDatanode(datanodeID);
|
||||
if (datanode == null || !datanode.isAlive) {
|
||||
throw new IOException(
|
||||
"processCacheReport from dead or unregistered datanode: " + datanode);
|
||||
}
|
||||
processCacheReportImpl(datanode, blockIds);
|
||||
} finally {
|
||||
endTime = Time.monotonicNow();
|
||||
namesystem.writeUnlock();
|
||||
}
|
||||
|
||||
// Log the block report processing stats from Namenode perspective
|
||||
final NameNodeMetrics metrics = NameNode.getNameNodeMetrics();
|
||||
if (metrics != null) {
|
||||
metrics.addCacheBlockReport((int) (endTime - startTime));
|
||||
}
|
||||
LOG.info("Processed cache report from "
|
||||
+ datanodeID + ", blocks: " + blockIds.size()
|
||||
+ ", processing time: " + (endTime - startTime) + " msecs");
|
||||
}
|
||||
|
||||
private void processCacheReportImpl(final DatanodeDescriptor datanode,
|
||||
final List<Long> blockIds) {
|
||||
CachedBlocksList cached = datanode.getCached();
|
||||
cached.clear();
|
||||
for (Iterator<Long> iter = blockIds.iterator(); iter.hasNext(); ) {
|
||||
Block block = new Block(iter.next());
|
||||
BlockInfo blockInfo = blockManager.getStoredBlock(block);
|
||||
if (blockInfo.getGenerationStamp() < block.getGenerationStamp()) {
|
||||
// The NameNode will eventually remove or update the out-of-date block.
|
||||
// Until then, we pretend that it isn't cached.
|
||||
LOG.warn("Genstamp in cache report disagrees with our genstamp for " +
|
||||
block + ": expected genstamp " + blockInfo.getGenerationStamp());
|
||||
continue;
|
||||
}
|
||||
if (!blockInfo.isComplete()) {
|
||||
LOG.warn("Ignoring block id " + block.getBlockId() + ", because " +
|
||||
"it is in not complete yet. It is in state " +
|
||||
blockInfo.getBlockUCState());
|
||||
continue;
|
||||
}
|
||||
Collection<DatanodeDescriptor> corruptReplicas =
|
||||
blockManager.getCorruptReplicas(blockInfo);
|
||||
if ((corruptReplicas != null) && corruptReplicas.contains(datanode)) {
|
||||
// The NameNode will eventually remove or update the corrupt block.
|
||||
// Until then, we pretend that it isn't cached.
|
||||
LOG.warn("Ignoring cached replica on " + datanode + " of " + block +
|
||||
" because it is corrupt.");
|
||||
continue;
|
||||
}
|
||||
CachedBlock cachedBlock =
|
||||
new CachedBlock(block.getBlockId(), (short)0, false);
|
||||
CachedBlock prevCachedBlock = cachedBlocks.get(cachedBlock);
|
||||
// Use the existing CachedBlock if it's present; otherwise,
|
||||
// insert a new one.
|
||||
if (prevCachedBlock != null) {
|
||||
cachedBlock = prevCachedBlock;
|
||||
} else {
|
||||
cachedBlocks.put(cachedBlock);
|
||||
}
|
||||
if (!cachedBlock.isPresent(datanode.getCached())) {
|
||||
datanode.getCached().add(cachedBlock);
|
||||
}
|
||||
if (cachedBlock.isPresent(datanode.getPendingCached())) {
|
||||
datanode.getPendingCached().remove(cachedBlock);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Saves the current state of the CacheManager to the DataOutput. Used
|
||||
* to persist CacheManager state in the FSImage.
|
||||
* @param out DataOutput to persist state
|
||||
* @param sdPath path of the storage directory
|
||||
* @throws IOException
|
||||
*/
|
||||
public void saveState(DataOutput out, String sdPath)
|
||||
throws IOException {
|
||||
out.writeLong(nextEntryId);
|
||||
savePools(out, sdPath);
|
||||
saveEntries(out, sdPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reloads CacheManager state from the passed DataInput. Used during namenode
|
||||
* startup to restore CacheManager state from an FSImage.
|
||||
* @param in DataInput from which to restore state
|
||||
* @throws IOException
|
||||
*/
|
||||
public void loadState(DataInput in) throws IOException {
|
||||
nextEntryId = in.readLong();
|
||||
// pools need to be loaded first since entries point to their parent pool
|
||||
loadPools(in);
|
||||
loadEntries(in);
|
||||
}
|
||||
|
||||
/**
|
||||
* Save cache pools to fsimage
|
||||
*/
|
||||
private void savePools(DataOutput out,
|
||||
String sdPath) throws IOException {
|
||||
StartupProgress prog = NameNode.getStartupProgress();
|
||||
Step step = new Step(StepType.CACHE_POOLS, sdPath);
|
||||
prog.beginStep(Phase.SAVING_CHECKPOINT, step);
|
||||
prog.setTotal(Phase.SAVING_CHECKPOINT, step, cachePools.size());
|
||||
Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
|
||||
out.writeInt(cachePools.size());
|
||||
for (CachePool pool: cachePools.values()) {
|
||||
pool.getInfo(true).writeTo(out);
|
||||
counter.increment();
|
||||
}
|
||||
prog.endStep(Phase.SAVING_CHECKPOINT, step);
|
||||
}
|
||||
|
||||
/*
|
||||
* Save cache entries to fsimage
|
||||
*/
|
||||
private void saveEntries(DataOutput out, String sdPath)
|
||||
throws IOException {
|
||||
StartupProgress prog = NameNode.getStartupProgress();
|
||||
Step step = new Step(StepType.CACHE_ENTRIES, sdPath);
|
||||
prog.beginStep(Phase.SAVING_CHECKPOINT, step);
|
||||
prog.setTotal(Phase.SAVING_CHECKPOINT, step, entriesById.size());
|
||||
Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
|
||||
out.writeInt(entriesById.size());
|
||||
for (PathBasedCacheEntry entry: entriesById.values()) {
|
||||
out.writeLong(entry.getEntryId());
|
||||
Text.writeString(out, entry.getPath());
|
||||
out.writeShort(entry.getReplication());
|
||||
Text.writeString(out, entry.getPool().getPoolName());
|
||||
counter.increment();
|
||||
}
|
||||
prog.endStep(Phase.SAVING_CHECKPOINT, step);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load cache pools from fsimage
|
||||
*/
|
||||
private void loadPools(DataInput in)
|
||||
throws IOException {
|
||||
StartupProgress prog = NameNode.getStartupProgress();
|
||||
Step step = new Step(StepType.CACHE_POOLS);
|
||||
prog.beginStep(Phase.LOADING_FSIMAGE, step);
|
||||
int numberOfPools = in.readInt();
|
||||
prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfPools);
|
||||
Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
|
||||
for (int i = 0; i < numberOfPools; i++) {
|
||||
addCachePool(CachePoolInfo.readFrom(in));
|
||||
counter.increment();
|
||||
}
|
||||
prog.endStep(Phase.LOADING_FSIMAGE, step);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load cache entries from the fsimage
|
||||
*/
|
||||
private void loadEntries(DataInput in) throws IOException {
|
||||
StartupProgress prog = NameNode.getStartupProgress();
|
||||
Step step = new Step(StepType.CACHE_ENTRIES);
|
||||
prog.beginStep(Phase.LOADING_FSIMAGE, step);
|
||||
int numberOfEntries = in.readInt();
|
||||
prog.setTotal(Phase.LOADING_FSIMAGE, step, numberOfEntries);
|
||||
Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
|
||||
for (int i = 0; i < numberOfEntries; i++) {
|
||||
long entryId = in.readLong();
|
||||
String path = Text.readString(in);
|
||||
short replication = in.readShort();
|
||||
String poolName = Text.readString(in);
|
||||
// Get pool reference by looking it up in the map
|
||||
CachePool pool = cachePools.get(poolName);
|
||||
if (pool == null) {
|
||||
throw new IOException("Entry refers to pool " + poolName +
|
||||
", which does not exist.");
|
||||
}
|
||||
PathBasedCacheEntry entry =
|
||||
new PathBasedCacheEntry(entryId, path, replication, pool);
|
||||
if (entriesById.put(entry.getEntryId(), entry) != null) {
|
||||
throw new IOException("An entry with ID " + entry.getEntryId() +
|
||||
" already exists");
|
||||
}
|
||||
List<PathBasedCacheEntry> entries = entriesByPath.get(entry.getPath());
|
||||
if (entries == null) {
|
||||
entries = new LinkedList<PathBasedCacheEntry>();
|
||||
entriesByPath.put(entry.getPath(), entries);
|
||||
}
|
||||
entries.add(entry);
|
||||
counter.increment();
|
||||
}
|
||||
prog.endStep(Phase.LOADING_FSIMAGE, step);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,197 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A CachePool describes a set of cache resources being managed by the NameNode.
|
||||
* User caching requests are billed to the cache pool specified in the request.
|
||||
*
|
||||
* This is an internal class, only used on the NameNode. For identifying or
|
||||
* describing a cache pool to clients, please use CachePoolInfo.
|
||||
*
|
||||
* CachePools must be accessed under the FSNamesystem lock.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class CachePool {
|
||||
public static final Log LOG = LogFactory.getLog(CachePool.class);
|
||||
|
||||
public static final int DEFAULT_WEIGHT = 100;
|
||||
|
||||
@Nonnull
|
||||
private final String poolName;
|
||||
|
||||
@Nonnull
|
||||
private String ownerName;
|
||||
|
||||
@Nonnull
|
||||
private String groupName;
|
||||
|
||||
/**
|
||||
* Cache pool permissions.
|
||||
*
|
||||
* READ permission means that you can list the cache directives in this pool.
|
||||
* WRITE permission means that you can add, remove, or modify cache directives
|
||||
* in this pool.
|
||||
* EXECUTE permission is unused.
|
||||
*/
|
||||
@Nonnull
|
||||
private FsPermission mode;
|
||||
|
||||
private int weight;
|
||||
|
||||
/**
|
||||
* Create a new cache pool based on a CachePoolInfo object and the defaults.
|
||||
* We will fill in information that was not supplied according to the
|
||||
* defaults.
|
||||
*/
|
||||
static CachePool createFromInfoAndDefaults(CachePoolInfo info)
|
||||
throws IOException {
|
||||
UserGroupInformation ugi = null;
|
||||
String ownerName = info.getOwnerName();
|
||||
if (ownerName == null) {
|
||||
if (ugi == null) {
|
||||
ugi = NameNode.getRemoteUser();
|
||||
}
|
||||
ownerName = ugi.getShortUserName();
|
||||
}
|
||||
String groupName = info.getGroupName();
|
||||
if (groupName == null) {
|
||||
if (ugi == null) {
|
||||
ugi = NameNode.getRemoteUser();
|
||||
}
|
||||
groupName = ugi.getPrimaryGroupName();
|
||||
}
|
||||
FsPermission mode = (info.getMode() == null) ?
|
||||
FsPermission.getCachePoolDefault() : info.getMode();
|
||||
Integer weight = (info.getWeight() == null) ?
|
||||
DEFAULT_WEIGHT : info.getWeight();
|
||||
return new CachePool(info.getPoolName(),
|
||||
ownerName, groupName, mode, weight);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new cache pool based on a CachePoolInfo object.
|
||||
* No fields in the CachePoolInfo can be blank.
|
||||
*/
|
||||
static CachePool createFromInfo(CachePoolInfo info) {
|
||||
return new CachePool(info.getPoolName(),
|
||||
info.getOwnerName(), info.getGroupName(),
|
||||
info.getMode(), info.getWeight());
|
||||
}
|
||||
|
||||
CachePool(String poolName, String ownerName, String groupName,
|
||||
FsPermission mode, int weight) {
|
||||
Preconditions.checkNotNull(poolName);
|
||||
Preconditions.checkNotNull(ownerName);
|
||||
Preconditions.checkNotNull(groupName);
|
||||
Preconditions.checkNotNull(mode);
|
||||
this.poolName = poolName;
|
||||
this.ownerName = ownerName;
|
||||
this.groupName = groupName;
|
||||
this.mode = new FsPermission(mode);
|
||||
this.weight = weight;
|
||||
}
|
||||
|
||||
public String getPoolName() {
|
||||
return poolName;
|
||||
}
|
||||
|
||||
public String getOwnerName() {
|
||||
return ownerName;
|
||||
}
|
||||
|
||||
public CachePool setOwnerName(String ownerName) {
|
||||
this.ownerName = ownerName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String getGroupName() {
|
||||
return groupName;
|
||||
}
|
||||
|
||||
public CachePool setGroupName(String groupName) {
|
||||
this.groupName = groupName;
|
||||
return this;
|
||||
}
|
||||
|
||||
public FsPermission getMode() {
|
||||
return mode;
|
||||
}
|
||||
|
||||
public CachePool setMode(FsPermission mode) {
|
||||
this.mode = new FsPermission(mode);
|
||||
return this;
|
||||
}
|
||||
|
||||
public int getWeight() {
|
||||
return weight;
|
||||
}
|
||||
|
||||
public CachePool setWeight(int weight) {
|
||||
this.weight = weight;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get information about this cache pool.
|
||||
*
|
||||
* @param fullInfo
|
||||
* If true, only the name will be returned (i.e., what you
|
||||
* would get if you didn't have read permission for this pool.)
|
||||
* @return
|
||||
* Cache pool information.
|
||||
*/
|
||||
public CachePoolInfo getInfo(boolean fullInfo) {
|
||||
CachePoolInfo info = new CachePoolInfo(poolName);
|
||||
if (!fullInfo) {
|
||||
return info;
|
||||
}
|
||||
return info.setOwnerName(ownerName).
|
||||
setGroupName(groupName).
|
||||
setMode(new FsPermission(mode)).
|
||||
setWeight(weight);
|
||||
}
|
||||
|
||||
public CachePoolInfo getInfo(FSPermissionChecker pc) {
|
||||
return getInfo(pc.checkPermission(this, FsAction.READ));
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
return new StringBuilder().
|
||||
append("{ ").append("poolName:").append(poolName).
|
||||
append(", ownerName:").append(ownerName).
|
||||
append(", groupName:").append(groupName).
|
||||
append(", mode:").append(mode).
|
||||
append(", weight:").append(weight).
|
||||
append(" }").toString();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,251 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
||||
import org.apache.hadoop.util.IntrusiveCollection;
|
||||
import org.apache.hadoop.util.LightWeightGSet;
|
||||
import org.apache.hadoop.util.IntrusiveCollection.Element;
|
||||
import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
|
||||
|
||||
/**
|
||||
* Represents a cached block.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||
public final class CachedBlock implements Element,
|
||||
LightWeightGSet.LinkedElement {
|
||||
private static final Object[] EMPTY_ARRAY = new Object[0];
|
||||
|
||||
/**
|
||||
* Block id.
|
||||
*/
|
||||
private final long blockId;
|
||||
|
||||
/**
|
||||
* Used to implement #{LightWeightGSet.LinkedElement}
|
||||
*/
|
||||
private LinkedElement nextElement;
|
||||
|
||||
/**
|
||||
* Bit 15: Mark
|
||||
* Bit 0-14: cache replication factor.
|
||||
*/
|
||||
private short replicationAndMark;
|
||||
|
||||
/**
|
||||
* Used to implement the CachedBlocksList.
|
||||
*
|
||||
* Since this CachedBlock can be in multiple CachedBlocksList objects,
|
||||
* we need to be able to store multiple 'prev' and 'next' pointers.
|
||||
* The triplets array does this.
|
||||
*
|
||||
* Each triplet contains a CachedBlockList object followed by a
|
||||
* prev pointer, followed by a next pointer.
|
||||
*/
|
||||
private Object[] triplets;
|
||||
|
||||
public CachedBlock(long blockId, short replication, boolean mark) {
|
||||
this.blockId = blockId;
|
||||
this.triplets = EMPTY_ARRAY;
|
||||
setReplicationAndMark(replication, mark);
|
||||
}
|
||||
|
||||
public long getBlockId() {
|
||||
return blockId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (int)(blockId^(blockId>>>32));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (o == null) { return false; }
|
||||
if (o == this) { return true; }
|
||||
if (o.getClass() != this.getClass()) {
|
||||
return false;
|
||||
}
|
||||
CachedBlock other = (CachedBlock)o;
|
||||
return other.blockId == blockId;
|
||||
}
|
||||
|
||||
public void setReplicationAndMark(short replication, boolean mark) {
|
||||
assert replication >= 0;
|
||||
replicationAndMark = (short)((replication << 1) | (mark ? 0x1 : 0x0));
|
||||
}
|
||||
|
||||
public boolean getMark() {
|
||||
return ((replicationAndMark & 0x1) != 0);
|
||||
}
|
||||
|
||||
public short getReplication() {
|
||||
return (short) (replicationAndMark >>> 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if this CachedBlock is present on the given list.
|
||||
*/
|
||||
public boolean isPresent(CachedBlocksList cachedBlocksList) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
CachedBlocksList list = (CachedBlocksList)triplets[i];
|
||||
if (list == cachedBlocksList) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a list of the datanodes which this block is cached,
|
||||
* planned to be cached, or planned to be uncached on.
|
||||
*
|
||||
* @param type If null, this parameter is ignored.
|
||||
* If it is non-null, we match only datanodes which
|
||||
* have it on this list.
|
||||
* See {@link DatanodeDescriptor#CachedBlocksList#Type}
|
||||
* for a description of all the lists.
|
||||
*
|
||||
* @return The list of datanodes. Modifying this list does not
|
||||
* alter the state of the CachedBlock.
|
||||
*/
|
||||
public List<DatanodeDescriptor> getDatanodes(Type type) {
|
||||
List<DatanodeDescriptor> nodes = new LinkedList<DatanodeDescriptor>();
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
CachedBlocksList list = (CachedBlocksList)triplets[i];
|
||||
if ((type == null) || (list.getType() == type)) {
|
||||
nodes.add(list.getDatanode());
|
||||
}
|
||||
}
|
||||
return nodes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void insertInternal(IntrusiveCollection<? extends Element> list, Element prev,
|
||||
Element next) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
if (triplets[i] == list) {
|
||||
throw new RuntimeException("Trying to re-insert an element that " +
|
||||
"is already in the list.");
|
||||
}
|
||||
}
|
||||
Object newTriplets[] = Arrays.copyOf(triplets, triplets.length + 3);
|
||||
newTriplets[triplets.length] = list;
|
||||
newTriplets[triplets.length + 1] = prev;
|
||||
newTriplets[triplets.length + 2] = next;
|
||||
triplets = newTriplets;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPrev(IntrusiveCollection<? extends Element> list, Element prev) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
if (triplets[i] == list) {
|
||||
triplets[i + 1] = prev;
|
||||
return;
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Called setPrev on an element that wasn't " +
|
||||
"in the list.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNext(IntrusiveCollection<? extends Element> list, Element next) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
if (triplets[i] == list) {
|
||||
triplets[i + 2] = next;
|
||||
return;
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Called setNext on an element that wasn't " +
|
||||
"in the list.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeInternal(IntrusiveCollection<? extends Element> list) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
if (triplets[i] == list) {
|
||||
Object[] newTriplets = new Object[triplets.length - 3];
|
||||
System.arraycopy(triplets, 0, newTriplets, 0, i);
|
||||
System.arraycopy(triplets, i + 3, newTriplets, i,
|
||||
triplets.length - (i + 3));
|
||||
triplets = newTriplets;
|
||||
return;
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Called remove on an element that wasn't " +
|
||||
"in the list.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Element getPrev(IntrusiveCollection<? extends Element> list) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
if (triplets[i] == list) {
|
||||
return (Element)triplets[i + 1];
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Called getPrev on an element that wasn't " +
|
||||
"in the list.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Element getNext(IntrusiveCollection<? extends Element> list) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
if (triplets[i] == list) {
|
||||
return (Element)triplets[i + 2];
|
||||
}
|
||||
}
|
||||
throw new RuntimeException("Called getNext on an element that wasn't " +
|
||||
"in the list.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInList(IntrusiveCollection<? extends Element> list) {
|
||||
for (int i = 0; i < triplets.length; i += 3) {
|
||||
if (triplets[i] == list) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuilder().append("{").
|
||||
append("blockId=").append(blockId).append(", ").
|
||||
append("replication=").append(getReplication()).append(", ").
|
||||
append("mark=").append(getMark()).append("}").
|
||||
toString();
|
||||
}
|
||||
|
||||
@Override // LightWeightGSet.LinkedElement
|
||||
public void setNext(LinkedElement next) {
|
||||
this.nextElement = next;
|
||||
}
|
||||
|
||||
@Override // LightWeightGSet.LinkedElement
|
||||
public LinkedElement getNext() {
|
||||
return nextElement;
|
||||
}
|
||||
}
|
|
@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongExce
|
|||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
|
||||
|
@ -2592,12 +2593,21 @@ public class FSDirectory implements Closeable {
|
|||
int childrenNum = node.isDirectory() ?
|
||||
node.asDirectory().getChildrenNum(snapshot) : 0;
|
||||
|
||||
return new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
|
||||
HdfsLocatedFileStatus status =
|
||||
new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
|
||||
blocksize, node.getModificationTime(snapshot),
|
||||
node.getAccessTime(snapshot), node.getFsPermission(snapshot),
|
||||
node.getUserName(snapshot), node.getGroupName(snapshot),
|
||||
node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
|
||||
node.getId(), loc, childrenNum);
|
||||
// Set caching information for the located blocks.
|
||||
if (loc != null) {
|
||||
CacheManager cacheManager = namesystem.getCacheManager();
|
||||
for (LocatedBlock lb: loc.getLocatedBlocks()) {
|
||||
cacheManager.setCachedLocations(lb);
|
||||
}
|
||||
}
|
||||
return status;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import static org.apache.hadoop.util.ExitUtil.terminate;
|
||||
import static org.apache.hadoop.util.Time.now;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -35,15 +36,18 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheEntry;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
|
||||
import static org.apache.hadoop.util.ExitUtil.terminate;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddPathBasedCacheDirectiveOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
|
||||
|
@ -55,12 +59,17 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemovePathBasedCacheDescriptorOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV1Op;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
|
||||
|
@ -69,9 +78,6 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV1Op;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op;
|
||||
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
||||
|
@ -948,6 +954,45 @@ public class FSEditLog implements LogsPurgeable {
|
|||
logEdit(op);
|
||||
}
|
||||
|
||||
void logAddPathBasedCacheDirective(PathBasedCacheDirective directive,
|
||||
boolean toLogRpcIds) {
|
||||
AddPathBasedCacheDirectiveOp op = AddPathBasedCacheDirectiveOp.getInstance(
|
||||
cache.get())
|
||||
.setPath(directive.getPath().toUri().getPath())
|
||||
.setReplication(directive.getReplication())
|
||||
.setPool(directive.getPool());
|
||||
logRpcIds(op, toLogRpcIds);
|
||||
logEdit(op);
|
||||
}
|
||||
|
||||
void logRemovePathBasedCacheDescriptor(Long id, boolean toLogRpcIds) {
|
||||
RemovePathBasedCacheDescriptorOp op =
|
||||
RemovePathBasedCacheDescriptorOp.getInstance(cache.get()).setId(id);
|
||||
logRpcIds(op, toLogRpcIds);
|
||||
logEdit(op);
|
||||
}
|
||||
|
||||
void logAddCachePool(CachePoolInfo pool, boolean toLogRpcIds) {
|
||||
AddCachePoolOp op =
|
||||
AddCachePoolOp.getInstance(cache.get()).setPool(pool);
|
||||
logRpcIds(op, toLogRpcIds);
|
||||
logEdit(op);
|
||||
}
|
||||
|
||||
void logModifyCachePool(CachePoolInfo info, boolean toLogRpcIds) {
|
||||
ModifyCachePoolOp op =
|
||||
ModifyCachePoolOp.getInstance(cache.get()).setInfo(info);
|
||||
logRpcIds(op, toLogRpcIds);
|
||||
logEdit(op);
|
||||
}
|
||||
|
||||
void logRemoveCachePool(String poolName, boolean toLogRpcIds) {
|
||||
RemoveCachePoolOp op =
|
||||
RemoveCachePoolOp.getInstance(cache.get()).setPoolName(poolName);
|
||||
logRpcIds(op, toLogRpcIds);
|
||||
logEdit(op);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all the journals this edit log is currently operating on.
|
||||
*/
|
||||
|
|
|
@ -30,16 +30,21 @@ 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.fs.Path;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddPathBasedCacheDirectiveOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.BlockListUpdatingOp;
|
||||
|
@ -52,7 +57,10 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteSnapshotOp;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemovePathBasedCacheDescriptorOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
|
||||
|
@ -76,6 +84,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Co
|
|||
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
|
||||
import org.apache.hadoop.hdfs.util.ChunkedArrayList;
|
||||
import org.apache.hadoop.hdfs.util.Holder;
|
||||
import org.apache.jasper.tagplugins.jstl.core.Remove;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
|
||||
|
@ -631,6 +640,54 @@ public class FSEditLogLoader {
|
|||
fsNamesys.setLastAllocatedBlockId(allocateBlockIdOp.blockId);
|
||||
break;
|
||||
}
|
||||
case OP_ADD_PATH_BASED_CACHE_DIRECTIVE: {
|
||||
AddPathBasedCacheDirectiveOp addOp = (AddPathBasedCacheDirectiveOp) op;
|
||||
PathBasedCacheDirective d = new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path(addOp.path)).
|
||||
setReplication(addOp.replication).
|
||||
setPool(addOp.pool).
|
||||
build();
|
||||
PathBasedCacheDescriptor descriptor =
|
||||
fsNamesys.getCacheManager().addDirective(d, null);
|
||||
if (toAddRetryCache) {
|
||||
fsNamesys.addCacheEntryWithPayload(op.rpcClientId, op.rpcCallId,
|
||||
descriptor);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR: {
|
||||
RemovePathBasedCacheDescriptorOp removeOp =
|
||||
(RemovePathBasedCacheDescriptorOp) op;
|
||||
fsNamesys.getCacheManager().removeDescriptor(removeOp.id, null);
|
||||
if (toAddRetryCache) {
|
||||
fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OP_ADD_CACHE_POOL: {
|
||||
AddCachePoolOp addOp = (AddCachePoolOp) op;
|
||||
fsNamesys.getCacheManager().addCachePool(addOp.info);
|
||||
if (toAddRetryCache) {
|
||||
fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OP_MODIFY_CACHE_POOL: {
|
||||
ModifyCachePoolOp modifyOp = (ModifyCachePoolOp) op;
|
||||
fsNamesys.getCacheManager().modifyCachePool(modifyOp.info);
|
||||
if (toAddRetryCache) {
|
||||
fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OP_REMOVE_CACHE_POOL: {
|
||||
RemoveCachePoolOp removeOp = (RemoveCachePoolOp) op;
|
||||
fsNamesys.getCacheManager().removeCachePool(removeOp.poolName);
|
||||
if (toAddRetryCache) {
|
||||
fsNamesys.addCacheEntry(op.rpcClientId, op.rpcCallId);
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new IOException("Invalid operation read " + op.opCode);
|
||||
}
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_PATH_BASED_CACHE_DIRECTIVE;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOW_SNAPSHOT;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CANCEL_DELEGATION_TOKEN;
|
||||
|
@ -32,7 +34,10 @@ import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_END_LOG
|
|||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_GET_DELEGATION_TOKEN;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_INVALID;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MKDIR;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MODIFY_CACHE_POOL;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REASSIGN_LEASE;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_CACHE_POOL;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_OLD;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_SNAPSHOT;
|
||||
|
@ -56,6 +61,7 @@ import java.io.DataOutput;
|
|||
import java.io.DataOutputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.EnumMap;
|
||||
import java.util.List;
|
||||
|
@ -73,6 +79,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
|
|||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DeprecatedUTF8;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
|
||||
|
@ -97,7 +104,9 @@ import org.xml.sax.ContentHandler;
|
|||
import org.xml.sax.SAXException;
|
||||
import org.xml.sax.helpers.AttributesImpl;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
|
||||
/**
|
||||
* Helper classes for reading the ops from an InputStream.
|
||||
|
@ -153,6 +162,13 @@ public abstract class FSEditLogOp {
|
|||
inst.put(OP_RENAME_SNAPSHOT, new RenameSnapshotOp());
|
||||
inst.put(OP_SET_GENSTAMP_V2, new SetGenstampV2Op());
|
||||
inst.put(OP_ALLOCATE_BLOCK_ID, new AllocateBlockIdOp());
|
||||
inst.put(OP_ADD_PATH_BASED_CACHE_DIRECTIVE,
|
||||
new AddPathBasedCacheDirectiveOp());
|
||||
inst.put(OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR,
|
||||
new RemovePathBasedCacheDescriptorOp());
|
||||
inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp());
|
||||
inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
|
||||
inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
|
||||
}
|
||||
|
||||
public FSEditLogOp get(FSEditLogOpCodes opcode) {
|
||||
|
@ -528,8 +544,7 @@ public abstract class FSEditLogOp {
|
|||
} else {
|
||||
this.blocks = new Block[0];
|
||||
}
|
||||
this.permissions =
|
||||
permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
|
||||
this.permissions = permissionStatusFromXml(st);
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
}
|
||||
|
@ -1208,8 +1223,7 @@ public abstract class FSEditLogOp {
|
|||
this.inodeId = Long.valueOf(st.getValue("INODEID"));
|
||||
this.path = st.getValue("PATH");
|
||||
this.timestamp = Long.valueOf(st.getValue("TIMESTAMP"));
|
||||
this.permissions =
|
||||
permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
|
||||
this.permissions = permissionStatusFromXml(st);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1940,8 +1954,7 @@ public abstract class FSEditLogOp {
|
|||
this.value = st.getValue("VALUE");
|
||||
this.mtime = Long.valueOf(st.getValue("MTIME"));
|
||||
this.atime = Long.valueOf(st.getValue("ATIME"));
|
||||
this.permissionStatus =
|
||||
permissionStatusFromXml(st.getChildren("PERMISSION_STATUS").get(0));
|
||||
this.permissionStatus = permissionStatusFromXml(st);
|
||||
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
|
@ -2848,6 +2861,317 @@ public abstract class FSEditLogOp {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@literal @AtMostOnce} for
|
||||
* {@link ClientProtocol#addPathBasedCacheDirective}
|
||||
*/
|
||||
static class AddPathBasedCacheDirectiveOp extends FSEditLogOp {
|
||||
String path;
|
||||
short replication;
|
||||
String pool;
|
||||
|
||||
public AddPathBasedCacheDirectiveOp() {
|
||||
super(OP_ADD_PATH_BASED_CACHE_DIRECTIVE);
|
||||
}
|
||||
|
||||
static AddPathBasedCacheDirectiveOp getInstance(OpInstanceCache cache) {
|
||||
return (AddPathBasedCacheDirectiveOp) cache
|
||||
.get(OP_ADD_PATH_BASED_CACHE_DIRECTIVE);
|
||||
}
|
||||
|
||||
public AddPathBasedCacheDirectiveOp setPath(String path) {
|
||||
this.path = path;
|
||||
return this;
|
||||
}
|
||||
|
||||
public AddPathBasedCacheDirectiveOp setReplication(short replication) {
|
||||
this.replication = replication;
|
||||
return this;
|
||||
}
|
||||
|
||||
public AddPathBasedCacheDirectiveOp setPool(String pool) {
|
||||
this.pool = pool;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
void readFields(DataInputStream in, int logVersion) throws IOException {
|
||||
this.path = FSImageSerialization.readString(in);
|
||||
this.replication = FSImageSerialization.readShort(in);
|
||||
this.pool = FSImageSerialization.readString(in);
|
||||
readRpcIds(in, logVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFields(DataOutputStream out) throws IOException {
|
||||
FSImageSerialization.writeString(path, out);
|
||||
FSImageSerialization.writeShort(replication, out);
|
||||
FSImageSerialization.writeString(pool, out);
|
||||
writeRpcIds(rpcClientId, rpcCallId, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "PATH", path);
|
||||
XMLUtils.addSaxString(contentHandler, "REPLICATION",
|
||||
Short.toString(replication));
|
||||
XMLUtils.addSaxString(contentHandler, "POOL", pool);
|
||||
appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
|
||||
}
|
||||
|
||||
@Override
|
||||
void fromXml(Stanza st) throws InvalidXmlException {
|
||||
path = st.getValue("PATH");
|
||||
replication = Short.parseShort(st.getValue("REPLICATION"));
|
||||
pool = st.getValue("POOL");
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("AddPathBasedCacheDirective [");
|
||||
builder.append("path=" + path + ",");
|
||||
builder.append("replication=" + replication + ",");
|
||||
builder.append("pool=" + pool);
|
||||
appendRpcIdsToString(builder, rpcClientId, rpcCallId);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@literal @AtMostOnce} for
|
||||
* {@link ClientProtocol#removePathBasedCacheDescriptor}
|
||||
*/
|
||||
static class RemovePathBasedCacheDescriptorOp extends FSEditLogOp {
|
||||
long id;
|
||||
|
||||
public RemovePathBasedCacheDescriptorOp() {
|
||||
super(OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR);
|
||||
}
|
||||
|
||||
static RemovePathBasedCacheDescriptorOp getInstance(OpInstanceCache cache) {
|
||||
return (RemovePathBasedCacheDescriptorOp) cache
|
||||
.get(OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR);
|
||||
}
|
||||
|
||||
public RemovePathBasedCacheDescriptorOp setId(long id) {
|
||||
this.id = id;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
void readFields(DataInputStream in, int logVersion) throws IOException {
|
||||
this.id = FSImageSerialization.readLong(in);
|
||||
readRpcIds(in, logVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFields(DataOutputStream out) throws IOException {
|
||||
FSImageSerialization.writeLong(id, out);
|
||||
writeRpcIds(rpcClientId, rpcCallId, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "ID", Long.toString(id));
|
||||
appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
|
||||
}
|
||||
|
||||
@Override
|
||||
void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.id = Long.parseLong(st.getValue("ID"));
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("RemovePathBasedCacheDescriptor [");
|
||||
builder.append("id=" + Long.toString(id));
|
||||
appendRpcIdsToString(builder, rpcClientId, rpcCallId);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/** {@literal @AtMostOnce} for {@link ClientProtocol#addCachePool} */
|
||||
static class AddCachePoolOp extends FSEditLogOp {
|
||||
CachePoolInfo info;
|
||||
|
||||
public AddCachePoolOp() {
|
||||
super(OP_ADD_CACHE_POOL);
|
||||
}
|
||||
|
||||
static AddCachePoolOp getInstance(OpInstanceCache cache) {
|
||||
return (AddCachePoolOp) cache.get(OP_ADD_CACHE_POOL);
|
||||
}
|
||||
|
||||
public AddCachePoolOp setPool(CachePoolInfo info) {
|
||||
this.info = info;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
void readFields(DataInputStream in, int logVersion) throws IOException {
|
||||
info = CachePoolInfo.readFrom(in);
|
||||
readRpcIds(in, logVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFields(DataOutputStream out) throws IOException {
|
||||
info .writeTo(out);
|
||||
writeRpcIds(rpcClientId, rpcCallId, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
info.writeXmlTo(contentHandler);
|
||||
appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
|
||||
}
|
||||
|
||||
@Override
|
||||
void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.info = CachePoolInfo.readXmlFrom(st);
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("AddCachePoolOp [");
|
||||
builder.append("poolName=" + info.getPoolName() + ",");
|
||||
builder.append("ownerName=" + info.getOwnerName() + ",");
|
||||
builder.append("groupName=" + info.getGroupName() + ",");
|
||||
builder.append("mode=" + Short.toString(info.getMode().toShort()) + ",");
|
||||
builder.append("weight=" + Integer.toString(info.getWeight()));
|
||||
appendRpcIdsToString(builder, rpcClientId, rpcCallId);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/** {@literal @AtMostOnce} for {@link ClientProtocol#modifyCachePool} */
|
||||
static class ModifyCachePoolOp extends FSEditLogOp {
|
||||
CachePoolInfo info;
|
||||
|
||||
public ModifyCachePoolOp() {
|
||||
super(OP_MODIFY_CACHE_POOL);
|
||||
}
|
||||
|
||||
static ModifyCachePoolOp getInstance(OpInstanceCache cache) {
|
||||
return (ModifyCachePoolOp) cache.get(OP_MODIFY_CACHE_POOL);
|
||||
}
|
||||
|
||||
public ModifyCachePoolOp setInfo(CachePoolInfo info) {
|
||||
this.info = info;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
void readFields(DataInputStream in, int logVersion) throws IOException {
|
||||
info = CachePoolInfo.readFrom(in);
|
||||
readRpcIds(in, logVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFields(DataOutputStream out) throws IOException {
|
||||
info.writeTo(out);
|
||||
writeRpcIds(rpcClientId, rpcCallId, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
cachePoolInfoToXml(contentHandler, info);
|
||||
appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
|
||||
}
|
||||
|
||||
@Override
|
||||
void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.info = cachePoolInfoFromXml(st);
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("ModifyCachePoolOp [");
|
||||
ArrayList<String> fields = new ArrayList<String>(5);
|
||||
if (info.getPoolName() != null) {
|
||||
fields.add("poolName=" + info.getPoolName());
|
||||
}
|
||||
if (info.getOwnerName() != null) {
|
||||
fields.add("ownerName=" + info.getOwnerName());
|
||||
}
|
||||
if (info.getGroupName() != null) {
|
||||
fields.add("groupName=" + info.getGroupName());
|
||||
}
|
||||
if (info.getMode() != null) {
|
||||
fields.add("mode=" + info.getMode().toString());
|
||||
}
|
||||
if (info.getWeight() != null) {
|
||||
fields.add("weight=" + info.getWeight());
|
||||
}
|
||||
builder.append(Joiner.on(",").join(fields));
|
||||
appendRpcIdsToString(builder, rpcClientId, rpcCallId);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/** {@literal @AtMostOnce} for {@link ClientProtocol#removeCachePool} */
|
||||
static class RemoveCachePoolOp extends FSEditLogOp {
|
||||
String poolName;
|
||||
|
||||
public RemoveCachePoolOp() {
|
||||
super(OP_REMOVE_CACHE_POOL);
|
||||
}
|
||||
|
||||
static RemoveCachePoolOp getInstance(OpInstanceCache cache) {
|
||||
return (RemoveCachePoolOp) cache.get(OP_REMOVE_CACHE_POOL);
|
||||
}
|
||||
|
||||
public RemoveCachePoolOp setPoolName(String poolName) {
|
||||
this.poolName = poolName;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
void readFields(DataInputStream in, int logVersion) throws IOException {
|
||||
poolName = FSImageSerialization.readString(in);
|
||||
readRpcIds(in, logVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFields(DataOutputStream out) throws IOException {
|
||||
FSImageSerialization.writeString(poolName, out);
|
||||
writeRpcIds(rpcClientId, rpcCallId, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "POOLNAME", poolName);
|
||||
appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
|
||||
}
|
||||
|
||||
@Override
|
||||
void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.poolName = st.getValue("POOLNAME");
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
builder.append("RemoveCachePoolOp [");
|
||||
builder.append("poolName=" + poolName);
|
||||
appendRpcIdsToString(builder, rpcClientId, rpcCallId);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
||||
|
||||
static private short readShort(DataInputStream in) throws IOException {
|
||||
return Short.parseShort(FSImageSerialization.readString(in));
|
||||
}
|
||||
|
@ -3235,16 +3559,65 @@ public abstract class FSEditLogOp {
|
|||
contentHandler.startElement("", "", "PERMISSION_STATUS", new AttributesImpl());
|
||||
XMLUtils.addSaxString(contentHandler, "USERNAME", perm.getUserName());
|
||||
XMLUtils.addSaxString(contentHandler, "GROUPNAME", perm.getGroupName());
|
||||
XMLUtils.addSaxString(contentHandler, "MODE",
|
||||
Short.valueOf(perm.getPermission().toShort()).toString());
|
||||
fsPermissionToXml(contentHandler, perm.getPermission());
|
||||
contentHandler.endElement("", "", "PERMISSION_STATUS");
|
||||
}
|
||||
|
||||
public static PermissionStatus permissionStatusFromXml(Stanza st)
|
||||
throws InvalidXmlException {
|
||||
String username = st.getValue("USERNAME");
|
||||
String groupname = st.getValue("GROUPNAME");
|
||||
Stanza status = st.getChildren("PERMISSION_STATUS").get(0);
|
||||
String username = status.getValue("USERNAME");
|
||||
String groupname = status.getValue("GROUPNAME");
|
||||
FsPermission mode = fsPermissionFromXml(status);
|
||||
return new PermissionStatus(username, groupname, mode);
|
||||
}
|
||||
|
||||
public static void fsPermissionToXml(ContentHandler contentHandler,
|
||||
FsPermission mode) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "MODE", Short.valueOf(mode.toShort())
|
||||
.toString());
|
||||
}
|
||||
|
||||
public static FsPermission fsPermissionFromXml(Stanza st)
|
||||
throws InvalidXmlException {
|
||||
short mode = Short.valueOf(st.getValue("MODE"));
|
||||
return new PermissionStatus(username, groupname, new FsPermission(mode));
|
||||
return new FsPermission(mode);
|
||||
}
|
||||
|
||||
public static void cachePoolInfoToXml(ContentHandler contentHandler,
|
||||
CachePoolInfo info) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "POOLNAME", info.getPoolName());
|
||||
if (info.getOwnerName() != null) {
|
||||
XMLUtils.addSaxString(contentHandler, "OWNERNAME", info.getOwnerName());
|
||||
}
|
||||
if (info.getGroupName() != null) {
|
||||
XMLUtils.addSaxString(contentHandler, "GROUPNAME", info.getGroupName());
|
||||
}
|
||||
if (info.getMode() != null) {
|
||||
fsPermissionToXml(contentHandler, info.getMode());
|
||||
}
|
||||
if (info.getWeight() != null) {
|
||||
XMLUtils.addSaxString(contentHandler, "WEIGHT",
|
||||
Integer.toString(info.getWeight()));
|
||||
}
|
||||
}
|
||||
|
||||
public static CachePoolInfo cachePoolInfoFromXml(Stanza st)
|
||||
throws InvalidXmlException {
|
||||
String poolName = st.getValue("POOLNAME");
|
||||
CachePoolInfo info = new CachePoolInfo(poolName);
|
||||
if (st.hasChildren("OWNERNAME")) {
|
||||
info.setOwnerName(st.getValue("OWNERNAME"));
|
||||
}
|
||||
if (st.hasChildren("GROUPNAME")) {
|
||||
info.setGroupName(st.getValue("GROUPNAME"));
|
||||
}
|
||||
if (st.hasChildren("MODE")) {
|
||||
info.setMode(FSEditLogOp.fsPermissionFromXml(st));
|
||||
}
|
||||
if (st.hasChildren("WEIGHT")) {
|
||||
info.setWeight(Integer.parseInt(st.getValue("WEIGHT")));
|
||||
}
|
||||
return info;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,7 +63,13 @@ public enum FSEditLogOpCodes {
|
|||
OP_ALLOW_SNAPSHOT ((byte) 29),
|
||||
OP_DISALLOW_SNAPSHOT ((byte) 30),
|
||||
OP_SET_GENSTAMP_V2 ((byte) 31),
|
||||
OP_ALLOCATE_BLOCK_ID ((byte) 32);
|
||||
OP_ALLOCATE_BLOCK_ID ((byte) 32),
|
||||
OP_ADD_PATH_BASED_CACHE_DIRECTIVE ((byte) 33),
|
||||
OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR ((byte) 34),
|
||||
OP_ADD_CACHE_POOL ((byte) 35),
|
||||
OP_MODIFY_CACHE_POOL ((byte) 36),
|
||||
OP_REMOVE_CACHE_POOL ((byte) 37);
|
||||
|
||||
private byte opCode;
|
||||
|
||||
/**
|
||||
|
|
|
@ -351,6 +351,8 @@ public class FSImageFormat {
|
|||
|
||||
loadSecretManagerState(in);
|
||||
|
||||
loadCacheManagerState(in);
|
||||
|
||||
// make sure to read to the end of file
|
||||
boolean eof = (in.read() == -1);
|
||||
assert eof : "Should have reached the end of image file " + curFile;
|
||||
|
@ -843,6 +845,14 @@ public class FSImageFormat {
|
|||
namesystem.loadSecretManagerState(in);
|
||||
}
|
||||
|
||||
private void loadCacheManagerState(DataInput in) throws IOException {
|
||||
int imgVersion = getLayoutVersion();
|
||||
if (!LayoutVersion.supports(Feature.CACHING, imgVersion)) {
|
||||
return;
|
||||
}
|
||||
namesystem.getCacheManager().loadState(in);
|
||||
}
|
||||
|
||||
private int getLayoutVersion() {
|
||||
return namesystem.getFSImage().getStorage().getLayoutVersion();
|
||||
}
|
||||
|
@ -985,6 +995,8 @@ public class FSImageFormat {
|
|||
context.checkCancelled();
|
||||
sourceNamesystem.saveSecretManagerState(out, sdPath);
|
||||
context.checkCancelled();
|
||||
sourceNamesystem.getCacheManager().saveState(out, sdPath);
|
||||
context.checkCancelled();
|
||||
out.flush();
|
||||
context.checkCancelled();
|
||||
fout.getChannel().force(true);
|
||||
|
|
|
@ -121,6 +121,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
|
||||
import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
|
||||
|
@ -144,6 +145,8 @@ import org.apache.hadoop.hdfs.HAUtil;
|
|||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
|
@ -155,6 +158,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
|||
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.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
|
||||
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||
|
@ -379,6 +383,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
FSDirectory dir;
|
||||
private final BlockManager blockManager;
|
||||
private final SnapshotManager snapshotManager;
|
||||
private final CacheManager cacheManager;
|
||||
private final DatanodeStatistics datanodeStatistics;
|
||||
|
||||
// Block pool ID used by this namenode
|
||||
|
@ -714,6 +719,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
this.dtSecretManager = createDelegationTokenSecretManager(conf);
|
||||
this.dir = new FSDirectory(fsImage, this, conf);
|
||||
this.snapshotManager = new SnapshotManager(dir);
|
||||
writeLock();
|
||||
try {
|
||||
this.cacheManager = new CacheManager(this, conf, blockManager);
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
this.safeMode = new SafeModeInfo(conf);
|
||||
this.auditLoggers = initAuditLoggers(conf);
|
||||
this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
|
||||
|
@ -906,6 +917,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
writeLock();
|
||||
try {
|
||||
if (blockManager != null) blockManager.close();
|
||||
cacheManager.deactivate();
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
|
@ -970,6 +982,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
//ResourceMonitor required only at ActiveNN. See HDFS-2914
|
||||
this.nnrmthread = new Daemon(new NameNodeResourceMonitor());
|
||||
nnrmthread.start();
|
||||
cacheManager.activate();
|
||||
blockManager.getDatanodeManager().setSendCachingCommands(true);
|
||||
} finally {
|
||||
writeUnlock();
|
||||
startingActiveService = false;
|
||||
|
@ -1015,6 +1029,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
// so that the tailer starts from the right spot.
|
||||
dir.fsImage.updateLastAppliedTxIdFromWritten();
|
||||
}
|
||||
cacheManager.deactivate();
|
||||
blockManager.getDatanodeManager().setSendCachingCommands(false);
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
|
@ -1588,8 +1604,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
length = Math.min(length, fileSize - offset);
|
||||
isUc = false;
|
||||
}
|
||||
return blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
|
||||
LocatedBlocks blocks =
|
||||
blockManager.createLocatedBlocks(inode.getBlocks(), fileSize,
|
||||
isUc, offset, length, needBlockToken, iip.isSnapshot());
|
||||
// Set caching information for the located blocks.
|
||||
for (LocatedBlock lb: blocks.getLocatedBlocks()) {
|
||||
cacheManager.setCachedLocations(lb);
|
||||
}
|
||||
return blocks;
|
||||
} finally {
|
||||
if (isReadOp) {
|
||||
readUnlock();
|
||||
|
@ -4062,15 +4084,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
*/
|
||||
HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
|
||||
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
|
||||
int xceiverCount, int xmitsInProgress, int failedVolumes)
|
||||
throws IOException {
|
||||
long cacheCapacity, long cacheUsed, int xceiverCount, int xmitsInProgress,
|
||||
int failedVolumes) throws IOException {
|
||||
readLock();
|
||||
try {
|
||||
final int maxTransfer = blockManager.getMaxReplicationStreams()
|
||||
- xmitsInProgress;
|
||||
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
|
||||
nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
|
||||
xceiverCount, maxTransfer, failedVolumes);
|
||||
cacheCapacity, cacheUsed, xceiverCount, maxTransfer, failedVolumes);
|
||||
return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
|
||||
} finally {
|
||||
readUnlock();
|
||||
|
@ -6527,6 +6549,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
public FSDirectory getFSDirectory() {
|
||||
return dir;
|
||||
}
|
||||
/** @return the cache manager. */
|
||||
public CacheManager getCacheManager() {
|
||||
return cacheManager;
|
||||
}
|
||||
|
||||
@Override // NameNodeMXBean
|
||||
public String getCorruptFiles() {
|
||||
|
@ -6903,6 +6929,215 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
}
|
||||
}
|
||||
|
||||
PathBasedCacheDescriptor addPathBasedCacheDirective(
|
||||
PathBasedCacheDirective directive) throws IOException {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
final FSPermissionChecker pc = isPermissionEnabled ?
|
||||
getPermissionChecker() : null;
|
||||
CacheEntryWithPayload cacheEntry =
|
||||
RetryCache.waitForCompletion(retryCache, null);
|
||||
if (cacheEntry != null && cacheEntry.isSuccess()) {
|
||||
return (PathBasedCacheDescriptor) cacheEntry.getPayload();
|
||||
}
|
||||
boolean success = false;
|
||||
PathBasedCacheDescriptor result = null;
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException(
|
||||
"Cannot add PathBasedCache directive", safeMode);
|
||||
}
|
||||
result = cacheManager.addDirective(directive, pc);
|
||||
getEditLog().logAddPathBasedCacheDirective(directive,
|
||||
cacheEntry != null);
|
||||
success = true;
|
||||
} finally {
|
||||
writeUnlock();
|
||||
if (success) {
|
||||
getEditLog().logSync();
|
||||
}
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(success, "addPathBasedCacheDirective", null, null, null);
|
||||
}
|
||||
RetryCache.setState(cacheEntry, success, result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
void removePathBasedCacheDescriptor(Long id) throws IOException {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
final FSPermissionChecker pc = isPermissionEnabled ?
|
||||
getPermissionChecker() : null;
|
||||
CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
||||
if (cacheEntry != null && cacheEntry.isSuccess()) {
|
||||
return;
|
||||
}
|
||||
boolean success = false;
|
||||
writeLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException(
|
||||
"Cannot remove PathBasedCache directives", safeMode);
|
||||
}
|
||||
cacheManager.removeDescriptor(id, pc);
|
||||
getEditLog().logRemovePathBasedCacheDescriptor(id, cacheEntry != null);
|
||||
success = true;
|
||||
} finally {
|
||||
writeUnlock();
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(success, "removePathBasedCacheDescriptor", null, null,
|
||||
null);
|
||||
}
|
||||
RetryCache.setState(cacheEntry, success);
|
||||
}
|
||||
getEditLog().logSync();
|
||||
}
|
||||
|
||||
BatchedListEntries<PathBasedCacheDescriptor> listPathBasedCacheDescriptors(
|
||||
long startId, String pool, String path) throws IOException {
|
||||
checkOperation(OperationCategory.READ);
|
||||
final FSPermissionChecker pc = isPermissionEnabled ?
|
||||
getPermissionChecker() : null;
|
||||
BatchedListEntries<PathBasedCacheDescriptor> results;
|
||||
readLock();
|
||||
boolean success = false;
|
||||
try {
|
||||
checkOperation(OperationCategory.READ);
|
||||
results =
|
||||
cacheManager.listPathBasedCacheDescriptors(startId, pool, path, pc);
|
||||
success = true;
|
||||
} finally {
|
||||
readUnlock();
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(success, "listPathBasedCacheDescriptors", null, null,
|
||||
null);
|
||||
}
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
public void addCachePool(CachePoolInfo req) throws IOException {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
final FSPermissionChecker pc = isPermissionEnabled ?
|
||||
getPermissionChecker() : null;
|
||||
CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
||||
if (cacheEntry != null && cacheEntry.isSuccess()) {
|
||||
return; // Return previous response
|
||||
}
|
||||
writeLock();
|
||||
boolean success = false;
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException(
|
||||
"Cannot add cache pool " + req.getPoolName(), safeMode);
|
||||
}
|
||||
if (pc != null) {
|
||||
pc.checkSuperuserPrivilege();
|
||||
}
|
||||
CachePoolInfo info = cacheManager.addCachePool(req);
|
||||
getEditLog().logAddCachePool(info, cacheEntry != null);
|
||||
success = true;
|
||||
} finally {
|
||||
writeUnlock();
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(success, "addCachePool", req.getPoolName(), null, null);
|
||||
}
|
||||
RetryCache.setState(cacheEntry, success);
|
||||
}
|
||||
|
||||
getEditLog().logSync();
|
||||
}
|
||||
|
||||
public void modifyCachePool(CachePoolInfo req) throws IOException {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
final FSPermissionChecker pc =
|
||||
isPermissionEnabled ? getPermissionChecker() : null;
|
||||
CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
||||
if (cacheEntry != null && cacheEntry.isSuccess()) {
|
||||
return; // Return previous response
|
||||
}
|
||||
writeLock();
|
||||
boolean success = false;
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException(
|
||||
"Cannot modify cache pool " + req.getPoolName(), safeMode);
|
||||
}
|
||||
if (pc != null) {
|
||||
pc.checkSuperuserPrivilege();
|
||||
}
|
||||
cacheManager.modifyCachePool(req);
|
||||
getEditLog().logModifyCachePool(req, cacheEntry != null);
|
||||
success = true;
|
||||
} finally {
|
||||
writeUnlock();
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(success, "modifyCachePool", req.getPoolName(), null, null);
|
||||
}
|
||||
RetryCache.setState(cacheEntry, success);
|
||||
}
|
||||
|
||||
getEditLog().logSync();
|
||||
}
|
||||
|
||||
public void removeCachePool(String cachePoolName) throws IOException {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
final FSPermissionChecker pc =
|
||||
isPermissionEnabled ? getPermissionChecker() : null;
|
||||
CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
|
||||
if (cacheEntry != null && cacheEntry.isSuccess()) {
|
||||
return; // Return previous response
|
||||
}
|
||||
writeLock();
|
||||
boolean success = false;
|
||||
try {
|
||||
checkOperation(OperationCategory.WRITE);
|
||||
if (isInSafeMode()) {
|
||||
throw new SafeModeException(
|
||||
"Cannot remove cache pool " + cachePoolName, safeMode);
|
||||
}
|
||||
if (pc != null) {
|
||||
pc.checkSuperuserPrivilege();
|
||||
}
|
||||
cacheManager.removeCachePool(cachePoolName);
|
||||
getEditLog().logRemoveCachePool(cachePoolName, cacheEntry != null);
|
||||
success = true;
|
||||
} finally {
|
||||
writeUnlock();
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(success, "removeCachePool", cachePoolName, null, null);
|
||||
}
|
||||
RetryCache.setState(cacheEntry, success);
|
||||
}
|
||||
|
||||
getEditLog().logSync();
|
||||
}
|
||||
|
||||
public BatchedListEntries<CachePoolInfo> listCachePools(String prevKey)
|
||||
throws IOException {
|
||||
final FSPermissionChecker pc =
|
||||
isPermissionEnabled ? getPermissionChecker() : null;
|
||||
BatchedListEntries<CachePoolInfo> results;
|
||||
checkOperation(OperationCategory.READ);
|
||||
boolean success = false;
|
||||
readLock();
|
||||
try {
|
||||
checkOperation(OperationCategory.READ);
|
||||
results = cacheManager.listCachePools(pc, prevKey);
|
||||
success = true;
|
||||
} finally {
|
||||
readUnlock();
|
||||
if (isAuditEnabled() && isExternalInvocation()) {
|
||||
logAuditEvent(success, "listCachePools", null, null, null);
|
||||
}
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
/**
|
||||
* Default AuditLogger implementation; used when no access logger is
|
||||
* defined in the config file. It can also be explicitly listed in the
|
||||
|
@ -6960,10 +7195,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
logAuditMessage(sb.toString());
|
||||
}
|
||||
}
|
||||
|
||||
public void logAuditMessage(String message) {
|
||||
auditLog.info(message);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -255,4 +255,30 @@ class FSPermissionChecker {
|
|||
throw new AccessControlException("Permission denied by sticky bit setting:" +
|
||||
" user=" + user + ", inode=" + inode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether a cache pool can be accessed by the current context
|
||||
*
|
||||
* @param pool CachePool being accessed
|
||||
* @param access type of action being performed on the cache pool
|
||||
* @return if the pool can be accessed
|
||||
*/
|
||||
public boolean checkPermission(CachePool pool, FsAction access) {
|
||||
FsPermission mode = pool.getMode();
|
||||
if (isSuperUser()) {
|
||||
return true;
|
||||
}
|
||||
if (user.equals(pool.getOwnerName())
|
||||
&& mode.getUserAction().implies(access)) {
|
||||
return true;
|
||||
}
|
||||
if (groups.contains(pool.getGroupName())
|
||||
&& mode.getGroupAction().implies(access)) {
|
||||
return true;
|
||||
}
|
||||
if (mode.getOtherAction().implies(access)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -686,8 +686,13 @@ public class NameNode implements NameNodeStatusMXBean {
|
|||
try {
|
||||
initializeGenericKeys(conf, nsId, namenodeId);
|
||||
initialize(conf);
|
||||
try {
|
||||
haContext.writeLock();
|
||||
state.prepareToEnterState(haContext);
|
||||
state.enterState(haContext);
|
||||
} finally {
|
||||
haContext.writeUnlock();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
this.stop();
|
||||
throw e;
|
||||
|
|
|
@ -36,6 +36,7 @@ import java.util.Set;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.HadoopIllegalArgumentException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BatchedRemoteIterator;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
|
@ -45,6 +46,7 @@ import org.apache.hadoop.fs.InvalidPathException;
|
|||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.ParentNotDirectoryException;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.UnresolvedLinkException;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.fs.permission.PermissionStatus;
|
||||
|
@ -59,6 +61,9 @@ import org.apache.hadoop.hdfs.DFSUtil;
|
|||
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
|
@ -956,13 +961,14 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
|
||||
@Override // DatanodeProtocol
|
||||
public HeartbeatResponse sendHeartbeat(DatanodeRegistration nodeReg,
|
||||
StorageReport[] report, int xmitsInProgress, int xceiverCount,
|
||||
StorageReport[] report, long dnCacheCapacity, long dnCacheUsed,
|
||||
int xmitsInProgress, int xceiverCount,
|
||||
int failedVolumes) throws IOException {
|
||||
verifyRequest(nodeReg);
|
||||
return namesystem.handleHeartbeat(nodeReg, report[0].getCapacity(),
|
||||
report[0].getDfsUsed(), report[0].getRemaining(),
|
||||
report[0].getBlockPoolUsed(), xceiverCount, xmitsInProgress,
|
||||
failedVolumes);
|
||||
report[0].getBlockPoolUsed(), dnCacheCapacity, dnCacheUsed,
|
||||
xceiverCount, xmitsInProgress, failedVolumes);
|
||||
}
|
||||
|
||||
@Override // DatanodeProtocol
|
||||
|
@ -982,6 +988,18 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DatanodeCommand cacheReport(DatanodeRegistration nodeReg,
|
||||
String poolId, List<Long> blockIds) throws IOException {
|
||||
verifyRequest(nodeReg);
|
||||
if (blockStateChangeLog.isDebugEnabled()) {
|
||||
blockStateChangeLog.debug("*BLOCK* NameNode.cacheReport: "
|
||||
+ "from " + nodeReg + " " + blockIds.size() + " blocks");
|
||||
}
|
||||
namesystem.getCacheManager().processCacheReport(nodeReg, blockIds);
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override // DatanodeProtocol
|
||||
public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
|
||||
StorageReceivedDeletedBlocks[] receivedAndDeletedBlocks) throws IOException {
|
||||
|
@ -1214,4 +1232,87 @@ class NameNodeRpcServer implements NamenodeProtocols {
|
|||
metrics.incrSnapshotDiffReportOps();
|
||||
return report;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PathBasedCacheDescriptor addPathBasedCacheDirective(
|
||||
PathBasedCacheDirective path) throws IOException {
|
||||
return namesystem.addPathBasedCacheDirective(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removePathBasedCacheDescriptor(Long id) throws IOException {
|
||||
namesystem.removePathBasedCacheDescriptor(id);
|
||||
}
|
||||
|
||||
private class ServerSidePathBasedCacheEntriesIterator
|
||||
extends BatchedRemoteIterator<Long, PathBasedCacheDescriptor> {
|
||||
|
||||
private final String pool;
|
||||
|
||||
private final String path;
|
||||
|
||||
public ServerSidePathBasedCacheEntriesIterator(Long firstKey, String pool,
|
||||
String path) {
|
||||
super(firstKey);
|
||||
this.pool = pool;
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BatchedEntries<PathBasedCacheDescriptor> makeRequest(
|
||||
Long nextKey) throws IOException {
|
||||
return namesystem.listPathBasedCacheDescriptors(nextKey, pool, path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long elementToPrevKey(PathBasedCacheDescriptor entry) {
|
||||
return entry.getEntryId();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<PathBasedCacheDescriptor> listPathBasedCacheDescriptors(long prevId,
|
||||
String pool, String path) throws IOException {
|
||||
return new ServerSidePathBasedCacheEntriesIterator(prevId, pool, path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void addCachePool(CachePoolInfo info) throws IOException {
|
||||
namesystem.addCachePool(info);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyCachePool(CachePoolInfo info) throws IOException {
|
||||
namesystem.modifyCachePool(info);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeCachePool(String cachePoolName) throws IOException {
|
||||
namesystem.removeCachePool(cachePoolName);
|
||||
}
|
||||
|
||||
private class ServerSideCachePoolIterator
|
||||
extends BatchedRemoteIterator<String, CachePoolInfo> {
|
||||
|
||||
public ServerSideCachePoolIterator(String prevKey) {
|
||||
super(prevKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BatchedEntries<CachePoolInfo> makeRequest(String prevKey)
|
||||
throws IOException {
|
||||
return namesystem.listCachePools(prevKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String elementToPrevKey(CachePoolInfo element) {
|
||||
return element.getPoolName();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public RemoteIterator<CachePoolInfo> listCachePools(String prevKey)
|
||||
throws IOException {
|
||||
return new ServerSideCachePoolIterator(prevKey);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -79,6 +79,8 @@ public class NameNodeMetrics {
|
|||
MutableCounterLong transactionsBatchedInSync;
|
||||
@Metric("Block report") MutableRate blockReport;
|
||||
MutableQuantiles[] blockReportQuantiles;
|
||||
@Metric("Cache report") MutableRate cacheReport;
|
||||
MutableQuantiles[] cacheReportQuantiles;
|
||||
|
||||
@Metric("Duration in SafeMode at startup in msec")
|
||||
MutableGaugeInt safeModeTime;
|
||||
|
@ -91,6 +93,7 @@ public class NameNodeMetrics {
|
|||
final int len = intervals.length;
|
||||
syncsQuantiles = new MutableQuantiles[len];
|
||||
blockReportQuantiles = new MutableQuantiles[len];
|
||||
cacheReportQuantiles = new MutableQuantiles[len];
|
||||
|
||||
for (int i = 0; i < len; i++) {
|
||||
int interval = intervals[i];
|
||||
|
@ -100,6 +103,9 @@ public class NameNodeMetrics {
|
|||
blockReportQuantiles[i] = registry.newQuantiles(
|
||||
"blockReport" + interval + "s",
|
||||
"Block report", "ops", "latency", interval);
|
||||
cacheReportQuantiles[i] = registry.newQuantiles(
|
||||
"cacheReport" + interval + "s",
|
||||
"Cache report", "ops", "latency", interval);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -229,6 +235,13 @@ public class NameNodeMetrics {
|
|||
}
|
||||
}
|
||||
|
||||
public void addCacheBlockReport(long latency) {
|
||||
cacheReport.add(latency);
|
||||
for (MutableQuantiles q : cacheReportQuantiles) {
|
||||
q.add(latency);
|
||||
}
|
||||
}
|
||||
|
||||
public void setSafeModeTime(long elapsed) {
|
||||
safeModeTime.set((int) elapsed);
|
||||
}
|
||||
|
|
|
@ -42,7 +42,17 @@ public enum StepType {
|
|||
/**
|
||||
* The namenode is performing an operation related to inodes.
|
||||
*/
|
||||
INODES("Inodes", "inodes");
|
||||
INODES("Inodes", "inodes"),
|
||||
|
||||
/**
|
||||
* The namenode is performing an operation related to cache pools.
|
||||
*/
|
||||
CACHE_POOLS("CachePools", "cache pools"),
|
||||
|
||||
/**
|
||||
* The namenode is performing an operation related to cache entries.
|
||||
*/
|
||||
CACHE_ENTRIES("CacheEntries", "cache entries");
|
||||
|
||||
private final String name, description;
|
||||
|
||||
|
|
|
@ -0,0 +1,50 @@
|
|||
/**
|
||||
* 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.protocol;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/****************************************************
|
||||
* A BlockIdCommand is an instruction to a datanode
|
||||
* regarding some blocks under its control.
|
||||
****************************************************/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
public class BlockIdCommand extends DatanodeCommand {
|
||||
final String poolId;
|
||||
final long blockIds[];
|
||||
|
||||
/**
|
||||
* Create BlockCommand for the given action
|
||||
* @param blocks blocks related to the action
|
||||
*/
|
||||
public BlockIdCommand(int action, String poolId, long[] blockIds) {
|
||||
super(action);
|
||||
this.poolId = poolId;
|
||||
this.blockIds= blockIds;
|
||||
}
|
||||
|
||||
public String getBlockPoolId() {
|
||||
return poolId;
|
||||
}
|
||||
|
||||
public long[] getBlockIds() {
|
||||
return blockIds;
|
||||
}
|
||||
}
|
|
@ -19,13 +19,14 @@
|
|||
package org.apache.hadoop.hdfs.server.protocol;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.io.retry.AtMostOnce;
|
||||
import org.apache.hadoop.io.retry.Idempotent;
|
||||
import org.apache.hadoop.security.KerberosInfo;
|
||||
|
||||
|
@ -74,6 +75,8 @@ public interface DatanodeProtocol {
|
|||
final static int DNA_RECOVERBLOCK = 6; // request a block recovery
|
||||
final static int DNA_ACCESSKEYUPDATE = 7; // update access key
|
||||
final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
|
||||
final static int DNA_CACHE = 9; // cache blocks
|
||||
final static int DNA_UNCACHE = 10; // uncache blocks
|
||||
|
||||
/**
|
||||
* Register Datanode.
|
||||
|
@ -104,6 +107,8 @@ public interface DatanodeProtocol {
|
|||
@Idempotent
|
||||
public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
|
||||
StorageReport[] reports,
|
||||
long dnCacheCapacity,
|
||||
long dnCacheUsed,
|
||||
int xmitsInProgress,
|
||||
int xceiverCount,
|
||||
int failedVolumes) throws IOException;
|
||||
|
@ -128,6 +133,24 @@ public interface DatanodeProtocol {
|
|||
public DatanodeCommand blockReport(DatanodeRegistration registration,
|
||||
String poolId, StorageBlockReport[] reports) throws IOException;
|
||||
|
||||
|
||||
/**
|
||||
* Communicates the complete list of locally cached blocks to the NameNode.
|
||||
*
|
||||
* This method is similar to
|
||||
* {@link #blockReport(DatanodeRegistration, String, StorageBlockReport[])},
|
||||
* which is used to communicated blocks stored on disk.
|
||||
*
|
||||
* @param The datanode registration.
|
||||
* @param poolId The block pool ID for the blocks.
|
||||
* @param blockIds A list of block IDs.
|
||||
* @return The DatanodeCommand.
|
||||
* @throws IOException
|
||||
*/
|
||||
@Idempotent
|
||||
public DatanodeCommand cacheReport(DatanodeRegistration registration,
|
||||
String poolId, List<Long> blockIds) throws IOException;
|
||||
|
||||
/**
|
||||
* blockReceivedAndDeleted() allows the DataNode to tell the NameNode about
|
||||
* recently-received and -deleted block data.
|
||||
|
|
|
@ -0,0 +1,775 @@
|
|||
/**
|
||||
* 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.tools;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.lang.WordUtils;
|
||||
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.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CachePool;
|
||||
import org.apache.hadoop.hdfs.tools.TableListing.Justification;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
|
||||
/**
|
||||
* This class implements command-line operations on the HDFS Cache.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class CacheAdmin extends Configured implements Tool {
|
||||
|
||||
/**
|
||||
* Maximum length for printed lines
|
||||
*/
|
||||
private static final int MAX_LINE_WIDTH = 80;
|
||||
|
||||
public CacheAdmin() {
|
||||
this(null);
|
||||
}
|
||||
|
||||
public CacheAdmin(Configuration conf) {
|
||||
super(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(String[] args) throws IOException {
|
||||
if (args.length == 0) {
|
||||
printUsage(false);
|
||||
return 1;
|
||||
}
|
||||
Command command = determineCommand(args[0]);
|
||||
if (command == null) {
|
||||
System.err.println("Can't understand command '" + args[0] + "'");
|
||||
if (!args[0].startsWith("-")) {
|
||||
System.err.println("Command names must start with dashes.");
|
||||
}
|
||||
printUsage(false);
|
||||
return 1;
|
||||
}
|
||||
List<String> argsList = new LinkedList<String>();
|
||||
for (int j = 1; j < args.length; j++) {
|
||||
argsList.add(args[j]);
|
||||
}
|
||||
return command.run(getConf(), argsList);
|
||||
}
|
||||
|
||||
public static void main(String[] argsArray) throws IOException {
|
||||
CacheAdmin cacheAdmin = new CacheAdmin(new Configuration());
|
||||
System.exit(cacheAdmin.run(argsArray));
|
||||
}
|
||||
|
||||
private static DistributedFileSystem getDFS(Configuration conf)
|
||||
throws IOException {
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
if (!(fs instanceof DistributedFileSystem)) {
|
||||
throw new IllegalArgumentException("FileSystem " + fs.getUri() +
|
||||
" is not an HDFS file system");
|
||||
}
|
||||
return (DistributedFileSystem)fs;
|
||||
}
|
||||
|
||||
/**
|
||||
* NN exceptions contain the stack trace as part of the exception message.
|
||||
* When it's a known error, pretty-print the error and squish the stack trace.
|
||||
*/
|
||||
private static String prettifyException(Exception e) {
|
||||
return e.getClass().getSimpleName() + ": "
|
||||
+ e.getLocalizedMessage().split("\n")[0];
|
||||
}
|
||||
|
||||
private static TableListing getOptionDescriptionListing() {
|
||||
TableListing listing = new TableListing.Builder()
|
||||
.addField("").addField("", true)
|
||||
.wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
|
||||
return listing;
|
||||
}
|
||||
|
||||
interface Command {
|
||||
String getName();
|
||||
String getShortUsage();
|
||||
String getLongUsage();
|
||||
int run(Configuration conf, List<String> args) throws IOException;
|
||||
}
|
||||
|
||||
private static class AddPathBasedCacheDirectiveCommand implements Command {
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-addDirective";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + getName() +
|
||||
" -path <path> -replication <replication> -pool <pool-name>]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
listing.addRow("<path>", "A path to cache. The path can be " +
|
||||
"a directory or a file.");
|
||||
listing.addRow("<replication>", "The cache replication factor to use. " +
|
||||
"Defaults to 1.");
|
||||
listing.addRow("<pool-name>", "The pool to which the directive will be " +
|
||||
"added. You must have write permission on the cache pool "
|
||||
+ "in order to add new directives.");
|
||||
return getShortUsage() + "\n" +
|
||||
"Add a new PathBasedCache directive.\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String path = StringUtils.popOptionWithArgument("-path", args);
|
||||
if (path == null) {
|
||||
System.err.println("You must specify a path with -path.");
|
||||
return 1;
|
||||
}
|
||||
short replication = 1;
|
||||
String replicationString =
|
||||
StringUtils.popOptionWithArgument("-replication", args);
|
||||
if (replicationString != null) {
|
||||
replication = Short.parseShort(replicationString);
|
||||
}
|
||||
String poolName = StringUtils.popOptionWithArgument("-pool", args);
|
||||
if (poolName == null) {
|
||||
System.err.println("You must specify a pool name with -pool.");
|
||||
return 1;
|
||||
}
|
||||
if (!args.isEmpty()) {
|
||||
System.err.println("Can't understand argument: " + args.get(0));
|
||||
return 1;
|
||||
}
|
||||
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
PathBasedCacheDirective directive = new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path(path)).
|
||||
setReplication(replication).
|
||||
setPool(poolName).
|
||||
build();
|
||||
try {
|
||||
PathBasedCacheDescriptor descriptor =
|
||||
dfs.addPathBasedCacheDirective(directive);
|
||||
System.out.println("Added PathBasedCache entry "
|
||||
+ descriptor.getEntryId());
|
||||
} catch (AddPathBasedCacheDirectiveException e) {
|
||||
System.err.println(prettifyException(e));
|
||||
return 2;
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static class RemovePathBasedCacheDirectiveCommand implements Command {
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-removeDirective";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + getName() + " <id>]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
listing.addRow("<id>", "The id of the cache directive to remove. " +
|
||||
"You must have write permission on the pool of the " +
|
||||
"directive in order to remove it. To see a list " +
|
||||
"of PathBasedCache directive IDs, use the -listDirectives command.");
|
||||
return getShortUsage() + "\n" +
|
||||
"Remove a cache directive.\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String idString= StringUtils.popFirstNonOption(args);
|
||||
if (idString == null) {
|
||||
System.err.println("You must specify a directive ID to remove.");
|
||||
return 1;
|
||||
}
|
||||
long id;
|
||||
try {
|
||||
id = Long.valueOf(idString);
|
||||
} catch (NumberFormatException e) {
|
||||
System.err.println("Invalid directive ID " + idString + ": expected " +
|
||||
"a numeric value.");
|
||||
return 1;
|
||||
}
|
||||
if (id <= 0) {
|
||||
System.err.println("Invalid directive ID " + id + ": ids must " +
|
||||
"be greater than 0.");
|
||||
return 1;
|
||||
}
|
||||
if (!args.isEmpty()) {
|
||||
System.err.println("Can't understand argument: " + args.get(0));
|
||||
System.err.println("Usage is " + getShortUsage());
|
||||
return 1;
|
||||
}
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
try {
|
||||
dfs.getClient().removePathBasedCacheDescriptor(id);
|
||||
System.out.println("Removed PathBasedCache directive " + id);
|
||||
} catch (RemovePathBasedCacheDescriptorException e) {
|
||||
System.err.println(prettifyException(e));
|
||||
return 2;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static class RemovePathBasedCacheDirectivesCommand implements Command {
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-removeDirectives";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + getName() + " <path>]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
listing.addRow("<path>", "The path of the cache directives to remove. " +
|
||||
"You must have write permission on the pool of the directive in order " +
|
||||
"to remove it. To see a list of cache directives, use the " +
|
||||
"-listDirectives command.");
|
||||
return getShortUsage() + "\n" +
|
||||
"Remove every cache directive with the specified path.\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String path = StringUtils.popOptionWithArgument("-path", args);
|
||||
if (path == null) {
|
||||
System.err.println("You must specify a path with -path.");
|
||||
return 1;
|
||||
}
|
||||
if (!args.isEmpty()) {
|
||||
System.err.println("Can't understand argument: " + args.get(0));
|
||||
System.err.println("Usage is " + getShortUsage());
|
||||
return 1;
|
||||
}
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
RemoteIterator<PathBasedCacheDescriptor> iter =
|
||||
dfs.listPathBasedCacheDescriptors(null, new Path(path));
|
||||
int exitCode = 0;
|
||||
while (iter.hasNext()) {
|
||||
PathBasedCacheDescriptor entry = iter.next();
|
||||
try {
|
||||
dfs.removePathBasedCacheDescriptor(entry);
|
||||
System.out.println("Removed PathBasedCache directive " +
|
||||
entry.getEntryId());
|
||||
} catch (RemovePathBasedCacheDescriptorException e) {
|
||||
System.err.println(prettifyException(e));
|
||||
exitCode = 2;
|
||||
}
|
||||
}
|
||||
if (exitCode == 0) {
|
||||
System.out.println("Removed every PathBasedCache directive with path " +
|
||||
path);
|
||||
}
|
||||
return exitCode;
|
||||
}
|
||||
}
|
||||
|
||||
private static class ListPathBasedCacheDirectiveCommand implements Command {
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-listDirectives";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + getName() + " [-path <path>] [-pool <pool>]]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
listing.addRow("<path>", "List only " +
|
||||
"PathBasedCache directives with this path. " +
|
||||
"Note that if there is a PathBasedCache directive for <path> " +
|
||||
"in a cache pool that we don't have read access for, it " +
|
||||
"will not be listed.");
|
||||
listing.addRow("<pool>", "List only path cache directives in that pool.");
|
||||
return getShortUsage() + "\n" +
|
||||
"List PathBasedCache directives.\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String pathFilter = StringUtils.popOptionWithArgument("-path", args);
|
||||
String poolFilter = StringUtils.popOptionWithArgument("-pool", args);
|
||||
if (!args.isEmpty()) {
|
||||
System.err.println("Can't understand argument: " + args.get(0));
|
||||
return 1;
|
||||
}
|
||||
TableListing tableListing = new TableListing.Builder().
|
||||
addField("ID", Justification.LEFT).
|
||||
addField("POOL", Justification.LEFT).
|
||||
addField("PATH", Justification.LEFT).
|
||||
build();
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
RemoteIterator<PathBasedCacheDescriptor> iter =
|
||||
dfs.listPathBasedCacheDescriptors(poolFilter, pathFilter != null ?
|
||||
new Path(pathFilter) : null);
|
||||
int numEntries = 0;
|
||||
while (iter.hasNext()) {
|
||||
PathBasedCacheDescriptor entry = iter.next();
|
||||
String row[] = new String[] {
|
||||
"" + entry.getEntryId(), entry.getPool(),
|
||||
entry.getPath().toUri().getPath(),
|
||||
};
|
||||
tableListing.addRow(row);
|
||||
numEntries++;
|
||||
}
|
||||
System.out.print(String.format("Found %d entr%s\n",
|
||||
numEntries, numEntries == 1 ? "y" : "ies"));
|
||||
if (numEntries > 0) {
|
||||
System.out.print(tableListing);
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static class AddCachePoolCommand implements Command {
|
||||
|
||||
private static final String NAME = "-addPool";
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + NAME + " <name> [-owner <owner>] " +
|
||||
"[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
|
||||
listing.addRow("<name>", "Name of the new pool.");
|
||||
listing.addRow("<owner>", "Username of the owner of the pool. " +
|
||||
"Defaults to the current user.");
|
||||
listing.addRow("<group>", "Group of the pool. " +
|
||||
"Defaults to the primary group name of the current user.");
|
||||
listing.addRow("<mode>", "UNIX-style permissions for the pool. " +
|
||||
"Permissions are specified in octal, e.g. 0755. " +
|
||||
"By default, this is set to " + String.format("0%03o",
|
||||
FsPermission.getCachePoolDefault().toShort()));
|
||||
listing.addRow("<weight>", "Weight of the pool. " +
|
||||
"This is a relative measure of the importance of the pool used " +
|
||||
"during cache resource management. By default, it is set to " +
|
||||
CachePool.DEFAULT_WEIGHT);
|
||||
|
||||
return getShortUsage() + "\n" +
|
||||
"Add a new cache pool.\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String owner = StringUtils.popOptionWithArgument("-owner", args);
|
||||
if (owner == null) {
|
||||
owner = UserGroupInformation.getCurrentUser().getShortUserName();
|
||||
}
|
||||
String group = StringUtils.popOptionWithArgument("-group", args);
|
||||
if (group == null) {
|
||||
group = UserGroupInformation.getCurrentUser().getGroupNames()[0];
|
||||
}
|
||||
String modeString = StringUtils.popOptionWithArgument("-mode", args);
|
||||
int mode;
|
||||
if (modeString == null) {
|
||||
mode = FsPermission.getCachePoolDefault().toShort();
|
||||
} else {
|
||||
mode = Integer.parseInt(modeString, 8);
|
||||
}
|
||||
String weightString = StringUtils.popOptionWithArgument("-weight", args);
|
||||
int weight;
|
||||
if (weightString == null) {
|
||||
weight = CachePool.DEFAULT_WEIGHT;
|
||||
} else {
|
||||
weight = Integer.parseInt(weightString);
|
||||
}
|
||||
String name = StringUtils.popFirstNonOption(args);
|
||||
if (name == null) {
|
||||
System.err.println("You must specify a name when creating a " +
|
||||
"cache pool.");
|
||||
return 1;
|
||||
}
|
||||
if (!args.isEmpty()) {
|
||||
System.err.print("Can't understand arguments: " +
|
||||
Joiner.on(" ").join(args) + "\n");
|
||||
System.err.println("Usage is " + getShortUsage());
|
||||
return 1;
|
||||
}
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
CachePoolInfo info = new CachePoolInfo(name).
|
||||
setOwnerName(owner).
|
||||
setGroupName(group).
|
||||
setMode(new FsPermission((short)mode)).
|
||||
setWeight(weight);
|
||||
try {
|
||||
dfs.addCachePool(info);
|
||||
} catch (IOException e) {
|
||||
throw new RemoteException(e.getClass().getName(), e.getMessage());
|
||||
}
|
||||
System.out.println("Successfully added cache pool " + name + ".");
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static class ModifyCachePoolCommand implements Command {
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-modifyPool";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + getName() + " <name> [-owner <owner>] " +
|
||||
"[-group <group>] [-mode <mode>] [-weight <weight>]]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
|
||||
listing.addRow("<name>", "Name of the pool to modify.");
|
||||
listing.addRow("<owner>", "Username of the owner of the pool");
|
||||
listing.addRow("<group>", "Groupname of the group of the pool.");
|
||||
listing.addRow("<mode>", "Unix-style permissions of the pool in octal.");
|
||||
listing.addRow("<weight>", "Weight of the pool.");
|
||||
|
||||
return getShortUsage() + "\n" +
|
||||
WordUtils.wrap("Modifies the metadata of an existing cache pool. " +
|
||||
"See usage of " + AddCachePoolCommand.NAME + " for more details",
|
||||
MAX_LINE_WIDTH) + "\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String owner = StringUtils.popOptionWithArgument("-owner", args);
|
||||
String group = StringUtils.popOptionWithArgument("-group", args);
|
||||
String modeString = StringUtils.popOptionWithArgument("-mode", args);
|
||||
Integer mode = (modeString == null) ?
|
||||
null : Integer.parseInt(modeString, 8);
|
||||
String weightString = StringUtils.popOptionWithArgument("-weight", args);
|
||||
Integer weight = (weightString == null) ?
|
||||
null : Integer.parseInt(weightString);
|
||||
String name = StringUtils.popFirstNonOption(args);
|
||||
if (name == null) {
|
||||
System.err.println("You must specify a name when creating a " +
|
||||
"cache pool.");
|
||||
return 1;
|
||||
}
|
||||
if (!args.isEmpty()) {
|
||||
System.err.print("Can't understand arguments: " +
|
||||
Joiner.on(" ").join(args) + "\n");
|
||||
System.err.println("Usage is " + getShortUsage());
|
||||
return 1;
|
||||
}
|
||||
boolean changed = false;
|
||||
CachePoolInfo info = new CachePoolInfo(name);
|
||||
if (owner != null) {
|
||||
info.setOwnerName(owner);
|
||||
changed = true;
|
||||
}
|
||||
if (group != null) {
|
||||
info.setGroupName(group);
|
||||
changed = true;
|
||||
}
|
||||
if (mode != null) {
|
||||
info.setMode(new FsPermission(mode.shortValue()));
|
||||
changed = true;
|
||||
}
|
||||
if (weight != null) {
|
||||
info.setWeight(weight);
|
||||
changed = true;
|
||||
}
|
||||
if (!changed) {
|
||||
System.err.println("You must specify at least one attribute to " +
|
||||
"change in the cache pool.");
|
||||
return 1;
|
||||
}
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
try {
|
||||
dfs.modifyCachePool(info);
|
||||
} catch (IOException e) {
|
||||
throw new RemoteException(e.getClass().getName(), e.getMessage());
|
||||
}
|
||||
System.out.print("Successfully modified cache pool " + name);
|
||||
String prefix = " to have ";
|
||||
if (owner != null) {
|
||||
System.out.print(prefix + "owner name " + owner);
|
||||
prefix = " and ";
|
||||
}
|
||||
if (group != null) {
|
||||
System.out.print(prefix + "group name " + group);
|
||||
prefix = " and ";
|
||||
}
|
||||
if (mode != null) {
|
||||
System.out.print(prefix + "mode " + new FsPermission(mode.shortValue()));
|
||||
prefix = " and ";
|
||||
}
|
||||
if (weight != null) {
|
||||
System.out.print(prefix + "weight " + weight);
|
||||
prefix = " and ";
|
||||
}
|
||||
System.out.print("\n");
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static class RemoveCachePoolCommand implements Command {
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-removePool";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + getName() + " <name>]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
return getShortUsage() + "\n" +
|
||||
WordUtils.wrap("Remove a cache pool. This also uncaches paths " +
|
||||
"associated with the pool.\n\n", MAX_LINE_WIDTH) +
|
||||
"<name> Name of the cache pool to remove.\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String name = StringUtils.popFirstNonOption(args);
|
||||
if (name == null) {
|
||||
System.err.println("You must specify a name when deleting a " +
|
||||
"cache pool.");
|
||||
return 1;
|
||||
}
|
||||
if (!args.isEmpty()) {
|
||||
System.err.print("Can't understand arguments: " +
|
||||
Joiner.on(" ").join(args) + "\n");
|
||||
System.err.println("Usage is " + getShortUsage());
|
||||
return 1;
|
||||
}
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
try {
|
||||
dfs.removeCachePool(name);
|
||||
} catch (IOException e) {
|
||||
throw new RemoteException(e.getClass().getName(), e.getMessage());
|
||||
}
|
||||
System.out.println("Successfully removed cache pool " + name + ".");
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static class ListCachePoolsCommand implements Command {
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-listPools";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[" + getName() + " [name]]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
listing.addRow("[name]", "If specified, list only the named cache pool.");
|
||||
|
||||
return getShortUsage() + "\n" +
|
||||
WordUtils.wrap("Display information about one or more cache pools, " +
|
||||
"e.g. name, owner, group, permissions, etc.", MAX_LINE_WIDTH) +
|
||||
"\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
String name = StringUtils.popFirstNonOption(args);
|
||||
if (!args.isEmpty()) {
|
||||
System.err.print("Can't understand arguments: " +
|
||||
Joiner.on(" ").join(args) + "\n");
|
||||
System.err.println("Usage is " + getShortUsage());
|
||||
return 1;
|
||||
}
|
||||
DistributedFileSystem dfs = getDFS(conf);
|
||||
TableListing listing = new TableListing.Builder().
|
||||
addField("NAME", Justification.LEFT).
|
||||
addField("OWNER", Justification.LEFT).
|
||||
addField("GROUP", Justification.LEFT).
|
||||
addField("MODE", Justification.LEFT).
|
||||
addField("WEIGHT", Justification.LEFT).
|
||||
build();
|
||||
int numResults = 0;
|
||||
try {
|
||||
RemoteIterator<CachePoolInfo> iter = dfs.listCachePools();
|
||||
while (iter.hasNext()) {
|
||||
CachePoolInfo info = iter.next();
|
||||
if (name == null || info.getPoolName().equals(name)) {
|
||||
listing.addRow(new String[] {
|
||||
info.getPoolName(),
|
||||
info.getOwnerName(),
|
||||
info.getGroupName(),
|
||||
info.getMode().toString(),
|
||||
info.getWeight().toString(),
|
||||
});
|
||||
++numResults;
|
||||
if (name != null) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RemoteException(e.getClass().getName(), e.getMessage());
|
||||
}
|
||||
System.out.print(String.format("Found %d result%s.\n", numResults,
|
||||
(numResults == 1 ? "" : "s")));
|
||||
if (numResults > 0) {
|
||||
System.out.print(listing);
|
||||
}
|
||||
// If there are no results, we return 1 (failure exit code);
|
||||
// otherwise we return 0 (success exit code).
|
||||
return (numResults == 0) ? 1 : 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static class HelpCommand implements Command {
|
||||
@Override
|
||||
public String getName() {
|
||||
return "-help";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getShortUsage() {
|
||||
return "[-help <command-name>]\n";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLongUsage() {
|
||||
TableListing listing = getOptionDescriptionListing();
|
||||
listing.addRow("<command-name>", "The command for which to get " +
|
||||
"detailed help. If no command is specified, print detailed help for " +
|
||||
"all commands");
|
||||
return getShortUsage() + "\n" +
|
||||
"Get detailed help about a command.\n\n" +
|
||||
listing.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int run(Configuration conf, List<String> args) throws IOException {
|
||||
if (args.size() == 0) {
|
||||
for (Command command : COMMANDS) {
|
||||
System.err.println(command.getLongUsage());
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
if (args.size() != 1) {
|
||||
System.out.println("You must give exactly one argument to -help.");
|
||||
return 0;
|
||||
}
|
||||
String commandName = args.get(0);
|
||||
commandName = commandName.replaceAll("^[-]*", "");
|
||||
Command command = determineCommand(commandName);
|
||||
if (command == null) {
|
||||
System.err.print("Sorry, I don't know the command '" +
|
||||
commandName + "'.\n");
|
||||
System.err.print("Valid command names are:\n");
|
||||
String separator = "";
|
||||
for (Command c : COMMANDS) {
|
||||
System.err.print(separator + c.getName());
|
||||
separator = ", ";
|
||||
}
|
||||
System.err.print("\n");
|
||||
return 1;
|
||||
}
|
||||
System.err.print(command.getLongUsage());
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
private static Command[] COMMANDS = {
|
||||
new AddPathBasedCacheDirectiveCommand(),
|
||||
new RemovePathBasedCacheDirectiveCommand(),
|
||||
new RemovePathBasedCacheDirectivesCommand(),
|
||||
new ListPathBasedCacheDirectiveCommand(),
|
||||
new AddCachePoolCommand(),
|
||||
new ModifyCachePoolCommand(),
|
||||
new RemoveCachePoolCommand(),
|
||||
new ListCachePoolsCommand(),
|
||||
new HelpCommand(),
|
||||
};
|
||||
|
||||
private static void printUsage(boolean longUsage) {
|
||||
System.err.println(
|
||||
"Usage: bin/hdfs cacheadmin [COMMAND]");
|
||||
for (Command command : COMMANDS) {
|
||||
if (longUsage) {
|
||||
System.err.print(command.getLongUsage());
|
||||
} else {
|
||||
System.err.print(" " + command.getShortUsage());
|
||||
}
|
||||
}
|
||||
System.err.println();
|
||||
}
|
||||
|
||||
private static Command determineCommand(String commandName) {
|
||||
for (int i = 0; i < COMMANDS.length; i++) {
|
||||
if (COMMANDS[i].getName().equals(commandName)) {
|
||||
return COMMANDS[i];
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,281 @@
|
|||
/**
|
||||
* 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.tools;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.commons.lang.WordUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* This class implements a "table listing" with column headers.
|
||||
*
|
||||
* Example:
|
||||
*
|
||||
* NAME OWNER GROUP MODE WEIGHT
|
||||
* pool1 andrew andrew rwxr-xr-x 100
|
||||
* pool2 andrew andrew rwxr-xr-x 100
|
||||
* pool3 andrew andrew rwxr-xr-x 100
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class TableListing {
|
||||
public enum Justification {
|
||||
LEFT,
|
||||
RIGHT;
|
||||
}
|
||||
|
||||
private static class Column {
|
||||
private final ArrayList<String> rows;
|
||||
private final Justification justification;
|
||||
private final boolean wrap;
|
||||
|
||||
private int wrapWidth = Integer.MAX_VALUE;
|
||||
private int maxWidth;
|
||||
|
||||
Column(String title, Justification justification, boolean wrap) {
|
||||
this.rows = new ArrayList<String>();
|
||||
this.justification = justification;
|
||||
this.wrap = wrap;
|
||||
this.maxWidth = 0;
|
||||
addRow(title);
|
||||
}
|
||||
|
||||
private void addRow(String val) {
|
||||
if ((val.length() + 1) > maxWidth) {
|
||||
maxWidth = val.length() + 1;
|
||||
}
|
||||
// Ceiling at wrapWidth, because it'll get wrapped
|
||||
if (maxWidth > wrapWidth) {
|
||||
maxWidth = wrapWidth;
|
||||
}
|
||||
rows.add(val);
|
||||
}
|
||||
|
||||
private int getMaxWidth() {
|
||||
return maxWidth;
|
||||
}
|
||||
|
||||
private void setWrapWidth(int width) {
|
||||
wrapWidth = width;
|
||||
// Ceiling the maxLength at wrapWidth
|
||||
if (maxWidth > wrapWidth) {
|
||||
maxWidth = wrapWidth;
|
||||
}
|
||||
// Else we need to traverse through and find the real maxWidth
|
||||
else {
|
||||
maxWidth = 0;
|
||||
for (int i=0; i<rows.size(); i++) {
|
||||
int length = rows.get(i).length();
|
||||
if (length > maxWidth) {
|
||||
maxWidth = length;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the ith row of the column as a set of wrapped strings, each at
|
||||
* most wrapWidth in length.
|
||||
*/
|
||||
String[] getRow(int idx) {
|
||||
String raw = rows.get(idx);
|
||||
// Line-wrap if it's too long
|
||||
String[] lines = new String[] {raw};
|
||||
if (wrap) {
|
||||
lines = WordUtils.wrap(lines[0], wrapWidth, "\n", true).split("\n");
|
||||
}
|
||||
for (int i=0; i<lines.length; i++) {
|
||||
if (justification == Justification.LEFT) {
|
||||
lines[i] = StringUtils.rightPad(lines[i], maxWidth);
|
||||
} else if (justification == Justification.RIGHT) {
|
||||
lines[i] = StringUtils.leftPad(lines[i], maxWidth);
|
||||
}
|
||||
}
|
||||
return lines;
|
||||
}
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private final LinkedList<Column> columns = new LinkedList<Column>();
|
||||
private boolean showHeader = true;
|
||||
private int wrapWidth = Integer.MAX_VALUE;
|
||||
|
||||
/**
|
||||
* Create a new Builder.
|
||||
*/
|
||||
public Builder() {
|
||||
}
|
||||
|
||||
public Builder addField(String title) {
|
||||
return addField(title, Justification.LEFT, false);
|
||||
}
|
||||
|
||||
public Builder addField(String title, Justification justification) {
|
||||
return addField(title, justification, false);
|
||||
}
|
||||
|
||||
public Builder addField(String title, boolean wrap) {
|
||||
return addField(title, Justification.LEFT, wrap);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new field to the Table under construction.
|
||||
*
|
||||
* @param title Field title.
|
||||
* @param justification Right or left justification. Defaults to left.
|
||||
* @param wrap Width at which to auto-wrap the content of the cell.
|
||||
* Defaults to Integer.MAX_VALUE.
|
||||
* @return This Builder object
|
||||
*/
|
||||
public Builder addField(String title, Justification justification,
|
||||
boolean wrap) {
|
||||
columns.add(new Column(title, justification, wrap));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether to hide column headers in table output
|
||||
*/
|
||||
public Builder hideHeaders() {
|
||||
this.showHeader = false;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether to show column headers in table output. This is the default.
|
||||
*/
|
||||
public Builder showHeaders() {
|
||||
this.showHeader = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the maximum width of a row in the TableListing. Must have one or
|
||||
* more wrappable fields for this to take effect.
|
||||
*/
|
||||
public Builder wrapWidth(int width) {
|
||||
this.wrapWidth = width;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new TableListing.
|
||||
*/
|
||||
public TableListing build() {
|
||||
return new TableListing(columns.toArray(new Column[0]), showHeader,
|
||||
wrapWidth);
|
||||
}
|
||||
}
|
||||
|
||||
private final Column columns[];
|
||||
|
||||
private int numRows;
|
||||
private boolean showHeader;
|
||||
private int wrapWidth;
|
||||
|
||||
TableListing(Column columns[], boolean showHeader, int wrapWidth) {
|
||||
this.columns = columns;
|
||||
this.numRows = 0;
|
||||
this.showHeader = showHeader;
|
||||
this.wrapWidth = wrapWidth;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new row.
|
||||
*
|
||||
* @param row The row of objects to add-- one per column.
|
||||
*/
|
||||
public void addRow(String... row) {
|
||||
if (row.length != columns.length) {
|
||||
throw new RuntimeException("trying to add a row with " + row.length +
|
||||
" columns, but we have " + columns.length + " columns.");
|
||||
}
|
||||
for (int i = 0; i < columns.length; i++) {
|
||||
columns[i].addRow(row[i]);
|
||||
}
|
||||
numRows++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder builder = new StringBuilder();
|
||||
// Calculate the widths of each column based on their maxWidths and
|
||||
// the wrapWidth for the entire table
|
||||
int width = (columns.length-1)*2; // inter-column padding
|
||||
for (int i=0; i<columns.length; i++) {
|
||||
width += columns[i].maxWidth;
|
||||
}
|
||||
// Decrease the column size of wrappable columns until the goal width
|
||||
// is reached, or we can't decrease anymore
|
||||
while (width > wrapWidth) {
|
||||
boolean modified = false;
|
||||
for (int i=0; i<columns.length; i++) {
|
||||
Column column = columns[i];
|
||||
if (column.wrap) {
|
||||
int maxWidth = column.getMaxWidth();
|
||||
if (maxWidth > 4) {
|
||||
column.setWrapWidth(maxWidth-1);
|
||||
modified = true;
|
||||
width -= 1;
|
||||
if (width <= wrapWidth) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!modified) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
int startrow = 0;
|
||||
if (!showHeader) {
|
||||
startrow = 1;
|
||||
}
|
||||
String[][] columnLines = new String[columns.length][];
|
||||
for (int i = startrow; i < numRows + 1; i++) {
|
||||
int maxColumnLines = 0;
|
||||
for (int j = 0; j < columns.length; j++) {
|
||||
columnLines[j] = columns[j].getRow(i);
|
||||
if (columnLines[j].length > maxColumnLines) {
|
||||
maxColumnLines = columnLines[j].length;
|
||||
}
|
||||
}
|
||||
|
||||
for (int c = 0; c < maxColumnLines; c++) {
|
||||
// First column gets no left-padding
|
||||
String prefix = "";
|
||||
for (int j = 0; j < columns.length; j++) {
|
||||
// Prepend padding
|
||||
builder.append(prefix);
|
||||
prefix = " ";
|
||||
if (columnLines[j].length > c) {
|
||||
builder.append(columnLines[j][c]);
|
||||
} else {
|
||||
builder.append(StringUtils.repeat(" ", columns[j].maxWidth));
|
||||
}
|
||||
}
|
||||
builder.append("\n");
|
||||
}
|
||||
}
|
||||
return builder.toString();
|
||||
}
|
||||
}
|
|
@ -126,7 +126,7 @@ class ImageLoaderCurrent implements ImageLoader {
|
|||
new SimpleDateFormat("yyyy-MM-dd HH:mm");
|
||||
private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
|
||||
-24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
|
||||
-40, -41, -42, -43, -44, -45, -46, -47 };
|
||||
-40, -41, -42, -43, -44, -45, -46, -47, -48 };
|
||||
private int imageVersion = 0;
|
||||
|
||||
private final Map<Long, String> subtreeMap = new HashMap<Long, String>();
|
||||
|
@ -216,6 +216,9 @@ class ImageLoaderCurrent implements ImageLoader {
|
|||
processDelegationTokens(in, v);
|
||||
}
|
||||
|
||||
if (LayoutVersion.supports(Feature.CACHING, imageVersion)) {
|
||||
processCacheManagerState(in, v);
|
||||
}
|
||||
v.leaveEnclosingElement(); // FSImage
|
||||
done = true;
|
||||
} finally {
|
||||
|
@ -227,6 +230,25 @@ class ImageLoaderCurrent implements ImageLoader {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Process CacheManager state from the fsimage.
|
||||
*/
|
||||
private void processCacheManagerState(DataInputStream in, ImageVisitor v)
|
||||
throws IOException {
|
||||
v.visit(ImageElement.CACHE_NEXT_ENTRY_ID, in.readLong());
|
||||
final int numPools = in.readInt();
|
||||
for (int i=0; i<numPools; i++) {
|
||||
v.visit(ImageElement.CACHE_POOL_NAME, Text.readString(in));
|
||||
processCachePoolPermission(in, v);
|
||||
v.visit(ImageElement.CACHE_POOL_WEIGHT, in.readInt());
|
||||
}
|
||||
final int numEntries = in.readInt();
|
||||
for (int i=0; i<numEntries; i++) {
|
||||
v.visit(ImageElement.CACHE_ENTRY_PATH, Text.readString(in));
|
||||
v.visit(ImageElement.CACHE_ENTRY_REPLICATION, in.readShort());
|
||||
v.visit(ImageElement.CACHE_ENTRY_POOL_NAME, Text.readString(in));
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Process the Delegation Token related section in fsimage.
|
||||
*
|
||||
|
@ -384,6 +406,22 @@ class ImageLoaderCurrent implements ImageLoader {
|
|||
v.leaveEnclosingElement(); // Permissions
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract CachePool permissions stored in the fsimage file.
|
||||
*
|
||||
* @param in Datastream to process
|
||||
* @param v Visitor to walk over inodes
|
||||
*/
|
||||
private void processCachePoolPermission(DataInputStream in, ImageVisitor v)
|
||||
throws IOException {
|
||||
v.visitEnclosingElement(ImageElement.PERMISSIONS);
|
||||
v.visit(ImageElement.CACHE_POOL_OWNER_NAME, Text.readString(in));
|
||||
v.visit(ImageElement.CACHE_POOL_GROUP_NAME, Text.readString(in));
|
||||
FsPermission fsp = new FsPermission(in.readShort());
|
||||
v.visit(ImageElement.CACHE_POOL_PERMISSION_STRING, fsp.toString());
|
||||
v.leaveEnclosingElement(); // Permissions
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the INode records stored in the fsimage.
|
||||
*
|
||||
|
|
|
@ -117,7 +117,19 @@ abstract class ImageVisitor {
|
|||
SNAPSHOT_DST_SNAPSHOT_ID,
|
||||
SNAPSHOT_LAST_SNAPSHOT_ID,
|
||||
SNAPSHOT_REF_INODE_ID,
|
||||
SNAPSHOT_REF_INODE
|
||||
SNAPSHOT_REF_INODE,
|
||||
|
||||
CACHE_NEXT_ENTRY_ID,
|
||||
CACHE_NUM_POOLS,
|
||||
CACHE_POOL_NAME,
|
||||
CACHE_POOL_OWNER_NAME,
|
||||
CACHE_POOL_GROUP_NAME,
|
||||
CACHE_POOL_PERMISSION_STRING,
|
||||
CACHE_POOL_WEIGHT,
|
||||
CACHE_NUM_ENTRIES,
|
||||
CACHE_ENTRY_PATH,
|
||||
CACHE_ENTRY_REPLICATION,
|
||||
CACHE_ENTRY_POOL_NAME
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -289,6 +289,8 @@ public class JsonUtil {
|
|||
m.put("dfsUsed", datanodeinfo.getDfsUsed());
|
||||
m.put("remaining", datanodeinfo.getRemaining());
|
||||
m.put("blockPoolUsed", datanodeinfo.getBlockPoolUsed());
|
||||
m.put("cacheCapacity", datanodeinfo.getCacheCapacity());
|
||||
m.put("cacheUsed", datanodeinfo.getCacheUsed());
|
||||
m.put("lastUpdate", datanodeinfo.getLastUpdate());
|
||||
m.put("xceiverCount", datanodeinfo.getXceiverCount());
|
||||
m.put("networkLocation", datanodeinfo.getNetworkLocation());
|
||||
|
@ -296,34 +298,56 @@ public class JsonUtil {
|
|||
return m;
|
||||
}
|
||||
|
||||
private static int getInt(Map<?, ?> m, String key, final int defaultValue) {
|
||||
Object value = m.get(key);
|
||||
if (value == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
return (int) (long) (Long) value;
|
||||
}
|
||||
|
||||
private static long getLong(Map<?, ?> m, String key, final long defaultValue) {
|
||||
Object value = m.get(key);
|
||||
if (value == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
return (long) (Long) value;
|
||||
}
|
||||
|
||||
private static String getString(Map<?, ?> m, String key,
|
||||
final String defaultValue) {
|
||||
Object value = m.get(key);
|
||||
if (value == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
return (String) value;
|
||||
}
|
||||
|
||||
/** Convert a Json map to an DatanodeInfo object. */
|
||||
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"),
|
||||
(String)m.get("hostName"),
|
||||
(String)m.get("storageID"),
|
||||
(int)(long)(Long)m.get("xferPort"),
|
||||
(int)(long)(Long)m.get("infoPort"),
|
||||
(int)(long)(Long)infoSecurePort,
|
||||
getInt(m, "infoSecurePort", 0),
|
||||
(int)(long)(Long)m.get("ipcPort"),
|
||||
|
||||
(Long)m.get("capacity"),
|
||||
(Long)m.get("dfsUsed"),
|
||||
(Long)m.get("remaining"),
|
||||
(Long)m.get("blockPoolUsed"),
|
||||
(Long)m.get("lastUpdate"),
|
||||
(int)(long)(Long)m.get("xceiverCount"),
|
||||
(String)m.get("networkLocation"),
|
||||
AdminStates.valueOf((String)m.get("adminState")));
|
||||
getLong(m, "capacity", 0l),
|
||||
getLong(m, "dfsUsed", 0l),
|
||||
getLong(m, "remaining", 0l),
|
||||
getLong(m, "blockPoolUsed", 0l),
|
||||
getLong(m, "cacheCapacity", 0l),
|
||||
getLong(m, "cacheUsed", 0l),
|
||||
getLong(m, "lastUpdate", 0l),
|
||||
getInt(m, "xceiverCount", 0),
|
||||
getString(m, "networkLocation", ""),
|
||||
AdminStates.valueOf(getString(m, "adminState", "NORMAL")));
|
||||
}
|
||||
|
||||
/** Convert a DatanodeInfo[] to a Json array. */
|
||||
|
|
|
@ -363,6 +363,91 @@ message IsFileClosedResponseProto {
|
|||
required bool result = 1;
|
||||
}
|
||||
|
||||
message PathBasedCacheDirectiveProto {
|
||||
required string path = 1;
|
||||
required uint32 replication = 2;
|
||||
required string pool = 3;
|
||||
}
|
||||
|
||||
message AddPathBasedCacheDirectiveRequestProto {
|
||||
required PathBasedCacheDirectiveProto directive = 1;
|
||||
}
|
||||
|
||||
message AddPathBasedCacheDirectiveResponseProto {
|
||||
required int64 descriptorId = 1;
|
||||
}
|
||||
|
||||
message RemovePathBasedCacheDescriptorRequestProto {
|
||||
required int64 descriptorId = 1;
|
||||
}
|
||||
|
||||
message RemovePathBasedCacheDescriptorResponseProto {
|
||||
}
|
||||
|
||||
message ListPathBasedCacheDescriptorsRequestProto {
|
||||
required int64 prevId = 1;
|
||||
optional string pool = 2;
|
||||
optional string path = 3;
|
||||
}
|
||||
|
||||
message ListPathBasedCacheDescriptorsElementProto {
|
||||
required int64 id = 1;
|
||||
required string pool = 2;
|
||||
required uint32 replication = 3;
|
||||
required string path = 4;
|
||||
}
|
||||
|
||||
message ListPathBasedCacheDescriptorsResponseProto {
|
||||
repeated ListPathBasedCacheDescriptorsElementProto elements = 1;
|
||||
required bool hasMore = 2;
|
||||
}
|
||||
|
||||
message AddCachePoolRequestProto {
|
||||
required string poolName = 1;
|
||||
optional string ownerName = 2;
|
||||
optional string groupName = 3;
|
||||
optional int32 mode = 4;
|
||||
optional int32 weight = 5;
|
||||
}
|
||||
|
||||
message AddCachePoolResponseProto { // void response
|
||||
}
|
||||
|
||||
message ModifyCachePoolRequestProto {
|
||||
required string poolName = 1;
|
||||
optional string ownerName = 2;
|
||||
optional string groupName = 3;
|
||||
optional int32 mode = 4;
|
||||
optional int32 weight = 5;
|
||||
}
|
||||
|
||||
message ModifyCachePoolResponseProto { // void response
|
||||
}
|
||||
|
||||
message RemoveCachePoolRequestProto {
|
||||
required string poolName = 1;
|
||||
}
|
||||
|
||||
message RemoveCachePoolResponseProto { // void response
|
||||
}
|
||||
|
||||
message ListCachePoolsRequestProto {
|
||||
required string prevPoolName = 1;
|
||||
}
|
||||
|
||||
message ListCachePoolsResponseProto {
|
||||
repeated ListCachePoolsResponseElementProto elements = 1;
|
||||
required bool hasMore = 2;
|
||||
}
|
||||
|
||||
message ListCachePoolsResponseElementProto {
|
||||
required string poolName = 1;
|
||||
required string ownerName = 2;
|
||||
required string groupName = 3;
|
||||
required int32 mode = 4;
|
||||
required int32 weight = 5;
|
||||
}
|
||||
|
||||
message GetFileLinkInfoRequestProto {
|
||||
required string src = 1;
|
||||
}
|
||||
|
@ -544,6 +629,20 @@ service ClientNamenodeProtocol {
|
|||
returns(ListCorruptFileBlocksResponseProto);
|
||||
rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
|
||||
rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
|
||||
rpc addPathBasedCacheDirective(AddPathBasedCacheDirectiveRequestProto)
|
||||
returns (AddPathBasedCacheDirectiveResponseProto);
|
||||
rpc removePathBasedCacheDescriptor(RemovePathBasedCacheDescriptorRequestProto)
|
||||
returns (RemovePathBasedCacheDescriptorResponseProto);
|
||||
rpc listPathBasedCacheDescriptors(ListPathBasedCacheDescriptorsRequestProto)
|
||||
returns (ListPathBasedCacheDescriptorsResponseProto);
|
||||
rpc addCachePool(AddCachePoolRequestProto)
|
||||
returns(AddCachePoolResponseProto);
|
||||
rpc modifyCachePool(ModifyCachePoolRequestProto)
|
||||
returns(ModifyCachePoolResponseProto);
|
||||
rpc removeCachePool(RemoveCachePoolRequestProto)
|
||||
returns(RemoveCachePoolResponseProto);
|
||||
rpc listCachePools(ListCachePoolsRequestProto)
|
||||
returns(ListCachePoolsResponseProto);
|
||||
rpc getFileLinkInfo(GetFileLinkInfoRequestProto)
|
||||
returns(GetFileLinkInfoResponseProto);
|
||||
rpc getContentSummary(GetContentSummaryRequestProto)
|
||||
|
|
|
@ -70,6 +70,7 @@ message DatanodeCommandProto {
|
|||
RegisterCommand = 5;
|
||||
UnusedUpgradeCommand = 6;
|
||||
NullDatanodeCommand = 7;
|
||||
BlockIdCommand = 8;
|
||||
}
|
||||
|
||||
required Type cmdType = 1; // Type of the command
|
||||
|
@ -82,6 +83,7 @@ message DatanodeCommandProto {
|
|||
optional FinalizeCommandProto finalizeCmd = 5;
|
||||
optional KeyUpdateCommandProto keyUpdateCmd = 6;
|
||||
optional RegisterCommandProto registerCmd = 7;
|
||||
optional BlockIdCommandProto blkIdCmd = 8;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -110,6 +112,20 @@ message BlockCommandProto {
|
|||
repeated DatanodeInfosProto targets = 4;
|
||||
}
|
||||
|
||||
/**
|
||||
* Command to instruct datanodes to perform certain action
|
||||
* on the given set of block IDs.
|
||||
*/
|
||||
message BlockIdCommandProto {
|
||||
enum Action {
|
||||
CACHE = 1;
|
||||
UNCACHE = 2;
|
||||
}
|
||||
required Action action = 1;
|
||||
required string blockPoolId = 2;
|
||||
repeated uint64 blockIds = 3 [packed=true];
|
||||
}
|
||||
|
||||
/**
|
||||
* List of blocks to be recovered by the datanode
|
||||
*/
|
||||
|
@ -163,6 +179,8 @@ message RegisterDatanodeResponseProto {
|
|||
* xmitsInProgress - number of transfers from this datanode to others
|
||||
* xceiverCount - number of active transceiver threads
|
||||
* failedVolumes - number of failed volumes
|
||||
* cacheCapacity - total cache capacity available at the datanode
|
||||
* cacheUsed - amount of cache used
|
||||
*/
|
||||
message HeartbeatRequestProto {
|
||||
required DatanodeRegistrationProto registration = 1; // Datanode info
|
||||
|
@ -170,6 +188,8 @@ message HeartbeatRequestProto {
|
|||
optional uint32 xmitsInProgress = 3 [ default = 0 ];
|
||||
optional uint32 xceiverCount = 4 [ default = 0 ];
|
||||
optional uint32 failedVolumes = 5 [ default = 0 ];
|
||||
optional uint64 dnCacheCapacity = 6 [ default = 0 ];
|
||||
optional uint64 dnCacheUsed = 7 [default = 0 ];
|
||||
}
|
||||
|
||||
message StorageReportProto {
|
||||
|
@ -202,9 +222,11 @@ message HeartbeatResponseProto {
|
|||
/**
|
||||
* registration - datanode registration information
|
||||
* blockPoolID - block pool ID of the reported blocks
|
||||
* blocks - each block is represented as two longs in the array.
|
||||
* blocks - each block is represented as multiple longs in the array.
|
||||
* first long represents block ID
|
||||
* second long represents length
|
||||
* third long represents gen stamp
|
||||
* fourth long (if under construction) represents replica state
|
||||
*/
|
||||
message BlockReportRequestProto {
|
||||
required DatanodeRegistrationProto registration = 1;
|
||||
|
@ -227,6 +249,21 @@ message BlockReportResponseProto {
|
|||
optional DatanodeCommandProto cmd = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* registration - datanode registration information
|
||||
* blockPoolId - block pool ID of the reported blocks
|
||||
* blocks - representation of blocks as longs for efficiency reasons
|
||||
*/
|
||||
message CacheReportRequestProto {
|
||||
required DatanodeRegistrationProto registration = 1;
|
||||
required string blockPoolId = 2;
|
||||
repeated uint64 blocks = 3 [packed=true];
|
||||
}
|
||||
|
||||
message CacheReportResponseProto {
|
||||
optional DatanodeCommandProto cmd = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Data structure to send received or deleted block information
|
||||
* from datanode to namenode.
|
||||
|
@ -344,6 +381,11 @@ service DatanodeProtocolService {
|
|||
*/
|
||||
rpc blockReport(BlockReportRequestProto) returns(BlockReportResponseProto);
|
||||
|
||||
/**
|
||||
* Report cached blocks at a datanode to the namenode
|
||||
*/
|
||||
rpc cacheReport(CacheReportRequestProto) returns(CacheReportResponseProto);
|
||||
|
||||
/**
|
||||
* Incremental block report from the DN. This contains info about recently
|
||||
* received and deleted blocks, as well as when blocks start being
|
||||
|
|
|
@ -83,6 +83,8 @@ message DatanodeInfoProto {
|
|||
}
|
||||
|
||||
optional AdminState adminState = 10 [default = NORMAL];
|
||||
optional uint64 cacheCapacity = 11 [default = 0];
|
||||
optional uint64 cacheUsed = 12 [default = 0];
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -127,6 +129,7 @@ message LocatedBlockProto {
|
|||
// their locations are not part of this object
|
||||
|
||||
required hadoop.common.TokenProto blockToken = 5;
|
||||
repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
|
||||
}
|
||||
|
||||
message DataEncryptionKeyProto {
|
||||
|
|
|
@ -1459,4 +1459,70 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.caching.enabled</name>
|
||||
<value>false</value>
|
||||
<description>
|
||||
Set to true to enable block caching. This flag enables the NameNode to
|
||||
maintain a mapping of cached blocks to DataNodes via processing DataNode
|
||||
cache reports. Based on these reports and addition and removal of caching
|
||||
directives, the NameNode will schedule caching and uncaching work.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.datanode.max.locked.memory</name>
|
||||
<value>0</value>
|
||||
<description>
|
||||
The amount of memory in bytes to use for caching of block replicas in
|
||||
memory on the datanode. The datanode's maximum locked memory soft ulimit
|
||||
(RLIMIT_MEMLOCK) must be set to at least this value, else the datanode
|
||||
will abort on startup.
|
||||
|
||||
By default, this parameter is set to 0, which disables in-memory caching.
|
||||
|
||||
If the native libraries are not available to the DataNode, this
|
||||
configuration has no effect.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.namenode.path.based.cache.refresh.interval.ms</name>
|
||||
<value>300000</value>
|
||||
<description>
|
||||
The amount of milliseconds between subsequent path cache rescans. Path
|
||||
cache rescans are when we calculate which blocks should be cached, and on
|
||||
what datanodes.
|
||||
|
||||
By default, this parameter is set to 300000, which is five minutes.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.datanode.fsdatasetcache.max.threads.per.volume</name>
|
||||
<value>4</value>
|
||||
<description>
|
||||
The maximum number of threads per volume to use for caching new data
|
||||
on the datanode. These threads consume both I/O and CPU. This can affect
|
||||
normal datanode operations.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.cachereport.intervalMsec</name>
|
||||
<value>10000</value>
|
||||
<description>
|
||||
Determines cache reporting interval in milliseconds. After this amount of
|
||||
time, the DataNode sends a full report of its cache state to the NameNode.
|
||||
The NameNode uses the cache report to update its map of cached blocks to
|
||||
DataNode locations.
|
||||
|
||||
This configuration has no effect if in-memory caching has been disabled by
|
||||
setting dfs.datanode.max.locked.memory to 0 (which is the default).
|
||||
|
||||
If the native libraries are not available to the DataNode, this
|
||||
configuration has no effect.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
</configuration>
|
||||
|
|
|
@ -0,0 +1,141 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.cli;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.cli.util.CLICommand;
|
||||
import org.apache.hadoop.cli.util.CLICommandCacheAdmin;
|
||||
import org.apache.hadoop.cli.util.CLICommandTypes;
|
||||
import org.apache.hadoop.cli.util.CLITestCmd;
|
||||
import org.apache.hadoop.cli.util.CacheAdminCmdExecutor;
|
||||
import org.apache.hadoop.cli.util.CommandExecutor;
|
||||
import org.apache.hadoop.cli.util.CommandExecutor.Result;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.tools.CacheAdmin;
|
||||
import org.apache.hadoop.security.authorize.PolicyProvider;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
public class TestCacheAdminCLI extends CLITestHelper {
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(TestCacheAdminCLI.class);
|
||||
|
||||
protected MiniDFSCluster dfsCluster = null;
|
||||
protected FileSystem fs = null;
|
||||
protected String namenode = null;
|
||||
|
||||
@Before
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
conf.setClass(PolicyProvider.POLICY_PROVIDER_CONFIG,
|
||||
HDFSPolicyProvider.class, PolicyProvider.class);
|
||||
|
||||
// Many of the tests expect a replication value of 1 in the output
|
||||
conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
|
||||
|
||||
dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||
|
||||
dfsCluster.waitClusterUp();
|
||||
namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
|
||||
username = System.getProperty("user.name");
|
||||
|
||||
fs = dfsCluster.getFileSystem();
|
||||
assertTrue("Not a HDFS: "+fs.getUri(),
|
||||
fs instanceof DistributedFileSystem);
|
||||
}
|
||||
|
||||
@After
|
||||
@Override
|
||||
public void tearDown() throws Exception {
|
||||
if (fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
if (dfsCluster != null) {
|
||||
dfsCluster.shutdown();
|
||||
}
|
||||
Thread.sleep(2000);
|
||||
super.tearDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTestFile() {
|
||||
return "testCacheAdminConf.xml";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TestConfigFileParser getConfigParser() {
|
||||
return new TestConfigFileParserCacheAdmin();
|
||||
}
|
||||
|
||||
private class TestConfigFileParserCacheAdmin extends
|
||||
CLITestHelper.TestConfigFileParser {
|
||||
@Override
|
||||
public void endElement(String uri, String localName, String qName)
|
||||
throws SAXException {
|
||||
if (qName.equals("cache-admin-command")) {
|
||||
if (testCommands != null) {
|
||||
testCommands.add(new CLITestCmdCacheAdmin(charString,
|
||||
new CLICommandCacheAdmin()));
|
||||
} else if (cleanupCommands != null) {
|
||||
cleanupCommands.add(new CLITestCmdCacheAdmin(charString,
|
||||
new CLICommandCacheAdmin()));
|
||||
}
|
||||
} else {
|
||||
super.endElement(uri, localName, qName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private class CLITestCmdCacheAdmin extends CLITestCmd {
|
||||
|
||||
public CLITestCmdCacheAdmin(String str, CLICommandTypes type) {
|
||||
super(str, type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CommandExecutor getExecutor(String tag)
|
||||
throws IllegalArgumentException {
|
||||
if (getType() instanceof CLICommandCacheAdmin) {
|
||||
return new CacheAdminCmdExecutor(tag, new CacheAdmin(conf));
|
||||
}
|
||||
return super.getExecutor(tag);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Result execute(CLICommand cmd) throws Exception {
|
||||
return cmd.getExecutor("").executeCommand(cmd.getCmd());
|
||||
}
|
||||
|
||||
@Test
|
||||
@Override
|
||||
public void testAll () {
|
||||
super.testAll();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
* <p/>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p/>
|
||||
* 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.cli.util;
|
||||
|
||||
public class CLICommandCacheAdmin implements CLICommandTypes {
|
||||
}
|
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.cli.util;
|
||||
|
||||
import org.apache.hadoop.hdfs.tools.CacheAdmin;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
public class CacheAdminCmdExecutor extends CommandExecutor {
|
||||
protected String namenode = null;
|
||||
protected CacheAdmin admin = null;
|
||||
|
||||
public CacheAdminCmdExecutor(String namenode, CacheAdmin admin) {
|
||||
this.namenode = namenode;
|
||||
this.admin = admin;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void execute(final String cmd) throws Exception {
|
||||
String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode);
|
||||
ToolRunner.run(admin, args);
|
||||
}
|
||||
}
|
|
@ -817,7 +817,7 @@ public class DFSTestUtil {
|
|||
DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
|
||||
DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
|
||||
DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT,
|
||||
1, 2, 3, 4, 5, 6, "local", adminState);
|
||||
1l, 2l, 3l, 4l, 0l, 0l, 5, 6, "local", adminState);
|
||||
}
|
||||
|
||||
public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
|
||||
|
@ -993,6 +993,20 @@ public class DFSTestUtil {
|
|||
locatedBlocks = DFSClientAdapter.callGetBlockLocations(
|
||||
cluster.getNameNodeRpc(nnIndex), filePath, 0L, bytes.length);
|
||||
} while (locatedBlocks.isUnderConstruction());
|
||||
// OP_ADD_CACHE_POOL 35
|
||||
filesystem.addCachePool(new CachePoolInfo("pool1"));
|
||||
// OP_MODIFY_CACHE_POOL 36
|
||||
filesystem.modifyCachePool(new CachePoolInfo("pool1").setWeight(99));
|
||||
// OP_ADD_PATH_BASED_CACHE_DIRECTIVE 33
|
||||
PathBasedCacheDescriptor pbcd = filesystem.addPathBasedCacheDirective(
|
||||
new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/path")).
|
||||
setPool("pool1").
|
||||
build());
|
||||
// OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR 34
|
||||
filesystem.removePathBasedCacheDescriptor(pbcd);
|
||||
// OP_REMOVE_CACHE_POOL 37
|
||||
filesystem.removeCachePool("pool1");
|
||||
}
|
||||
|
||||
public static void abortStream(DFSOutputStream out) throws IOException {
|
||||
|
|
|
@ -61,4 +61,15 @@ public class LogVerificationAppender extends AppenderSkeleton {
|
|||
}
|
||||
return count;
|
||||
}
|
||||
|
||||
public int countLinesWithMessage(final String text) {
|
||||
int count = 0;
|
||||
for (LoggingEvent e: getLog()) {
|
||||
String msg = e.getRenderedMessage();
|
||||
if (msg != null && msg.contains(text)) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
return count;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
|
|||
import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assume.assumeTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -30,6 +31,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
|
||||
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
@ -106,4 +109,38 @@ public class TestDatanodeConfig {
|
|||
throw new IOException("Bad URI", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testMemlockLimit() throws Exception {
|
||||
assumeTrue(NativeIO.isAvailable());
|
||||
final long memlockLimit = NativeIO.getMemlockLimit();
|
||||
|
||||
// Can't increase the memlock limit past the maximum.
|
||||
assumeTrue(memlockLimit != Long.MAX_VALUE);
|
||||
|
||||
Configuration conf = cluster.getConfiguration(0);
|
||||
long prevLimit = conf.
|
||||
getLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT);
|
||||
try {
|
||||
// Try starting the DN with limit configured to the ulimit
|
||||
conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
memlockLimit);
|
||||
DataNode dn = null;
|
||||
dn = DataNode.createDataNode(new String[]{}, conf);
|
||||
dn.shutdown();
|
||||
// Try starting the DN with a limit > ulimit
|
||||
conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
memlockLimit+1);
|
||||
try {
|
||||
dn = DataNode.createDataNode(new String[]{}, conf);
|
||||
} catch (RuntimeException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"more than the datanode's available RLIMIT_MEMLOCK", e);
|
||||
}
|
||||
} finally {
|
||||
conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
prevLimit);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,57 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.protocolPB;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.EmptyPathError;
|
||||
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.AddPathBasedCacheDirectiveRequestProto;
|
||||
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.PathBasedCacheDirectiveProto;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
public class TestClientNamenodeProtocolServerSideTranslatorPB {
|
||||
|
||||
@Test
|
||||
public void testAddPathBasedCacheDirectiveEmptyPathError() throws Exception {
|
||||
ClientProtocol server = mock(ClientProtocol.class);
|
||||
RpcController controller = mock(RpcController.class);
|
||||
AddPathBasedCacheDirectiveRequestProto request =
|
||||
AddPathBasedCacheDirectiveRequestProto.newBuilder().
|
||||
setDirective(PathBasedCacheDirectiveProto.newBuilder().
|
||||
setPath("").
|
||||
setPool("pool").
|
||||
setReplication(1).
|
||||
build()).
|
||||
build();
|
||||
ClientNamenodeProtocolServerSideTranslatorPB translator =
|
||||
new ClientNamenodeProtocolServerSideTranslatorPB(server);
|
||||
try {
|
||||
translator.addPathBasedCacheDirective(controller, request);
|
||||
fail("Expected ServiceException");
|
||||
} catch (ServiceException e) {
|
||||
assertNotNull(e.getCause());
|
||||
assertTrue(e.getCause() instanceof EmptyPathError);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -98,7 +98,9 @@ public class TestBlockManager {
|
|||
cluster.add(dn);
|
||||
dn.updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
0L, 0L,
|
||||
0, 0);
|
||||
bm.getDatanodeManager().checkIfClusterIsNowMultiRack(dn);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,151 @@
|
|||
/*
|
||||
* 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 java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
|
||||
import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestCachedBlocksList {
|
||||
public static final Log LOG = LogFactory.getLog(TestCachedBlocksList.class);
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testSingleList() {
|
||||
DatanodeDescriptor dn = new DatanodeDescriptor(
|
||||
new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002, 5003));
|
||||
CachedBlock[] blocks = new CachedBlock[] {
|
||||
new CachedBlock(0L, (short)1, true),
|
||||
new CachedBlock(1L, (short)1, true),
|
||||
new CachedBlock(2L, (short)1, true),
|
||||
};
|
||||
// check that lists are empty
|
||||
Assert.assertTrue("expected pending cached list to start off empty.",
|
||||
!dn.getPendingCached().iterator().hasNext());
|
||||
Assert.assertTrue("expected cached list to start off empty.",
|
||||
!dn.getCached().iterator().hasNext());
|
||||
Assert.assertTrue("expected pending uncached list to start off empty.",
|
||||
!dn.getPendingUncached().iterator().hasNext());
|
||||
// add a block to the back
|
||||
Assert.assertTrue(dn.getCached().add(blocks[0]));
|
||||
Assert.assertTrue("expected pending cached list to still be empty.",
|
||||
!dn.getPendingCached().iterator().hasNext());
|
||||
Assert.assertEquals("failed to insert blocks[0]", blocks[0],
|
||||
dn.getCached().iterator().next());
|
||||
Assert.assertTrue("expected pending uncached list to still be empty.",
|
||||
!dn.getPendingUncached().iterator().hasNext());
|
||||
// add another block to the back
|
||||
Assert.assertTrue(dn.getCached().add(blocks[1]));
|
||||
Iterator<CachedBlock> iter = dn.getCached().iterator();
|
||||
Assert.assertEquals(blocks[0], iter.next());
|
||||
Assert.assertEquals(blocks[1], iter.next());
|
||||
Assert.assertTrue(!iter.hasNext());
|
||||
// add a block to the front
|
||||
Assert.assertTrue(dn.getCached().addFirst(blocks[2]));
|
||||
iter = dn.getCached().iterator();
|
||||
Assert.assertEquals(blocks[2], iter.next());
|
||||
Assert.assertEquals(blocks[0], iter.next());
|
||||
Assert.assertEquals(blocks[1], iter.next());
|
||||
Assert.assertTrue(!iter.hasNext());
|
||||
// remove a block from the middle
|
||||
Assert.assertTrue(dn.getCached().remove(blocks[0]));
|
||||
iter = dn.getCached().iterator();
|
||||
Assert.assertEquals(blocks[2], iter.next());
|
||||
Assert.assertEquals(blocks[1], iter.next());
|
||||
Assert.assertTrue(!iter.hasNext());
|
||||
// remove all blocks
|
||||
dn.getCached().clear();
|
||||
Assert.assertTrue("expected cached list to be empty after clear.",
|
||||
!dn.getPendingCached().iterator().hasNext());
|
||||
}
|
||||
|
||||
private void testAddElementsToList(CachedBlocksList list,
|
||||
CachedBlock[] blocks) {
|
||||
Assert.assertTrue("expected list to start off empty.",
|
||||
!list.iterator().hasNext());
|
||||
for (CachedBlock block : blocks) {
|
||||
Assert.assertTrue(list.add(block));
|
||||
}
|
||||
}
|
||||
|
||||
private void testRemoveElementsFromList(Random r,
|
||||
CachedBlocksList list, CachedBlock[] blocks) {
|
||||
int i = 0;
|
||||
for (Iterator<CachedBlock> iter = list.iterator(); iter.hasNext(); ) {
|
||||
Assert.assertEquals(blocks[i], iter.next());
|
||||
i++;
|
||||
}
|
||||
if (r.nextBoolean()) {
|
||||
LOG.info("Removing via iterator");
|
||||
for (Iterator<CachedBlock> iter = list.iterator(); iter.hasNext() ;) {
|
||||
iter.next();
|
||||
iter.remove();
|
||||
}
|
||||
} else {
|
||||
LOG.info("Removing in pseudo-random order");
|
||||
CachedBlock[] remainingBlocks = Arrays.copyOf(blocks, blocks.length);
|
||||
for (int removed = 0; removed < remainingBlocks.length; ) {
|
||||
int toRemove = r.nextInt(remainingBlocks.length);
|
||||
if (remainingBlocks[toRemove] != null) {
|
||||
Assert.assertTrue(list.remove(remainingBlocks[toRemove]));
|
||||
remainingBlocks[toRemove] = null;
|
||||
removed++;
|
||||
}
|
||||
}
|
||||
}
|
||||
Assert.assertTrue("expected list to be empty after everything " +
|
||||
"was removed.", !list.iterator().hasNext());
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testMultipleLists() {
|
||||
DatanodeDescriptor[] datanodes = new DatanodeDescriptor[] {
|
||||
new DatanodeDescriptor(
|
||||
new DatanodeID("127.0.0.1", "localhost", "abcd", 5000, 5001, 5002, 5003)),
|
||||
new DatanodeDescriptor(
|
||||
new DatanodeID("127.0.1.1", "localhost", "efgh", 6000, 6001, 6002, 6003)),
|
||||
};
|
||||
CachedBlocksList[] lists = new CachedBlocksList[] {
|
||||
datanodes[0].getPendingCached(),
|
||||
datanodes[0].getCached(),
|
||||
datanodes[1].getPendingCached(),
|
||||
datanodes[1].getCached(),
|
||||
datanodes[1].getPendingUncached(),
|
||||
};
|
||||
final int NUM_BLOCKS = 8000;
|
||||
CachedBlock[] blocks = new CachedBlock[NUM_BLOCKS];
|
||||
for (int i = 0; i < NUM_BLOCKS; i++) {
|
||||
blocks[i] = new CachedBlock(i, (short)i, true);
|
||||
}
|
||||
Random r = new Random(654);
|
||||
for (CachedBlocksList list : lists) {
|
||||
testAddElementsToList(list, blocks);
|
||||
}
|
||||
for (CachedBlocksList list : lists) {
|
||||
testRemoveElementsFromList(r, list, blocks);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -103,7 +103,7 @@ public class TestOverReplicatedBlocks {
|
|||
String corruptMachineName = corruptDataNode.getXferAddr();
|
||||
for (DatanodeDescriptor datanode : hm.getDatanodes()) {
|
||||
if (!corruptMachineName.equals(datanode.getXferAddr())) {
|
||||
datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0, 0);
|
||||
datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0L, 0L, 0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -118,7 +118,7 @@ public class TestReplicationPolicy {
|
|||
for (int i=0; i < NUM_OF_DATANODES; i++) {
|
||||
dataNodes[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -135,7 +135,8 @@ public class TestReplicationPolicy {
|
|||
public void testChooseTarget1() throws Exception {
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4, 0); // overloaded
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
0L, 0L, 4, 0); // overloaded
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
targets = chooseTarget(0);
|
||||
|
@ -165,7 +166,7 @@ public class TestReplicationPolicy {
|
|||
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
private static DatanodeDescriptor[] chooseTarget(int numOfReplicas) {
|
||||
|
@ -285,7 +286,8 @@ public class TestReplicationPolicy {
|
|||
// make data node 0 to be not qualified to choose
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); // no space
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L,
|
||||
0L, 0L, 0, 0); // no space
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
targets = chooseTarget(0);
|
||||
|
@ -318,7 +320,7 @@ public class TestReplicationPolicy {
|
|||
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -335,7 +337,7 @@ public class TestReplicationPolicy {
|
|||
for(int i=0; i<2; i++) {
|
||||
dataNodes[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
|
@ -363,7 +365,7 @@ public class TestReplicationPolicy {
|
|||
for(int i=0; i<2; i++) {
|
||||
dataNodes[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -425,7 +427,7 @@ public class TestReplicationPolicy {
|
|||
for(int i=0; i<2; i++) {
|
||||
dataNodes[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
final LogVerificationAppender appender = new LogVerificationAppender();
|
||||
|
@ -450,7 +452,7 @@ public class TestReplicationPolicy {
|
|||
for(int i=0; i<2; i++) {
|
||||
dataNodes[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -133,7 +133,7 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
for(int i=0; i<NUM_OF_DATANODES; i++) {
|
||||
dataNodes[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -199,7 +199,8 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
public void testChooseTarget1() throws Exception {
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4, 0); // overloaded
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
0L, 0L, 4, 0); // overloaded
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
targets = chooseTarget(0);
|
||||
|
@ -232,7 +233,7 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeDescriptor[] targets) {
|
||||
|
@ -299,7 +300,8 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
// make data node 0 to be not qualified to choose
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); // no space
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L,
|
||||
0L, 0L, 0, 0); // no space
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
targets = chooseTarget(0);
|
||||
|
@ -330,7 +332,7 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -348,7 +350,7 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
for(int i=0; i<3; i++) {
|
||||
dataNodes[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
|
@ -576,11 +578,11 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
|
||||
dataNodes[0].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
|
||||
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
|
||||
dataNodesInBoundaryCase[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
|
@ -611,7 +613,7 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
|
||||
dataNodesInBoundaryCase[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
|
||||
chosenNodes.add(dataNodesInBoundaryCase[0]);
|
||||
|
@ -651,7 +653,7 @@ public class TestReplicationPolicyWithNodeGroup {
|
|||
for(int i=0; i<NUM_OF_DATANODES_MORE_TARGETS; i++) {
|
||||
dataNodesInMoreTargetsCase[i].updateHeartbeat(
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
|
||||
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
|
||||
}
|
||||
|
||||
DatanodeDescriptor[] targets;
|
||||
|
|
|
@ -452,9 +452,9 @@ public class TestJspHelper {
|
|||
DatanodeID dnId2 = new DatanodeID("127.0.0.2", "localhost2", "storage2",
|
||||
1235, 2346, 3457, 4568);
|
||||
DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1", 1024,
|
||||
100, 924, 100, 10, 2);
|
||||
100, 924, 100, 5l, 3l, 10, 2);
|
||||
DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2", 2500,
|
||||
200, 1848, 200, 20, 1);
|
||||
200, 1848, 200, 10l, 2l, 20, 1);
|
||||
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
|
||||
live.add(dnDesc1);
|
||||
live.add(dnDesc2);
|
||||
|
|
|
@ -24,6 +24,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
|
@ -465,6 +466,11 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|||
return new BlockListAsLongs(blocks, null);
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public List<Long> getCacheReport(String bpid) {
|
||||
return new LinkedList<Long>();
|
||||
}
|
||||
|
||||
@Override // FSDatasetMBean
|
||||
public long getCapacity() {
|
||||
return storage.getCapacity();
|
||||
|
@ -490,6 +496,16 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|||
return storage.getNumFailedVolumes();
|
||||
}
|
||||
|
||||
@Override // FSDatasetMBean
|
||||
public long getDnCacheUsed() {
|
||||
return 0l;
|
||||
}
|
||||
|
||||
@Override // FSDatasetMBean
|
||||
public long getDnCacheCapacity() {
|
||||
return 0l;
|
||||
}
|
||||
|
||||
@Override // FsDatasetSpi
|
||||
public synchronized long getLength(ExtendedBlock b) throws IOException {
|
||||
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
|
||||
|
@ -559,6 +575,18 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
|
|||
}
|
||||
}
|
||||
|
||||
@Override // FSDatasetSpi
|
||||
public void cache(String bpid, long[] cacheBlks) {
|
||||
throw new UnsupportedOperationException(
|
||||
"SimulatedFSDataset does not support cache operation!");
|
||||
}
|
||||
|
||||
@Override // FSDatasetSpi
|
||||
public void uncache(String bpid, long[] uncacheBlks) {
|
||||
throw new UnsupportedOperationException(
|
||||
"SimulatedFSDataset does not support uncache operation!");
|
||||
}
|
||||
|
||||
private BInfo getBInfo(final ExtendedBlock b) {
|
||||
final Map<Block, BInfo> map = blockMap.get(b.getBlockPoolId());
|
||||
return map == null? null: map.get(b.getLocalBlock());
|
||||
|
|
|
@ -126,6 +126,8 @@ public class TestBPOfferService {
|
|||
.when(mock).sendHeartbeat(
|
||||
Mockito.any(DatanodeRegistration.class),
|
||||
Mockito.any(StorageReport[].class),
|
||||
Mockito.anyLong(),
|
||||
Mockito.anyLong(),
|
||||
Mockito.anyInt(),
|
||||
Mockito.anyInt(),
|
||||
Mockito.anyInt());
|
||||
|
|
|
@ -154,6 +154,8 @@ public class TestBlockRecovery {
|
|||
when(namenode.sendHeartbeat(
|
||||
Mockito.any(DatanodeRegistration.class),
|
||||
Mockito.any(StorageReport[].class),
|
||||
Mockito.anyLong(),
|
||||
Mockito.anyLong(),
|
||||
Mockito.anyInt(),
|
||||
Mockito.anyInt(),
|
||||
Mockito.anyInt()))
|
||||
|
|
|
@ -0,0 +1,269 @@
|
|||
/**
|
||||
* 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.datanode;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assume.assumeTrue;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyLong;
|
||||
import static org.mockito.Matchers.anyInt;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.FileChannel;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.HdfsBlockLocation;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.LogVerificationAppender;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.Block;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSImage;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
|
||||
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
||||
import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
|
||||
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestFsDatasetCache {
|
||||
|
||||
// Most Linux installs allow a default of 64KB locked memory
|
||||
private static final long CACHE_CAPACITY = 64 * 1024;
|
||||
private static final long BLOCK_SIZE = 4096;
|
||||
|
||||
private static Configuration conf;
|
||||
private static MiniDFSCluster cluster = null;
|
||||
private static FileSystem fs;
|
||||
private static NameNode nn;
|
||||
private static FSImage fsImage;
|
||||
private static DataNode dn;
|
||||
private static FsDatasetSpi<?> fsd;
|
||||
private static DatanodeProtocolClientSideTranslatorPB spyNN;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
assumeTrue(!Path.WINDOWS);
|
||||
assumeTrue(NativeIO.isAvailable());
|
||||
conf = new HdfsConfiguration();
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
||||
conf.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
|
||||
CACHE_CAPACITY);
|
||||
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
|
||||
fs = cluster.getFileSystem();
|
||||
nn = cluster.getNameNode();
|
||||
fsImage = nn.getFSImage();
|
||||
dn = cluster.getDataNodes().get(0);
|
||||
fsd = dn.getFSDataset();
|
||||
|
||||
spyNN = DataNodeTestUtils.spyOnBposToNN(dn, nn);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (fs != null) {
|
||||
fs.close();
|
||||
}
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
private static void setHeartbeatResponse(DatanodeCommand[] cmds)
|
||||
throws IOException {
|
||||
HeartbeatResponse response = new HeartbeatResponse(
|
||||
cmds,
|
||||
new NNHAStatusHeartbeat(HAServiceState.ACTIVE,
|
||||
fsImage.getLastAppliedOrWrittenTxId()));
|
||||
doReturn(response).when(spyNN).sendHeartbeat(
|
||||
(DatanodeRegistration) any(),
|
||||
(StorageReport[]) any(), anyLong(), anyLong(),
|
||||
anyInt(), anyInt(), anyInt());
|
||||
}
|
||||
|
||||
private static DatanodeCommand[] cacheBlock(HdfsBlockLocation loc) {
|
||||
return cacheBlocks(new HdfsBlockLocation[] {loc});
|
||||
}
|
||||
|
||||
private static DatanodeCommand[] cacheBlocks(HdfsBlockLocation[] locs) {
|
||||
return new DatanodeCommand[] {
|
||||
getResponse(locs, DatanodeProtocol.DNA_CACHE)
|
||||
};
|
||||
}
|
||||
|
||||
private static DatanodeCommand[] uncacheBlock(HdfsBlockLocation loc) {
|
||||
return uncacheBlocks(new HdfsBlockLocation[] {loc});
|
||||
}
|
||||
|
||||
private static DatanodeCommand[] uncacheBlocks(HdfsBlockLocation[] locs) {
|
||||
return new DatanodeCommand[] {
|
||||
getResponse(locs, DatanodeProtocol.DNA_UNCACHE)
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a cache or uncache DatanodeCommand from an array of locations
|
||||
*/
|
||||
private static DatanodeCommand getResponse(HdfsBlockLocation[] locs,
|
||||
int action) {
|
||||
String bpid = locs[0].getLocatedBlock().getBlock().getBlockPoolId();
|
||||
long[] blocks = new long[locs.length];
|
||||
for (int i=0; i<locs.length; i++) {
|
||||
blocks[i] = locs[i].getLocatedBlock().getBlock().getBlockId();
|
||||
}
|
||||
return new BlockIdCommand(action, bpid, blocks);
|
||||
}
|
||||
|
||||
private static long[] getBlockSizes(HdfsBlockLocation[] locs)
|
||||
throws Exception {
|
||||
long[] sizes = new long[locs.length];
|
||||
for (int i=0; i<locs.length; i++) {
|
||||
HdfsBlockLocation loc = locs[i];
|
||||
String bpid = loc.getLocatedBlock().getBlock().getBlockPoolId();
|
||||
Block block = loc.getLocatedBlock().getBlock().getLocalBlock();
|
||||
ExtendedBlock extBlock = new ExtendedBlock(bpid, block);
|
||||
FileChannel blockChannel =
|
||||
((FileInputStream)fsd.getBlockInputStream(extBlock, 0)).getChannel();
|
||||
sizes[i] = blockChannel.size();
|
||||
}
|
||||
return sizes;
|
||||
}
|
||||
|
||||
/**
|
||||
* Blocks until cache usage hits the expected new value.
|
||||
*/
|
||||
private long verifyExpectedCacheUsage(final long expected) throws Exception {
|
||||
long cacheUsed = fsd.getDnCacheUsed();
|
||||
while (cacheUsed != expected) {
|
||||
cacheUsed = fsd.getDnCacheUsed();
|
||||
Thread.sleep(100);
|
||||
}
|
||||
assertEquals("Unexpected amount of cache used", expected, cacheUsed);
|
||||
return cacheUsed;
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testCacheAndUncacheBlock() throws Exception {
|
||||
final int NUM_BLOCKS = 5;
|
||||
|
||||
// Write a test file
|
||||
final Path testFile = new Path("/testCacheBlock");
|
||||
final long testFileLen = BLOCK_SIZE*NUM_BLOCKS;
|
||||
DFSTestUtil.createFile(fs, testFile, testFileLen, (short)1, 0xABBAl);
|
||||
|
||||
// Get the details of the written file
|
||||
HdfsBlockLocation[] locs =
|
||||
(HdfsBlockLocation[])fs.getFileBlockLocations(testFile, 0, testFileLen);
|
||||
assertEquals("Unexpected number of blocks", NUM_BLOCKS, locs.length);
|
||||
final long[] blockSizes = getBlockSizes(locs);
|
||||
|
||||
// Check initial state
|
||||
final long cacheCapacity = fsd.getDnCacheCapacity();
|
||||
long cacheUsed = fsd.getDnCacheUsed();
|
||||
long current = 0;
|
||||
assertEquals("Unexpected cache capacity", CACHE_CAPACITY, cacheCapacity);
|
||||
assertEquals("Unexpected amount of cache used", current, cacheUsed);
|
||||
|
||||
// Cache each block in succession, checking each time
|
||||
for (int i=0; i<NUM_BLOCKS; i++) {
|
||||
setHeartbeatResponse(cacheBlock(locs[i]));
|
||||
current = verifyExpectedCacheUsage(current + blockSizes[i]);
|
||||
}
|
||||
|
||||
// Uncache each block in succession, again checking each time
|
||||
for (int i=0; i<NUM_BLOCKS; i++) {
|
||||
setHeartbeatResponse(uncacheBlock(locs[i]));
|
||||
current = verifyExpectedCacheUsage(current - blockSizes[i]);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testFilesExceedMaxLockedMemory() throws Exception {
|
||||
// Create some test files that will exceed total cache capacity
|
||||
// Don't forget that meta files take up space too!
|
||||
final int numFiles = 4;
|
||||
final long fileSize = CACHE_CAPACITY / numFiles;
|
||||
final Path[] testFiles = new Path[4];
|
||||
final HdfsBlockLocation[][] fileLocs = new HdfsBlockLocation[numFiles][];
|
||||
final long[] fileSizes = new long[numFiles];
|
||||
for (int i=0; i<numFiles; i++) {
|
||||
testFiles[i] = new Path("/testFilesExceedMaxLockedMemory-" + i);
|
||||
DFSTestUtil.createFile(fs, testFiles[i], fileSize, (short)1, 0xDFAl);
|
||||
fileLocs[i] = (HdfsBlockLocation[])fs.getFileBlockLocations(
|
||||
testFiles[i], 0, fileSize);
|
||||
// Get the file size (sum of blocks)
|
||||
long[] sizes = getBlockSizes(fileLocs[i]);
|
||||
for (int j=0; j<sizes.length; j++) {
|
||||
fileSizes[i] += sizes[j];
|
||||
}
|
||||
}
|
||||
|
||||
// Cache the first n-1 files
|
||||
long current = 0;
|
||||
for (int i=0; i<numFiles-1; i++) {
|
||||
setHeartbeatResponse(cacheBlocks(fileLocs[i]));
|
||||
current = verifyExpectedCacheUsage(current + fileSizes[i]);
|
||||
}
|
||||
final long oldCurrent = current;
|
||||
|
||||
// nth file should hit a capacity exception
|
||||
final LogVerificationAppender appender = new LogVerificationAppender();
|
||||
final Logger logger = Logger.getRootLogger();
|
||||
logger.addAppender(appender);
|
||||
setHeartbeatResponse(cacheBlocks(fileLocs[numFiles-1]));
|
||||
int lines = 0;
|
||||
while (lines == 0) {
|
||||
Thread.sleep(100);
|
||||
lines = appender.countLinesWithMessage(
|
||||
DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY + " exceeded");
|
||||
}
|
||||
|
||||
// Uncache the cached part of the nth file
|
||||
setHeartbeatResponse(uncacheBlocks(fileLocs[numFiles-1]));
|
||||
while (fsd.getDnCacheUsed() != oldCurrent) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
|
||||
// Uncache the n-1 files
|
||||
for (int i=0; i<numFiles-1; i++) {
|
||||
setHeartbeatResponse(uncacheBlocks(fileLocs[i]));
|
||||
current = verifyExpectedCacheUsage(current - fileSizes[i]);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -845,8 +845,8 @@ public class NNThroughputBenchmark implements Tool {
|
|||
// TODO:FEDERATION currently a single block pool is supported
|
||||
StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(),
|
||||
false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
|
||||
DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
|
||||
rep, 0, 0, 0).getCommands();
|
||||
DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration, rep,
|
||||
0L, 0L, 0, 0, 0).getCommands();
|
||||
if(cmds != null) {
|
||||
for (DatanodeCommand cmd : cmds ) {
|
||||
if(LOG.isDebugEnabled()) {
|
||||
|
@ -893,7 +893,7 @@ public class NNThroughputBenchmark implements Tool {
|
|||
StorageReport[] rep = { new StorageReport(dnRegistration.getStorageID(),
|
||||
false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
|
||||
DatanodeCommand[] cmds = nameNodeProto.sendHeartbeat(dnRegistration,
|
||||
rep, 0, 0, 0).getCommands();
|
||||
rep, 0L, 0L, 0, 0, 0).getCommands();
|
||||
if (cmds != null) {
|
||||
for (DatanodeCommand cmd : cmds) {
|
||||
if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {
|
||||
|
|
|
@ -111,7 +111,8 @@ public class NameNodeAdapter {
|
|||
public static HeartbeatResponse sendHeartBeat(DatanodeRegistration nodeReg,
|
||||
DatanodeDescriptor dd, FSNamesystem namesystem) throws IOException {
|
||||
return namesystem.handleHeartbeat(nodeReg, dd.getCapacity(),
|
||||
dd.getDfsUsed(), dd.getRemaining(), dd.getBlockPoolUsed(), 0, 0, 0);
|
||||
dd.getDfsUsed(), dd.getRemaining(), dd.getBlockPoolUsed(),
|
||||
dd.getCacheCapacity(), dd.getCacheRemaining(), 0, 0, 0);
|
||||
}
|
||||
|
||||
public static boolean setReplication(final FSNamesystem ns,
|
||||
|
|
|
@ -39,8 +39,11 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
||||
import org.apache.hadoop.hdfs.server.common.Util;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
||||
|
@ -229,6 +232,26 @@ public class OfflineEditsViewerHelper {
|
|||
// OP_UPDATE_MASTER_KEY 21
|
||||
// done by getDelegationTokenSecretManager().startThreads();
|
||||
|
||||
// OP_ADD_CACHE_POOL 35
|
||||
final String pool = "poolparty";
|
||||
dfs.addCachePool(new CachePoolInfo(pool));
|
||||
// OP_MODIFY_CACHE_POOL 36
|
||||
dfs.modifyCachePool(new CachePoolInfo(pool)
|
||||
.setOwnerName("carlton")
|
||||
.setGroupName("party")
|
||||
.setMode(new FsPermission((short)0700))
|
||||
.setWeight(1989));
|
||||
// OP_ADD_PATH_BASED_CACHE_DIRECTIVE 33
|
||||
PathBasedCacheDescriptor descriptor =
|
||||
dfs.addPathBasedCacheDirective(new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/bar")).
|
||||
setReplication((short)1).
|
||||
setPool(pool).
|
||||
build());
|
||||
// OP_REMOVE_PATH_BASED_CACHE_DESCRIPTOR 34
|
||||
dfs.removePathBasedCacheDescriptor(descriptor);
|
||||
// OP_REMOVE_CACHE_POOL 37
|
||||
dfs.removeCachePool(pool);
|
||||
// sync to disk, otherwise we parse partial edits
|
||||
cluster.getNameNode().getFSImage().getEditLog().logSync();
|
||||
|
||||
|
|
|
@ -142,7 +142,8 @@ public class TestDeadDatanode {
|
|||
// that asks datanode to register again
|
||||
StorageReport[] rep = { new StorageReport(reg.getStorageID(), false, 0, 0,
|
||||
0, 0) };
|
||||
DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, rep, 0, 0, 0).getCommands();
|
||||
DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, rep, 0L, 0L, 0, 0, 0)
|
||||
.getCommands();
|
||||
assertEquals(1, cmd.length);
|
||||
assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
|
||||
.getAction());
|
||||
|
|
|
@ -413,7 +413,7 @@ public class TestNamenodeRetryCache {
|
|||
|
||||
LightWeightCache<CacheEntry, CacheEntry> cacheSet =
|
||||
(LightWeightCache<CacheEntry, CacheEntry>) namesystem.getRetryCache().getCacheSet();
|
||||
assertEquals(14, cacheSet.size());
|
||||
assertEquals(19, cacheSet.size());
|
||||
|
||||
Map<CacheEntry, CacheEntry> oldEntries =
|
||||
new HashMap<CacheEntry, CacheEntry>();
|
||||
|
@ -432,7 +432,7 @@ public class TestNamenodeRetryCache {
|
|||
assertTrue(namesystem.hasRetryCache());
|
||||
cacheSet = (LightWeightCache<CacheEntry, CacheEntry>) namesystem
|
||||
.getRetryCache().getCacheSet();
|
||||
assertEquals(14, cacheSet.size());
|
||||
assertEquals(19, cacheSet.size());
|
||||
iter = cacheSet.iterator();
|
||||
while (iter.hasNext()) {
|
||||
CacheEntry entry = iter.next();
|
||||
|
|
|
@ -0,0 +1,743 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CACHING_ENABLED_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CACHEREPORT_INTERVAL_MSEC_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS;
|
||||
import static junit.framework.Assert.assertTrue;
|
||||
import static junit.framework.Assert.fail;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
|
||||
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.FileSystem;
|
||||
import org.apache.hadoop.fs.FileSystemTestHelper;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPathNameError;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.InvalidPoolNameError;
|
||||
import org.apache.hadoop.hdfs.protocol.AddPathBasedCacheDirectiveException.PoolWritePermissionDeniedError;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDescriptor;
|
||||
import org.apache.hadoop.hdfs.protocol.PathBasedCacheDirective;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.InvalidIdException;
|
||||
import org.apache.hadoop.hdfs.protocol.RemovePathBasedCacheDescriptorException.NoSuchIdException;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||
import org.apache.hadoop.io.nativeio.NativeIO;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.util.GSet;
|
||||
import org.junit.After;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
|
||||
public class TestPathBasedCacheRequests {
|
||||
static final Log LOG = LogFactory.getLog(TestPathBasedCacheRequests.class);
|
||||
|
||||
private static final UserGroupInformation unprivilegedUser =
|
||||
UserGroupInformation.createRemoteUser("unprivilegedUser");
|
||||
|
||||
static private Configuration conf;
|
||||
static private MiniDFSCluster cluster;
|
||||
static private DistributedFileSystem dfs;
|
||||
static private NamenodeProtocols proto;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
conf = new HdfsConfiguration();
|
||||
// set low limits here for testing purposes
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_POOLS_NUM_RESPONSES, 2);
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_CACHE_DESCRIPTORS_NUM_RESPONSES, 2);
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
dfs = cluster.getFileSystem();
|
||||
proto = cluster.getNameNodeRpc();
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws Exception {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testBasicPoolOperations() throws Exception {
|
||||
final String poolName = "pool1";
|
||||
CachePoolInfo info = new CachePoolInfo(poolName).
|
||||
setOwnerName("bob").setGroupName("bobgroup").
|
||||
setMode(new FsPermission((short)0755)).setWeight(150);
|
||||
|
||||
// Add a pool
|
||||
dfs.addCachePool(info);
|
||||
|
||||
// Do some bad addCachePools
|
||||
try {
|
||||
dfs.addCachePool(info);
|
||||
fail("added the pool with the same name twice");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("pool1 already exists", ioe);
|
||||
}
|
||||
try {
|
||||
dfs.addCachePool(new CachePoolInfo(""));
|
||||
fail("added empty pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
dfs.addCachePool(null);
|
||||
fail("added null pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
|
||||
}
|
||||
try {
|
||||
proto.addCachePool(new CachePoolInfo(""));
|
||||
fail("added empty pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
proto.addCachePool(null);
|
||||
fail("added null pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
|
||||
}
|
||||
|
||||
// Modify the pool
|
||||
info.setOwnerName("jane").setGroupName("janegroup")
|
||||
.setMode(new FsPermission((short)0700)).setWeight(314);
|
||||
dfs.modifyCachePool(info);
|
||||
|
||||
// Do some invalid modify pools
|
||||
try {
|
||||
dfs.modifyCachePool(new CachePoolInfo("fool"));
|
||||
fail("modified non-existent cache pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("fool does not exist", ioe);
|
||||
}
|
||||
try {
|
||||
dfs.modifyCachePool(new CachePoolInfo(""));
|
||||
fail("modified empty pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
dfs.modifyCachePool(null);
|
||||
fail("modified null pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
|
||||
}
|
||||
try {
|
||||
proto.modifyCachePool(new CachePoolInfo(""));
|
||||
fail("modified empty pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
proto.modifyCachePool(null);
|
||||
fail("modified null pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("CachePoolInfo is null", ioe);
|
||||
}
|
||||
|
||||
// Remove the pool
|
||||
dfs.removeCachePool(poolName);
|
||||
// Do some bad removePools
|
||||
try {
|
||||
dfs.removeCachePool("pool99");
|
||||
fail("expected to get an exception when " +
|
||||
"removing a non-existent pool.");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("can't remove " +
|
||||
"non-existent cache pool", ioe);
|
||||
}
|
||||
try {
|
||||
dfs.removeCachePool(poolName);
|
||||
Assert.fail("expected to get an exception when " +
|
||||
"removing a non-existent pool.");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("can't remove " +
|
||||
"non-existent cache pool", ioe);
|
||||
}
|
||||
try {
|
||||
dfs.removeCachePool("");
|
||||
fail("removed empty pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
dfs.removeCachePool(null);
|
||||
fail("removed null pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
proto.removeCachePool("");
|
||||
fail("removed empty pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
proto.removeCachePool(null);
|
||||
fail("removed null pool");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("invalid empty cache pool name",
|
||||
ioe);
|
||||
}
|
||||
|
||||
info = new CachePoolInfo("pool2");
|
||||
dfs.addCachePool(info);
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testCreateAndModifyPools() throws Exception {
|
||||
String poolName = "pool1";
|
||||
String ownerName = "abc";
|
||||
String groupName = "123";
|
||||
FsPermission mode = new FsPermission((short)0755);
|
||||
int weight = 150;
|
||||
dfs.addCachePool(new CachePoolInfo(poolName).
|
||||
setOwnerName(ownerName).setGroupName(groupName).
|
||||
setMode(mode).setWeight(weight));
|
||||
|
||||
RemoteIterator<CachePoolInfo> iter = dfs.listCachePools();
|
||||
CachePoolInfo info = iter.next();
|
||||
assertEquals(poolName, info.getPoolName());
|
||||
assertEquals(ownerName, info.getOwnerName());
|
||||
assertEquals(groupName, info.getGroupName());
|
||||
|
||||
ownerName = "def";
|
||||
groupName = "456";
|
||||
mode = new FsPermission((short)0700);
|
||||
weight = 151;
|
||||
dfs.modifyCachePool(new CachePoolInfo(poolName).
|
||||
setOwnerName(ownerName).setGroupName(groupName).
|
||||
setMode(mode).setWeight(weight));
|
||||
|
||||
iter = dfs.listCachePools();
|
||||
info = iter.next();
|
||||
assertEquals(poolName, info.getPoolName());
|
||||
assertEquals(ownerName, info.getOwnerName());
|
||||
assertEquals(groupName, info.getGroupName());
|
||||
assertEquals(mode, info.getMode());
|
||||
assertEquals(Integer.valueOf(weight), info.getWeight());
|
||||
|
||||
dfs.removeCachePool(poolName);
|
||||
iter = dfs.listCachePools();
|
||||
assertFalse("expected no cache pools after deleting pool", iter.hasNext());
|
||||
|
||||
proto.listCachePools(null);
|
||||
|
||||
try {
|
||||
proto.removeCachePool("pool99");
|
||||
Assert.fail("expected to get an exception when " +
|
||||
"removing a non-existent pool.");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("can't remove non-existent",
|
||||
ioe);
|
||||
}
|
||||
try {
|
||||
proto.removeCachePool(poolName);
|
||||
Assert.fail("expected to get an exception when " +
|
||||
"removing a non-existent pool.");
|
||||
} catch (IOException ioe) {
|
||||
GenericTestUtils.assertExceptionContains("can't remove non-existent",
|
||||
ioe);
|
||||
}
|
||||
|
||||
iter = dfs.listCachePools();
|
||||
assertFalse("expected no cache pools after deleting pool", iter.hasNext());
|
||||
}
|
||||
|
||||
private static void validateListAll(
|
||||
RemoteIterator<PathBasedCacheDescriptor> iter,
|
||||
PathBasedCacheDescriptor... descriptors) throws Exception {
|
||||
for (PathBasedCacheDescriptor descriptor: descriptors) {
|
||||
assertTrue("Unexpectedly few elements", iter.hasNext());
|
||||
assertEquals("Unexpected descriptor", descriptor, iter.next());
|
||||
}
|
||||
assertFalse("Unexpectedly many list elements", iter.hasNext());
|
||||
}
|
||||
|
||||
private static PathBasedCacheDescriptor addAsUnprivileged(
|
||||
final PathBasedCacheDirective directive) throws Exception {
|
||||
return unprivilegedUser
|
||||
.doAs(new PrivilegedExceptionAction<PathBasedCacheDescriptor>() {
|
||||
@Override
|
||||
public PathBasedCacheDescriptor run() throws IOException {
|
||||
DistributedFileSystem myDfs =
|
||||
(DistributedFileSystem) FileSystem.get(conf);
|
||||
return myDfs.addPathBasedCacheDirective(directive);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testAddRemoveDirectives() throws Exception {
|
||||
proto.addCachePool(new CachePoolInfo("pool1").
|
||||
setMode(new FsPermission((short)0777)));
|
||||
proto.addCachePool(new CachePoolInfo("pool2").
|
||||
setMode(new FsPermission((short)0777)));
|
||||
proto.addCachePool(new CachePoolInfo("pool3").
|
||||
setMode(new FsPermission((short)0777)));
|
||||
proto.addCachePool(new CachePoolInfo("pool4").
|
||||
setMode(new FsPermission((short)0)));
|
||||
|
||||
PathBasedCacheDirective alpha = new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/alpha")).
|
||||
setPool("pool1").
|
||||
build();
|
||||
PathBasedCacheDirective beta = new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/beta")).
|
||||
setPool("pool2").
|
||||
build();
|
||||
PathBasedCacheDirective delta = new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/delta")).
|
||||
setPool("pool1").
|
||||
build();
|
||||
|
||||
PathBasedCacheDescriptor alphaD = addAsUnprivileged(alpha);
|
||||
PathBasedCacheDescriptor alphaD2 = addAsUnprivileged(alpha);
|
||||
assertFalse("Expected to get unique descriptors when re-adding an "
|
||||
+ "existing PathBasedCacheDirective",
|
||||
alphaD.getEntryId() == alphaD2.getEntryId());
|
||||
PathBasedCacheDescriptor betaD = addAsUnprivileged(beta);
|
||||
|
||||
try {
|
||||
addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/unicorn")).
|
||||
setPool("no_such_pool").
|
||||
build());
|
||||
fail("expected an error when adding to a non-existent pool.");
|
||||
} catch (IOException ioe) {
|
||||
assertTrue(ioe instanceof InvalidPoolNameError);
|
||||
}
|
||||
|
||||
try {
|
||||
addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/blackhole")).
|
||||
setPool("pool4").
|
||||
build());
|
||||
fail("expected an error when adding to a pool with " +
|
||||
"mode 0 (no permissions for anyone).");
|
||||
} catch (IOException ioe) {
|
||||
assertTrue(ioe instanceof PoolWritePermissionDeniedError);
|
||||
}
|
||||
|
||||
try {
|
||||
addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/illegal:path/")).
|
||||
setPool("pool1").
|
||||
build());
|
||||
fail("expected an error when adding a malformed path " +
|
||||
"to the cache directives.");
|
||||
} catch (IllegalArgumentException e) {
|
||||
// expected
|
||||
}
|
||||
|
||||
try {
|
||||
addAsUnprivileged(new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/emptypoolname")).
|
||||
setReplication((short)1).
|
||||
setPool("").
|
||||
build());
|
||||
Assert.fail("expected an error when adding a PathBasedCache " +
|
||||
"directive with an empty pool name.");
|
||||
} catch (IOException ioe) {
|
||||
Assert.assertTrue(ioe instanceof InvalidPoolNameError);
|
||||
}
|
||||
|
||||
PathBasedCacheDescriptor deltaD = addAsUnprivileged(delta);
|
||||
|
||||
// We expect the following to succeed, because DistributedFileSystem
|
||||
// qualifies the path.
|
||||
PathBasedCacheDescriptor relativeD = addAsUnprivileged(
|
||||
new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("relative")).
|
||||
setPool("pool1").
|
||||
build());
|
||||
|
||||
RemoteIterator<PathBasedCacheDescriptor> iter;
|
||||
iter = dfs.listPathBasedCacheDescriptors(null, null);
|
||||
validateListAll(iter, alphaD, alphaD2, betaD, deltaD, relativeD);
|
||||
iter = dfs.listPathBasedCacheDescriptors("pool3", null);
|
||||
Assert.assertFalse(iter.hasNext());
|
||||
iter = dfs.listPathBasedCacheDescriptors("pool1", null);
|
||||
validateListAll(iter, alphaD, alphaD2, deltaD, relativeD);
|
||||
iter = dfs.listPathBasedCacheDescriptors("pool2", null);
|
||||
validateListAll(iter, betaD);
|
||||
|
||||
dfs.removePathBasedCacheDescriptor(betaD);
|
||||
iter = dfs.listPathBasedCacheDescriptors("pool2", null);
|
||||
Assert.assertFalse(iter.hasNext());
|
||||
|
||||
try {
|
||||
dfs.removePathBasedCacheDescriptor(betaD);
|
||||
Assert.fail("expected an error when removing a non-existent ID");
|
||||
} catch (IOException ioe) {
|
||||
Assert.assertTrue(ioe instanceof NoSuchIdException);
|
||||
}
|
||||
|
||||
try {
|
||||
proto.removePathBasedCacheDescriptor(-42l);
|
||||
Assert.fail("expected an error when removing a negative ID");
|
||||
} catch (IOException ioe) {
|
||||
Assert.assertTrue(ioe instanceof InvalidIdException);
|
||||
}
|
||||
try {
|
||||
proto.removePathBasedCacheDescriptor(43l);
|
||||
Assert.fail("expected an error when removing a non-existent ID");
|
||||
} catch (IOException ioe) {
|
||||
Assert.assertTrue(ioe instanceof NoSuchIdException);
|
||||
}
|
||||
|
||||
dfs.removePathBasedCacheDescriptor(alphaD);
|
||||
dfs.removePathBasedCacheDescriptor(alphaD2);
|
||||
dfs.removePathBasedCacheDescriptor(deltaD);
|
||||
dfs.removePathBasedCacheDescriptor(relativeD);
|
||||
iter = dfs.listPathBasedCacheDescriptors(null, null);
|
||||
assertFalse(iter.hasNext());
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testCacheManagerRestart() throws Exception {
|
||||
cluster.shutdown();
|
||||
cluster = null;
|
||||
HdfsConfiguration conf = createCachingConf();
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
|
||||
|
||||
cluster.waitActive();
|
||||
DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
|
||||
// Create and validate a pool
|
||||
final String pool = "poolparty";
|
||||
String groupName = "partygroup";
|
||||
FsPermission mode = new FsPermission((short)0777);
|
||||
int weight = 747;
|
||||
dfs.addCachePool(new CachePoolInfo(pool)
|
||||
.setGroupName(groupName)
|
||||
.setMode(mode)
|
||||
.setWeight(weight));
|
||||
RemoteIterator<CachePoolInfo> pit = dfs.listCachePools();
|
||||
assertTrue("No cache pools found", pit.hasNext());
|
||||
CachePoolInfo info = pit.next();
|
||||
assertEquals(pool, info.getPoolName());
|
||||
assertEquals(groupName, info.getGroupName());
|
||||
assertEquals(mode, info.getMode());
|
||||
assertEquals(weight, (int)info.getWeight());
|
||||
assertFalse("Unexpected # of cache pools found", pit.hasNext());
|
||||
|
||||
// Create some cache entries
|
||||
int numEntries = 10;
|
||||
String entryPrefix = "/party-";
|
||||
for (int i=0; i<numEntries; i++) {
|
||||
dfs.addPathBasedCacheDirective(
|
||||
new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path(entryPrefix + i)).setPool(pool).build());
|
||||
}
|
||||
RemoteIterator<PathBasedCacheDescriptor> dit
|
||||
= dfs.listPathBasedCacheDescriptors(null, null);
|
||||
for (int i=0; i<numEntries; i++) {
|
||||
assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
|
||||
PathBasedCacheDescriptor cd = dit.next();
|
||||
assertEquals(i+1, cd.getEntryId());
|
||||
assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
|
||||
assertEquals(pool, cd.getPool());
|
||||
}
|
||||
assertFalse("Unexpected # of cache descriptors found", dit.hasNext());
|
||||
|
||||
// Restart namenode
|
||||
cluster.restartNameNode();
|
||||
|
||||
// Check that state came back up
|
||||
pit = dfs.listCachePools();
|
||||
assertTrue("No cache pools found", pit.hasNext());
|
||||
info = pit.next();
|
||||
assertEquals(pool, info.getPoolName());
|
||||
assertEquals(pool, info.getPoolName());
|
||||
assertEquals(groupName, info.getGroupName());
|
||||
assertEquals(mode, info.getMode());
|
||||
assertEquals(weight, (int)info.getWeight());
|
||||
assertFalse("Unexpected # of cache pools found", pit.hasNext());
|
||||
|
||||
dit = dfs.listPathBasedCacheDescriptors(null, null);
|
||||
for (int i=0; i<numEntries; i++) {
|
||||
assertTrue("Unexpected # of cache entries: " + i, dit.hasNext());
|
||||
PathBasedCacheDescriptor cd = dit.next();
|
||||
assertEquals(i+1, cd.getEntryId());
|
||||
assertEquals(entryPrefix + i, cd.getPath().toUri().getPath());
|
||||
assertEquals(pool, cd.getPool());
|
||||
}
|
||||
assertFalse("Unexpected # of cache descriptors found", dit.hasNext());
|
||||
}
|
||||
|
||||
private static void waitForCachedBlocks(NameNode nn,
|
||||
final int expectedCachedBlocks, final int expectedCachedReplicas)
|
||||
throws Exception {
|
||||
final FSNamesystem namesystem = nn.getNamesystem();
|
||||
final CacheManager cacheManager = namesystem.getCacheManager();
|
||||
LOG.info("Waiting for " + expectedCachedBlocks + " blocks with " +
|
||||
expectedCachedReplicas + " replicas.");
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
int numCachedBlocks = 0, numCachedReplicas = 0;
|
||||
namesystem.readLock();
|
||||
try {
|
||||
GSet<CachedBlock, CachedBlock> cachedBlocks =
|
||||
cacheManager.getCachedBlocks();
|
||||
if (cachedBlocks != null) {
|
||||
for (Iterator<CachedBlock> iter = cachedBlocks.iterator();
|
||||
iter.hasNext(); ) {
|
||||
CachedBlock cachedBlock = iter.next();
|
||||
numCachedBlocks++;
|
||||
numCachedReplicas += cachedBlock.getDatanodes(Type.CACHED).size();
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
namesystem.readUnlock();
|
||||
}
|
||||
if ((numCachedBlocks == expectedCachedBlocks) &&
|
||||
(numCachedReplicas == expectedCachedReplicas)) {
|
||||
return true;
|
||||
} else {
|
||||
LOG.info("cached blocks: have " + numCachedBlocks +
|
||||
" / " + expectedCachedBlocks);
|
||||
LOG.info("cached replicas: have " + numCachedReplicas +
|
||||
" / " + expectedCachedReplicas);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}, 500, 60000);
|
||||
}
|
||||
|
||||
private static final long BLOCK_SIZE = 512;
|
||||
private static final int NUM_DATANODES = 4;
|
||||
|
||||
// Most Linux installs will allow non-root users to lock 64KB.
|
||||
private static final long CACHE_CAPACITY = 64 * 1024 / NUM_DATANODES;
|
||||
|
||||
/**
|
||||
* Return true if we can test DN caching.
|
||||
*/
|
||||
private static boolean canTestDatanodeCaching() {
|
||||
if (!NativeIO.isAvailable()) {
|
||||
// Need NativeIO in order to cache blocks on the DN.
|
||||
return false;
|
||||
}
|
||||
if (NativeIO.getMemlockLimit() < CACHE_CAPACITY) {
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private static HdfsConfiguration createCachingConf() {
|
||||
HdfsConfiguration conf = new HdfsConfiguration();
|
||||
conf.setLong(DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
||||
conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, CACHE_CAPACITY);
|
||||
conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
|
||||
conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, true);
|
||||
conf.setLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, 1000);
|
||||
conf.setLong(DFS_NAMENODE_PATH_BASED_CACHE_REFRESH_INTERVAL_MS, 1000);
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Test(timeout=120000)
|
||||
public void testWaitForCachedReplicas() throws Exception {
|
||||
Assume.assumeTrue(canTestDatanodeCaching());
|
||||
HdfsConfiguration conf = createCachingConf();
|
||||
FileSystemTestHelper helper = new FileSystemTestHelper();
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
|
||||
|
||||
try {
|
||||
cluster.waitActive();
|
||||
DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
NameNode namenode = cluster.getNameNode();
|
||||
NamenodeProtocols nnRpc = namenode.getRpcServer();
|
||||
Path rootDir = helper.getDefaultWorkingDirectory(dfs);
|
||||
// Create the pool
|
||||
final String pool = "friendlyPool";
|
||||
nnRpc.addCachePool(new CachePoolInfo("friendlyPool"));
|
||||
// Create some test files
|
||||
final int numFiles = 2;
|
||||
final int numBlocksPerFile = 2;
|
||||
final List<String> paths = new ArrayList<String>(numFiles);
|
||||
for (int i=0; i<numFiles; i++) {
|
||||
Path p = new Path(rootDir, "testCachePaths-" + i);
|
||||
FileSystemTestHelper.createFile(dfs, p, numBlocksPerFile,
|
||||
(int)BLOCK_SIZE);
|
||||
paths.add(p.toUri().getPath());
|
||||
}
|
||||
// Check the initial statistics at the namenode
|
||||
waitForCachedBlocks(namenode, 0, 0);
|
||||
// Cache and check each path in sequence
|
||||
int expected = 0;
|
||||
for (int i=0; i<numFiles; i++) {
|
||||
PathBasedCacheDirective directive =
|
||||
new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path(paths.get(i))).
|
||||
setPool(pool).
|
||||
build();
|
||||
PathBasedCacheDescriptor descriptor =
|
||||
nnRpc.addPathBasedCacheDirective(directive);
|
||||
assertEquals("Descriptor does not match requested path",
|
||||
new Path(paths.get(i)), descriptor.getPath());
|
||||
assertEquals("Descriptor does not match requested pool", pool,
|
||||
descriptor.getPool());
|
||||
expected += numBlocksPerFile;
|
||||
waitForCachedBlocks(namenode, expected, expected);
|
||||
}
|
||||
// Uncache and check each path in sequence
|
||||
RemoteIterator<PathBasedCacheDescriptor> entries =
|
||||
nnRpc.listPathBasedCacheDescriptors(0, null, null);
|
||||
for (int i=0; i<numFiles; i++) {
|
||||
PathBasedCacheDescriptor descriptor = entries.next();
|
||||
nnRpc.removePathBasedCacheDescriptor(descriptor.getEntryId());
|
||||
expected -= numBlocksPerFile;
|
||||
waitForCachedBlocks(namenode, expected, expected);
|
||||
}
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=120000)
|
||||
public void testAddingPathBasedCacheDirectivesWhenCachingIsDisabled()
|
||||
throws Exception {
|
||||
Assume.assumeTrue(canTestDatanodeCaching());
|
||||
HdfsConfiguration conf = createCachingConf();
|
||||
conf.setBoolean(DFS_NAMENODE_CACHING_ENABLED_KEY, false);
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
|
||||
|
||||
try {
|
||||
cluster.waitActive();
|
||||
DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
NameNode namenode = cluster.getNameNode();
|
||||
// Create the pool
|
||||
String pool = "pool1";
|
||||
namenode.getRpcServer().addCachePool(new CachePoolInfo(pool));
|
||||
// Create some test files
|
||||
final int numFiles = 2;
|
||||
final int numBlocksPerFile = 2;
|
||||
final List<String> paths = new ArrayList<String>(numFiles);
|
||||
for (int i=0; i<numFiles; i++) {
|
||||
Path p = new Path("/testCachePaths-" + i);
|
||||
FileSystemTestHelper.createFile(dfs, p, numBlocksPerFile,
|
||||
(int)BLOCK_SIZE);
|
||||
paths.add(p.toUri().getPath());
|
||||
}
|
||||
// Check the initial statistics at the namenode
|
||||
waitForCachedBlocks(namenode, 0, 0);
|
||||
// Cache and check each path in sequence
|
||||
int expected = 0;
|
||||
for (int i=0; i<numFiles; i++) {
|
||||
PathBasedCacheDirective directive =
|
||||
new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path(paths.get(i))).
|
||||
setPool(pool).
|
||||
build();
|
||||
dfs.addPathBasedCacheDirective(directive);
|
||||
waitForCachedBlocks(namenode, expected, 0);
|
||||
}
|
||||
Thread.sleep(20000);
|
||||
waitForCachedBlocks(namenode, expected, 0);
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=120000)
|
||||
public void testWaitForCachedReplicasInDirectory() throws Exception {
|
||||
Assume.assumeTrue(canTestDatanodeCaching());
|
||||
HdfsConfiguration conf = createCachingConf();
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
|
||||
|
||||
try {
|
||||
cluster.waitActive();
|
||||
DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
NameNode namenode = cluster.getNameNode();
|
||||
// Create the pool
|
||||
final String pool = "friendlyPool";
|
||||
dfs.addCachePool(new CachePoolInfo(pool));
|
||||
// Create some test files
|
||||
final List<Path> paths = new LinkedList<Path>();
|
||||
paths.add(new Path("/foo/bar"));
|
||||
paths.add(new Path("/foo/baz"));
|
||||
paths.add(new Path("/foo2/bar2"));
|
||||
paths.add(new Path("/foo2/baz2"));
|
||||
dfs.mkdir(new Path("/foo"), FsPermission.getDirDefault());
|
||||
dfs.mkdir(new Path("/foo2"), FsPermission.getDirDefault());
|
||||
final int numBlocksPerFile = 2;
|
||||
for (Path path : paths) {
|
||||
FileSystemTestHelper.createFile(dfs, path, numBlocksPerFile,
|
||||
(int)BLOCK_SIZE, (short)3, false);
|
||||
}
|
||||
waitForCachedBlocks(namenode, 0, 0);
|
||||
// cache entire directory
|
||||
PathBasedCacheDescriptor descriptor = dfs.addPathBasedCacheDirective(
|
||||
new PathBasedCacheDirective.Builder().
|
||||
setPath(new Path("/foo")).
|
||||
setReplication((short)2).
|
||||
setPool(pool).
|
||||
build());
|
||||
assertEquals("Descriptor does not match requested pool", pool,
|
||||
descriptor.getPool());
|
||||
waitForCachedBlocks(namenode, 4, 8);
|
||||
// remove and watch numCached go to 0
|
||||
dfs.removePathBasedCacheDescriptor(descriptor);
|
||||
waitForCachedBlocks(namenode, 0, 0);
|
||||
} finally {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue