|
@@ -27,7 +27,11 @@ import org.apache.hadoop.net.NetworkTopology;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
|
|
|
-import java.io.*;
|
|
|
+import java.io.BufferedWriter;
|
|
|
+import java.io.File;
|
|
|
+import java.io.FileWriter;
|
|
|
+import java.io.IOException;
|
|
|
+import java.io.PrintWriter;
|
|
|
import java.util.*;
|
|
|
import java.util.Map.Entry;
|
|
|
|
|
@@ -147,7 +151,7 @@ class FSNamesystem implements FSConstants {
|
|
|
// Used for handling lock-leases
|
|
|
// Mapping: leaseHolder -> Lease
|
|
|
//
|
|
|
- private Map<UTF8, Lease> leases = new TreeMap<UTF8, Lease>();
|
|
|
+ private Map<StringBytesWritable, Lease> leases = new TreeMap<StringBytesWritable, Lease>();
|
|
|
// Set of: Lease
|
|
|
private SortedSet<Lease> sortedLeases = new TreeSet<Lease>();
|
|
|
|
|
@@ -566,7 +570,7 @@ class FSNamesystem implements FSConstants {
|
|
|
*/
|
|
|
private void verifyReplication(String src,
|
|
|
short replication,
|
|
|
- UTF8 clientName
|
|
|
+ String clientName
|
|
|
) throws IOException {
|
|
|
String text = "file " + src
|
|
|
+ ((clientName != null) ? " on client " + clientName : "")
|
|
@@ -590,15 +594,15 @@ class FSNamesystem implements FSConstants {
|
|
|
* @return Return an array that consists of the block, plus a set
|
|
|
* of machines
|
|
|
* @throws IOException if the filename is invalid
|
|
|
- * {@link FSDirectory#isValidToCreate(UTF8)}.
|
|
|
+ * {@link FSDirectory#isValidToCreate(String)}.
|
|
|
*/
|
|
|
- public LocatedBlock startFile(UTF8 src,
|
|
|
- UTF8 holder,
|
|
|
- UTF8 clientMachine,
|
|
|
- boolean overwrite,
|
|
|
- short replication,
|
|
|
- long blockSize
|
|
|
- ) throws IOException {
|
|
|
+ public synchronized LocatedBlock startFile(String src,
|
|
|
+ String holder,
|
|
|
+ String clientMachine,
|
|
|
+ boolean overwrite,
|
|
|
+ short replication,
|
|
|
+ long blockSize
|
|
|
+ ) throws IOException {
|
|
|
|
|
|
//
|
|
|
// Create file into pendingCreates and get the first blockId
|
|
@@ -612,7 +616,7 @@ class FSNamesystem implements FSConstants {
|
|
|
//
|
|
|
try {
|
|
|
DatanodeDescriptor clientNode =
|
|
|
- host2DataNodeMap.getDatanodeByHost(clientMachine.toString());
|
|
|
+ host2DataNodeMap.getDatanodeByHost(clientMachine);
|
|
|
DatanodeDescriptor targets[] = replicator.chooseTarget(replication,
|
|
|
clientNode, null, blockSize);
|
|
|
if (targets.length < this.minReplication) {
|
|
@@ -639,18 +643,18 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public synchronized Block startFileInternal(UTF8 src,
|
|
|
- UTF8 holder,
|
|
|
- UTF8 clientMachine,
|
|
|
- boolean overwrite,
|
|
|
- short replication,
|
|
|
- long blockSize
|
|
|
- ) throws IOException {
|
|
|
+ public synchronized Block startFileInternal(String src,
|
|
|
+ String holder,
|
|
|
+ String clientMachine,
|
|
|
+ boolean overwrite,
|
|
|
+ short replication,
|
|
|
+ long blockSize
|
|
|
+ ) throws IOException {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: file "
|
|
|
+src+" for "+holder+" at "+clientMachine);
|
|
|
if (isInSafeMode())
|
|
|
throw new SafeModeException("Cannot create file" + src, safeMode);
|
|
|
- if (!isValidName(src.toString())) {
|
|
|
+ if (!isValidName(src)) {
|
|
|
throw new IOException("Invalid file name: " + src);
|
|
|
}
|
|
|
try {
|
|
@@ -660,7 +664,7 @@ class FSNamesystem implements FSConstants {
|
|
|
// If the file exists in pendingCreate, then it must be in our
|
|
|
// leases. Find the appropriate lease record.
|
|
|
//
|
|
|
- Lease lease = leases.get(holder);
|
|
|
+ Lease lease = getLease(holder);
|
|
|
//
|
|
|
// We found the lease for this file. And surprisingly the original
|
|
|
// holder is trying to recreate this file. This should never occur.
|
|
@@ -674,8 +678,7 @@ class FSNamesystem implements FSConstants {
|
|
|
//
|
|
|
// Find the original holder.
|
|
|
//
|
|
|
- UTF8 oldholder = pendingFile.getClientName();
|
|
|
- lease = leases.get(oldholder);
|
|
|
+ lease = getLease(pendingFile.getClientName());
|
|
|
if (lease == null) {
|
|
|
throw new AlreadyBeingCreatedException(
|
|
|
"failed to create file " + src + " for " + holder +
|
|
@@ -689,13 +692,13 @@ class FSNamesystem implements FSConstants {
|
|
|
//
|
|
|
if (lease.expiredSoftLimit()) {
|
|
|
lease.releaseLocks();
|
|
|
- leases.remove(lease.holder);
|
|
|
+ removeLease(lease.getHolder());
|
|
|
LOG.info("Removing lease " + lease + " ");
|
|
|
if (!sortedLeases.remove(lease)) {
|
|
|
LOG.error("Unknown failure trying to remove " + lease +
|
|
|
" from lease set.");
|
|
|
}
|
|
|
- } else {
|
|
|
+ } else {
|
|
|
throw new AlreadyBeingCreatedException(
|
|
|
"failed to create file " + src + " for " + holder +
|
|
|
" on client " + clientMachine +
|
|
@@ -706,7 +709,7 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- verifyReplication(src.toString(), replication, clientMachine);
|
|
|
+ verifyReplication(src, replication, clientMachine);
|
|
|
} catch(IOException e) {
|
|
|
throw new IOException("failed to create "+e.getMessage());
|
|
|
}
|
|
@@ -721,7 +724,7 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
|
|
|
DatanodeDescriptor clientNode =
|
|
|
- host2DataNodeMap.getDatanodeByHost(clientMachine.toString());
|
|
|
+ host2DataNodeMap.getDatanodeByHost(clientMachine);
|
|
|
|
|
|
// Reserve space for this pending file
|
|
|
pendingCreates.put(src,
|
|
@@ -733,10 +736,10 @@ class FSNamesystem implements FSConstants {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
|
|
|
+"add "+src+" to pendingCreates for "+holder);
|
|
|
synchronized (leases) {
|
|
|
- Lease lease = leases.get(holder);
|
|
|
+ Lease lease = getLease(holder);
|
|
|
if (lease == null) {
|
|
|
lease = new Lease(holder);
|
|
|
- leases.put(holder, lease);
|
|
|
+ putLease(holder, lease);
|
|
|
sortedLeases.add(lease);
|
|
|
} else {
|
|
|
sortedLeases.remove(lease);
|
|
@@ -766,9 +769,9 @@ class FSNamesystem implements FSConstants {
|
|
|
* are replicated. Will return an empty 2-elt array if we want the
|
|
|
* client to "try again later".
|
|
|
*/
|
|
|
- public LocatedBlock getAdditionalBlock(UTF8 src,
|
|
|
- UTF8 clientName
|
|
|
- ) throws IOException {
|
|
|
+ public synchronized LocatedBlock getAdditionalBlock(String src,
|
|
|
+ String clientName
|
|
|
+ ) throws IOException {
|
|
|
long fileLength, blockSize;
|
|
|
int replication;
|
|
|
DatanodeDescriptor clientNode = null;
|
|
@@ -790,9 +793,9 @@ class FSNamesystem implements FSConstants {
|
|
|
throw new LeaseExpiredException("No lease on " + src);
|
|
|
}
|
|
|
if (!pendingFile.getClientName().equals(clientName)) {
|
|
|
- throw new LeaseExpiredException("Lease mismatch on " + src +
|
|
|
- " owned by " + pendingFile.getClientName() +
|
|
|
- " and appended by " + clientName);
|
|
|
+ throw new LeaseExpiredException("Lease mismatch on " + src + " owned by "
|
|
|
+ + pendingFile.getClientName()
|
|
|
+ + " and appended by " + clientName);
|
|
|
}
|
|
|
|
|
|
//
|
|
@@ -825,7 +828,7 @@ class FSNamesystem implements FSConstants {
|
|
|
/**
|
|
|
* The client would like to let go of the given block
|
|
|
*/
|
|
|
- public synchronized boolean abandonBlock(Block b, UTF8 src) {
|
|
|
+ public synchronized boolean abandonBlock(Block b, String src) throws IOException {
|
|
|
//
|
|
|
// Remove the block from the pending creates list
|
|
|
//
|
|
@@ -843,26 +846,25 @@ class FSNamesystem implements FSConstants {
|
|
|
/**
|
|
|
* Abandon the entire file in progress
|
|
|
*/
|
|
|
- public synchronized void abandonFileInProgress(UTF8 src,
|
|
|
- UTF8 holder
|
|
|
+ public synchronized void abandonFileInProgress(String src,
|
|
|
+ String holder
|
|
|
) throws IOException {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.abandonFileInProgress:" + src);
|
|
|
synchronized (leases) {
|
|
|
// find the lease
|
|
|
- Lease lease = leases.get(holder);
|
|
|
+ Lease lease = getLease(holder);
|
|
|
if (lease != null) {
|
|
|
// remove the file from the lease
|
|
|
if (lease.completedCreate(src)) {
|
|
|
// if we found the file in the lease, remove it from pendingCreates
|
|
|
internalReleaseCreate(src, holder);
|
|
|
} else {
|
|
|
- LOG.info("Attempt by " + holder.toString() +
|
|
|
- " to release someone else's create lock on " +
|
|
|
- src.toString());
|
|
|
+ LOG.info("Attempt by " + holder +
|
|
|
+ " to release someone else's create lock on " + src);
|
|
|
}
|
|
|
} else {
|
|
|
LOG.info("Attempt to release a lock from an unknown lease holder "
|
|
|
- + holder.toString() + " for " + src.toString());
|
|
|
+ + holder + " for " + src);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -873,20 +875,20 @@ class FSNamesystem implements FSConstants {
|
|
|
* Before we return, we make sure that all the file's blocks have
|
|
|
* been reported by datanodes and are replicated correctly.
|
|
|
*/
|
|
|
- public int completeFile(UTF8 src, UTF8 holder) throws IOException {
|
|
|
+ public int completeFile(String src, String holder) throws IOException {
|
|
|
int status = completeFileInternal(src, holder);
|
|
|
getEditLog().logSync();
|
|
|
return status;
|
|
|
}
|
|
|
|
|
|
- private synchronized int completeFileInternal(UTF8 src,
|
|
|
- UTF8 holder) throws IOException {
|
|
|
+ private synchronized int completeFileInternal(String src,
|
|
|
+ String holder) throws IOException {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
|
|
|
if (isInSafeMode())
|
|
|
throw new SafeModeException("Cannot complete file " + src, safeMode);
|
|
|
FileUnderConstruction pendingFile = pendingCreates.get(src);
|
|
|
|
|
|
- Block[] fileBlocks = dir.getFileBlocks(src.toString());
|
|
|
+ Block[] fileBlocks = dir.getFileBlocks(src);
|
|
|
if (fileBlocks!= null || pendingFile == null) {
|
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.completeFile: "
|
|
|
+ "failed to complete " + src
|
|
@@ -933,11 +935,11 @@ class FSNamesystem implements FSConstants {
|
|
|
+ " is removed from pendingCreates");
|
|
|
|
|
|
synchronized (leases) {
|
|
|
- Lease lease = leases.get(holder);
|
|
|
+ Lease lease = getLease(holder);
|
|
|
if (lease != null) {
|
|
|
lease.completedCreate(src);
|
|
|
if (!lease.hasLocks()) {
|
|
|
- leases.remove(holder);
|
|
|
+ removeLease(holder);
|
|
|
sortedLeases.remove(lease);
|
|
|
}
|
|
|
}
|
|
@@ -971,7 +973,7 @@ class FSNamesystem implements FSConstants {
|
|
|
/**
|
|
|
* Allocate a block at the given pending filename
|
|
|
*/
|
|
|
- private Block allocateBlock(UTF8 src) {
|
|
|
+ synchronized Block allocateBlock(String src) throws IOException {
|
|
|
Block b = null;
|
|
|
do {
|
|
|
b = new Block(FSNamesystem.randBlockId.nextLong(), 0);
|
|
@@ -1093,7 +1095,7 @@ class FSNamesystem implements FSConstants {
|
|
|
// are made, edit namespace and return to client.
|
|
|
////////////////////////////////////////////////////////////////
|
|
|
|
|
|
- public boolean renameTo(UTF8 src, UTF8 dst) throws IOException {
|
|
|
+ public boolean renameTo(String src, String dst) throws IOException {
|
|
|
boolean status = renameToInternal(src, dst);
|
|
|
getEditLog().logSync();
|
|
|
return status;
|
|
@@ -1102,11 +1104,11 @@ class FSNamesystem implements FSConstants {
|
|
|
/**
|
|
|
* Change the indicated filename.
|
|
|
*/
|
|
|
- private synchronized boolean renameToInternal(UTF8 src, UTF8 dst) throws IOException {
|
|
|
+ public synchronized boolean renameToInternal(String src, String dst) throws IOException {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src + " to " + dst);
|
|
|
if (isInSafeMode())
|
|
|
throw new SafeModeException("Cannot rename " + src, safeMode);
|
|
|
- if (!isValidName(dst.toString())) {
|
|
|
+ if (!isValidName(dst)) {
|
|
|
throw new IOException("Invalid name: " + dst);
|
|
|
}
|
|
|
return dir.renameTo(src, dst);
|
|
@@ -1116,7 +1118,7 @@ class FSNamesystem implements FSConstants {
|
|
|
* Remove the indicated filename from the namespace. This may
|
|
|
* invalidate some blocks that make up the file.
|
|
|
*/
|
|
|
- public boolean delete(UTF8 src) throws IOException {
|
|
|
+ public boolean delete(String src) throws IOException {
|
|
|
boolean status = deleteInternal(src);
|
|
|
getEditLog().logSync();
|
|
|
return status;
|
|
@@ -1126,7 +1128,7 @@ class FSNamesystem implements FSConstants {
|
|
|
* Remove the indicated filename from the namespace. This may
|
|
|
* invalidate some blocks that make up the file.
|
|
|
*/
|
|
|
- private synchronized boolean deleteInternal(UTF8 src) throws IOException {
|
|
|
+ private synchronized boolean deleteInternal(String src) throws IOException {
|
|
|
NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src);
|
|
|
if (isInSafeMode())
|
|
|
throw new SafeModeException("Cannot delete " + src, safeMode);
|
|
@@ -1152,8 +1154,8 @@ class FSNamesystem implements FSConstants {
|
|
|
/**
|
|
|
* Return whether the given filename exists
|
|
|
*/
|
|
|
- public boolean exists(UTF8 src) {
|
|
|
- if (dir.getFileBlocks(src.toString()) != null || dir.isDir(src)) {
|
|
|
+ public boolean exists(String src) {
|
|
|
+ if (dir.getFileBlocks(src) != null || dir.isDir(src)) {
|
|
|
return true;
|
|
|
} else {
|
|
|
return false;
|
|
@@ -1163,7 +1165,7 @@ class FSNamesystem implements FSConstants {
|
|
|
/**
|
|
|
* Whether the given name is a directory
|
|
|
*/
|
|
|
- public boolean isDir(UTF8 src) {
|
|
|
+ public boolean isDir(String src) {
|
|
|
return dir.isDir(src);
|
|
|
}
|
|
|
|
|
@@ -1226,13 +1228,13 @@ class FSNamesystem implements FSConstants {
|
|
|
* expire, all the corresponding locks can be released.
|
|
|
*************************************************************/
|
|
|
class Lease implements Comparable<Lease> {
|
|
|
- public UTF8 holder;
|
|
|
- public long lastUpdate;
|
|
|
- private Collection<UTF8> locks = new TreeSet<UTF8>();
|
|
|
- private Collection<UTF8> creates = new TreeSet<UTF8>();
|
|
|
+ private StringBytesWritable holder;
|
|
|
+ private long lastUpdate;
|
|
|
+ private Collection<StringBytesWritable> locks = new TreeSet<StringBytesWritable>();
|
|
|
+ private Collection<StringBytesWritable> creates = new TreeSet<StringBytesWritable>();
|
|
|
|
|
|
- public Lease(UTF8 holder) {
|
|
|
- this.holder = holder;
|
|
|
+ public Lease(String holder) throws IOException {
|
|
|
+ this.holder = new StringBytesWritable(holder);
|
|
|
renew();
|
|
|
}
|
|
|
public void renew() {
|
|
@@ -1256,27 +1258,28 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
return false;
|
|
|
}
|
|
|
- public void obtained(UTF8 src) {
|
|
|
- locks.add(src);
|
|
|
+ public void obtained(String src) throws IOException {
|
|
|
+ locks.add(new StringBytesWritable(src));
|
|
|
}
|
|
|
- public void released(UTF8 src) {
|
|
|
- locks.remove(src);
|
|
|
+ public void released(String src) throws IOException {
|
|
|
+ locks.remove(new StringBytesWritable(src));
|
|
|
}
|
|
|
- public void startedCreate(UTF8 src) {
|
|
|
- creates.add(src);
|
|
|
+ public void startedCreate(String src) throws IOException {
|
|
|
+ creates.add(new StringBytesWritable(src));
|
|
|
}
|
|
|
- public boolean completedCreate(UTF8 src) {
|
|
|
- return creates.remove(src);
|
|
|
+ public boolean completedCreate(String src) throws IOException {
|
|
|
+ return creates.remove(new StringBytesWritable(src));
|
|
|
}
|
|
|
public boolean hasLocks() {
|
|
|
return (locks.size() + creates.size()) > 0;
|
|
|
}
|
|
|
- public void releaseLocks() {
|
|
|
- for (Iterator<UTF8> it = locks.iterator(); it.hasNext();)
|
|
|
- internalReleaseLock(it.next(), holder);
|
|
|
+ public void releaseLocks() throws IOException {
|
|
|
+ String holderStr = holder.getString();
|
|
|
+ for (Iterator<StringBytesWritable> it = locks.iterator(); it.hasNext();)
|
|
|
+ internalReleaseLock(it.next().getString(), holderStr);
|
|
|
locks.clear();
|
|
|
- for (Iterator<UTF8> it = creates.iterator(); it.hasNext();)
|
|
|
- internalReleaseCreate(it.next(), holder);
|
|
|
+ for (Iterator<StringBytesWritable> it = creates.iterator(); it.hasNext();)
|
|
|
+ internalReleaseCreate(it.next().getString(), holderStr);
|
|
|
creates.clear();
|
|
|
}
|
|
|
|
|
@@ -1318,7 +1321,12 @@ class FSNamesystem implements FSConstants {
|
|
|
public int hashCode() {
|
|
|
return holder.hashCode();
|
|
|
}
|
|
|
+
|
|
|
+ String getHolder() throws IOException {
|
|
|
+ return holder.getString();
|
|
|
+ }
|
|
|
}
|
|
|
+
|
|
|
/******************************************************
|
|
|
* LeaseMonitor checks for leases that have expired,
|
|
|
* and disposes of them.
|
|
@@ -1355,6 +1363,18 @@ class FSNamesystem implements FSConstants {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private Lease getLease(String holder) throws IOException {
|
|
|
+ return leases.get(new StringBytesWritable(holder));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void putLease(String holder, Lease lease) throws IOException {
|
|
|
+ leases.put(new StringBytesWritable(holder), lease);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void removeLease(String holder) throws IOException {
|
|
|
+ leases.remove(new StringBytesWritable(holder));
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* Get a lock (perhaps exclusive) on the given file
|
|
@@ -1366,23 +1386,7 @@ class FSNamesystem implements FSConstants {
|
|
|
boolean exclusive) throws IOException {
|
|
|
if (isInSafeMode())
|
|
|
throw new SafeModeException("Cannot lock file " + src, safeMode);
|
|
|
- int result = dir.obtainLock(src, holder, exclusive);
|
|
|
- if (result == COMPLETE_SUCCESS) {
|
|
|
- synchronized (leases) {
|
|
|
- Lease lease = leases.get(holder);
|
|
|
- if (lease == null) {
|
|
|
- lease = new Lease(holder);
|
|
|
- leases.put(holder, lease);
|
|
|
- sortedLeases.add(lease);
|
|
|
- } else {
|
|
|
- sortedLeases.remove(lease);
|
|
|
- lease.renew();
|
|
|
- sortedLeases.add(lease);
|
|
|
- }
|
|
|
- lease.obtained(src);
|
|
|
- }
|
|
|
- }
|
|
|
- return result;
|
|
|
+ return OPERATION_FAILED;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1391,22 +1395,9 @@ class FSNamesystem implements FSConstants {
|
|
|
/** @deprecated */
|
|
|
@Deprecated
|
|
|
public synchronized int releaseLock(UTF8 src, UTF8 holder) {
|
|
|
- int result = internalReleaseLock(src, holder);
|
|
|
- if (result == COMPLETE_SUCCESS) {
|
|
|
- synchronized (leases) {
|
|
|
- Lease lease = leases.get(holder);
|
|
|
- if (lease != null) {
|
|
|
- lease.released(src);
|
|
|
- if (!lease.hasLocks()) {
|
|
|
- leases.remove(holder);
|
|
|
- sortedLeases.remove(lease);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return result;
|
|
|
+ return OPERATION_FAILED;
|
|
|
}
|
|
|
- private int internalReleaseLock(UTF8 src, UTF8 holder) {
|
|
|
+ private int internalReleaseLock(String src, String holder) throws IOException {
|
|
|
return dir.releaseLock(src, holder);
|
|
|
}
|
|
|
|
|
@@ -1415,17 +1406,15 @@ class FSNamesystem implements FSConstants {
|
|
|
* @param src The filename
|
|
|
* @param holder The datanode that was creating the file
|
|
|
*/
|
|
|
- private void internalReleaseCreate(UTF8 src, UTF8 holder) {
|
|
|
+ private void internalReleaseCreate(String src, String holder) throws IOException {
|
|
|
boolean status = pendingCreates.remove(src);
|
|
|
if (status) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.internalReleaseCreate: "
|
|
|
- + src
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.internalReleaseCreate: " + src
|
|
|
+ " is removed from pendingCreates for "
|
|
|
+ holder + " (failure)");
|
|
|
} else {
|
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.internalReleaseCreate: "
|
|
|
- + "attempt to release a create lock on "
|
|
|
- + src.toString()
|
|
|
+ + "attempt to release a create lock on "+ src
|
|
|
+ " that was not in pedingCreates");
|
|
|
}
|
|
|
}
|
|
@@ -1433,11 +1422,11 @@ class FSNamesystem implements FSConstants {
|
|
|
/**
|
|
|
* Renew the lease(s) held by the given client
|
|
|
*/
|
|
|
- public void renewLease(UTF8 holder) throws IOException {
|
|
|
+ public void renewLease(String holder) throws IOException {
|
|
|
synchronized (leases) {
|
|
|
if (isInSafeMode())
|
|
|
throw new SafeModeException("Cannot renew lease for " + holder, safeMode);
|
|
|
- Lease lease = leases.get(holder);
|
|
|
+ Lease lease = getLease(holder);
|
|
|
if (lease != null) {
|
|
|
sortedLeases.remove(lease);
|
|
|
lease.renew();
|
|
@@ -1450,7 +1439,7 @@ class FSNamesystem implements FSConstants {
|
|
|
* Get a listing of all files at 'src'. The Object[] array
|
|
|
* exists so we can return file attributes (soon to be implemented)
|
|
|
*/
|
|
|
- public DFSFileInfo[] getListing(UTF8 src) {
|
|
|
+ public DFSFileInfo[] getListing(String src) {
|
|
|
return dir.getListing(src);
|
|
|
}
|
|
|
|
|
@@ -1483,8 +1472,8 @@ class FSNamesystem implements FSConstants {
|
|
|
* @see DataNode#register()
|
|
|
*/
|
|
|
public void registerDatanode(DatanodeRegistration nodeReg,
|
|
|
- String networkLocation
|
|
|
- ) throws IOException {
|
|
|
+ String networkLocation
|
|
|
+ ) throws IOException {
|
|
|
registerDatanodeInternal(nodeReg, networkLocation);
|
|
|
getEditLog().logSync();
|
|
|
}
|