HADOOP-10571. Use Log.*(Object, Throwable) overload to log exceptions.

Contributed by Andras Bokor.
This commit is contained in:
Steve Loughran 2018-02-14 16:20:14 +00:00
parent 042ef2fa7b
commit f20dc0d577
20 changed files with 587 additions and 767 deletions

View File

@ -139,7 +139,7 @@ public boolean reportChecksumFailure(Path p, FSDataInputStream in,
LOG.warn("Ignoring failure of renameTo"); LOG.warn("Ignoring failure of renameTo");
} }
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Error moving bad file " + p + ": " + e); LOG.warn("Error moving bad file " + p, e);
} }
return false; return false;
} }

View File

@ -888,9 +888,8 @@ private boolean becomeActive() {
Stat oldBreadcrumbStat = fenceOldActive(); Stat oldBreadcrumbStat = fenceOldActive();
writeBreadCrumbNode(oldBreadcrumbStat); writeBreadCrumbNode(oldBreadcrumbStat);
if (LOG.isDebugEnabled()) { LOG.debug("Becoming active for {}", this);
LOG.debug("Becoming active for " + this);
}
appClient.becomeActive(); appClient.becomeActive();
state = State.ACTIVE; state = State.ACTIVE;
return true; return true;
@ -910,8 +909,8 @@ private void writeBreadCrumbNode(Stat oldBreadcrumbStat)
throws KeeperException, InterruptedException { throws KeeperException, InterruptedException {
Preconditions.checkState(appData != null, "no appdata"); Preconditions.checkState(appData != null, "no appdata");
LOG.info("Writing znode " + zkBreadCrumbPath + LOG.info("Writing znode {} to indicate that the local " +
" to indicate that the local node is the most recent active..."); "node is the most recent active...", zkBreadCrumbPath);
if (oldBreadcrumbStat == null) { if (oldBreadcrumbStat == null) {
// No previous active, just create the node // No previous active, just create the node
createWithRetries(zkBreadCrumbPath, appData, zkAcl, createWithRetries(zkBreadCrumbPath, appData, zkAcl,
@ -948,9 +947,8 @@ private void tryDeleteOwnBreadCrumbNode() {
deleteWithRetries(zkBreadCrumbPath, stat.getVersion()); deleteWithRetries(zkBreadCrumbPath, stat.getVersion());
} catch (Exception e) { } catch (Exception e) {
LOG.warn("Unable to delete our own bread-crumb of being active at " + LOG.warn("Unable to delete our own bread-crumb of being active at {}." +
zkBreadCrumbPath + ": " + e.getLocalizedMessage() + ". " + ". Expecting to be fenced by the next active.", zkBreadCrumbPath, e);
"Expecting to be fenced by the next active.");
} }
} }
@ -984,7 +982,7 @@ public byte[] run() throws KeeperException, InterruptedException {
throw ke; throw ke;
} }
LOG.info("Old node exists: " + StringUtils.byteToHexString(data)); LOG.info("Old node exists: {}", StringUtils.byteToHexString(data));
if (Arrays.equals(data, appData)) { if (Arrays.equals(data, appData)) {
LOG.info("But old node has our own data, so don't need to fence it."); LOG.info("But old node has our own data, so don't need to fence it.");
} else { } else {
@ -995,9 +993,7 @@ public byte[] run() throws KeeperException, InterruptedException {
private void becomeStandby() { private void becomeStandby() {
if (state != State.STANDBY) { if (state != State.STANDBY) {
if (LOG.isDebugEnabled()) { LOG.debug("Becoming standby for {}", this);
LOG.debug("Becoming standby for " + this);
}
state = State.STANDBY; state = State.STANDBY;
appClient.becomeStandby(); appClient.becomeStandby();
} }
@ -1005,9 +1001,7 @@ private void becomeStandby() {
private void enterNeutralMode() { private void enterNeutralMode() {
if (state != State.NEUTRAL) { if (state != State.NEUTRAL) {
if (LOG.isDebugEnabled()) { LOG.debug("Entering neutral mode for {}", this);
LOG.debug("Entering neutral mode for " + this);
}
state = State.NEUTRAL; state = State.NEUTRAL;
appClient.enterNeutralMode(); appClient.enterNeutralMode();
} }
@ -1124,7 +1118,7 @@ private interface ZKAction<T> {
private synchronized boolean isStaleClient(Object ctx) { private synchronized boolean isStaleClient(Object ctx) {
Preconditions.checkNotNull(ctx); Preconditions.checkNotNull(ctx);
if (zkClient != (ZooKeeper)ctx) { if (zkClient != (ZooKeeper)ctx) {
LOG.warn("Ignoring stale result from old client with sessionId " + LOG.warn("Ignoring stale result from old client with sessionId {}",
String.format("0x%08x", ((ZooKeeper)ctx).getSessionId())); String.format("0x%08x", ((ZooKeeper)ctx).getSessionId()));
return true; return true;
} }
@ -1162,8 +1156,8 @@ private void waitForZKConnectionEvent(int connectionTimeoutMs)
throws KeeperException, IOException { throws KeeperException, IOException {
try { try {
if (!hasReceivedEvent.await(connectionTimeoutMs, TimeUnit.MILLISECONDS)) { if (!hasReceivedEvent.await(connectionTimeoutMs, TimeUnit.MILLISECONDS)) {
LOG.error("Connection timed out: couldn't connect to ZooKeeper in " LOG.error("Connection timed out: couldn't connect to ZooKeeper in " +
+ connectionTimeoutMs + " milliseconds"); "{} milliseconds", connectionTimeoutMs);
zk.close(); zk.close();
throw KeeperException.create(Code.CONNECTIONLOSS); throw KeeperException.create(Code.CONNECTIONLOSS);
} }

View File

@ -123,7 +123,7 @@ private void preFailoverChecks(HAServiceTarget from,
toSvcStatus = toSvc.getServiceStatus(); toSvcStatus = toSvc.getServiceStatus();
} catch (IOException e) { } catch (IOException e) {
String msg = "Unable to get service state for " + target; String msg = "Unable to get service state for " + target;
LOG.error(msg + ": " + e.getLocalizedMessage()); LOG.error(msg, e);
throw new FailoverFailedException(msg, e); throw new FailoverFailedException(msg, e);
} }
@ -139,7 +139,7 @@ private void preFailoverChecks(HAServiceTarget from,
target + " is not ready to become active: " + target + " is not ready to become active: " +
notReadyReason); notReadyReason);
} else { } else {
LOG.warn("Service is not ready to become active, but forcing: " + LOG.warn("Service is not ready to become active, but forcing: {}",
notReadyReason); notReadyReason);
} }
} }
@ -172,11 +172,11 @@ boolean tryGracefulFence(HAServiceTarget svc) {
proxy.transitionToStandby(createReqInfo()); proxy.transitionToStandby(createReqInfo());
return true; return true;
} catch (ServiceFailedException sfe) { } catch (ServiceFailedException sfe) {
LOG.warn("Unable to gracefully make " + svc + " standby (" + LOG.warn("Unable to gracefully make {} standby ({})",
sfe.getMessage() + ")"); svc, sfe.getMessage());
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.warn("Unable to gracefully make " + svc + LOG.warn("Unable to gracefully make {} standby (unable to connect)",
" standby (unable to connect)", ioe); svc, ioe);
} finally { } finally {
if (proxy != null) { if (proxy != null) {
RPC.stopProxy(proxy); RPC.stopProxy(proxy);
@ -227,13 +227,13 @@ public void failover(HAServiceTarget fromSvc,
toSvc.getProxy(conf, rpcTimeoutToNewActive), toSvc.getProxy(conf, rpcTimeoutToNewActive),
createReqInfo()); createReqInfo());
} catch (ServiceFailedException sfe) { } catch (ServiceFailedException sfe) {
LOG.error("Unable to make " + toSvc + " active (" + LOG.error("Unable to make {} active ({}). Failing back.",
sfe.getMessage() + "). Failing back."); toSvc, sfe.getMessage());
failed = true; failed = true;
cause = sfe; cause = sfe;
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.error("Unable to make " + toSvc + LOG.error("Unable to make {} active (unable to connect). Failing back.",
" active (unable to connect). Failing back.", ioe); toSvc, ioe);
failed = true; failed = true;
cause = ioe; cause = ioe;
} }

View File

@ -204,12 +204,11 @@ private void doHealthChecks() throws InterruptedException {
healthy = true; healthy = true;
} catch (Throwable t) { } catch (Throwable t) {
if (isHealthCheckFailedException(t)) { if (isHealthCheckFailedException(t)) {
LOG.warn("Service health check failed for " + targetToMonitor LOG.warn("Service health check failed for {}", targetToMonitor, t);
+ ": " + t.getMessage());
enterState(State.SERVICE_UNHEALTHY); enterState(State.SERVICE_UNHEALTHY);
} else { } else {
LOG.warn("Transport-level exception trying to monitor health of " + LOG.warn("Transport-level exception trying to monitor health of {}",
targetToMonitor + ": " + t.getCause() + " " + t.getLocalizedMessage()); targetToMonitor, t);
RPC.stopProxy(proxy); RPC.stopProxy(proxy);
proxy = null; proxy = null;
enterState(State.SERVICE_NOT_RESPONDING); enterState(State.SERVICE_NOT_RESPONDING);
@ -246,7 +245,7 @@ private synchronized void setLastServiceStatus(HAServiceStatus status) {
private synchronized void enterState(State newState) { private synchronized void enterState(State newState) {
if (newState != state) { if (newState != state) {
LOG.info("Entering state " + newState); LOG.info("Entering state {}", newState);
state = newState; state = newState;
synchronized (callbacks) { synchronized (callbacks) {
for (Callback cb : callbacks) { for (Callback cb : callbacks) {

View File

@ -72,9 +72,7 @@ public static RetryPolicy getDefaultRetryPolicy(
retryPolicySpecKey, defaultRetryPolicySpec retryPolicySpecKey, defaultRetryPolicySpec
); );
if (LOG.isDebugEnabled()) { LOG.debug("multipleLinearRandomRetry = {}", multipleLinearRandomRetry);
LOG.debug("multipleLinearRandomRetry = " + multipleLinearRandomRetry);
}
if (multipleLinearRandomRetry == null) { if (multipleLinearRandomRetry == null) {
//no retry //no retry
@ -124,10 +122,9 @@ public RetryAction shouldRetry(Exception e, int retries, int failovers,
p = RetryPolicies.TRY_ONCE_THEN_FAIL; p = RetryPolicies.TRY_ONCE_THEN_FAIL;
} }
if (LOG.isDebugEnabled()) { LOG.debug("RETRY {}) policy={}", retries,
LOG.debug("RETRY " + retries + ") policy=" p.getClass().getSimpleName(), e);
+ p.getClass().getSimpleName() + ", exception=" + e);
}
return p.shouldRetry(e, retries, failovers, isMethodIdempotent); return p.shouldRetry(e, retries, failovers, isMethodIdempotent);
} }

View File

@ -179,8 +179,7 @@ public static String[] getIPs(String strInterface,
netIf = getSubinterface(strInterface); netIf = getSubinterface(strInterface);
} }
} catch (SocketException e) { } catch (SocketException e) {
LOG.warn("I/O error finding interface " + strInterface + LOG.warn("I/O error finding interface {}", strInterface, e);
": " + e.getMessage());
return new String[] { cachedHostAddress }; return new String[] { cachedHostAddress };
} }
if (netIf == null) { if (netIf == null) {
@ -265,7 +264,7 @@ public static String[] getHosts(String strInterface,
} }
if (hosts.isEmpty()) { if (hosts.isEmpty()) {
LOG.warn("Unable to determine hostname for interface " + LOG.warn("Unable to determine hostname for interface {}",
strInterface); strInterface);
hosts.add(cachedHostname); hosts.add(cachedHostname);
} }
@ -283,8 +282,8 @@ private static String resolveLocalHostname() {
try { try {
localhost = InetAddress.getLocalHost().getCanonicalHostName(); localhost = InetAddress.getLocalHost().getCanonicalHostName();
} catch (UnknownHostException e) { } catch (UnknownHostException e) {
LOG.warn("Unable to determine local hostname " LOG.warn("Unable to determine local hostname -falling back to '{}'",
+ "-falling back to \"" + LOCALHOST + "\"", e); LOCALHOST, e);
localhost = LOCALHOST; localhost = LOCALHOST;
} }
return localhost; return localhost;
@ -306,15 +305,15 @@ private static String resolveLocalHostIPAddress() {
try { try {
address = InetAddress.getLocalHost().getHostAddress(); address = InetAddress.getLocalHost().getHostAddress();
} catch (UnknownHostException e) { } catch (UnknownHostException e) {
LOG.warn("Unable to determine address of the host" LOG.warn("Unable to determine address of the host " +
+ "-falling back to \"" + LOCALHOST + "\" address", e); "-falling back to '{}' address", LOCALHOST, e);
try { try {
address = InetAddress.getByName(LOCALHOST).getHostAddress(); address = InetAddress.getByName(LOCALHOST).getHostAddress();
} catch (UnknownHostException noLocalHostAddressException) { } catch (UnknownHostException noLocalHostAddressException) {
//at this point, deep trouble //at this point, deep trouble
LOG.error("Unable to determine local loopback address " LOG.error("Unable to determine local loopback address of '{}' " +
+ "of \"" + LOCALHOST + "\" " + "-this system's network configuration is unsupported",
"-this system's network configuration is unsupported", e); LOCALHOST, e);
address = null; address = null;
} }
} }
@ -431,8 +430,8 @@ public static List<InetAddress> getIPsAsInetAddressList(String strInterface,
netIf = getSubinterface(strInterface); netIf = getSubinterface(strInterface);
} }
} catch (SocketException e) { } catch (SocketException e) {
LOG.warn("I/O error finding interface " + strInterface + LOG.warn("I/O error finding interface {}: {}",
": " + e.getMessage()); strInterface, e.getMessage());
return Arrays.asList(InetAddress.getByName(cachedHostAddress)); return Arrays.asList(InetAddress.getByName(cachedHostAddress));
} }
if (netIf == null) { if (netIf == null) {

View File

@ -194,9 +194,7 @@ public void start() {
serviceStart(); serviceStart();
if (isInState(STATE.STARTED)) { if (isInState(STATE.STARTED)) {
//if the service started (and isn't now in a later state), notify //if the service started (and isn't now in a later state), notify
if (LOG.isDebugEnabled()) { LOG.debug("Service {} is started", getName());
LOG.debug("Service " + getName() + " is started");
}
notifyListeners(); notifyListeners();
} }
} catch (Exception e) { } catch (Exception e) {
@ -235,12 +233,10 @@ public void stop() {
} }
} else { } else {
//already stopped: note it //already stopped: note it
if (LOG.isDebugEnabled()) {
LOG.debug("Ignoring re-entrant call to stop()"); LOG.debug("Ignoring re-entrant call to stop()");
} }
} }
} }
}
/** /**
* Relay to {@link #stop()} * Relay to {@link #stop()}
@ -258,9 +254,7 @@ public final void close() throws IOException {
* @param exception the exception * @param exception the exception
*/ */
protected final void noteFailure(Exception exception) { protected final void noteFailure(Exception exception) {
if (LOG.isDebugEnabled()) { LOG.debug("noteFailure {}" + exception);
LOG.debug("noteFailure " + exception, (Throwable) null);
}
if (exception == null) { if (exception == null) {
//make sure failure logic doesn't itself cause problems //make sure failure logic doesn't itself cause problems
return; return;
@ -270,10 +264,8 @@ protected final void noteFailure(Exception exception) {
if (failureCause == null) { if (failureCause == null) {
failureCause = exception; failureCause = exception;
failureState = getServiceState(); failureState = getServiceState();
LOG.info("Service " + getName() LOG.info("Service {} failed in state {}",
+ " failed in state " + failureState getName(), failureState, exception);
+ "; cause: " + exception,
exception);
} }
} }
} }
@ -418,8 +410,7 @@ private void notifyListeners() {
listeners.notifyListeners(this); listeners.notifyListeners(this);
globalListeners.notifyListeners(this); globalListeners.notifyListeners(this);
} catch (Throwable e) { } catch (Throwable e) {
LOG.warn("Exception while notifying listeners of " + this + ": " + e, LOG.warn("Exception while notifying listeners of {}", this, e);
e);
} }
} }
@ -449,10 +440,8 @@ private STATE enterState(STATE newState) {
assert stateModel != null : "null state in " + name + " " + this.getClass(); assert stateModel != null : "null state in " + name + " " + this.getClass();
STATE oldState = stateModel.enterState(newState); STATE oldState = stateModel.enterState(newState);
if (oldState != newState) { if (oldState != newState) {
if (LOG.isDebugEnabled()) { LOG.debug("Service: {} entered state {}", getName(), getServiceState());
LOG.debug(
"Service: " + getName() + " entered state " + getServiceState());
}
recordLifecycleEvent(); recordLifecycleEvent();
} }
return oldState; return oldState;

View File

@ -81,9 +81,7 @@ public static Exception stopQuietly(Log log, Service service) {
try { try {
stop(service); stop(service);
} catch (Exception e) { } catch (Exception e) {
log.warn("When stopping the service " + service.getName() log.warn("When stopping the service " + service.getName(), e);
+ " : " + e,
e);
return e; return e;
} }
return null; return null;
@ -103,7 +101,7 @@ public static Exception stopQuietly(Logger log, Service service) {
try { try {
stop(service); stop(service);
} catch (Exception e) { } catch (Exception e) {
log.warn("When stopping the service {} : {}", service.getName(), e, e); log.warn("When stopping the service {}", service.getName(), e);
return e; return e;
} }
return null; return null;

View File

@ -57,8 +57,7 @@ public void testStopQuietlyWhenServiceStopThrowsException() throws Exception {
ServiceOperations.stopQuietly(logger, service); ServiceOperations.stopQuietly(logger, service);
assertThat(logCapturer.getOutput(), assertThat(logCapturer.getOutput(),
containsString("When stopping the service " + service.getName() containsString("When stopping the service " + service.getName()));
+ " : " + e));
verify(e, times(1)).printStackTrace(Mockito.any(PrintWriter.class)); verify(e, times(1)).printStackTrace(Mockito.any(PrintWriter.class));
} }

View File

@ -17,8 +17,6 @@
*/ */
package org.apache.hadoop.hdfs.nfs.nfs3; package org.apache.hadoop.hdfs.nfs.nfs3;
import org.apache.commons.logging.LogFactory;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.nio.file.FileSystemException; import java.nio.file.FileSystemException;
@ -32,7 +30,6 @@
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
@ -50,12 +47,15 @@
import com.google.common.cache.LoadingCache; import com.google.common.cache.LoadingCache;
import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification; import com.google.common.cache.RemovalNotification;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* A cache saves DFSClient objects for different users. * A cache saves DFSClient objects for different users.
*/ */
class DFSClientCache { class DFSClientCache {
private static final Log LOG = LogFactory.getLog(DFSClientCache.class); private static final Logger LOG =
LoggerFactory.getLogger(DFSClientCache.class);
/** /**
* Cache that maps User id to the corresponding DFSClient. * Cache that maps User id to the corresponding DFSClient.
*/ */
@ -169,8 +169,8 @@ private void prepareAddressMap() throws IOException {
URI value = namenodeUriMap.get(namenodeId); URI value = namenodeUriMap.get(namenodeId);
// if a unique nnid, add it to the map // if a unique nnid, add it to the map
if (value == null) { if (value == null) {
LOG.info("Added export:" + exportPath + " FileSystem URI:" + exportURI LOG.info("Added export: {} FileSystem URI: {} with namenodeId: {}",
+ " with namenodeId:" + namenodeId); exportPath, exportPath, namenodeId);
namenodeUriMap.put(namenodeId, exportURI); namenodeUriMap.put(namenodeId, exportURI);
} else { } else {
// if the nnid already exists, it better be the for the same namenode // if the nnid already exists, it better be the for the same namenode
@ -194,7 +194,7 @@ public synchronized void run() {
try { try {
closeAll(true); closeAll(true);
} catch (IOException e) { } catch (IOException e) {
LOG.info("DFSClientCache.closeAll() threw an exception:\n", e); LOG.info("DFSClientCache.closeAll() threw an exception", e);
} }
} }
} }
@ -269,10 +269,7 @@ UserGroupInformation getUserGroupInformation(
UserGroupInformation ugi = UserGroupInformation ugi =
UserGroupInformation.createProxyUser(effectiveUser, realUser); UserGroupInformation.createProxyUser(effectiveUser, realUser);
if (LOG.isDebugEnabled()){ LOG.debug("Created ugi: {} for username: {}", ugi, effectiveUser);
LOG.debug(String.format("Created ugi:" +
" %s for username: %s", ugi, effectiveUser));
}
return ugi; return ugi;
} }
@ -329,8 +326,7 @@ DFSClient getDfsClient(String userName, int namenodeId) {
try { try {
client = clientCache.get(new DfsClientKey(userName, namenodeId)); client = clientCache.get(new DfsClientKey(userName, namenodeId));
} catch (ExecutionException e) { } catch (ExecutionException e) {
LOG.error("Failed to create DFSClient for user:" + userName + " Cause:" LOG.error("Failed to create DFSClient for user: {}", userName, e);
+ e);
} }
return client; return client;
} }
@ -343,8 +339,7 @@ FSDataInputStream getDfsInputStream(String userName, String inodePath,
try { try {
s = inputstreamCache.get(k); s = inputstreamCache.get(k);
} catch (ExecutionException e) { } catch (ExecutionException e) {
LOG.warn("Failed to create DFSInputStream for user:" + userName LOG.warn("Failed to create DFSInputStream for user: {}", userName, e);
+ " Cause:" + e);
} }
return s; return s;
} }

View File

@ -31,8 +31,6 @@
import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
@ -61,13 +59,15 @@
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* OpenFileCtx saves the context of one HDFS file output stream. Access to it is * OpenFileCtx saves the context of one HDFS file output stream. Access to it is
* synchronized by its member lock. * synchronized by its member lock.
*/ */
class OpenFileCtx { class OpenFileCtx {
public static final Log LOG = LogFactory.getLog(OpenFileCtx.class); public static final Logger LOG = LoggerFactory.getLogger(OpenFileCtx.class);
// Pending writes water mark for dump, 1MB // Pending writes water mark for dump, 1MB
private static long DUMP_WRITE_WATER_MARK = 1024 * 1024; private static long DUMP_WRITE_WATER_MARK = 1024 * 1024;
@ -210,10 +210,8 @@ boolean hasPendingWork() {
/** Increase or decrease the memory occupation of non-sequential writes */ /** Increase or decrease the memory occupation of non-sequential writes */
private long updateNonSequentialWriteInMemory(long count) { private long updateNonSequentialWriteInMemory(long count) {
long newValue = nonSequentialWriteInMemory.addAndGet(count); long newValue = nonSequentialWriteInMemory.addAndGet(count);
if (LOG.isDebugEnabled()) { LOG.debug("Update nonSequentialWriteInMemory by {} new value: {}",
LOG.debug("Update nonSequentialWriteInMemory by " + count + " new value: " count, newValue);
+ newValue);
}
Preconditions.checkState(newValue >= 0, Preconditions.checkState(newValue >= 0,
"nonSequentialWriteInMemory is negative " + newValue "nonSequentialWriteInMemory is negative " + newValue
@ -273,9 +271,7 @@ private long getFlushedOffset() {
// Check if need to dump the new writes // Check if need to dump the new writes
private void waitForDump() { private void waitForDump() {
if (!enabledDump) { if (!enabledDump) {
if (LOG.isDebugEnabled()) {
LOG.debug("Do nothing, dump is disabled."); LOG.debug("Do nothing, dump is disabled.");
}
return; return;
} }
@ -286,9 +282,7 @@ private void waitForDump() {
// wake up the dumper thread to dump the data // wake up the dumper thread to dump the data
synchronized (this) { synchronized (this) {
if (nonSequentialWriteInMemory.get() >= DUMP_WRITE_WATER_MARK) { if (nonSequentialWriteInMemory.get() >= DUMP_WRITE_WATER_MARK) {
if (LOG.isDebugEnabled()) {
LOG.debug("Asking dumper to dump..."); LOG.debug("Asking dumper to dump...");
}
if (dumpThread == null) { if (dumpThread == null) {
dumpThread = new Daemon(new Dumper()); dumpThread = new Daemon(new Dumper());
dumpThread.start(); dumpThread.start();
@ -312,7 +306,7 @@ class Dumper implements Runnable {
private void dump() { private void dump() {
// Create dump outputstream for the first time // Create dump outputstream for the first time
if (dumpOut == null) { if (dumpOut == null) {
LOG.info("Create dump file: " + dumpFilePath); LOG.info("Create dump file: {}", dumpFilePath);
File dumpFile = new File(dumpFilePath); File dumpFile = new File(dumpFilePath);
try { try {
synchronized (this) { synchronized (this) {
@ -322,13 +316,14 @@ private void dump() {
dumpOut = new FileOutputStream(dumpFile); dumpOut = new FileOutputStream(dumpFile);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("Got failure when creating dump stream " + dumpFilePath, e); LOG.error("Got failure when creating dump stream {}",
dumpFilePath, e);
enabledDump = false; enabledDump = false;
if (dumpOut != null) { if (dumpOut != null) {
try { try {
dumpOut.close(); dumpOut.close();
} catch (IOException e1) { } catch (IOException e1) {
LOG.error("Can't close dump stream " + dumpFilePath, e); LOG.error("Can't close dump stream {}", dumpFilePath, e);
} }
} }
return; return;
@ -340,17 +335,15 @@ private void dump() {
try { try {
raf = new RandomAccessFile(dumpFilePath, "r"); raf = new RandomAccessFile(dumpFilePath, "r");
} catch (FileNotFoundException e) { } catch (FileNotFoundException e) {
LOG.error("Can't get random access to file " + dumpFilePath); LOG.error("Can't get random access to file {}", dumpFilePath);
// Disable dump // Disable dump
enabledDump = false; enabledDump = false;
return; return;
} }
} }
if (LOG.isDebugEnabled()) { LOG.debug("Start dump. Before dump, nonSequentialWriteInMemory == {}",
LOG.debug("Start dump. Before dump, nonSequentialWriteInMemory == " nonSequentialWriteInMemory.get());
+ nonSequentialWriteInMemory.get());
}
Iterator<OffsetRange> it = pendingWrites.keySet().iterator(); Iterator<OffsetRange> it = pendingWrites.keySet().iterator();
while (activeState && it.hasNext() while (activeState && it.hasNext()
@ -367,18 +360,16 @@ private void dump() {
updateNonSequentialWriteInMemory(-dumpedDataSize); updateNonSequentialWriteInMemory(-dumpedDataSize);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("Dump data failed: " + writeCtx + " with error: " + e LOG.error("Dump data failed: {} OpenFileCtx state: {}",
+ " OpenFileCtx state: " + activeState); writeCtx, activeState, e);
// Disable dump // Disable dump
enabledDump = false; enabledDump = false;
return; return;
} }
} }
if (LOG.isDebugEnabled()) { LOG.debug("After dump, nonSequentialWriteInMemory == {}",
LOG.debug("After dump, nonSequentialWriteInMemory == " nonSequentialWriteInMemory.get());
+ nonSequentialWriteInMemory.get());
}
} }
@Override @Override
@ -393,26 +384,22 @@ public void run() {
OpenFileCtx.this.notifyAll(); OpenFileCtx.this.notifyAll();
try { try {
OpenFileCtx.this.wait(); OpenFileCtx.this.wait();
if (LOG.isDebugEnabled()) {
LOG.debug("Dumper woke up"); LOG.debug("Dumper woke up");
}
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.info("Dumper is interrupted, dumpFilePath= " LOG.info("Dumper is interrupted, dumpFilePath = {}",
+ OpenFileCtx.this.dumpFilePath); OpenFileCtx.this.dumpFilePath);
} }
} }
} }
if (LOG.isDebugEnabled()) { LOG.debug("Dumper checking OpenFileCtx activeState: {} " +
LOG.debug("Dumper checking OpenFileCtx activeState: " + activeState "enabledDump: {}", activeState, enabledDump);
+ " enabledDump: " + enabledDump);
}
} catch (Throwable t) { } catch (Throwable t) {
// unblock threads with new request // unblock threads with new request
synchronized (OpenFileCtx.this) { synchronized (OpenFileCtx.this) {
OpenFileCtx.this.notifyAll(); OpenFileCtx.this.notifyAll();
} }
LOG.info("Dumper get Throwable: " + t + ". dumpFilePath: " LOG.info("Dumper got Throwable. dumpFilePath: {}",
+ OpenFileCtx.this.dumpFilePath, t); OpenFileCtx.this.dumpFilePath, t);
activeState = false; activeState = false;
} }
} }
@ -428,8 +415,8 @@ private WriteCtx checkRepeatedWriteRequest(WRITE3Request request,
return null; return null;
} else { } else {
if (xid != writeCtx.getXid()) { if (xid != writeCtx.getXid()) {
LOG.warn("Got a repeated request, same range, with a different xid: " LOG.warn("Got a repeated request, same range, with a different xid: " +
+ xid + " xid in old request: " + writeCtx.getXid()); "{} xid in old request: {}", xid, writeCtx.getXid());
//TODO: better handling. //TODO: better handling.
} }
return writeCtx; return writeCtx;
@ -441,8 +428,8 @@ public void receivedNewWrite(DFSClient dfsClient, WRITE3Request request,
IdMappingServiceProvider iug) { IdMappingServiceProvider iug) {
if (!activeState) { if (!activeState) {
LOG.info("OpenFileCtx is inactive, fileId: " LOG.info("OpenFileCtx is inactive, fileId: {}",
+ request.getHandle().dumpFileHandle()); request.getHandle().dumpFileHandle());
WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr); WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO, WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO,
fileWcc, 0, request.getStableHow(), Nfs3Constant.WRITE_COMMIT_VERF); fileWcc, 0, request.getStableHow(), Nfs3Constant.WRITE_COMMIT_VERF);
@ -460,15 +447,11 @@ public void receivedNewWrite(DFSClient dfsClient, WRITE3Request request,
xid); xid);
if (existantWriteCtx != null) { if (existantWriteCtx != null) {
if (!existantWriteCtx.getReplied()) { if (!existantWriteCtx.getReplied()) {
if (LOG.isDebugEnabled()) { LOG.debug("Repeated write request which hasn't been served: " +
LOG.debug("Repeated write request which hasn't been served: xid=" "xid={}, drop it.", xid);
+ xid + ", drop it.");
}
} else { } else {
if (LOG.isDebugEnabled()) { LOG.debug("Repeated write request which is already served: xid={}" +
LOG.debug("Repeated write request which is already served: xid=" ", resend response.", xid);
+ xid + ", resend response.");
}
WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr); WccData fileWcc = new WccData(latestAttr.getWccAttr(), latestAttr);
WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK, WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
fileWcc, request.getCount(), request.getStableHow(), fileWcc, request.getCount(), request.getStableHow(),
@ -489,13 +472,11 @@ public static void alterWriteRequest(WRITE3Request request, long cachedOffset) {
long offset = request.getOffset(); long offset = request.getOffset();
int count = request.getCount(); int count = request.getCount();
long smallerCount = offset + count - cachedOffset; long smallerCount = offset + count - cachedOffset;
if (LOG.isDebugEnabled()) { LOG.debug("Got overwrite with appended data [{}-{}),"
LOG.debug(String.format("Got overwrite with appended data [%d-%d)," + " current offset {}," + " drop the overlapped section [{}-{})"
+ " current offset %d," + " drop the overlapped section [%d-%d)" + " and append new data [{}-{}).", offset, (offset + count),
+ " and append new data [%d-%d).", offset, (offset + count), cachedOffset, offset, cachedOffset, cachedOffset,
cachedOffset, offset, cachedOffset, cachedOffset, (offset (offset + count));
+ count)));
}
ByteBuffer data = request.getData(); ByteBuffer data = request.getData();
Preconditions.checkState(data.position() == 0, Preconditions.checkState(data.position() == 0,
@ -538,10 +519,8 @@ private synchronized WriteCtx addWritesToCache(WRITE3Request request,
long cachedOffset = nextOffset.get(); long cachedOffset = nextOffset.get();
int originalCount = WriteCtx.INVALID_ORIGINAL_COUNT; int originalCount = WriteCtx.INVALID_ORIGINAL_COUNT;
if (LOG.isDebugEnabled()) { LOG.debug("requested offset={} and current offset={}",
LOG.debug("requested offset=" + offset + " and current offset=" offset, cachedOffset);
+ cachedOffset);
}
// Ignore write request with range below the current offset // Ignore write request with range below the current offset
if (offset + count <= cachedOffset) { if (offset + count <= cachedOffset) {
@ -576,8 +555,8 @@ private synchronized WriteCtx addWritesToCache(WRITE3Request request,
// Fail non-append call // Fail non-append call
if (offset < cachedOffset) { if (offset < cachedOffset) {
LOG.warn("(offset,count,nextOffset): " + "(" + offset + "," + count + "," LOG.warn("(offset,count,nextOffset): ({},{},{})",
+ nextOffset + ")"); offset, count, nextOffset);
return null; return null;
} else { } else {
DataState dataState = offset == cachedOffset ? WriteCtx.DataState.NO_DUMP DataState dataState = offset == cachedOffset ? WriteCtx.DataState.NO_DUMP
@ -586,10 +565,8 @@ private synchronized WriteCtx addWritesToCache(WRITE3Request request,
request.getOffset(), request.getCount(), originalCount, request.getOffset(), request.getCount(), originalCount,
request.getStableHow(), request.getData(), channel, xid, false, request.getStableHow(), request.getData(), channel, xid, false,
dataState); dataState);
if (LOG.isDebugEnabled()) { LOG.debug("Add new write to the list with nextOffset {}" +
LOG.debug("Add new write to the list with nextOffset " + cachedOffset " and requested offset={}", cachedOffset, offset);
+ " and requested offset=" + offset);
}
if (writeCtx.getDataState() == WriteCtx.DataState.ALLOW_DUMP) { if (writeCtx.getDataState() == WriteCtx.DataState.ALLOW_DUMP) {
// update the memory size // update the memory size
updateNonSequentialWriteInMemory(count); updateNonSequentialWriteInMemory(count);
@ -598,14 +575,12 @@ private synchronized WriteCtx addWritesToCache(WRITE3Request request,
WriteCtx oldWriteCtx = checkRepeatedWriteRequest(request, channel, xid); WriteCtx oldWriteCtx = checkRepeatedWriteRequest(request, channel, xid);
if (oldWriteCtx == null) { if (oldWriteCtx == null) {
pendingWrites.put(new OffsetRange(offset, offset + count), writeCtx); pendingWrites.put(new OffsetRange(offset, offset + count), writeCtx);
if (LOG.isDebugEnabled()) { LOG.debug("New write buffered with xid {} nextOffset {}" +
LOG.debug("New write buffered with xid " + xid + " nextOffset " "req offset={} mapsize={}",
+ cachedOffset + " req offset=" + offset + " mapsize=" xid, cachedOffset, offset, pendingWrites.size());
+ pendingWrites.size());
}
} else { } else {
LOG.warn("Got a repeated request, same range, with xid: " + xid LOG.warn("Got a repeated request, same range, with xid: " +
+ " nextOffset " + +cachedOffset + " req offset=" + offset); "{} nextOffset {} req offset={}", xid, cachedOffset, offset);
} }
return writeCtx; return writeCtx;
} }
@ -625,9 +600,7 @@ private void processOverWrite(DFSClient dfsClient, WRITE3Request request,
response = new WRITE3Response(Nfs3Status.NFS3ERR_INVAL, wccData, 0, response = new WRITE3Response(Nfs3Status.NFS3ERR_INVAL, wccData, 0,
WriteStableHow.UNSTABLE, Nfs3Constant.WRITE_COMMIT_VERF); WriteStableHow.UNSTABLE, Nfs3Constant.WRITE_COMMIT_VERF);
} else { } else {
if (LOG.isDebugEnabled()) {
LOG.debug("Process perfectOverWrite"); LOG.debug("Process perfectOverWrite");
}
// TODO: let executor handle perfect overwrite // TODO: let executor handle perfect overwrite
response = processPerfectOverWrite(dfsClient, offset, count, stableHow, response = processPerfectOverWrite(dfsClient, offset, count, stableHow,
request.getData().array(), request.getData().array(),
@ -652,18 +625,14 @@ private synchronized boolean checkAndStartWrite(
if (writeCtx.getOffset() == nextOffset.get()) { if (writeCtx.getOffset() == nextOffset.get()) {
if (!asyncStatus) { if (!asyncStatus) {
if (LOG.isDebugEnabled()) { LOG.debug("Trigger the write back task. Current nextOffset: {}",
LOG.debug("Trigger the write back task. Current nextOffset: " nextOffset.get());
+ nextOffset.get());
}
asyncStatus = true; asyncStatus = true;
asyncWriteBackStartOffset = writeCtx.getOffset(); asyncWriteBackStartOffset = writeCtx.getOffset();
asyncDataService.execute(new AsyncDataService.WriteBackTask(this)); asyncDataService.execute(new AsyncDataService.WriteBackTask(this));
} else { } else {
if (LOG.isDebugEnabled()) {
LOG.debug("The write back thread is working."); LOG.debug("The write back thread is working.");
} }
}
return true; return true;
} else { } else {
return false; return false;
@ -694,15 +663,13 @@ private void receivedNewWriteInternal(DFSClient dfsClient,
// responses of the previous batch. So here send response immediately // responses of the previous batch. So here send response immediately
// for unstable non-sequential write // for unstable non-sequential write
if (stableHow != WriteStableHow.UNSTABLE) { if (stableHow != WriteStableHow.UNSTABLE) {
LOG.info("Have to change stable write to unstable write: " LOG.info("Have to change stable write to unstable write: {}",
+ request.getStableHow()); request.getStableHow());
stableHow = WriteStableHow.UNSTABLE; stableHow = WriteStableHow.UNSTABLE;
} }
if (LOG.isDebugEnabled()) { LOG.debug("UNSTABLE write request, send response for offset: {}",
LOG.debug("UNSTABLE write request, send response for offset: " writeCtx.getOffset());
+ writeCtx.getOffset());
}
WccData fileWcc = new WccData(preOpAttr, latestAttr); WccData fileWcc = new WccData(preOpAttr, latestAttr);
WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK, WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
fileWcc, count, stableHow, Nfs3Constant.WRITE_COMMIT_VERF); fileWcc, count, stableHow, Nfs3Constant.WRITE_COMMIT_VERF);
@ -738,8 +705,8 @@ private WRITE3Response processPerfectOverWrite(DFSClient dfsClient,
LOG.info("The FSDataOutputStream has been closed. " LOG.info("The FSDataOutputStream has been closed. "
+ "Continue processing the perfect overwrite."); + "Continue processing the perfect overwrite.");
} catch (IOException e) { } catch (IOException e) {
LOG.info("hsync failed when processing possible perfect overwrite, path=" LOG.info("hsync failed when processing possible perfect overwrite, " +
+ path + " error: " + e); "path={} error: {}", path, e.toString());
return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow, return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow,
Nfs3Constant.WRITE_COMMIT_VERF); Nfs3Constant.WRITE_COMMIT_VERF);
} }
@ -748,18 +715,18 @@ private WRITE3Response processPerfectOverWrite(DFSClient dfsClient,
fis = dfsClient.createWrappedInputStream(dfsClient.open(path)); fis = dfsClient.createWrappedInputStream(dfsClient.open(path));
readCount = fis.read(offset, readbuffer, 0, count); readCount = fis.read(offset, readbuffer, 0, count);
if (readCount < count) { if (readCount < count) {
LOG.error("Can't read back " + count + " bytes, partial read size: " LOG.error("Can't read back {} bytes, partial read size: {}",
+ readCount); count, readCount);
return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow, return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow,
Nfs3Constant.WRITE_COMMIT_VERF); Nfs3Constant.WRITE_COMMIT_VERF);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.info("Read failed when processing possible perfect overwrite, path=" LOG.info("Read failed when processing possible perfect overwrite, " +
+ path, e); "path={}", path, e);
return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow, return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow,
Nfs3Constant.WRITE_COMMIT_VERF); Nfs3Constant.WRITE_COMMIT_VERF);
} finally { } finally {
IOUtils.cleanup(LOG, fis); IOUtils.cleanupWithLogger(LOG, fis);
} }
// Compare with the request // Compare with the request
@ -776,8 +743,8 @@ private WRITE3Response processPerfectOverWrite(DFSClient dfsClient,
dfsClient.setTimes(path, Time.monotonicNow(), -1); dfsClient.setTimes(path, Time.monotonicNow(), -1);
postOpAttr = Nfs3Utils.getFileAttr(dfsClient, path, iug); postOpAttr = Nfs3Utils.getFileAttr(dfsClient, path, iug);
} catch (IOException e) { } catch (IOException e) {
LOG.info("Got error when processing perfect overwrite, path=" + path LOG.info("Got error when processing perfect overwrite, path={} " +
+ " error: " + e); "error: {}", path, e.toString());
return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow, return new WRITE3Response(Nfs3Status.NFS3ERR_IO, wccData, 0, stableHow,
Nfs3Constant.WRITE_COMMIT_VERF); Nfs3Constant.WRITE_COMMIT_VERF);
} }
@ -810,9 +777,7 @@ public COMMIT_STATUS checkCommit(DFSClient dfsClient, long commitOffset,
COMMIT_STATUS ret = checkCommitInternal(commitOffset, channel, xid, COMMIT_STATUS ret = checkCommitInternal(commitOffset, channel, xid,
preOpAttr, fromRead); preOpAttr, fromRead);
if (LOG.isDebugEnabled()) { LOG.debug("Got commit status: {}", ret.name());
LOG.debug("Got commit status: " + ret.name());
}
// Do the sync outside the lock // Do the sync outside the lock
if (ret == COMMIT_STATUS.COMMIT_DO_SYNC if (ret == COMMIT_STATUS.COMMIT_DO_SYNC
|| ret == COMMIT_STATUS.COMMIT_FINISHED) { || ret == COMMIT_STATUS.COMMIT_FINISHED) {
@ -828,7 +793,7 @@ public COMMIT_STATUS checkCommit(DFSClient dfsClient, long commitOffset,
ret = COMMIT_STATUS.COMMIT_ERROR; ret = COMMIT_STATUS.COMMIT_ERROR;
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("Got stream error during data sync: " + e); LOG.error("Got stream error during data sync", e);
// Do nothing. Stream will be closed eventually by StreamMonitor. // Do nothing. Stream will be closed eventually by StreamMonitor.
// status = Nfs3Status.NFS3ERR_IO; // status = Nfs3Status.NFS3ERR_IO;
ret = COMMIT_STATUS.COMMIT_ERROR; ret = COMMIT_STATUS.COMMIT_ERROR;
@ -867,9 +832,7 @@ private COMMIT_STATUS handleSpecialWait(boolean fromRead, long commitOffset,
CommitCtx commitCtx = new CommitCtx(commitOffset, channel, xid, preOpAttr); CommitCtx commitCtx = new CommitCtx(commitOffset, channel, xid, preOpAttr);
pendingCommits.put(commitOffset, commitCtx); pendingCommits.put(commitOffset, commitCtx);
} }
if (LOG.isDebugEnabled()) {
LOG.debug("return COMMIT_SPECIAL_WAIT"); LOG.debug("return COMMIT_SPECIAL_WAIT");
}
return COMMIT_STATUS.COMMIT_SPECIAL_WAIT; return COMMIT_STATUS.COMMIT_SPECIAL_WAIT;
} }
@ -886,10 +849,8 @@ synchronized COMMIT_STATUS checkCommitInternal(long commitOffset,
} }
long flushed = getFlushedOffset(); long flushed = getFlushedOffset();
if (LOG.isDebugEnabled()) { LOG.debug("getFlushedOffset={} commitOffset={} nextOffset={}",
LOG.debug("getFlushedOffset=" + flushed + " commitOffset=" + commitOffset flushed, commitOffset, nextOffset.get());
+ "nextOffset=" + nextOffset.get());
}
if (pendingWrites.isEmpty()) { if (pendingWrites.isEmpty()) {
if (aixCompatMode) { if (aixCompatMode) {
@ -898,10 +859,8 @@ synchronized COMMIT_STATUS checkCommitInternal(long commitOffset,
return COMMIT_STATUS.COMMIT_FINISHED; return COMMIT_STATUS.COMMIT_FINISHED;
} else { } else {
if (flushed < nextOffset.get()) { if (flushed < nextOffset.get()) {
if (LOG.isDebugEnabled()) {
LOG.debug("get commit while still writing to the requested offset," LOG.debug("get commit while still writing to the requested offset,"
+ " with empty queue"); + " with empty queue");
}
return handleSpecialWait(fromRead, nextOffset.get(), channel, xid, return handleSpecialWait(fromRead, nextOffset.get(), channel, xid,
preOpAttr); preOpAttr);
} else { } else {
@ -920,18 +879,14 @@ synchronized COMMIT_STATUS checkCommitInternal(long commitOffset,
if (co <= flushed) { if (co <= flushed) {
return COMMIT_STATUS.COMMIT_DO_SYNC; return COMMIT_STATUS.COMMIT_DO_SYNC;
} else if (co < nextOffset.get()) { } else if (co < nextOffset.get()) {
if (LOG.isDebugEnabled()) {
LOG.debug("get commit while still writing to the requested offset"); LOG.debug("get commit while still writing to the requested offset");
}
return handleSpecialWait(fromRead, co, channel, xid, preOpAttr); return handleSpecialWait(fromRead, co, channel, xid, preOpAttr);
} else { } else {
// co >= nextOffset // co >= nextOffset
if (checkSequential(co, nextOffset.get())) { if (checkSequential(co, nextOffset.get())) {
return handleSpecialWait(fromRead, co, channel, xid, preOpAttr); return handleSpecialWait(fromRead, co, channel, xid, preOpAttr);
} else { } else {
if (LOG.isDebugEnabled()) {
LOG.debug("return COMMIT_SPECIAL_SUCCESS"); LOG.debug("return COMMIT_SPECIAL_SUCCESS");
}
return COMMIT_STATUS.COMMIT_SPECIAL_SUCCESS; return COMMIT_STATUS.COMMIT_SPECIAL_SUCCESS;
} }
} }
@ -993,8 +948,8 @@ public synchronized boolean streamCleanup(FileHandle handle,
// Check the stream timeout // Check the stream timeout
if (checkStreamTimeout(streamTimeout)) { if (checkStreamTimeout(streamTimeout)) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("stream can be closed for fileId: " LOG.debug("stream can be closed for fileId: {}",
+ handle.dumpFileHandle()); handle.dumpFileHandle());
} }
flag = true; flag = true;
} }
@ -1009,10 +964,8 @@ public synchronized boolean streamCleanup(FileHandle handle,
*/ */
private synchronized WriteCtx offerNextToWrite() { private synchronized WriteCtx offerNextToWrite() {
if (pendingWrites.isEmpty()) { if (pendingWrites.isEmpty()) {
if (LOG.isDebugEnabled()) { LOG.debug("The async write task has no pending writes, fileId: {}",
LOG.debug("The async write task has no pending writes, fileId: " latestAttr.getFileId());
+ latestAttr.getFileId());
}
// process pending commit again to handle this race: a commit is added // process pending commit again to handle this race: a commit is added
// to pendingCommits map just after the last doSingleWrite returns. // to pendingCommits map just after the last doSingleWrite returns.
// There is no pending write and the commit should be handled by the // There is no pending write and the commit should be handled by the
@ -1029,49 +982,35 @@ private synchronized WriteCtx offerNextToWrite() {
OffsetRange range = lastEntry.getKey(); OffsetRange range = lastEntry.getKey();
WriteCtx toWrite = lastEntry.getValue(); WriteCtx toWrite = lastEntry.getValue();
if (LOG.isTraceEnabled()) { LOG.trace("range.getMin()={} nextOffset={}",
LOG.trace("range.getMin()=" + range.getMin() + " nextOffset=" range.getMin(), nextOffset);
+ nextOffset);
}
long offset = nextOffset.get(); long offset = nextOffset.get();
if (range.getMin() > offset) { if (range.getMin() > offset) {
if (LOG.isDebugEnabled()) {
LOG.debug("The next sequential write has not arrived yet"); LOG.debug("The next sequential write has not arrived yet");
}
processCommits(nextOffset.get()); // handle race processCommits(nextOffset.get()); // handle race
this.asyncStatus = false; this.asyncStatus = false;
} else if (range.getMax() <= offset) { } else if (range.getMax() <= offset) {
if (LOG.isDebugEnabled()) { LOG.debug("Remove write {} which is already written from the list",
LOG.debug("Remove write " + range.toString() range);
+ " which is already written from the list");
}
// remove the WriteCtx from cache // remove the WriteCtx from cache
pendingWrites.remove(range); pendingWrites.remove(range);
} else if (range.getMin() < offset && range.getMax() > offset) { } else if (range.getMin() < offset && range.getMax() > offset) {
LOG.warn("Got an overlapping write " + range.toString() LOG.warn("Got an overlapping write {}, nextOffset={}. " +
+ ", nextOffset=" + offset "Remove and trim it", range, offset);
+ ". Remove and trim it");
pendingWrites.remove(range); pendingWrites.remove(range);
trimWriteRequest(toWrite, offset); trimWriteRequest(toWrite, offset);
// update nextOffset // update nextOffset
nextOffset.addAndGet(toWrite.getCount()); nextOffset.addAndGet(toWrite.getCount());
if (LOG.isDebugEnabled()) { LOG.debug("Change nextOffset (after trim) to {}", nextOffset.get());
LOG.debug("Change nextOffset (after trim) to " + nextOffset.get());
}
return toWrite; return toWrite;
} else { } else {
if (LOG.isDebugEnabled()) { LOG.debug("Remove write {} from the list", range);
LOG.debug("Remove write " + range.toString()
+ " from the list");
}
// after writing, remove the WriteCtx from cache // after writing, remove the WriteCtx from cache
pendingWrites.remove(range); pendingWrites.remove(range);
// update nextOffset // update nextOffset
nextOffset.addAndGet(toWrite.getCount()); nextOffset.addAndGet(toWrite.getCount());
if (LOG.isDebugEnabled()) { LOG.debug("Change nextOffset to {}", nextOffset.get());
LOG.debug("Change nextOffset to " + nextOffset.get());
}
return toWrite; return toWrite;
} }
return null; return null;
@ -1095,9 +1034,9 @@ void executeWriteBack() {
} }
} }
if (!activeState && LOG.isDebugEnabled()) { if (!activeState) {
LOG.debug("The openFileCtx is not active anymore, fileId: " LOG.debug("The openFileCtx is not active anymore, fileId: {}",
+ latestAttr.getFileId()); latestAttr.getFileId());
} }
} finally { } finally {
// Make sure to reset asyncStatus to false unless a race happens // Make sure to reset asyncStatus to false unless a race happens
@ -1105,11 +1044,12 @@ void executeWriteBack() {
if (startOffset == asyncWriteBackStartOffset) { if (startOffset == asyncWriteBackStartOffset) {
asyncStatus = false; asyncStatus = false;
} else { } else {
LOG.info("Another async task is already started before this one" LOG.info("Another async task is already started before this one " +
+ " is finalized. fileId: " + latestAttr.getFileId() "is finalized. fileId: {} asyncStatus: {} " +
+ " asyncStatus: " + asyncStatus + " original startOffset: " "original startOffset: {} " +
+ startOffset + " new startOffset: " + asyncWriteBackStartOffset "new startOffset: {}. Won't change asyncStatus here.",
+ ". Won't change asyncStatus here."); latestAttr.getFileId(), asyncStatus,
startOffset, asyncWriteBackStartOffset);
} }
} }
} }
@ -1132,8 +1072,8 @@ private void processCommits(long offset) {
status = Nfs3Status.NFS3_OK; status = Nfs3Status.NFS3_OK;
} catch (ClosedChannelException cce) { } catch (ClosedChannelException cce) {
if (!pendingWrites.isEmpty()) { if (!pendingWrites.isEmpty()) {
LOG.error("Can't sync for fileId: " + latestAttr.getFileId() LOG.error("Can't sync for fileId: {}. " +
+ ". Channel closed with writes pending.", cce); "Channel closed with writes pending", latestAttr.getFileId(), cce);
} }
status = Nfs3Status.NFS3ERR_IO; status = Nfs3Status.NFS3ERR_IO;
} catch (IOException e) { } catch (IOException e) {
@ -1152,8 +1092,8 @@ private void processCommits(long offset) {
} }
if (latestAttr.getSize() != offset) { if (latestAttr.getSize() != offset) {
LOG.error("After sync, the expect file size: " + offset LOG.error("After sync, the expect file size: {}, " +
+ ", however actual file size is: " + latestAttr.getSize()); "however actual file size is: {}", offset, latestAttr.getSize());
status = Nfs3Status.NFS3ERR_IO; status = Nfs3Status.NFS3ERR_IO;
} }
WccData wccData = new WccData(Nfs3Utils.getWccAttr(latestAttr), latestAttr); WccData wccData = new WccData(Nfs3Utils.getWccAttr(latestAttr), latestAttr);
@ -1172,9 +1112,9 @@ private void processCommits(long offset) {
new VerifierNone()), commit.getXid()); new VerifierNone()), commit.getXid());
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("FileId: " + latestAttr.getFileId() + " Service time: " LOG.debug("FileId: {} Service time: {}ns. " +
+ Nfs3Utils.getElapsedTime(commit.startTime) "Sent response for commit: {}", latestAttr.getFileId(),
+ "ns. Sent response for commit: " + commit); Nfs3Utils.getElapsedTime(commit.startTime), commit);
} }
entry = pendingCommits.firstEntry(); entry = pendingCommits.firstEntry();
} }
@ -1190,8 +1130,8 @@ private void doSingleWrite(final WriteCtx writeCtx) {
FileHandle handle = writeCtx.getHandle(); FileHandle handle = writeCtx.getHandle();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("do write, fileHandle " + handle.dumpFileHandle() + " offset: " LOG.debug("do write, fileHandle {} offset: {} length: {} stableHow: {}",
+ offset + " length: " + count + " stableHow: " + stableHow.name()); handle.dumpFileHandle(), offset, count, stableHow.name());
} }
try { try {
@ -1215,10 +1155,10 @@ private void doSingleWrite(final WriteCtx writeCtx) {
writeCtx.setDataState(WriteCtx.DataState.NO_DUMP); writeCtx.setDataState(WriteCtx.DataState.NO_DUMP);
updateNonSequentialWriteInMemory(-count); updateNonSequentialWriteInMemory(-count);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("After writing " + handle.dumpFileHandle() LOG.debug("After writing {} at offset {}, " +
+ " at offset " + offset "updated the memory count, new value: {}",
+ ", updated the memory count, new value: " handle.dumpFileHandle(), offset,
+ nonSequentialWriteInMemory.get()); nonSequentialWriteInMemory.get());
} }
} }
} }
@ -1226,7 +1166,7 @@ private void doSingleWrite(final WriteCtx writeCtx) {
if (!writeCtx.getReplied()) { if (!writeCtx.getReplied()) {
if (stableHow != WriteStableHow.UNSTABLE) { if (stableHow != WriteStableHow.UNSTABLE) {
LOG.info("Do sync for stable write: " + writeCtx); LOG.info("Do sync for stable write: {}", writeCtx);
try { try {
if (stableHow == WriteStableHow.DATA_SYNC) { if (stableHow == WriteStableHow.DATA_SYNC) {
fos.hsync(); fos.hsync();
@ -1237,7 +1177,7 @@ private void doSingleWrite(final WriteCtx writeCtx) {
fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)); fos.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));
} }
} catch (IOException e) { } catch (IOException e) {
LOG.error("hsync failed with writeCtx: " + writeCtx, e); LOG.error("hsync failed with writeCtx: {}", writeCtx, e);
throw e; throw e;
} }
} }
@ -1245,8 +1185,8 @@ private void doSingleWrite(final WriteCtx writeCtx) {
WccAttr preOpAttr = latestAttr.getWccAttr(); WccAttr preOpAttr = latestAttr.getWccAttr();
WccData fileWcc = new WccData(preOpAttr, latestAttr); WccData fileWcc = new WccData(preOpAttr, latestAttr);
if (writeCtx.getOriginalCount() != WriteCtx.INVALID_ORIGINAL_COUNT) { if (writeCtx.getOriginalCount() != WriteCtx.INVALID_ORIGINAL_COUNT) {
LOG.warn("Return original count: " + writeCtx.getOriginalCount() LOG.warn("Return original count: {} instead of real data count: {}",
+ " instead of real data count: " + count); writeCtx.getOriginalCount(), count);
count = writeCtx.getOriginalCount(); count = writeCtx.getOriginalCount();
} }
WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK, WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3_OK,
@ -1260,8 +1200,8 @@ private void doSingleWrite(final WriteCtx writeCtx) {
processCommits(writeCtx.getOffset() + writeCtx.getCount()); processCommits(writeCtx.getOffset() + writeCtx.getCount());
} catch (IOException e) { } catch (IOException e) {
LOG.error("Error writing to fileHandle " + handle.dumpFileHandle() LOG.error("Error writing to fileHandle {} at offset {} and length {}",
+ " at offset " + offset + " and length " + count, e); handle.dumpFileHandle(), offset, count, e);
if (!writeCtx.getReplied()) { if (!writeCtx.getReplied()) {
WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO); WRITE3Response response = new WRITE3Response(Nfs3Status.NFS3ERR_IO);
Nfs3Utils.writeChannel(channel, response.serialize( Nfs3Utils.writeChannel(channel, response.serialize(
@ -1269,8 +1209,8 @@ private void doSingleWrite(final WriteCtx writeCtx) {
// Keep stream open. Either client retries or SteamMonitor closes it. // Keep stream open. Either client retries or SteamMonitor closes it.
} }
LOG.info("Clean up open file context for fileId: " LOG.info("Clean up open file context for fileId: {}",
+ latestAttr.getFileId()); latestAttr.getFileId());
cleanup(); cleanup();
} }
} }
@ -1297,17 +1237,16 @@ synchronized void cleanup() {
fos.close(); fos.close();
} }
} catch (IOException e) { } catch (IOException e) {
LOG.info("Can't close stream for fileId: " + latestAttr.getFileId() LOG.info("Can't close stream for fileId: {}, error: {}",
+ ", error: " + e); latestAttr.getFileId(), e.toString());
} }
// Reply error for pending writes // Reply error for pending writes
LOG.info("There are " + pendingWrites.size() + " pending writes."); LOG.info("There are {} pending writes.", pendingWrites.size());
WccAttr preOpAttr = latestAttr.getWccAttr(); WccAttr preOpAttr = latestAttr.getWccAttr();
while (!pendingWrites.isEmpty()) { while (!pendingWrites.isEmpty()) {
OffsetRange key = pendingWrites.firstKey(); OffsetRange key = pendingWrites.firstKey();
LOG.info("Fail pending write: " + key.toString() LOG.info("Fail pending write: {}, nextOffset={}", key, nextOffset.get());
+ ", nextOffset=" + nextOffset.get());
WriteCtx writeCtx = pendingWrites.remove(key); WriteCtx writeCtx = pendingWrites.remove(key);
if (!writeCtx.getReplied()) { if (!writeCtx.getReplied()) {
@ -1325,11 +1264,12 @@ synchronized void cleanup() {
try { try {
dumpOut.close(); dumpOut.close();
} catch (IOException e) { } catch (IOException e) {
LOG.error("Failed to close outputstream of dump file" + dumpFilePath, e); LOG.error("Failed to close outputstream of dump file {}",
dumpFilePath, e);
} }
File dumpFile = new File(dumpFilePath); File dumpFile = new File(dumpFilePath);
if (dumpFile.exists() && !dumpFile.delete()) { if (dumpFile.exists() && !dumpFile.delete()) {
LOG.error("Failed to delete dumpfile: " + dumpFile); LOG.error("Failed to delete dumpfile: {}", dumpFile);
} }
} }
if (raf != null) { if (raf != null) {

View File

@ -28,8 +28,6 @@
import java.nio.charset.Charset; import java.nio.charset.Charset;
import java.util.EnumSet; import java.util.EnumSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException; import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
@ -137,6 +135,8 @@
import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.channel.ChannelHandlerContext;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* RPC program corresponding to nfs daemon. See {@link Nfs3}. * RPC program corresponding to nfs daemon. See {@link Nfs3}.
@ -146,7 +146,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
public static final FsPermission umask = new FsPermission( public static final FsPermission umask = new FsPermission(
(short) DEFAULT_UMASK); (short) DEFAULT_UMASK);
static final Log LOG = LogFactory.getLog(RpcProgramNfs3.class); static final Logger LOG = LoggerFactory.getLogger(RpcProgramNfs3.class);
private final NfsConfiguration config; private final NfsConfiguration config;
private final WriteManager writeManager; private final WriteManager writeManager;
@ -204,7 +204,7 @@ public RpcProgramNfs3(NfsConfiguration config, DatagramSocket registrationSocket
NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY); NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
superuser = config.get(NfsConfigKeys.NFS_SUPERUSER_KEY, superuser = config.get(NfsConfigKeys.NFS_SUPERUSER_KEY,
NfsConfigKeys.NFS_SUPERUSER_DEFAULT); NfsConfigKeys.NFS_SUPERUSER_DEFAULT);
LOG.info("Configured HDFS superuser is " + superuser); LOG.info("Configured HDFS superuser is {}", superuser);
if (!enableDump) { if (!enableDump) {
writeDumpDir = null; writeDumpDir = null;
@ -230,13 +230,13 @@ public static RpcProgramNfs3 createRpcProgramNfs3(NfsConfiguration config,
private void clearDirectory(String writeDumpDir) throws IOException { private void clearDirectory(String writeDumpDir) throws IOException {
File dumpDir = new File(writeDumpDir); File dumpDir = new File(writeDumpDir);
if (dumpDir.exists()) { if (dumpDir.exists()) {
LOG.info("Delete current dump directory " + writeDumpDir); LOG.info("Delete current dump directory {}", writeDumpDir);
if (!(FileUtil.fullyDelete(dumpDir))) { if (!(FileUtil.fullyDelete(dumpDir))) {
throw new IOException("Cannot remove current dump directory: " throw new IOException("Cannot remove current dump directory: "
+ dumpDir); + dumpDir);
} }
} }
LOG.info("Create new dump directory " + writeDumpDir); LOG.info("Create new dump directory {}", writeDumpDir);
if (!dumpDir.mkdirs()) { if (!dumpDir.mkdirs()) {
throw new IOException("Cannot create dump directory " + dumpDir); throw new IOException("Cannot create dump directory " + dumpDir);
} }
@ -298,9 +298,7 @@ private int mapErrorStatus(IOException e) {
@Override @Override
public NFS3Response nullProcedure() { public NFS3Response nullProcedure() {
if (LOG.isDebugEnabled()) {
LOG.debug("NFS NULL"); LOG.debug("NFS NULL");
}
return new NFS3Response(Nfs3Status.NFS3_OK); return new NFS3Response(Nfs3Status.NFS3_OK);
} }
@ -331,10 +329,9 @@ GETATTR3Response getattr(XDR xdr, SecurityHandler securityHandler,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("GETATTR for fileHandle: " + handle.dumpFileHandle() LOG.debug("GETATTR for fileHandle: {} client: {}",
+ " client: " + remoteAddress); handle.dumpFileHandle(), remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -346,7 +343,7 @@ GETATTR3Response getattr(XDR xdr, SecurityHandler securityHandler,
try { try {
attrs = writeManager.getFileAttr(dfsClient, handle, iug); attrs = writeManager.getFileAttr(dfsClient, handle, iug);
} catch (RemoteException r) { } catch (RemoteException r) {
LOG.warn("Exception ", r); LOG.warn("Exception", r);
IOException io = r.unwrapRemoteException(); IOException io = r.unwrapRemoteException();
/** /**
* AuthorizationException can be thrown if the user can't be proxy'ed. * AuthorizationException can be thrown if the user can't be proxy'ed.
@ -357,13 +354,13 @@ GETATTR3Response getattr(XDR xdr, SecurityHandler securityHandler,
return new GETATTR3Response(Nfs3Status.NFS3ERR_IO); return new GETATTR3Response(Nfs3Status.NFS3ERR_IO);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.info("Can't get file attribute, fileId=" + handle.getFileId(), e); LOG.info("Can't get file attribute, fileId={}", handle.getFileId(), e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
response.setStatus(status); response.setStatus(status);
return response; return response;
} }
if (attrs == null) { if (attrs == null) {
LOG.error("Can't get path for fileId: " + handle.getFileId()); LOG.error("Can't get path for fileId: {}", handle.getFileId());
response.setStatus(Nfs3Status.NFS3ERR_STALE); response.setStatus(Nfs3Status.NFS3ERR_STALE);
return response; return response;
} }
@ -378,9 +375,7 @@ private void setattrInternal(DFSClient dfsClient, String fileIdPath,
EnumSet<SetAttrField> updateFields = newAttr.getUpdateFields(); EnumSet<SetAttrField> updateFields = newAttr.getUpdateFields();
if (setMode && updateFields.contains(SetAttrField.MODE)) { if (setMode && updateFields.contains(SetAttrField.MODE)) {
if (LOG.isDebugEnabled()) { LOG.debug("set new mode: {}", newAttr.getMode());
LOG.debug("set new mode: " + newAttr.getMode());
}
dfsClient.setPermission(fileIdPath, dfsClient.setPermission(fileIdPath,
new FsPermission((short) (newAttr.getMode()))); new FsPermission((short) (newAttr.getMode())));
} }
@ -398,9 +393,7 @@ private void setattrInternal(DFSClient dfsClient, String fileIdPath,
long mtime = updateFields.contains(SetAttrField.MTIME) ? newAttr.getMtime() long mtime = updateFields.contains(SetAttrField.MTIME) ? newAttr.getMtime()
.getMilliSeconds() : -1; .getMilliSeconds() : -1;
if (atime != -1 || mtime != -1) { if (atime != -1 || mtime != -1) {
if (LOG.isDebugEnabled()) { LOG.debug("set atime: {} mtime: {}", atime, mtime);
LOG.debug("set atime: " + +atime + " mtime: " + mtime);
}
dfsClient.setTimes(fileIdPath, mtime, atime); dfsClient.setTimes(fileIdPath, mtime, atime);
} }
} }
@ -427,10 +420,9 @@ SETATTR3Response setattr(XDR xdr, SecurityHandler securityHandler,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS SETATTR fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS SETATTR fileHandle: {} client: {}",
+ " client: " + remoteAddress); handle.dumpFileHandle(), remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -439,8 +431,8 @@ SETATTR3Response setattr(XDR xdr, SecurityHandler securityHandler,
} }
if (request.getAttr().getUpdateFields().contains(SetAttrField.SIZE)) { if (request.getAttr().getUpdateFields().contains(SetAttrField.SIZE)) {
LOG.error("Setting file size is not supported when setattr, fileId: " LOG.error("Setting file size is not supported when setattr, fileId: {}",
+ handle.getFileId()); handle.getFileId());
response.setStatus(Nfs3Status.NFS3ERR_INVAL); response.setStatus(Nfs3Status.NFS3ERR_INVAL);
return response; return response;
} }
@ -450,7 +442,7 @@ SETATTR3Response setattr(XDR xdr, SecurityHandler securityHandler,
try { try {
preOpAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug); preOpAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
if (preOpAttr == null) { if (preOpAttr == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
response.setStatus(Nfs3Status.NFS3ERR_STALE); response.setStatus(Nfs3Status.NFS3ERR_STALE);
return response; return response;
} }
@ -474,13 +466,13 @@ SETATTR3Response setattr(XDR xdr, SecurityHandler securityHandler,
WccData wccData = new WccData(preOpWcc, postOpAttr); WccData wccData = new WccData(preOpWcc, postOpAttr);
return new SETATTR3Response(Nfs3Status.NFS3_OK, wccData); return new SETATTR3Response(Nfs3Status.NFS3_OK, wccData);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
WccData wccData = null; WccData wccData = null;
try { try {
wccData = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(preOpAttr), wccData = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(preOpAttr),
dfsClient, fileIdPath, iug); dfsClient, fileIdPath, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.info("Can't get postOpAttr for fileIdPath: " + fileIdPath, e1); LOG.info("Can't get postOpAttr for fileIdPath: {}", fileIdPath, e1);
} }
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
@ -515,10 +507,9 @@ LOOKUP3Response lookup(XDR xdr, SecurityHandler securityHandler,
String fileName = request.getName(); String fileName = request.getName();
int namenodeId = dirHandle.getNamenodeId(); int namenodeId = dirHandle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS LOOKUP dir fileHandle: " + dirHandle.dumpFileHandle() LOG.debug("NFS LOOKUP dir fileHandle: {} name: {} client: {}",
+ " name: " + fileName + " client: " + remoteAddress); dirHandle.dumpFileHandle(), fileName, remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -530,10 +521,8 @@ LOOKUP3Response lookup(XDR xdr, SecurityHandler securityHandler,
Nfs3FileAttributes postOpObjAttr = writeManager.getFileAttr(dfsClient, Nfs3FileAttributes postOpObjAttr = writeManager.getFileAttr(dfsClient,
dirHandle, fileName, namenodeId); dirHandle, fileName, namenodeId);
if (postOpObjAttr == null) { if (postOpObjAttr == null) {
if (LOG.isDebugEnabled()) { LOG.debug("NFS LOOKUP fileId: {} name: {} does not exist",
LOG.debug("NFS LOOKUP fileId: " + dirHandle.getFileId() + " name: " dirHandle.getFileId(), fileName);
+ fileName + " does not exist");
}
Nfs3FileAttributes postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, Nfs3FileAttributes postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient,
dirFileIdPath, iug); dirFileIdPath, iug);
return new LOOKUP3Response(Nfs3Status.NFS3ERR_NOENT, null, null, return new LOOKUP3Response(Nfs3Status.NFS3ERR_NOENT, null, null,
@ -543,7 +532,7 @@ LOOKUP3Response lookup(XDR xdr, SecurityHandler securityHandler,
Nfs3FileAttributes postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, Nfs3FileAttributes postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient,
dirFileIdPath, iug); dirFileIdPath, iug);
if (postOpDirAttr == null) { if (postOpDirAttr == null) {
LOG.info("Can't get path for dir fileId: " + dirHandle.getFileId()); LOG.info("Can't get path for dir fileId: {}", dirHandle.getFileId());
return new LOOKUP3Response(Nfs3Status.NFS3ERR_STALE); return new LOOKUP3Response(Nfs3Status.NFS3ERR_STALE);
} }
FileHandle fileHandle = FileHandle fileHandle =
@ -552,7 +541,7 @@ LOOKUP3Response lookup(XDR xdr, SecurityHandler securityHandler,
postOpDirAttr); postOpDirAttr);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new LOOKUP3Response(status); return new LOOKUP3Response(status);
} }
@ -592,16 +581,15 @@ ACCESS3Response access(XDR xdr, SecurityHandler securityHandler,
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS ACCESS fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS ACCESS fileHandle: {} client: {}",
+ " client: " + remoteAddress); handle.dumpFileHandle(), remoteAddress);
} }
Nfs3FileAttributes attrs; Nfs3FileAttributes attrs;
try { try {
attrs = writeManager.getFileAttr(dfsClient, handle, iug); attrs = writeManager.getFileAttr(dfsClient, handle, iug);
if (attrs == null) { if (attrs == null) {
LOG.error("Can't get path for fileId: " + handle.getFileId()); LOG.error("Can't get path for fileId: {}", handle.getFileId());
return new ACCESS3Response(Nfs3Status.NFS3ERR_STALE); return new ACCESS3Response(Nfs3Status.NFS3ERR_STALE);
} }
if(iug.getUserName(securityHandler.getUid(), "unknown").equals(superuser)) { if(iug.getUserName(securityHandler.getUid(), "unknown").equals(superuser)) {
@ -616,7 +604,7 @@ ACCESS3Response access(XDR xdr, SecurityHandler securityHandler,
return new ACCESS3Response(Nfs3Status.NFS3_OK, attrs, access); return new ACCESS3Response(Nfs3Status.NFS3_OK, attrs, access);
} catch (RemoteException r) { } catch (RemoteException r) {
LOG.warn("Exception ", r); LOG.warn("Exception", r);
IOException io = r.unwrapRemoteException(); IOException io = r.unwrapRemoteException();
/** /**
* AuthorizationException can be thrown if the user can't be proxy'ed. * AuthorizationException can be thrown if the user can't be proxy'ed.
@ -627,7 +615,7 @@ ACCESS3Response access(XDR xdr, SecurityHandler securityHandler,
return new ACCESS3Response(Nfs3Status.NFS3ERR_IO); return new ACCESS3Response(Nfs3Status.NFS3ERR_IO);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new ACCESS3Response(status); return new ACCESS3Response(status);
} }
@ -660,10 +648,9 @@ READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS READLINK fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS READLINK fileHandle: {} client: {}",
+ " client: " + remoteAddress); handle.dumpFileHandle(), remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -678,24 +665,23 @@ READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler,
Nfs3FileAttributes postOpAttr = Nfs3Utils.getFileAttr(dfsClient, Nfs3FileAttributes postOpAttr = Nfs3Utils.getFileAttr(dfsClient,
fileIdPath, iug); fileIdPath, iug);
if (postOpAttr == null) { if (postOpAttr == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new READLINK3Response(Nfs3Status.NFS3ERR_STALE); return new READLINK3Response(Nfs3Status.NFS3ERR_STALE);
} }
if (postOpAttr.getType() != NfsFileType.NFSLNK.toValue()) { if (postOpAttr.getType() != NfsFileType.NFSLNK.toValue()) {
LOG.error("Not a symlink, fileId: " + handle.getFileId()); LOG.error("Not a symlink, fileId: {}", handle.getFileId());
return new READLINK3Response(Nfs3Status.NFS3ERR_INVAL); return new READLINK3Response(Nfs3Status.NFS3ERR_INVAL);
} }
if (target == null) { if (target == null) {
LOG.error("Symlink target should not be null, fileId: " LOG.error("Symlink target should not be null, fileId: {}",
+ handle.getFileId()); handle.getFileId());
return new READLINK3Response(Nfs3Status.NFS3ERR_SERVERFAULT); return new READLINK3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
} }
int rtmax = config.getInt(NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY, int rtmax = config.getInt(NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY,
NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT); NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT);
if (rtmax < target.getBytes(Charset.forName("UTF-8")).length) { if (rtmax < target.getBytes(Charset.forName("UTF-8")).length) {
LOG.error("Link size: " LOG.error("Link size: {} is larger than max transfer size: {}",
+ target.getBytes(Charset.forName("UTF-8")).length target.getBytes(Charset.forName("UTF-8")).length, rtmax);
+ " is larger than max transfer size: " + rtmax);
return new READLINK3Response(Nfs3Status.NFS3ERR_IO, postOpAttr, return new READLINK3Response(Nfs3Status.NFS3ERR_IO, postOpAttr,
new byte[0]); new byte[0]);
} }
@ -704,7 +690,7 @@ READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler,
target.getBytes(Charset.forName("UTF-8"))); target.getBytes(Charset.forName("UTF-8")));
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Readlink error: " + e.getClass(), e); LOG.warn("Readlink error", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new READLINK3Response(status); return new READLINK3Response(status);
} }
@ -741,10 +727,9 @@ READ3Response read(XDR xdr, SecurityHandler securityHandler,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS READ fileHandle: " + handle.dumpFileHandle()+ " offset: " LOG.debug("NFS READ fileHandle: {} offset: {} count: {} client: {}",
+ offset + " count: " + count + " client: " + remoteAddress); handle.dumpFileHandle(), offset, count, remoteAddress);
} }
DFSClient dfsClient = clientCache.getDfsClient(userName, namenodeId); DFSClient dfsClient = clientCache.getDfsClient(userName, namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT); response.setStatus(Nfs3Status.NFS3ERR_SERVERFAULT);
@ -760,15 +745,12 @@ READ3Response read(XDR xdr, SecurityHandler securityHandler,
attrs = Nfs3Utils.getFileAttr(dfsClient, attrs = Nfs3Utils.getFileAttr(dfsClient,
Nfs3Utils.getFileIdPath(handle), iug); Nfs3Utils.getFileIdPath(handle), iug);
} catch (IOException e) { } catch (IOException e) {
if (LOG.isDebugEnabled()) { LOG.debug("Get error accessing file, fileId: {}",
LOG.debug("Get error accessing file, fileId: " + handle.getFileId(), e); handle.getFileId(), e);
}
return new READ3Response(Nfs3Status.NFS3ERR_IO); return new READ3Response(Nfs3Status.NFS3ERR_IO);
} }
if (attrs == null) { if (attrs == null) {
if (LOG.isDebugEnabled()) { LOG.debug("Can't get path for fileId: {}", handle.getFileId());
LOG.debug("Can't get path for fileId: " + handle.getFileId());
}
return new READ3Response(Nfs3Status.NFS3ERR_NOENT); return new READ3Response(Nfs3Status.NFS3ERR_NOENT);
} }
int access = Nfs3Utils.getAccessRightsForUserGroup( int access = Nfs3Utils.getAccessRightsForUserGroup(
@ -787,8 +769,8 @@ READ3Response read(XDR xdr, SecurityHandler securityHandler,
// optimized later by reading from the cache. // optimized later by reading from the cache.
int ret = writeManager.commitBeforeRead(dfsClient, handle, offset + count); int ret = writeManager.commitBeforeRead(dfsClient, handle, offset + count);
if (ret != Nfs3Status.NFS3_OK) { if (ret != Nfs3Status.NFS3_OK) {
LOG.warn("commitBeforeRead didn't succeed with ret=" + ret LOG.warn("commitBeforeRead didn't succeed with ret={}. " +
+ ". Read may not get most recent data."); "Read may not get most recent data.", ret);
} }
try { try {
@ -828,9 +810,8 @@ READ3Response read(XDR xdr, SecurityHandler securityHandler,
attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle), attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle),
iug); iug);
if (readCount < count) { if (readCount < count) {
LOG.info("Partial read. Asked offset: " + offset + " count: " + count LOG.info("Partial read. Asked offset: {} count: {} and read back: {} " +
+ " and read back: " + readCount + " file size: " "file size: {}", offset, count, readCount, attrs.getSize());
+ attrs.getSize());
} }
// HDFS returns -1 for read beyond file size. // HDFS returns -1 for read beyond file size.
if (readCount < 0) { if (readCount < 0) {
@ -841,8 +822,7 @@ READ3Response read(XDR xdr, SecurityHandler securityHandler,
ByteBuffer.wrap(readbuffer)); ByteBuffer.wrap(readbuffer));
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Read error: " + e.getClass() + " offset: " + offset LOG.warn("Read error. Offset: {} count: {}", offset, count, e);
+ " count: " + count, e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new READ3Response(status); return new READ3Response(status);
} }
@ -884,11 +864,11 @@ WRITE3Response write(XDR xdr, Channel channel, int xid,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS WRITE fileHandle: " + handle.dumpFileHandle() + " offset: " LOG.debug("NFS WRITE fileHandle: {} offset: {} length: {} " +
+ offset + " length: " + count + " stableHow: " + stableHow.getValue() "stableHow: {} xid: {} client: {}",
+ " xid: " + xid + " client: " + remoteAddress); handle.dumpFileHandle(), offset, count, stableHow.getValue(), xid,
remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -900,7 +880,7 @@ WRITE3Response write(XDR xdr, Channel channel, int xid,
try { try {
preOpAttr = writeManager.getFileAttr(dfsClient, handle, iug); preOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
if (preOpAttr == null) { if (preOpAttr == null) {
LOG.error("Can't get path for fileId: " + handle.getFileId()); LOG.error("Can't get path for fileId: {}", handle.getFileId());
return new WRITE3Response(Nfs3Status.NFS3ERR_STALE); return new WRITE3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -910,22 +890,20 @@ WRITE3Response write(XDR xdr, Channel channel, int xid,
Nfs3Constant.WRITE_COMMIT_VERF); Nfs3Constant.WRITE_COMMIT_VERF);
} }
if (LOG.isDebugEnabled()) { LOG.debug("requested offset={} and current filesize={}",
LOG.debug("requested offset=" + offset + " and current filesize=" offset, preOpAttr.getSize());
+ preOpAttr.getSize());
}
writeManager.handleWrite(dfsClient, request, channel, xid, preOpAttr); writeManager.handleWrite(dfsClient, request, channel, xid, preOpAttr);
} catch (IOException e) { } catch (IOException e) {
LOG.info("Error writing to fileId " + handle.getFileId() + " at offset " LOG.info("Error writing to fileId {} at offset {} and length {}",
+ offset + " and length " + data.length, e); handle.getFileId(), offset, data.length, e);
// Try to return WccData // Try to return WccData
Nfs3FileAttributes postOpAttr = null; Nfs3FileAttributes postOpAttr = null;
try { try {
postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug); postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId(), e1); LOG.info("Can't get postOpAttr for fileId: {}", e1);
} }
WccAttr attr = preOpAttr == null ? null : Nfs3Utils.getWccAttr(preOpAttr); WccAttr attr = preOpAttr == null ? null : Nfs3Utils.getWccAttr(preOpAttr);
WccData fileWcc = new WccData(attr, postOpAttr); WccData fileWcc = new WccData(attr, postOpAttr);
@ -961,10 +939,9 @@ CREATE3Response create(XDR xdr, SecurityHandler securityHandler,
String fileName = request.getName(); String fileName = request.getName();
int namenodeId = dirHandle.getNamenodeId(); int namenodeId = dirHandle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS CREATE dir fileHandle: " + dirHandle.dumpFileHandle() LOG.debug("NFS CREATE dir fileHandle: {} filename: {} client: {}",
+ " filename: " + fileName + " client: " + remoteAddress); dirHandle.dumpFileHandle(), fileName, remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -976,8 +953,8 @@ CREATE3Response create(XDR xdr, SecurityHandler securityHandler,
if ((createMode != Nfs3Constant.CREATE_EXCLUSIVE) if ((createMode != Nfs3Constant.CREATE_EXCLUSIVE)
&& request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE) && request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE)
&& request.getObjAttr().getSize() != 0) { && request.getObjAttr().getSize() != 0) {
LOG.error("Setting file size is not supported when creating file: " LOG.error("Setting file size is not supported when creating file: {} " +
+ fileName + " dir fileId: " + dirHandle.getFileId()); "dir fileId: {}", fileName, dirHandle.getFileId());
return new CREATE3Response(Nfs3Status.NFS3ERR_INVAL); return new CREATE3Response(Nfs3Status.NFS3ERR_INVAL);
} }
@ -990,7 +967,7 @@ CREATE3Response create(XDR xdr, SecurityHandler securityHandler,
try { try {
preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
if (preOpDirAttr == null) { if (preOpDirAttr == null) {
LOG.error("Can't get path for dirHandle: " + dirHandle); LOG.error("Can't get path for dirHandle: {}", dirHandle);
return new CREATE3Response(Nfs3Status.NFS3ERR_STALE); return new CREATE3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -1041,10 +1018,8 @@ preOpDirAttr, new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
fos.close(); fos.close();
fos = null; fos = null;
} else { } else {
if (LOG.isDebugEnabled()) { LOG.debug("Opened stream for file: {}, fileId: {}",
LOG.debug("Opened stream for file: " + fileName + ", fileId: " fileName, fileHandle.getFileId());
+ fileHandle.getFileId());
}
} }
} catch (IOException e) { } catch (IOException e) {
@ -1053,8 +1028,8 @@ preOpDirAttr, new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
try { try {
fos.close(); fos.close();
} catch (IOException e1) { } catch (IOException e1) {
LOG.error("Can't close stream for dirFileId: " + dirHandle.getFileId() LOG.error("Can't close stream for dirFileId: {} filename: {}",
+ " filename: " + fileName, e1); dirHandle.getFileId(), fileName, e1);
} }
} }
if (dirWcc == null) { if (dirWcc == null) {
@ -1062,8 +1037,8 @@ preOpDirAttr, new WccData(Nfs3Utils.getWccAttr(preOpDirAttr),
dirWcc = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(preOpDirAttr), dirWcc = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(preOpDirAttr),
dfsClient, dirFileIdPath, iug); dfsClient, dirFileIdPath, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.error("Can't get postOpDirAttr for dirFileId: " LOG.error("Can't get postOpDirAttr for dirFileId: {}",
+ dirHandle.getFileId(), e1); dirHandle.getFileId(), e1);
} }
} }
@ -1105,13 +1080,12 @@ MKDIR3Response mkdir(XDR xdr, SecurityHandler securityHandler,
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS MKDIR dirHandle: " + dirHandle.dumpFileHandle() LOG.debug("NFS MKDIR dirHandle: {} filename: {} client: {}",
+ " filename: " + fileName + " client: " + remoteAddress); dirHandle.dumpFileHandle(), fileName, remoteAddress);
} }
if (request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE)) { if (request.getObjAttr().getUpdateFields().contains(SetAttrField.SIZE)) {
LOG.error("Setting file size is not supported when mkdir: " + fileName LOG.error("Setting file size is not supported when mkdir: " +
+ " in dirHandle" + dirHandle); "{} in dirHandle {}", fileName, dirHandle);
return new MKDIR3Response(Nfs3Status.NFS3ERR_INVAL); return new MKDIR3Response(Nfs3Status.NFS3ERR_INVAL);
} }
@ -1123,7 +1097,7 @@ MKDIR3Response mkdir(XDR xdr, SecurityHandler securityHandler,
try { try {
preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
if (preOpDirAttr == null) { if (preOpDirAttr == null) {
LOG.info("Can't get path for dir fileId: " + dirHandle.getFileId()); LOG.info("Can't get path for dir fileId: {}", dirHandle.getFileId());
return new MKDIR3Response(Nfs3Status.NFS3ERR_STALE); return new MKDIR3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -1158,13 +1132,13 @@ MKDIR3Response mkdir(XDR xdr, SecurityHandler securityHandler,
return new MKDIR3Response(Nfs3Status.NFS3_OK, new FileHandle( return new MKDIR3Response(Nfs3Status.NFS3_OK, new FileHandle(
postOpObjAttr.getFileId(), namenodeId), postOpObjAttr, dirWcc); postOpObjAttr.getFileId(), namenodeId), postOpObjAttr, dirWcc);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
// Try to return correct WccData // Try to return correct WccData
if (postOpDirAttr == null) { if (postOpDirAttr == null) {
try { try {
postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e); LOG.info("Can't get postOpDirAttr for {}", dirFileIdPath, e);
} }
} }
@ -1202,10 +1176,9 @@ REMOVE3Response remove(XDR xdr, SecurityHandler securityHandler,
String fileName = request.getName(); String fileName = request.getName();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS REMOVE dir fileHandle: " + dirHandle.dumpFileHandle() LOG.debug("NFS REMOVE dir fileHandle: {} fileName: {} client: {}",
+ " fileName: " + fileName + " client: " + remoteAddress); dirHandle.dumpFileHandle(), fileName, remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -1219,7 +1192,7 @@ REMOVE3Response remove(XDR xdr, SecurityHandler securityHandler,
try { try {
preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
if (preOpDirAttr == null) { if (preOpDirAttr == null) {
LOG.info("Can't get path for dir fileId: " + dirHandle.getFileId()); LOG.info("Can't get path for dir fileId: {}", dirHandle.getFileId());
return new REMOVE3Response(Nfs3Status.NFS3ERR_STALE); return new REMOVE3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -1247,13 +1220,13 @@ REMOVE3Response remove(XDR xdr, SecurityHandler securityHandler,
} }
return new REMOVE3Response(Nfs3Status.NFS3_OK, dirWcc); return new REMOVE3Response(Nfs3Status.NFS3_OK, dirWcc);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
// Try to return correct WccData // Try to return correct WccData
if (postOpDirAttr == null) { if (postOpDirAttr == null) {
try { try {
postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e1); LOG.info("Can't get postOpDirAttr for {}", dirFileIdPath, e1);
} }
} }
@ -1285,10 +1258,9 @@ RMDIR3Response rmdir(XDR xdr, SecurityHandler securityHandler,
String fileName = request.getName(); String fileName = request.getName();
int namenodeId = dirHandle.getNamenodeId(); int namenodeId = dirHandle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS RMDIR dir fileHandle: " + dirHandle.dumpFileHandle() LOG.debug("NFS RMDIR dir fileHandle: {} fileName: {} client: {}",
+ " fileName: " + fileName + " client: " + remoteAddress); dirHandle.dumpFileHandle(), fileName, remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -1302,7 +1274,7 @@ RMDIR3Response rmdir(XDR xdr, SecurityHandler securityHandler,
try { try {
preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); preOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
if (preOpDirAttr == null) { if (preOpDirAttr == null) {
LOG.info("Can't get path for dir fileId: " + dirHandle.getFileId()); LOG.info("Can't get path for dir fileId: {}", dirHandle.getFileId());
return new RMDIR3Response(Nfs3Status.NFS3ERR_STALE); return new RMDIR3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -1334,13 +1306,13 @@ RMDIR3Response rmdir(XDR xdr, SecurityHandler securityHandler,
return new RMDIR3Response(Nfs3Status.NFS3_OK, dirWcc); return new RMDIR3Response(Nfs3Status.NFS3_OK, dirWcc);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
// Try to return correct WccData // Try to return correct WccData
if (postOpDirAttr == null) { if (postOpDirAttr == null) {
try { try {
postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.info("Can't get postOpDirAttr for " + dirFileIdPath, e1); LOG.info("Can't get postOpDirAttr for {}", dirFileIdPath, e1);
} }
} }
@ -1376,11 +1348,10 @@ RENAME3Response rename(XDR xdr, SecurityHandler securityHandler,
int toNamenodeId = toHandle.getNamenodeId(); int toNamenodeId = toHandle.getNamenodeId();
String toName = request.getToName(); String toName = request.getToName();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS RENAME from: " + fromHandle.dumpFileHandle() LOG.debug("NFS RENAME from: {}/{} to: {}/{} client: {}",
+ "/" + fromName + " to: " + toHandle.dumpFileHandle() fromHandle.dumpFileHandle(), fromName, toHandle.dumpFileHandle(),
+ "/" + toName + " client: " + remoteAddress); toName, remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), fromNamenodeId); clientCache.getDfsClient(securityHandler.getUser(), fromNamenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -1403,14 +1374,15 @@ RENAME3Response rename(XDR xdr, SecurityHandler securityHandler,
try { try {
fromPreOpAttr = Nfs3Utils.getFileAttr(dfsClient, fromDirFileIdPath, iug); fromPreOpAttr = Nfs3Utils.getFileAttr(dfsClient, fromDirFileIdPath, iug);
if (fromPreOpAttr == null) { if (fromPreOpAttr == null) {
LOG.info("Can't get path for fromHandle fileId: " LOG.info("Can't get path for fromHandle fileId: {}",
+ fromHandle.getFileId()); fromHandle.getFileId());
return new RENAME3Response(Nfs3Status.NFS3ERR_STALE); return new RENAME3Response(Nfs3Status.NFS3ERR_STALE);
} }
toPreOpAttr = Nfs3Utils.getFileAttr(dfsClient, toDirFileIdPath, iug); toPreOpAttr = Nfs3Utils.getFileAttr(dfsClient, toDirFileIdPath, iug);
if (toPreOpAttr == null) { if (toPreOpAttr == null) {
LOG.info("Can't get path for toHandle fileId: " + toHandle.getFileId()); LOG.info("Can't get path for toHandle fileId: {}",
toHandle.getFileId());
return new RENAME3Response(Nfs3Status.NFS3ERR_STALE); return new RENAME3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -1434,7 +1406,7 @@ RENAME3Response rename(XDR xdr, SecurityHandler securityHandler,
dfsClient, toDirFileIdPath, iug); dfsClient, toDirFileIdPath, iug);
return new RENAME3Response(Nfs3Status.NFS3_OK, fromDirWcc, toDirWcc); return new RENAME3Response(Nfs3Status.NFS3_OK, fromDirWcc, toDirWcc);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
// Try to return correct WccData // Try to return correct WccData
try { try {
fromDirWcc = Nfs3Utils.createWccData( fromDirWcc = Nfs3Utils.createWccData(
@ -1443,8 +1415,8 @@ RENAME3Response rename(XDR xdr, SecurityHandler securityHandler,
toDirWcc = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(toPreOpAttr), toDirWcc = Nfs3Utils.createWccData(Nfs3Utils.getWccAttr(toPreOpAttr),
dfsClient, toDirFileIdPath, iug); dfsClient, toDirFileIdPath, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.info("Can't get postOpDirAttr for " + fromDirFileIdPath + " or" LOG.info("Can't get postOpDirAttr for {} or {}",
+ toDirFileIdPath, e1); fromDirFileIdPath, toDirFileIdPath, e1);
} }
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
@ -1484,10 +1456,8 @@ SYMLINK3Response symlink(XDR xdr, SecurityHandler securityHandler,
// Don't do any name check to source path, just leave it to HDFS // Don't do any name check to source path, just leave it to HDFS
String linkIdPath = linkDirIdPath + "/" + name; String linkIdPath = linkDirIdPath + "/" + name;
if (LOG.isDebugEnabled()) { LOG.debug("NFS SYMLINK, target: {} link: {} namenodeId: {} client: {}",
LOG.debug("NFS SYMLINK, target: " + symData + " link: " + linkIdPath symData, linkIdPath, namenodeId, remoteAddress);
+ " namenodeId: " + namenodeId + " client: " + remoteAddress);
}
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
@ -1515,7 +1485,7 @@ SYMLINK3Response symlink(XDR xdr, SecurityHandler securityHandler,
objAttr.getFileId(), namenodeId), objAttr, dirWcc); objAttr.getFileId(), namenodeId), objAttr, dirWcc);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception: " + e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
response.setStatus(status); response.setStatus(status);
return response; return response;
@ -1542,9 +1512,8 @@ private DirectoryListing listPaths(DFSClient dfsClient, String dirFileIdPath,
throw io; throw io;
} }
// This happens when startAfter was just deleted // This happens when startAfter was just deleted
LOG.info("Cookie couldn't be found: " LOG.info("Cookie couldn't be found: {}, do listing from beginning",
+ new String(startAfter, Charset.forName("UTF-8")) new String(startAfter, Charset.forName("UTF-8")));
+ ", do listing from beginning");
dlisting = dfsClient dlisting = dfsClient
.listPaths(dirFileIdPath, HdfsFileStatus.EMPTY_NAME); .listPaths(dirFileIdPath, HdfsFileStatus.EMPTY_NAME);
} }
@ -1577,21 +1546,19 @@ public READDIR3Response readdir(XDR xdr, SecurityHandler securityHandler,
long cookie = request.getCookie(); long cookie = request.getCookie();
if (cookie < 0) { if (cookie < 0) {
LOG.error("Invalid READDIR request, with negative cookie: " + cookie); LOG.error("Invalid READDIR request, with negative cookie: {}", cookie);
return new READDIR3Response(Nfs3Status.NFS3ERR_INVAL); return new READDIR3Response(Nfs3Status.NFS3ERR_INVAL);
} }
long count = request.getCount(); long count = request.getCount();
if (count <= 0) { if (count <= 0) {
LOG.info("Nonpositive count in invalid READDIR request: " + count); LOG.info("Nonpositive count in invalid READDIR request: {}", count);
return new READDIR3Response(Nfs3Status.NFS3_OK); return new READDIR3Response(Nfs3Status.NFS3_OK);
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS READDIR fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS READDIR fileHandle: {} cookie: {} count: {} client: {}",
+ " cookie: " + cookie + " count: " + count + " client: " handle.dumpFileHandle(), cookie, count, remoteAddress);
+ remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -1607,12 +1574,12 @@ public READDIR3Response readdir(XDR xdr, SecurityHandler securityHandler,
String dirFileIdPath = Nfs3Utils.getFileIdPath(handle); String dirFileIdPath = Nfs3Utils.getFileIdPath(handle);
dirStatus = dfsClient.getFileInfo(dirFileIdPath); dirStatus = dfsClient.getFileInfo(dirFileIdPath);
if (dirStatus == null) { if (dirStatus == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new READDIR3Response(Nfs3Status.NFS3ERR_STALE); return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
} }
if (!dirStatus.isDirectory()) { if (!dirStatus.isDirectory()) {
LOG.error("Can't readdir for regular file, fileId: " LOG.error("Can't readdir for regular file, fileId: {}",
+ handle.getFileId()); handle.getFileId());
return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR); return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR);
} }
long cookieVerf = request.getCookieVerf(); long cookieVerf = request.getCookieVerf();
@ -1631,8 +1598,9 @@ public READDIR3Response readdir(XDR xdr, SecurityHandler securityHandler,
LOG.warn("AIX compatibility mode enabled, ignoring cookieverf " + LOG.warn("AIX compatibility mode enabled, ignoring cookieverf " +
"mismatches."); "mismatches.");
} else { } else {
LOG.error("CookieVerf mismatch. request cookieVerf: " + cookieVerf LOG.error("CookieVerf mismatch. request cookieVerf: {} " +
+ " dir cookieVerf: " + dirStatus.getModificationTime()); "dir cookieVerf: {}",
cookieVerf, dirStatus.getModificationTime());
return new READDIR3Response( return new READDIR3Response(
Nfs3Status.NFS3ERR_BAD_COOKIE, Nfs3Status.NFS3ERR_BAD_COOKIE,
Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug)); Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug));
@ -1664,11 +1632,11 @@ public READDIR3Response readdir(XDR xdr, SecurityHandler securityHandler,
dlisting = listPaths(dfsClient, dirFileIdPath, startAfter); dlisting = listPaths(dfsClient, dirFileIdPath, startAfter);
postOpAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); postOpAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
if (postOpAttr == null) { if (postOpAttr == null) {
LOG.error("Can't get path for fileId: " + handle.getFileId()); LOG.error("Can't get path for fileId: {}", handle.getFileId());
return new READDIR3Response(Nfs3Status.NFS3ERR_STALE); return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new READDIR3Response(status); return new READDIR3Response(status);
} }
@ -1742,26 +1710,28 @@ READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler,
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
long cookie = request.getCookie(); long cookie = request.getCookie();
if (cookie < 0) { if (cookie < 0) {
LOG.error("Invalid READDIRPLUS request, with negative cookie: " + cookie); LOG.error("Invalid READDIRPLUS request, with negative cookie: {}",
cookie);
return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL); return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL);
} }
long dirCount = request.getDirCount(); long dirCount = request.getDirCount();
if (dirCount <= 0) { if (dirCount <= 0) {
LOG.info("Nonpositive dircount in invalid READDIRPLUS request: " + dirCount); LOG.info("Nonpositive dircount in invalid READDIRPLUS request: {}",
dirCount);
return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL); return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL);
} }
int maxCount = request.getMaxCount(); int maxCount = request.getMaxCount();
if (maxCount <= 0) { if (maxCount <= 0) {
LOG.info("Nonpositive maxcount in invalid READDIRPLUS request: " + maxCount); LOG.info("Nonpositive maxcount in invalid READDIRPLUS request: {}",
maxCount);
return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL); return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_INVAL);
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS READDIRPLUS fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS READDIRPLUS fileHandle: {} cookie: {} dirCount: {} " +
+ " cookie: " + cookie + " dirCount: " + dirCount + " maxCount: " "maxCount: {} client: {}",
+ maxCount + " client: " + remoteAddress); handle.dumpFileHandle(), cookie, dirCount, maxCount, remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -1777,12 +1747,12 @@ READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler,
String dirFileIdPath = Nfs3Utils.getFileIdPath(handle); String dirFileIdPath = Nfs3Utils.getFileIdPath(handle);
dirStatus = dfsClient.getFileInfo(dirFileIdPath); dirStatus = dfsClient.getFileInfo(dirFileIdPath);
if (dirStatus == null) { if (dirStatus == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE); return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
} }
if (!dirStatus.isDirectory()) { if (!dirStatus.isDirectory()) {
LOG.error("Can't readdirplus for regular file, fileId: " LOG.error("Can't readdirplus for regular file, fileId: {}",
+ handle.getFileId()); handle.getFileId());
return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR); return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR);
} }
long cookieVerf = request.getCookieVerf(); long cookieVerf = request.getCookieVerf();
@ -1799,8 +1769,9 @@ READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler,
LOG.warn("AIX compatibility mode enabled, ignoring cookieverf " + LOG.warn("AIX compatibility mode enabled, ignoring cookieverf " +
"mismatches."); "mismatches.");
} else { } else {
LOG.error("cookieverf mismatch. request cookieverf: " + cookieVerf LOG.error("cookieverf mismatch. request cookieverf: {} " +
+ " dir cookieverf: " + dirStatus.getModificationTime()); "dir cookieverf: {}",
cookieVerf, dirStatus.getModificationTime());
return new READDIRPLUS3Response( return new READDIRPLUS3Response(
Nfs3Status.NFS3ERR_BAD_COOKIE, Nfs3Status.NFS3ERR_BAD_COOKIE,
Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug), Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug),
@ -1833,11 +1804,11 @@ READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler,
dlisting = listPaths(dfsClient, dirFileIdPath, startAfter); dlisting = listPaths(dfsClient, dirFileIdPath, startAfter);
postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug); postOpDirAttr = Nfs3Utils.getFileAttr(dfsClient, dirFileIdPath, iug);
if (postOpDirAttr == null) { if (postOpDirAttr == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE); return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new READDIRPLUS3Response(status); return new READDIRPLUS3Response(status);
} }
@ -1865,7 +1836,7 @@ READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler,
try { try {
attr = writeManager.getFileAttr(dfsClient, childHandle, iug); attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
} catch (IOException e) { } catch (IOException e) {
LOG.error("Can't get file attributes for fileId: " + fileId, e); LOG.error("Can't get file attributes for fileId: {}", fileId, e);
continue; continue;
} }
entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId, entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
@ -1882,7 +1853,7 @@ READDIRPLUS3Response readdirplus(XDR xdr, SecurityHandler securityHandler,
try { try {
attr = writeManager.getFileAttr(dfsClient, childHandle, iug); attr = writeManager.getFileAttr(dfsClient, childHandle, iug);
} catch (IOException e) { } catch (IOException e) {
LOG.error("Can't get file attributes for fileId: " + fileId, e); LOG.error("Can't get file attributes for fileId: {}", fileId, e);
continue; continue;
} }
entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId, entries[i] = new READDIRPLUS3Response.EntryPlus3(fileId,
@ -1923,10 +1894,9 @@ FSSTAT3Response fsstat(XDR xdr, SecurityHandler securityHandler,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS FSSTAT fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS FSSTAT fileHandle: {} client: {}",
+ " client: " + remoteAddress); handle.dumpFileHandle(), remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -1942,7 +1912,7 @@ FSSTAT3Response fsstat(XDR xdr, SecurityHandler securityHandler,
Nfs3FileAttributes attrs = writeManager.getFileAttr(dfsClient, handle, Nfs3FileAttributes attrs = writeManager.getFileAttr(dfsClient, handle,
iug); iug);
if (attrs == null) { if (attrs == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new FSSTAT3Response(Nfs3Status.NFS3ERR_STALE); return new FSSTAT3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -1957,7 +1927,7 @@ FSSTAT3Response fsstat(XDR xdr, SecurityHandler securityHandler,
return new FSSTAT3Response(Nfs3Status.NFS3_OK, attrs, totalBytes, return new FSSTAT3Response(Nfs3Status.NFS3_OK, attrs, totalBytes,
freeBytes, freeBytes, maxFsObjects, maxFsObjects, maxFsObjects, 0); freeBytes, freeBytes, maxFsObjects, maxFsObjects, maxFsObjects, 0);
} catch (RemoteException r) { } catch (RemoteException r) {
LOG.warn("Exception ", r); LOG.warn("Exception", r);
IOException io = r.unwrapRemoteException(); IOException io = r.unwrapRemoteException();
/** /**
* AuthorizationException can be thrown if the user can't be proxy'ed. * AuthorizationException can be thrown if the user can't be proxy'ed.
@ -1968,7 +1938,7 @@ FSSTAT3Response fsstat(XDR xdr, SecurityHandler securityHandler,
return new FSSTAT3Response(Nfs3Status.NFS3ERR_IO); return new FSSTAT3Response(Nfs3Status.NFS3ERR_IO);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new FSSTAT3Response(status); return new FSSTAT3Response(status);
} }
@ -2000,10 +1970,9 @@ FSINFO3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS FSINFO fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS FSINFO fileHandle: {} client: {}", remoteAddress,
+" client: " + remoteAddress); handle.dumpFileHandle(), remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -2025,7 +1994,7 @@ FSINFO3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
Nfs3FileAttributes attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3FileAttributes attrs = Nfs3Utils.getFileAttr(dfsClient,
Nfs3Utils.getFileIdPath(handle), iug); Nfs3Utils.getFileIdPath(handle), iug);
if (attrs == null) { if (attrs == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new FSINFO3Response(Nfs3Status.NFS3ERR_STALE); return new FSINFO3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -2035,7 +2004,7 @@ FSINFO3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
return new FSINFO3Response(Nfs3Status.NFS3_OK, attrs, rtmax, rtmax, 1, return new FSINFO3Response(Nfs3Status.NFS3_OK, attrs, rtmax, rtmax, 1,
wtmax, wtmax, 1, dtperf, Long.MAX_VALUE, new NfsTime(1), fsProperty); wtmax, wtmax, 1, dtperf, Long.MAX_VALUE, new NfsTime(1), fsProperty);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new FSINFO3Response(status); return new FSINFO3Response(status);
} }
@ -2069,10 +2038,9 @@ PATHCONF3Response pathconf(XDR xdr, SecurityHandler securityHandler,
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS PATHCONF fileHandle: " + handle.dumpFileHandle() LOG.debug("NFS PATHCONF fileHandle: {} client: {}",
+ " client: " + remoteAddress); handle.dumpFileHandle(), remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -2084,14 +2052,14 @@ PATHCONF3Response pathconf(XDR xdr, SecurityHandler securityHandler,
attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle), attrs = Nfs3Utils.getFileAttr(dfsClient, Nfs3Utils.getFileIdPath(handle),
iug); iug);
if (attrs == null) { if (attrs == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new PATHCONF3Response(Nfs3Status.NFS3ERR_STALE); return new PATHCONF3Response(Nfs3Status.NFS3ERR_STALE);
} }
return new PATHCONF3Response(Nfs3Status.NFS3_OK, attrs, 0, return new PATHCONF3Response(Nfs3Status.NFS3_OK, attrs, 0,
HdfsServerConstants.MAX_PATH_LENGTH, true, false, false, true); HdfsServerConstants.MAX_PATH_LENGTH, true, false, false, true);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
int status = mapErrorStatus(e); int status = mapErrorStatus(e);
return new PATHCONF3Response(status); return new PATHCONF3Response(status);
} }
@ -2123,11 +2091,10 @@ COMMIT3Response commit(XDR xdr, Channel channel, int xid,
FileHandle handle = request.getHandle(); FileHandle handle = request.getHandle();
int namenodeId = handle.getNamenodeId(); int namenodeId = handle.getNamenodeId();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("NFS COMMIT fileHandle: " + handle.dumpFileHandle() + " offset=" LOG.debug("NFS COMMIT fileHandle: {} offset={} count={} client: {}",
+ request.getOffset() + " count=" + request.getCount() + " client: " handle.dumpFileHandle(), request.getOffset(), request.getCount(),
+ remoteAddress); remoteAddress);
} }
DFSClient dfsClient = DFSClient dfsClient =
clientCache.getDfsClient(securityHandler.getUser(), namenodeId); clientCache.getDfsClient(securityHandler.getUser(), namenodeId);
if (dfsClient == null) { if (dfsClient == null) {
@ -2140,7 +2107,7 @@ COMMIT3Response commit(XDR xdr, Channel channel, int xid,
try { try {
preOpAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug); preOpAttr = Nfs3Utils.getFileAttr(dfsClient, fileIdPath, iug);
if (preOpAttr == null) { if (preOpAttr == null) {
LOG.info("Can't get path for fileId: " + handle.getFileId()); LOG.info("Can't get path for fileId: {}", handle.getFileId());
return new COMMIT3Response(Nfs3Status.NFS3ERR_STALE); return new COMMIT3Response(Nfs3Status.NFS3ERR_STALE);
} }
@ -2158,12 +2125,12 @@ COMMIT3Response commit(XDR xdr, Channel channel, int xid,
preOpAttr, namenodeId); preOpAttr, namenodeId);
return null; return null;
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception ", e); LOG.warn("Exception", e);
Nfs3FileAttributes postOpAttr = null; Nfs3FileAttributes postOpAttr = null;
try { try {
postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug); postOpAttr = writeManager.getFileAttr(dfsClient, handle, iug);
} catch (IOException e1) { } catch (IOException e1) {
LOG.info("Can't get postOpAttr for fileId: " + handle.getFileId(), e1); LOG.info("Can't get postOpAttr for fileId: {}", handle.getFileId(), e1);
} }
WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr); WccData fileWcc = new WccData(Nfs3Utils.getWccAttr(preOpAttr), postOpAttr);
@ -2205,8 +2172,8 @@ public void handleInternal(ChannelHandlerContext ctx, RpcInfo info) {
if (nfsproc3 != NFSPROC3.NULL) { if (nfsproc3 != NFSPROC3.NULL) {
if (credentials.getFlavor() != AuthFlavor.AUTH_SYS if (credentials.getFlavor() != AuthFlavor.AUTH_SYS
&& credentials.getFlavor() != AuthFlavor.RPCSEC_GSS) { && credentials.getFlavor() != AuthFlavor.RPCSEC_GSS) {
LOG.info("Wrong RPC AUTH flavor, " + credentials.getFlavor() LOG.info("Wrong RPC AUTH flavor, {} is not AUTH_SYS or RPCSEC_GSS.",
+ " is not AUTH_SYS or RPCSEC_GSS."); credentials.getFlavor());
XDR reply = new XDR(); XDR reply = new XDR();
RpcDeniedReply rdr = new RpcDeniedReply(xid, RpcDeniedReply rdr = new RpcDeniedReply(xid,
RpcReply.ReplyState.MSG_ACCEPTED, RpcReply.ReplyState.MSG_ACCEPTED,
@ -2226,12 +2193,13 @@ public void handleInternal(ChannelHandlerContext ctx, RpcInfo info) {
xid); xid);
if (entry != null) { // in cache if (entry != null) { // in cache
if (entry.isCompleted()) { if (entry.isCompleted()) {
LOG.info("Sending the cached reply to retransmitted request " + xid); LOG.info("Sending the cached reply to retransmitted request {}",
xid);
RpcUtil.sendRpcResponse(ctx, entry.getResponse()); RpcUtil.sendRpcResponse(ctx, entry.getResponse());
return; return;
} else { // else request is in progress } else { // else request is in progress
LOG.info("Retransmitted request, transaction still in progress " LOG.info("Retransmitted request, transaction still in progress {}",
+ xid); xid);
// Ignore the request and do nothing // Ignore the request and do nothing
return; return;
} }
@ -2261,18 +2229,12 @@ public void handleInternal(ChannelHandlerContext ctx, RpcInfo info) {
response = readlink(xdr, info); response = readlink(xdr, info);
metrics.addReadlink(Nfs3Utils.getElapsedTime(startTime)); metrics.addReadlink(Nfs3Utils.getElapsedTime(startTime));
} else if (nfsproc3 == NFSPROC3.READ) { } else if (nfsproc3 == NFSPROC3.READ) {
if (LOG.isDebugEnabled()) { LOG.debug("{}{}", Nfs3Utils.READ_RPC_START, xid);
LOG.debug(Nfs3Utils.READ_RPC_START + xid);
}
response = read(xdr, info); response = read(xdr, info);
if (LOG.isDebugEnabled() && (nfsproc3 == NFSPROC3.READ)) { LOG.debug("{}{}", Nfs3Utils.READ_RPC_END, xid);
LOG.debug(Nfs3Utils.READ_RPC_END + xid);
}
metrics.addRead(Nfs3Utils.getElapsedTime(startTime)); metrics.addRead(Nfs3Utils.getElapsedTime(startTime));
} else if (nfsproc3 == NFSPROC3.WRITE) { } else if (nfsproc3 == NFSPROC3.WRITE) {
if (LOG.isDebugEnabled()) { LOG.debug("{}{}", Nfs3Utils.WRITE_RPC_START, xid);
LOG.debug(Nfs3Utils.WRITE_RPC_START + xid);
}
response = write(xdr, info); response = write(xdr, info);
// Write end debug trace is in Nfs3Utils.writeChannel // Write end debug trace is in Nfs3Utils.writeChannel
} else if (nfsproc3 == NFSPROC3.CREATE) { } else if (nfsproc3 == NFSPROC3.CREATE) {
@ -2323,10 +2285,8 @@ RpcAcceptedReply.AcceptState.PROC_UNAVAIL, new VerifierNone()).write(
out); out);
} }
if (response == null) { if (response == null) {
if (LOG.isDebugEnabled()) { LOG.debug("No sync response, expect an async response for request XID={}",
LOG.debug("No sync response, expect an async response for request XID=" rpcCall.getXid());
+ rpcCall.getXid());
}
return; return;
} }
// TODO: currently we just return VerifierNone // TODO: currently we just return VerifierNone

View File

@ -477,7 +477,7 @@ private static Tracer createTracer(Configuration conf) {
HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT)) { HdfsClientConfigKeys.Read.ShortCircuit.DEFAULT)) {
String reason = DomainSocket.getLoadingFailureReason(); String reason = DomainSocket.getLoadingFailureReason();
if (reason != null) { if (reason != null) {
LOG.warn("File descriptor passing is disabled because " + reason); LOG.warn("File descriptor passing is disabled because {}", reason);
this.fileDescriptorPassingDisabledReason = reason; this.fileDescriptorPassingDisabledReason = reason;
} else { } else {
LOG.info("File descriptor passing is enabled."); LOG.info("File descriptor passing is enabled.");
@ -493,7 +493,7 @@ private static Tracer createTracer(Configuration conf) {
try { try {
hostName = getHostName(conf); hostName = getHostName(conf);
LOG.info("Configured hostname is " + hostName); LOG.info("Configured hostname is {}", hostName);
startDataNode(dataDirs, resources); startDataNode(dataDirs, resources);
} catch (IOException ie) { } catch (IOException ie) {
shutdown(); shutdown();
@ -533,7 +533,7 @@ public String reconfigurePropertyImpl(String property, String newVal)
case DFS_DATANODE_DATA_DIR_KEY: { case DFS_DATANODE_DATA_DIR_KEY: {
IOException rootException = null; IOException rootException = null;
try { try {
LOG.info("Reconfiguring " + property + " to " + newVal); LOG.info("Reconfiguring {} to {}", property, newVal);
this.refreshVolumes(newVal); this.refreshVolumes(newVal);
return getConf().get(DFS_DATANODE_DATA_DIR_KEY); return getConf().get(DFS_DATANODE_DATA_DIR_KEY);
} catch (IOException e) { } catch (IOException e) {
@ -545,7 +545,7 @@ public String reconfigurePropertyImpl(String property, String newVal)
new BlockReportOptions.Factory().setIncremental(false).build()); new BlockReportOptions.Factory().setIncremental(false).build());
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Exception while sending the block report after refreshing" LOG.warn("Exception while sending the block report after refreshing"
+ " volumes " + property + " to " + newVal, e); + " volumes {} to {}", property, newVal, e);
if (rootException == null) { if (rootException == null) {
rootException = e; rootException = e;
} }
@ -561,7 +561,7 @@ public String reconfigurePropertyImpl(String property, String newVal)
case DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY: { case DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY: {
ReconfigurationException rootException = null; ReconfigurationException rootException = null;
try { try {
LOG.info("Reconfiguring " + property + " to " + newVal); LOG.info("Reconfiguring {} to {}", property, newVal);
int movers; int movers;
if (newVal == null) { if (newVal == null) {
// set to default // set to default
@ -696,8 +696,8 @@ ChangedVolumes parseChangedVolumes(String newVolumes) throws IOException {
// New conf doesn't have the storage location which available in // New conf doesn't have the storage location which available in
// the current storage locations. Add to the deactivateLocations list. // the current storage locations. Add to the deactivateLocations list.
if (!found) { if (!found) {
LOG.info("Deactivation request received for active volume: " LOG.info("Deactivation request received for active volume: {}",
+ dir.getRoot().toString()); dir.getRoot());
results.deactivateLocations.add( results.deactivateLocations.add(
StorageLocation.parse(dir.getRoot().toString())); StorageLocation.parse(dir.getRoot().toString()));
} }
@ -724,8 +724,8 @@ ChangedVolumes parseChangedVolumes(String newVolumes) throws IOException {
// New conf doesn't have this failed storage location. // New conf doesn't have this failed storage location.
// Add to the deactivate locations list. // Add to the deactivate locations list.
if (!found) { if (!found) {
LOG.info("Deactivation request received for failed volume: " LOG.info("Deactivation request received for failed volume: {}",
+ failedStorageLocation); failedStorageLocation);
results.deactivateLocations.add(StorageLocation.parse( results.deactivateLocations.add(StorageLocation.parse(
failedStorageLocation)); failedStorageLocation));
} }
@ -760,7 +760,7 @@ private synchronized void refreshVolumes(String newVolumes) throws IOException {
throw new IOException("Attempt to remove all volumes."); throw new IOException("Attempt to remove all volumes.");
} }
if (!changedVolumes.newLocations.isEmpty()) { if (!changedVolumes.newLocations.isEmpty()) {
LOG.info("Adding new volumes: " + LOG.info("Adding new volumes: {}",
Joiner.on(",").join(changedVolumes.newLocations)); Joiner.on(",").join(changedVolumes.newLocations));
// Add volumes for each Namespace // Add volumes for each Namespace
@ -794,16 +794,16 @@ public IOException call() {
errorMessageBuilder.append( errorMessageBuilder.append(
String.format("FAILED TO ADD: %s: %s%n", String.format("FAILED TO ADD: %s: %s%n",
volume, ioe.getMessage())); volume, ioe.getMessage()));
LOG.error("Failed to add volume: " + volume, ioe); LOG.error("Failed to add volume: {}", volume, ioe);
} else { } else {
effectiveVolumes.add(volume.toString()); effectiveVolumes.add(volume.toString());
LOG.info("Successfully added volume: " + volume); LOG.info("Successfully added volume: {}", volume);
} }
} catch (Exception e) { } catch (Exception e) {
errorMessageBuilder.append( errorMessageBuilder.append(
String.format("FAILED to ADD: %s: %s%n", volume, String.format("FAILED to ADD: %s: %s%n", volume,
e.toString())); e.toString()));
LOG.error("Failed to add volume: " + volume, e); LOG.error("Failed to add volume: {}", volume, e);
} }
} }
} }
@ -812,7 +812,7 @@ public IOException call() {
removeVolumes(changedVolumes.deactivateLocations); removeVolumes(changedVolumes.deactivateLocations);
} catch (IOException e) { } catch (IOException e) {
errorMessageBuilder.append(e.getMessage()); errorMessageBuilder.append(e.getMessage());
LOG.error("Failed to remove volume: " + e.getMessage(), e); LOG.error("Failed to remove volume", e);
} }
if (errorMessageBuilder.length() > 0) { if (errorMessageBuilder.length() > 0) {
@ -967,16 +967,17 @@ private void startPlugins(Configuration conf) {
ServicePlugin.class); ServicePlugin.class);
} catch (RuntimeException e) { } catch (RuntimeException e) {
String pluginsValue = conf.get(DFS_DATANODE_PLUGINS_KEY); String pluginsValue = conf.get(DFS_DATANODE_PLUGINS_KEY);
LOG.error("Unable to load DataNode plugins. Specified list of plugins: " + LOG.error("Unable to load DataNode plugins. " +
"Specified list of plugins: {}",
pluginsValue, e); pluginsValue, e);
throw e; throw e;
} }
for (ServicePlugin p: plugins) { for (ServicePlugin p: plugins) {
try { try {
p.start(this); p.start(this);
LOG.info("Started plug-in " + p); LOG.info("Started plug-in {}", p);
} catch (Throwable t) { } catch (Throwable t) {
LOG.warn("ServicePlugin " + p + " could not be started", t); LOG.warn("ServicePlugin {} could not be started", p, t);
} }
} }
} }
@ -1026,7 +1027,7 @@ private void initIpcServer() throws IOException {
traceAdminService, traceAdminService,
ipcServer); ipcServer);
LOG.info("Opened IPC server at " + ipcServer.getListenerAddress()); LOG.info("Opened IPC server at {}", ipcServer.getListenerAddress());
// set service-level authorization security policy // set service-level authorization security policy
if (getConf().getBoolean( if (getConf().getBoolean(
@ -1085,7 +1086,8 @@ private synchronized void initDirectoryScanner(Configuration conf) {
directoryScanner = new DirectoryScanner(this, data, conf); directoryScanner = new DirectoryScanner(this, data, conf);
directoryScanner.start(); directoryScanner.start();
} else { } else {
LOG.info("Periodic Directory Tree Verification scan is disabled because " + LOG.info("Periodic Directory Tree Verification scan " +
"is disabled because {}",
reason); reason);
} }
} }
@ -1139,7 +1141,7 @@ private void initDataXceiver() throws IOException {
dnConf.getTransferSocketRecvBufferSize()); dnConf.getTransferSocketRecvBufferSize());
} }
streamingAddr = tcpPeerServer.getStreamingAddr(); streamingAddr = tcpPeerServer.getStreamingAddr();
LOG.info("Opened streaming server at " + streamingAddr); LOG.info("Opened streaming server at {}", streamingAddr);
this.threadGroup = new ThreadGroup("dataXceiverServer"); this.threadGroup = new ThreadGroup("dataXceiverServer");
xserver = new DataXceiverServer(tcpPeerServer, getConf(), this); xserver = new DataXceiverServer(tcpPeerServer, getConf(), this);
this.dataXceiverServer = new Daemon(threadGroup, xserver); this.dataXceiverServer = new Daemon(threadGroup, xserver);
@ -1157,7 +1159,7 @@ private void initDataXceiver() throws IOException {
if (domainPeerServer != null) { if (domainPeerServer != null) {
this.localDataXceiverServer = new Daemon(threadGroup, this.localDataXceiverServer = new Daemon(threadGroup,
new DataXceiverServer(domainPeerServer, getConf(), this)); new DataXceiverServer(domainPeerServer, getConf(), this));
LOG.info("Listening on UNIX domain socket: " + LOG.info("Listening on UNIX domain socket: {}",
domainPeerServer.getBindPath()); domainPeerServer.getBindPath());
} }
} }
@ -1175,7 +1177,7 @@ private static DomainPeerServer getDomainPeerServer(Configuration conf,
(!conf.getBoolean(HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, (!conf.getBoolean(HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT))) { HdfsClientConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT))) {
LOG.warn("Although short-circuit local reads are configured, " + LOG.warn("Although short-circuit local reads are configured, " +
"they are disabled because you didn't configure " + "they are disabled because you didn't configure {}",
DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY); DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY);
} }
return null; return null;
@ -1205,8 +1207,8 @@ public void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint,
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid, bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid,
isOnTransientStorage); isOnTransientStorage);
} else { } else {
LOG.error("Cannot find BPOfferService for reporting block received for bpid=" LOG.error("Cannot find BPOfferService for reporting block received " +
+ block.getBlockPoolId()); "for bpid={}", block.getBlockPoolId());
} }
} }
@ -1217,8 +1219,8 @@ protected void notifyNamenodeReceivingBlock(
if(bpos != null) { if(bpos != null) {
bpos.notifyNamenodeReceivingBlock(block, storageUuid); bpos.notifyNamenodeReceivingBlock(block, storageUuid);
} else { } else {
LOG.error("Cannot find BPOfferService for reporting block receiving for bpid=" LOG.error("Cannot find BPOfferService for reporting block receiving " +
+ block.getBlockPoolId()); "for bpid={}", block.getBlockPoolId());
} }
} }
@ -1239,7 +1241,7 @@ public void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid)
public void reportBadBlocks(ExtendedBlock block) throws IOException{ public void reportBadBlocks(ExtendedBlock block) throws IOException{
FsVolumeSpi volume = getFSDataset().getVolume(block); FsVolumeSpi volume = getFSDataset().getVolume(block);
if (volume == null) { if (volume == null) {
LOG.warn("Cannot find FsVolumeSpi to report bad block: " + block); LOG.warn("Cannot find FsVolumeSpi to report bad block: {}", block);
return; return;
} }
reportBadBlocks(block, volume); reportBadBlocks(block, volume);
@ -1382,7 +1384,7 @@ void startDataNode(List<StorageLocation> dataDirectories,
} }
} }
} }
LOG.info("Starting DataNode with maxLockedMemory = " + LOG.info("Starting DataNode with maxLockedMemory = {}",
dnConf.maxLockedMemory); dnConf.maxLockedMemory);
int volFailuresTolerated = dnConf.getVolFailuresTolerated(); int volFailuresTolerated = dnConf.getVolFailuresTolerated();
@ -1409,8 +1411,8 @@ void startDataNode(List<StorageLocation> dataDirectories,
// Login is done by now. Set the DN user name. // Login is done by now. Set the DN user name.
dnUserName = UserGroupInformation.getCurrentUser().getUserName(); dnUserName = UserGroupInformation.getCurrentUser().getUserName();
LOG.info("dnUserName = " + dnUserName); LOG.info("dnUserName = {}", dnUserName);
LOG.info("supergroup = " + supergroup); LOG.info("supergroup = {}", supergroup);
initIpcServer(); initIpcServer();
metrics = DataNodeMetrics.create(getConf(), getDisplayName()); metrics = DataNodeMetrics.create(getConf(), getDisplayName());
@ -1514,7 +1516,7 @@ synchronized void checkDatanodeUuid() throws IOException {
if (storage.getDatanodeUuid() == null) { if (storage.getDatanodeUuid() == null) {
storage.setDatanodeUuid(generateUuid()); storage.setDatanodeUuid(generateUuid());
storage.writeAll(); storage.writeAll();
LOG.info("Generated and persisted new Datanode UUID " + LOG.info("Generated and persisted new Datanode UUID {}",
storage.getDatanodeUuid()); storage.getDatanodeUuid());
} }
} }
@ -1583,11 +1585,11 @@ private synchronized void registerBlockPoolWithSecretManager(
if (!blockPoolTokenSecretManager.isBlockPoolRegistered(blockPoolId)) { if (!blockPoolTokenSecretManager.isBlockPoolRegistered(blockPoolId)) {
long blockKeyUpdateInterval = keys.getKeyUpdateInterval(); long blockKeyUpdateInterval = keys.getKeyUpdateInterval();
long blockTokenLifetime = keys.getTokenLifetime(); long blockTokenLifetime = keys.getTokenLifetime();
LOG.info("Block token params received from NN: for block pool " + LOG.info("Block token params received from NN: " +
blockPoolId + " keyUpdateInterval=" "for block pool {} keyUpdateInterval={} min(s), " +
+ blockKeyUpdateInterval / (60 * 1000) "tokenLifetime={} min(s)",
+ " min(s), tokenLifetime=" + blockTokenLifetime / (60 * 1000) blockPoolId, blockKeyUpdateInterval / (60 * 1000),
+ " min(s)"); blockTokenLifetime / (60 * 1000));
final boolean enableProtobuf = getConf().getBoolean( final boolean enableProtobuf = getConf().getBoolean(
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_PROTOBUF_ENABLE, DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_PROTOBUF_ENABLE,
DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_PROTOBUF_ENABLE_DEFAULT); DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_PROTOBUF_ENABLE_DEFAULT);
@ -1690,9 +1692,10 @@ private void initStorage(final NamespaceInfo nsInfo) throws IOException {
storage.recoverTransitionRead(this, nsInfo, dataDirs, startOpt); storage.recoverTransitionRead(this, nsInfo, dataDirs, startOpt);
} }
final StorageInfo bpStorage = storage.getBPStorage(bpid); final StorageInfo bpStorage = storage.getBPStorage(bpid);
LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID() LOG.info("Setting up storage: nsid={};bpid={};lv={};" +
+ ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion() "nsInfo={};dnuuid={}",
+ ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid()); bpStorage.getNamespaceID(), bpid, storage.getLayoutVersion(),
nsInfo, storage.getDatanodeUuid());
} }
// If this is a newly formatted DataNode then assign a new DatanodeUuid. // If this is a newly formatted DataNode then assign a new DatanodeUuid.
@ -1802,9 +1805,8 @@ public static InterDatanodeProtocol createInterDataNodeProtocolProxy(
final boolean connectToDnViaHostname) throws IOException { final boolean connectToDnViaHostname) throws IOException {
final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname); final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
final InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr); final InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
if (LOG.isDebugEnabled()) { LOG.debug("Connecting to datanode {} addr={}",
LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr); dnAddr, addr);
}
final UserGroupInformation loginUgi = UserGroupInformation.getLoginUser(); final UserGroupInformation loginUgi = UserGroupInformation.getLoginUser();
try { try {
return loginUgi return loginUgi
@ -1868,20 +1870,15 @@ public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
checkBlockToken(block, token, BlockTokenIdentifier.AccessMode.READ); checkBlockToken(block, token, BlockTokenIdentifier.AccessMode.READ);
Preconditions.checkNotNull(data, "Storage not yet initialized"); Preconditions.checkNotNull(data, "Storage not yet initialized");
BlockLocalPathInfo info = data.getBlockLocalPathInfo(block); BlockLocalPathInfo info = data.getBlockLocalPathInfo(block);
if (LOG.isDebugEnabled()) {
if (info != null) { if (info != null) {
if (LOG.isTraceEnabled()) { LOG.trace("getBlockLocalPathInfo successful " +
LOG.trace("getBlockLocalPathInfo successful block=" + block "block={} blockfile {} metafile {}",
+ " blockfile " + info.getBlockPath() + " metafile " block, info.getBlockPath(), info.getMetaPath());
+ info.getMetaPath());
}
} else { } else {
if (LOG.isTraceEnabled()) { LOG.trace("getBlockLocalPathInfo for block={} " +
LOG.trace("getBlockLocalPathInfo for block=" + block "returning null", block);
+ " returning null");
}
}
} }
metrics.incrBlocksGetLocalPathInfo(); metrics.incrBlocksGetLocalPathInfo();
return info; return info;
} }
@ -1939,9 +1936,7 @@ private void checkBlockToken(ExtendedBlock block,
ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier()); ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
DataInputStream in = new DataInputStream(buf); DataInputStream in = new DataInputStream(buf);
id.readFields(in); id.readFields(in);
if (LOG.isDebugEnabled()) { LOG.debug("Got: {}", id);
LOG.debug("Got: " + id.toString());
}
blockPoolTokenSecretManager.checkAccess(id, null, block, accessMode, blockPoolTokenSecretManager.checkAccess(id, null, block, accessMode,
null, null); null, null);
} }
@ -1959,9 +1954,9 @@ public void shutdown() {
for (ServicePlugin p : plugins) { for (ServicePlugin p : plugins) {
try { try {
p.stop(); p.stop();
LOG.info("Stopped plug-in " + p); LOG.info("Stopped plug-in {}", p);
} catch (Throwable t) { } catch (Throwable t) {
LOG.warn("ServicePlugin " + p + " could not be stopped", t); LOG.warn("ServicePlugin {} could not be stopped", p, t);
} }
} }
} }
@ -1984,7 +1979,7 @@ public void shutdown() {
this.dataXceiverServer.interrupt(); this.dataXceiverServer.interrupt();
} catch (Exception e) { } catch (Exception e) {
// Ignore, since the out of band messaging is advisory. // Ignore, since the out of band messaging is advisory.
LOG.trace("Exception interrupting DataXceiverServer: ", e); LOG.trace("Exception interrupting DataXceiverServer", e);
} }
} }
@ -2038,7 +2033,7 @@ public void shutdown() {
this.threadGroup.interrupt(); this.threadGroup.interrupt();
break; break;
} }
LOG.info("Waiting for threadgroup to exit, active threads is " + LOG.info("Waiting for threadgroup to exit, active threads is {}",
this.threadGroup.activeCount()); this.threadGroup.activeCount());
if (this.threadGroup.activeCount() == 0) { if (this.threadGroup.activeCount() == 0) {
break; break;
@ -2085,7 +2080,7 @@ public void shutdown() {
try { try {
this.blockPoolManager.shutDownAll(bposArray); this.blockPoolManager.shutDownAll(bposArray);
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
LOG.warn("Received exception in BlockPoolManager#shutDownAll: ", ie); LOG.warn("Received exception in BlockPoolManager#shutDownAll", ie);
} }
} }
@ -2093,7 +2088,7 @@ public void shutdown() {
try { try {
this.storage.unlockAll(); this.storage.unlockAll();
} catch (IOException ie) { } catch (IOException ie) {
LOG.warn("Exception when unlocking storage: " + ie, ie); LOG.warn("Exception when unlocking storage", ie);
} }
} }
if (data != null) { if (data != null) {
@ -2140,8 +2135,8 @@ public void checkDiskErrorAsync(FsVolumeSpi volume) {
private void handleDiskError(String failedVolumes) { private void handleDiskError(String failedVolumes) {
final boolean hasEnoughResources = data.hasEnoughResource(); final boolean hasEnoughResources = data.hasEnoughResource();
LOG.warn("DataNode.handleDiskError on : [" + failedVolumes + LOG.warn("DataNode.handleDiskError on: " +
"] Keep Running: " + hasEnoughResources); "[{}] Keep Running: {}", failedVolumes, hasEnoughResources);
// If we have enough active valid volumes then we do not want to // If we have enough active valid volumes then we do not want to
// shutdown the DN completely. // shutdown the DN completely.
@ -2438,15 +2433,13 @@ private class DataTransfer implements Runnable {
String[] targetStorageIds, ExtendedBlock b, String[] targetStorageIds, ExtendedBlock b,
BlockConstructionStage stage, final String clientname) { BlockConstructionStage stage, final String clientname) {
if (DataTransferProtocol.LOG.isDebugEnabled()) { if (DataTransferProtocol.LOG.isDebugEnabled()) {
DataTransferProtocol.LOG.debug(getClass().getSimpleName() + ": " DataTransferProtocol.LOG.debug("{}: {} (numBytes={}), stage={}, " +
+ b + " (numBytes=" + b.getNumBytes() + ")" "clientname={}, targets={}, target storage types={}, " +
+ ", stage=" + stage "target storage IDs={}", getClass().getSimpleName(), b,
+ ", clientname=" + clientname b.getNumBytes(), stage, clientname, Arrays.asList(targets),
+ ", targets=" + Arrays.asList(targets) targetStorageTypes == null ? "[]" :
+ ", target storage types=" + (targetStorageTypes == null ? "[]" : Arrays.asList(targetStorageTypes),
Arrays.asList(targetStorageTypes)) targetStorageIds == null ? "[]" : Arrays.asList(targetStorageIds));
+ ", target storage IDs=" + (targetStorageIds == null ? "[]" :
Arrays.asList(targetStorageIds)));
} }
this.targets = targets; this.targets = targets;
this.targetStorageTypes = targetStorageTypes; this.targetStorageTypes = targetStorageTypes;
@ -2475,9 +2468,7 @@ public void run() {
try { try {
final String dnAddr = targets[0].getXferAddr(connectToDnViaHostname); final String dnAddr = targets[0].getXferAddr(connectToDnViaHostname);
InetSocketAddress curTarget = NetUtils.createSocketAddr(dnAddr); InetSocketAddress curTarget = NetUtils.createSocketAddr(dnAddr);
if (LOG.isDebugEnabled()) { LOG.debug("Connecting to datanode {}", dnAddr);
LOG.debug("Connecting to datanode " + dnAddr);
}
sock = newSocket(); sock = newSocket();
NetUtils.connect(sock, curTarget, dnConf.socketTimeout); NetUtils.connect(sock, curTarget, dnConf.socketTimeout);
sock.setTcpNoDelay(dnConf.getDataTransferServerTcpNoDelay()); sock.setTcpNoDelay(dnConf.getDataTransferServerTcpNoDelay());
@ -2521,17 +2512,15 @@ public void run() {
blockSender.sendBlock(out, unbufOut, null); blockSender.sendBlock(out, unbufOut, null);
// no response necessary // no response necessary
LOG.info(getClass().getSimpleName() + ", at " LOG.info("{}, at {}: Transmitted {} (numBytes={}) to {}",
+ DataNode.this.getDisplayName() + ": Transmitted " + b getClass().getSimpleName(), DataNode.this.getDisplayName(),
+ " (numBytes=" + b.getNumBytes() + ") to " + curTarget); b, b.getNumBytes(), curTarget);
// read ack // read ack
if (isClient) { if (isClient) {
DNTransferAckProto closeAck = DNTransferAckProto.parseFrom( DNTransferAckProto closeAck = DNTransferAckProto.parseFrom(
PBHelperClient.vintPrefixed(in)); PBHelperClient.vintPrefixed(in));
if (LOG.isDebugEnabled()) { LOG.debug("{}: close-ack={}", getClass().getSimpleName(), closeAck);
LOG.debug(getClass().getSimpleName() + ": close-ack=" + closeAck);
}
if (closeAck.getStatus() != Status.SUCCESS) { if (closeAck.getStatus() != Status.SUCCESS) {
if (closeAck.getStatus() == Status.ERROR_ACCESS_TOKEN) { if (closeAck.getStatus() == Status.ERROR_ACCESS_TOKEN) {
throw new InvalidBlockTokenException( throw new InvalidBlockTokenException(
@ -2550,17 +2539,11 @@ public void run() {
// Add the block to the front of the scanning queue if metadata file // Add the block to the front of the scanning queue if metadata file
// is corrupt. We already add the block to front of scanner if the // is corrupt. We already add the block to front of scanner if the
// peer disconnects. // peer disconnects.
LOG.info("Adding block: " + b + " for scanning"); LOG.info("Adding block: {} for scanning", b);
blockScanner.markSuspectBlock(data.getVolume(b).getStorageID(), b); blockScanner.markSuspectBlock(data.getVolume(b).getStorageID(), b);
} }
LOG.warn(bpReg + ":Failed to transfer " + b + " to " + LOG.warn("{}:Failed to transfer {} to {} got",
targets[0] + " got ", ie); bpReg, b, targets[0], ie);
// disk check moved to FileIoProvider
IOException cause = DatanodeUtil.getCauseIfDiskError(ie);
if (cause != null) { // possible disk error
LOG.warn("IOException in DataTransfer#run() "+ ie.getMessage() +". "
+ "Cause is ", cause);
}
} finally { } finally {
decrementXmitsInProgress(); decrementXmitsInProgress();
IOUtils.closeStream(blockSender); IOUtils.closeStream(blockSender);
@ -2691,14 +2674,9 @@ public static List<StorageLocation> getStorageLocations(Configuration conf) {
final StorageLocation location; final StorageLocation location;
try { try {
location = StorageLocation.parse(locationString); location = StorageLocation.parse(locationString);
} catch (IOException ioe) { } catch (IOException | SecurityException ioe) {
LOG.error("Failed to initialize storage directory " + locationString LOG.error("Failed to initialize storage directory {}." +
+ ". Exception details: " + ioe); "Exception details: {}", locationString, ioe.toString());
// Ignore the exception.
continue;
} catch (SecurityException se) {
LOG.error("Failed to initialize storage directory " + locationString
+ ". Exception details: " + se);
// Ignore the exception. // Ignore the exception.
continue; continue;
} }
@ -2745,7 +2723,7 @@ void join() {
wait(2000); wait(2000);
} }
} catch (InterruptedException ex) { } catch (InterruptedException ex) {
LOG.warn("Received exception in Datanode#join: " + ex); LOG.warn("Received exception in Datanode#join: {}", ex.toString());
} }
} }
} }
@ -2950,9 +2928,7 @@ private void checkReadAccess(final ExtendedBlock block) throws IOException {
} }
for (TokenIdentifier tokenId : tokenIds) { for (TokenIdentifier tokenId : tokenIds) {
BlockTokenIdentifier id = (BlockTokenIdentifier) tokenId; BlockTokenIdentifier id = (BlockTokenIdentifier) tokenId;
if (LOG.isDebugEnabled()) { LOG.debug("Got: {}", id);
LOG.debug("Got: " + id.toString());
}
blockPoolTokenSecretManager.checkAccess(id, null, block, blockPoolTokenSecretManager.checkAccess(id, null, block,
BlockTokenIdentifier.AccessMode.READ, null, null); BlockTokenIdentifier.AccessMode.READ, null, null);
} }
@ -3165,11 +3141,11 @@ public void refreshNamenodes() throws IOException {
public void deleteBlockPool(String blockPoolId, boolean force) public void deleteBlockPool(String blockPoolId, boolean force)
throws IOException { throws IOException {
checkSuperuserPrivilege(); checkSuperuserPrivilege();
LOG.info("deleteBlockPool command received for block pool " + blockPoolId LOG.info("deleteBlockPool command received for block pool {}, " +
+ ", force=" + force); "force={}", blockPoolId, force);
if (blockPoolManager.get(blockPoolId) != null) { if (blockPoolManager.get(blockPoolId) != null) {
LOG.warn("The block pool "+blockPoolId+ LOG.warn("The block pool {} is still running, cannot be deleted.",
" is still running, cannot be deleted."); blockPoolId);
throw new IOException( throw new IOException(
"The block pool is still running. First do a refreshNamenodes to " + "The block pool is still running. First do a refreshNamenodes to " +
"shutdown the block pool service"); "shutdown the block pool service");
@ -3181,8 +3157,8 @@ public void deleteBlockPool(String blockPoolId, boolean force)
@Override // ClientDatanodeProtocol @Override // ClientDatanodeProtocol
public synchronized void shutdownDatanode(boolean forUpgrade) throws IOException { public synchronized void shutdownDatanode(boolean forUpgrade) throws IOException {
checkSuperuserPrivilege(); checkSuperuserPrivilege();
LOG.info("shutdownDatanode command received (upgrade=" + forUpgrade + LOG.info("shutdownDatanode command received (upgrade={}). " +
"). Shutting down Datanode..."); "Shutting down Datanode...", forUpgrade);
// Shutdown can be called only once. // Shutdown can be called only once.
if (shutdownInProgress) { if (shutdownInProgress) {
@ -3381,12 +3357,9 @@ private void handleVolumeFailures(Set<FsVolumeSpi> unhealthyVolumes) {
// Remove all unhealthy volumes from DataNode. // Remove all unhealthy volumes from DataNode.
removeVolumes(unhealthyLocations, false); removeVolumes(unhealthyLocations, false);
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Error occurred when removing unhealthy storage dirs: " LOG.warn("Error occurred when removing unhealthy storage dirs", e);
+ e.getMessage(), e);
}
if (LOG.isDebugEnabled()) {
LOG.debug(sb.toString());
} }
LOG.debug("{}", sb);
// send blockreport regarding volume failure // send blockreport regarding volume failure
handleDiskError(sb.toString()); handleDiskError(sb.toString());
} }
@ -3568,7 +3541,7 @@ public String getDiskBalancerSetting(String key) throws IOException {
case DiskBalancerConstants.DISKBALANCER_BANDWIDTH : case DiskBalancerConstants.DISKBALANCER_BANDWIDTH :
return Long.toString(this.diskBalancer.getBandwidth()); return Long.toString(this.diskBalancer.getBandwidth());
default: default:
LOG.error("Disk Balancer - Unknown key in get balancer setting. Key: " + LOG.error("Disk Balancer - Unknown key in get balancer setting. Key: {}",
key); key);
throw new DiskBalancerException("Unknown key", throw new DiskBalancerException("Unknown key",
DiskBalancerException.Result.UNKNOWN_KEY); DiskBalancerException.Result.UNKNOWN_KEY);

View File

@ -148,10 +148,8 @@ private DataXceiver(Peer peer, DataNode datanode,
(colonIdx < 0) ? remoteAddress : remoteAddress.substring(0, colonIdx); (colonIdx < 0) ? remoteAddress : remoteAddress.substring(0, colonIdx);
localAddress = peer.getLocalAddressString(); localAddress = peer.getLocalAddressString();
if (LOG.isDebugEnabled()) { LOG.debug("Number of active connections is: {}",
LOG.debug("Number of active connections is: " datanode.getXceiverCount());
+ datanode.getXceiverCount());
}
} }
/** /**
@ -187,7 +185,7 @@ public void sendOOB() throws IOException, InterruptedException {
// This doesn't need to be in a critical section. Althogh the client // This doesn't need to be in a critical section. Althogh the client
// can resue the connection to issue a different request, trying sending // can resue the connection to issue a different request, trying sending
// an OOB through the recently closed block receiver is harmless. // an OOB through the recently closed block receiver is harmless.
LOG.info("Sending OOB to peer: " + peer); LOG.info("Sending OOB to peer: {}", peer);
br.sendOOB(); br.sendOOB();
} }
@ -199,7 +197,7 @@ public void stopWriter() {
} }
xceiver.interrupt(); xceiver.interrupt();
} }
LOG.info("Stopped the writer: " + peer); LOG.info("Stopped the writer: {}", peer);
} }
/** /**
@ -239,14 +237,15 @@ public void run() {
} catch (InvalidMagicNumberException imne) { } catch (InvalidMagicNumberException imne) {
if (imne.isHandshake4Encryption()) { if (imne.isHandshake4Encryption()) {
LOG.info("Failed to read expected encryption handshake from client " + LOG.info("Failed to read expected encryption handshake from client " +
"at " + peer.getRemoteAddressString() + ". Perhaps the client " + "at {}. Perhaps the client " +
"is running an older version of Hadoop which does not support " + "is running an older version of Hadoop which does not support " +
"encryption", imne); "encryption", peer.getRemoteAddressString(), imne);
} else { } else {
LOG.info("Failed to read expected SASL data transfer protection " + LOG.info("Failed to read expected SASL data transfer protection " +
"handshake from client at " + peer.getRemoteAddressString() + "handshake from client at {}" +
". Perhaps the client is running an older version of Hadoop " + ". Perhaps the client is running an older version of Hadoop " +
"which does not support SASL data transfer protection", imne); "which does not support SASL data transfer protection",
peer.getRemoteAddressString(), imne);
} }
return; return;
} }
@ -302,7 +301,7 @@ public void run() {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace(s, t); LOG.trace(s, t);
} else { } else {
LOG.info(s + "; " + t); LOG.info("{}; {}", s, t.toString());
} }
} else if (op == Op.READ_BLOCK && t instanceof SocketTimeoutException) { } else if (op == Op.READ_BLOCK && t instanceof SocketTimeoutException) {
String s1 = String s1 =
@ -311,23 +310,19 @@ public void run() {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace(s1, t); LOG.trace(s1, t);
} else { } else {
LOG.info(s1 + "; " + t); LOG.info("{}; {}", s1, t.toString());
} }
} else if (t instanceof InvalidToken) { } else if (t instanceof InvalidToken) {
// The InvalidToken exception has already been logged in // The InvalidToken exception has already been logged in
// checkAccess() method and this is not a server error. // checkAccess() method and this is not a server error.
if (LOG.isTraceEnabled()) {
LOG.trace(s, t); LOG.trace(s, t);
}
} else { } else {
LOG.error(s, t); LOG.error(s, t);
} }
} finally { } finally {
collectThreadLocalStates(); collectThreadLocalStates();
if (LOG.isDebugEnabled()) { LOG.debug("{}:Number of active connections is: {}",
LOG.debug(datanode.getDisplayName() + ":Number of active connections is: " datanode.getDisplayName(), datanode.getXceiverCount());
+ datanode.getXceiverCount());
}
updateCurrentThreadName("Cleaning up"); updateCurrentThreadName("Cleaning up");
if (peer != null) { if (peer != null) {
dataXceiverServer.closePeer(peer); dataXceiverServer.closePeer(peer);
@ -405,21 +400,22 @@ public void requestShortCircuitFds(final ExtendedBlock blk,
DomainSocket sock = peer.getDomainSocket(); DomainSocket sock = peer.getDomainSocket();
sock.sendFileDescriptors(fds, buf, 0, buf.length); sock.sendFileDescriptors(fds, buf, 0, buf.length);
if (supportsReceiptVerification) { if (supportsReceiptVerification) {
LOG.trace("Reading receipt verification byte for " + slotId); LOG.trace("Reading receipt verification byte for {}", slotId);
int val = sock.getInputStream().read(); int val = sock.getInputStream().read();
if (val < 0) { if (val < 0) {
throw new EOFException(); throw new EOFException();
} }
} else { } else {
LOG.trace("Receipt verification is not enabled on the DataNode. " + LOG.trace("Receipt verification is not enabled on the DataNode. " +
"Not verifying " + slotId); "Not verifying {}", slotId);
} }
success = true; success = true;
} }
} finally { } finally {
if ((!success) && (registeredSlotId != null)) { if ((!success) && (registeredSlotId != null)) {
LOG.info("Unregistering " + registeredSlotId + " because the " + LOG.info("Unregistering {} because the " +
"requestShortCircuitFdsForRead operation failed."); "requestShortCircuitFdsForRead operation failed.",
registeredSlotId);
datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId); datanode.shortCircuitRegistry.unregisterSlot(registeredSlotId);
} }
if (ClientTraceLog.isInfoEnabled()) { if (ClientTraceLog.isInfoEnabled()) {
@ -548,7 +544,7 @@ public void requestShortCircuitShm(String clientName) throws IOException {
// bad behavior inside the poll() call. See HADOOP-11802 for details. // bad behavior inside the poll() call. See HADOOP-11802 for details.
try { try {
LOG.warn("Failed to send success response back to the client. " + LOG.warn("Failed to send success response back to the client. " +
"Shutting down socket for " + shmInfo.getShmId() + "."); "Shutting down socket for {}", shmInfo.getShmId());
sock.shutdown(); sock.shutdown();
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Failed to shut down socket in error handler", e); LOG.warn("Failed to shut down socket in error handler", e);
@ -616,9 +612,9 @@ public void readBlock(final ExtendedBlock block,
ClientReadStatusProto stat = ClientReadStatusProto.parseFrom( ClientReadStatusProto stat = ClientReadStatusProto.parseFrom(
PBHelperClient.vintPrefixed(in)); PBHelperClient.vintPrefixed(in));
if (!stat.hasStatus()) { if (!stat.hasStatus()) {
LOG.warn("Client " + peer.getRemoteAddressString() + LOG.warn("Client {} did not send a valid status code " +
" did not send a valid status code after reading. " + "after reading. Will close connection.",
"Will close connection."); peer.getRemoteAddressString());
IOUtils.closeStream(out); IOUtils.closeStream(out);
} }
} catch (IOException ioe) { } catch (IOException ioe) {
@ -633,10 +629,8 @@ public void readBlock(final ExtendedBlock block,
datanode.metrics.incrBlocksRead(); datanode.metrics.incrBlocksRead();
datanode.metrics.incrTotalReadTime(duration); datanode.metrics.incrTotalReadTime(duration);
} catch ( SocketException ignored ) { } catch ( SocketException ignored ) {
if (LOG.isTraceEnabled()) { LOG.trace("{}:Ignoring exception while serving {} to {}",
LOG.trace(dnR + ":Ignoring exception while serving " + block + " to " + dnR, block, remoteAddress, ignored);
remoteAddress, ignored);
}
// Its ok for remote side to close the connection anytime. // Its ok for remote side to close the connection anytime.
datanode.metrics.incrBlocksRead(); datanode.metrics.incrBlocksRead();
IOUtils.closeStream(out); IOUtils.closeStream(out);
@ -645,8 +639,8 @@ public void readBlock(final ExtendedBlock block,
* Earlier version shutdown() datanode if there is disk error. * Earlier version shutdown() datanode if there is disk error.
*/ */
if (!(ioe instanceof SocketTimeoutException)) { if (!(ioe instanceof SocketTimeoutException)) {
LOG.warn(dnR + ":Got exception while serving " + block + " to " LOG.warn("{}:Got exception while serving {} to {}",
+ remoteAddress, ioe); dnR, block, remoteAddress, ioe);
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
} }
throw ioe; throw ioe;
@ -721,17 +715,16 @@ public void writeBlock(final ExtendedBlock block,
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("opWriteBlock: stage=" + stage + ", clientname=" + clientname LOG.debug("opWriteBlock: stage={}, clientname={}\n " +
+ "\n block =" + block + ", newGs=" + latestGenerationStamp "block ={}, newGs={}, bytesRcvd=[{}, {}]\n " +
+ ", bytesRcvd=[" + minBytesRcvd + ", " + maxBytesRcvd + "]" "targets={}; pipelineSize={}, srcDataNode={}, pinning={}",
+ "\n targets=" + Arrays.asList(targets) stage, clientname, block, latestGenerationStamp, minBytesRcvd,
+ "; pipelineSize=" + pipelineSize + ", srcDataNode=" + srcDataNode maxBytesRcvd, Arrays.asList(targets), pipelineSize, srcDataNode,
+ ", pinning=" + pinning); pinning);
LOG.debug("isDatanode=" + isDatanode LOG.debug("isDatanode={}, isClient={}, isTransfer={}",
+ ", isClient=" + isClient isDatanode, isClient, isTransfer);
+ ", isTransfer=" + isTransfer); LOG.debug("writeBlock receive buf size {} tcp no delay {}",
LOG.debug("writeBlock receive buf size " + peer.getReceiveBufferSize() + peer.getReceiveBufferSize(), peer.getTcpNoDelay());
" tcp no delay " + peer.getTcpNoDelay());
} }
// We later mutate block's generation stamp and length, but we need to // We later mutate block's generation stamp and length, but we need to
@ -741,8 +734,8 @@ public void writeBlock(final ExtendedBlock block,
if (block.getNumBytes() == 0) { if (block.getNumBytes() == 0) {
block.setNumBytes(dataXceiverServer.estimateBlockSize); block.setNumBytes(dataXceiverServer.estimateBlockSize);
} }
LOG.info("Receiving " + block + " src: " + remoteAddress + " dest: " LOG.info("Receiving {} src: {} dest: {}",
+ localAddress); block, remoteAddress, localAddress);
DataOutputStream mirrorOut = null; // stream to next target DataOutputStream mirrorOut = null; // stream to next target
DataInputStream mirrorIn = null; // reply from next target DataInputStream mirrorIn = null; // reply from next target
@ -778,9 +771,7 @@ public void writeBlock(final ExtendedBlock block,
InetSocketAddress mirrorTarget = null; InetSocketAddress mirrorTarget = null;
// Connect to backup machine // Connect to backup machine
mirrorNode = targets[0].getXferAddr(connectToDnViaHostname); mirrorNode = targets[0].getXferAddr(connectToDnViaHostname);
if (LOG.isDebugEnabled()) { LOG.debug("Connecting to datanode {}", mirrorNode);
LOG.debug("Connecting to datanode " + mirrorNode);
}
mirrorTarget = NetUtils.createSocketAddr(mirrorNode); mirrorTarget = NetUtils.createSocketAddr(mirrorNode);
mirrorSock = datanode.newSocket(); mirrorSock = datanode.newSocket();
try { try {
@ -844,11 +835,10 @@ public void writeBlock(final ExtendedBlock block,
BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(mirrorIn)); BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(mirrorIn));
mirrorInStatus = connectAck.getStatus(); mirrorInStatus = connectAck.getStatus();
firstBadLink = connectAck.getFirstBadLink(); firstBadLink = connectAck.getFirstBadLink();
if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) { if (mirrorInStatus != SUCCESS) {
LOG.debug("Datanode " + targets.length + LOG.debug("Datanode {} got response for connect" +
" got response for connect ack " + "ack from downstream datanode with firstbadlink as {}",
" from downstream datanode with firstbadlink as " + targets.length, firstBadLink);
firstBadLink);
} }
} }
@ -869,13 +859,12 @@ public void writeBlock(final ExtendedBlock block,
IOUtils.closeSocket(mirrorSock); IOUtils.closeSocket(mirrorSock);
mirrorSock = null; mirrorSock = null;
if (isClient) { if (isClient) {
LOG.error(datanode + ":Exception transfering block " + LOG.error("{}:Exception transfering block {} to mirror {}",
block + " to mirror " + mirrorNode + ": " + e); datanode, block, mirrorNode, e);
throw e; throw e;
} else { } else {
LOG.info(datanode + ":Exception transfering " + LOG.info("{}:Exception transfering {} to mirror {}- continuing " +
block + " to mirror " + mirrorNode + "without the mirror", datanode, block, mirrorNode, e);
"- continuing without the mirror", e);
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
} }
} }
@ -883,10 +872,9 @@ public void writeBlock(final ExtendedBlock block,
// send connect-ack to source for clients and not transfer-RBW/Finalized // send connect-ack to source for clients and not transfer-RBW/Finalized
if (isClient && !isTransfer) { if (isClient && !isTransfer) {
if (LOG.isDebugEnabled() || mirrorInStatus != SUCCESS) { if (mirrorInStatus != SUCCESS) {
LOG.debug("Datanode " + targets.length + LOG.debug("Datanode {} forwarding connect ack to upstream " +
" forwarding connect ack to upstream firstbadlink is " + "firstbadlink is {}", targets.length, firstBadLink);
firstBadLink);
} }
BlockOpResponseProto.newBuilder() BlockOpResponseProto.newBuilder()
.setStatus(mirrorInStatus) .setStatus(mirrorInStatus)
@ -904,9 +892,7 @@ public void writeBlock(final ExtendedBlock block,
// send close-ack for transfer-RBW/Finalized // send close-ack for transfer-RBW/Finalized
if (isTransfer) { if (isTransfer) {
if (LOG.isTraceEnabled()) {
LOG.trace("TRANSFER: send close-ack"); LOG.trace("TRANSFER: send close-ack");
}
writeResponse(SUCCESS, null, replyOut); writeResponse(SUCCESS, null, replyOut);
} }
} }
@ -924,15 +910,16 @@ public void writeBlock(final ExtendedBlock block,
if (isDatanode || if (isDatanode ||
stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) { stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
datanode.closeBlock(block, null, storageUuid, isOnTransientStorage); datanode.closeBlock(block, null, storageUuid, isOnTransientStorage);
LOG.info("Received " + block + " src: " + remoteAddress + " dest: " LOG.info("Received {} src: {} dest: {} of size {}",
+ localAddress + " of size " + block.getNumBytes()); block, remoteAddress, localAddress, block.getNumBytes());
} }
if(isClient) { if(isClient) {
size = block.getNumBytes(); size = block.getNumBytes();
} }
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.info("opWriteBlock " + block + " received exception " + ioe); LOG.info("opWriteBlock {} received exception {}",
block, ioe.toString());
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
throw ioe; throw ioe;
} finally { } finally {
@ -970,7 +957,8 @@ public void transferBlock(final ExtendedBlock blk,
targetStorageTypes, targetStorageIds, clientName); targetStorageTypes, targetStorageIds, clientName);
writeResponse(Status.SUCCESS, null, out); writeResponse(Status.SUCCESS, null, out);
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.info("transferBlock " + blk + " received exception " + ioe); LOG.info("transferBlock {} received exception {}",
blk, ioe.toString());
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
throw ioe; throw ioe;
} finally { } finally {
@ -1005,7 +993,8 @@ public void blockChecksum(ExtendedBlock block,
.writeDelimitedTo(out); .writeDelimitedTo(out);
out.flush(); out.flush();
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.info("blockChecksum " + block + " received exception " + ioe); LOG.info("blockChecksum {} received exception {}",
block, ioe.toString());
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
throw ioe; throw ioe;
} finally { } finally {
@ -1046,8 +1035,8 @@ public void blockGroupChecksum(final StripedBlockInfo stripedBlockInfo,
.writeDelimitedTo(out); .writeDelimitedTo(out);
out.flush(); out.flush();
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.info("blockChecksum " + stripedBlockInfo.getBlock() + LOG.info("blockChecksum {} received exception {}",
" received exception " + ioe); stripedBlockInfo.getBlock(), ioe.toString());
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
throw ioe; throw ioe;
} finally { } finally {
@ -1105,10 +1094,10 @@ public void copyBlock(final ExtendedBlock block,
datanode.metrics.incrBlocksRead(); datanode.metrics.incrBlocksRead();
datanode.metrics.incrTotalReadTime(duration); datanode.metrics.incrTotalReadTime(duration);
LOG.info("Copied " + block + " to " + peer.getRemoteAddressString()); LOG.info("Copied {} to {}", block, peer.getRemoteAddressString());
} catch (IOException ioe) { } catch (IOException ioe) {
isOpSuccess = false; isOpSuccess = false;
LOG.info("opCopyBlock " + block + " received exception " + ioe); LOG.info("opCopyBlock {} received exception {}", block, ioe.toString());
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
throw ioe; throw ioe;
} finally { } finally {
@ -1163,16 +1152,14 @@ public void replaceBlock(final ExtendedBlock block,
ReplicaInfo oldReplica = datanode.data.moveBlockAcrossStorage(block, ReplicaInfo oldReplica = datanode.data.moveBlockAcrossStorage(block,
storageType, storageId); storageType, storageId);
if (oldReplica != null) { if (oldReplica != null) {
LOG.info("Moved " + block + " from StorageType " LOG.info("Moved {} from StorageType {} to {}",
+ oldReplica.getVolume().getStorageType() + " to " + storageType); block, oldReplica.getVolume().getStorageType(), storageType);
} }
} else { } else {
block.setNumBytes(dataXceiverServer.estimateBlockSize); block.setNumBytes(dataXceiverServer.estimateBlockSize);
// get the output stream to the proxy // get the output stream to the proxy
final String dnAddr = proxySource.getXferAddr(connectToDnViaHostname); final String dnAddr = proxySource.getXferAddr(connectToDnViaHostname);
if (LOG.isDebugEnabled()) { LOG.debug("Connecting to datanode {}", dnAddr);
LOG.debug("Connecting to datanode " + dnAddr);
}
InetSocketAddress proxyAddr = NetUtils.createSocketAddr(dnAddr); InetSocketAddress proxyAddr = NetUtils.createSocketAddr(dnAddr);
proxySock = datanode.newSocket(); proxySock = datanode.newSocket();
NetUtils.connect(proxySock, proxyAddr, dnConf.socketTimeout); NetUtils.connect(proxySock, proxyAddr, dnConf.socketTimeout);
@ -1229,8 +1216,8 @@ public void replaceBlock(final ExtendedBlock block,
datanode.notifyNamenodeReceivedBlock( datanode.notifyNamenodeReceivedBlock(
block, delHint, r.getStorageUuid(), r.isOnTransientStorage()); block, delHint, r.getStorageUuid(), r.isOnTransientStorage());
LOG.info("Moved " + block + " from " + peer.getRemoteAddressString() LOG.info("Moved {} from {}, delHint={}",
+ ", delHint=" + delHint); block, peer.getRemoteAddressString(), delHint);
} }
} catch (IOException ioe) { } catch (IOException ioe) {
opStatus = ERROR; opStatus = ERROR;
@ -1260,7 +1247,8 @@ public void replaceBlock(final ExtendedBlock block,
try { try {
sendResponse(opStatus, errMsg); sendResponse(opStatus, errMsg);
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.warn("Error writing reply back to " + peer.getRemoteAddressString()); LOG.warn("Error writing reply back to {}",
peer.getRemoteAddressString());
incrDatanodeNetworkErrors(); incrDatanodeNetworkErrors();
} }
IOUtils.closeStream(proxyOut); IOUtils.closeStream(proxyOut);
@ -1408,10 +1396,8 @@ private void checkAccess(OutputStream out, final boolean reply,
final String[] storageIds) throws IOException { final String[] storageIds) throws IOException {
checkAndWaitForBP(blk); checkAndWaitForBP(blk);
if (datanode.isBlockTokenEnabled) { if (datanode.isBlockTokenEnabled) {
if (LOG.isDebugEnabled()) { LOG.debug("Checking block access token for block '{}' with mode '{}'",
LOG.debug("Checking block access token for block '" + blk.getBlockId() blk.getBlockId(), mode);
+ "' with mode '" + mode + "'");
}
try { try {
datanode.blockPoolTokenSecretManager.checkAccess(t, null, blk, mode, datanode.blockPoolTokenSecretManager.checkAccess(t, null, blk, mode,
storageTypes, storageIds); storageTypes, storageIds);
@ -1429,9 +1415,9 @@ private void checkAccess(OutputStream out, final boolean reply,
resp.build().writeDelimitedTo(out); resp.build().writeDelimitedTo(out);
out.flush(); out.flush();
} }
LOG.warn("Block token verification failed: op=" + op LOG.warn("Block token verification failed: op={}, " +
+ ", remoteAddress=" + remoteAddress "remoteAddress={}, message={}",
+ ", message=" + e.getLocalizedMessage()); op, remoteAddress, e.getLocalizedMessage());
throw e; throw e;
} finally { } finally {
IOUtils.closeStream(out); IOUtils.closeStream(out);

View File

@ -6185,7 +6185,7 @@ public String getCorruptFiles() {
LOG.debug("Get corrupt file blocks returned error: " + e.getMessage()); LOG.debug("Get corrupt file blocks returned error: " + e.getMessage());
} }
} catch (IOException e) { } catch (IOException e) {
LOG.warn("Get corrupt file blocks returned error: " + e.getMessage()); LOG.warn("Get corrupt file blocks returned error", e);
} }
return JSON.toString(list); return JSON.toString(list);
} }

View File

@ -28,8 +28,6 @@
import java.util.concurrent.*; import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.ServiceFailedException; import org.apache.hadoop.ha.ServiceFailedException;
@ -51,6 +49,8 @@
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* Thread which runs inside the NN when it's in Standby state, * Thread which runs inside the NN when it's in Standby state,
@ -60,7 +60,8 @@
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class StandbyCheckpointer { public class StandbyCheckpointer {
private static final Log LOG = LogFactory.getLog(StandbyCheckpointer.class); private static final Logger LOG =
LoggerFactory.getLogger(StandbyCheckpointer.class);
private static final long PREVENT_AFTER_CANCEL_MS = 2*60*1000L; private static final long PREVENT_AFTER_CANCEL_MS = 2*60*1000L;
private final CheckpointConf checkpointConf; private final CheckpointConf checkpointConf;
private final Configuration conf; private final Configuration conf;
@ -136,8 +137,8 @@ private static boolean checkAddress(URL addr) {
public void start() { public void start() {
LOG.info("Starting standby checkpoint thread...\n" + LOG.info("Starting standby checkpoint thread...\n" +
"Checkpointing active NN to possible NNs: " + activeNNAddresses + "\n" + "Checkpointing active NN to possible NNs: {}\n" +
"Serving checkpoints at " + myNNAddress); "Serving checkpoints at {}", activeNNAddresses, myNNAddress);
thread.start(); thread.start();
} }
@ -177,8 +178,8 @@ private void doCheckpoint(boolean sendCheckpoint) throws InterruptedException, I
assert thisCheckpointTxId >= prevCheckpointTxId; assert thisCheckpointTxId >= prevCheckpointTxId;
if (thisCheckpointTxId == prevCheckpointTxId) { if (thisCheckpointTxId == prevCheckpointTxId) {
LOG.info("A checkpoint was triggered but the Standby Node has not " + LOG.info("A checkpoint was triggered but the Standby Node has not " +
"received any transactions since the last checkpoint at txid " + "received any transactions since the last checkpoint at txid {}. " +
thisCheckpointTxId + ". Skipping..."); "Skipping...", thisCheckpointTxId);
return; return;
} }
@ -253,8 +254,7 @@ public TransferFsImage.TransferResult call()
} }
} catch (ExecutionException e) { } catch (ExecutionException e) {
ioe = new IOException("Exception during image upload: " + e.getMessage(), ioe = new IOException("Exception during image upload", e);
e.getCause());
break; break;
} catch (InterruptedException e) { } catch (InterruptedException e) {
ie = e; ie = e;
@ -401,15 +401,15 @@ private void doWork() {
if (needCheckpoint) { if (needCheckpoint) {
LOG.info("Triggering a rollback fsimage for rolling upgrade."); LOG.info("Triggering a rollback fsimage for rolling upgrade.");
} else if (uncheckpointed >= checkpointConf.getTxnCount()) { } else if (uncheckpointed >= checkpointConf.getTxnCount()) {
LOG.info("Triggering checkpoint because there have been " + LOG.info("Triggering checkpoint because there have been {} txns " +
uncheckpointed + " txns since the last checkpoint, which " + "since the last checkpoint, " +
"exceeds the configured threshold " + "which exceeds the configured threshold {}",
checkpointConf.getTxnCount()); uncheckpointed, checkpointConf.getTxnCount());
needCheckpoint = true; needCheckpoint = true;
} else if (secsSinceLast >= checkpointConf.getPeriod()) { } else if (secsSinceLast >= checkpointConf.getPeriod()) {
LOG.info("Triggering checkpoint because it has been " + LOG.info("Triggering checkpoint because it has been {} seconds " +
secsSinceLast + " seconds since the last checkpoint, which " + "since the last checkpoint, which exceeds the configured " +
"exceeds the configured interval " + checkpointConf.getPeriod()); "interval {}", secsSinceLast, checkpointConf.getPeriod());
needCheckpoint = true; needCheckpoint = true;
} }
@ -442,7 +442,7 @@ private void doWork() {
LOG.info("Checkpoint finished successfully."); LOG.info("Checkpoint finished successfully.");
} }
} catch (SaveNamespaceCancelledException ce) { } catch (SaveNamespaceCancelledException ce) {
LOG.info("Checkpoint was cancelled: " + ce.getMessage()); LOG.info("Checkpoint was cancelled: {}", ce.getMessage());
canceledCount++; canceledCount++;
} catch (InterruptedException ie) { } catch (InterruptedException ie) {
LOG.info("Interrupted during checkpointing", ie); LOG.info("Interrupted during checkpointing", ie);

View File

@ -1332,7 +1332,7 @@ public URI getURI(int nnIndex) {
try { try {
uri = new URI("hdfs://" + hostPort); uri = new URI("hdfs://" + hostPort);
} catch (URISyntaxException e) { } catch (URISyntaxException e) {
NameNode.LOG.warn("unexpected URISyntaxException: " + e ); NameNode.LOG.warn("unexpected URISyntaxException", e);
} }
return uri; return uri;
} }

View File

@ -32,13 +32,13 @@
import org.apache.commons.cli.OptionBuilder; import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options; import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException; import org.apache.commons.cli.ParseException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.eclipse.jetty.util.ajax.JSON; import org.eclipse.jetty.util.ajax.JSON;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** /**
* This class drives the creation of a mini-cluster on the local machine. By * This class drives the creation of a mini-cluster on the local machine. By
@ -58,8 +58,8 @@
* $HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/share/hadoop/hdfs/hadoop-hdfs-0.24.0-SNAPSHOT-tests.jar org.apache.hadoop.test.MiniDFSClusterManager -options... * $HADOOP_HOME/bin/hadoop jar $HADOOP_HOME/share/hadoop/hdfs/hadoop-hdfs-0.24.0-SNAPSHOT-tests.jar org.apache.hadoop.test.MiniDFSClusterManager -options...
*/ */
public class MiniDFSClusterManager { public class MiniDFSClusterManager {
private static final Log LOG = private static final Logger LOG =
LogFactory.getLog(MiniDFSClusterManager.class); LoggerFactory.getLogger(MiniDFSClusterManager.class);
private MiniDFSCluster dfs; private MiniDFSCluster dfs;
private String writeDetails; private String writeDetails;
@ -146,8 +146,8 @@ public void start() throws IOException, FileNotFoundException {
.build(); .build();
dfs.waitActive(); dfs.waitActive();
LOG.info("Started MiniDFSCluster -- namenode on port " LOG.info("Started MiniDFSCluster -- namenode on port {}",
+ dfs.getNameNodePort()); dfs.getNameNodePort());
if (writeConfig != null) { if (writeConfig != null) {
FileOutputStream fos = new FileOutputStream(new File(writeConfig)); FileOutputStream fos = new FileOutputStream(new File(writeConfig));
@ -180,7 +180,7 @@ private boolean parseArguments(String[] args) {
CommandLineParser parser = new GnuParser(); CommandLineParser parser = new GnuParser();
cli = parser.parse(options, args); cli = parser.parse(options, args);
} catch(ParseException e) { } catch(ParseException e) {
LOG.warn("options parsing failed: "+e.getMessage()); LOG.warn("options parsing failed", e);
new HelpFormatter().printHelp("...", options); new HelpFormatter().printHelp("...", options);
return false; return false;
} }
@ -192,7 +192,7 @@ private boolean parseArguments(String[] args) {
if (cli.getArgs().length > 0) { if (cli.getArgs().length > 0) {
for (String arg : cli.getArgs()) { for (String arg : cli.getArgs()) {
LOG.error("Unrecognized option: " + arg); LOG.error("Unrecognized option: {}", arg);
new HelpFormatter().printHelp("...", options); new HelpFormatter().printHelp("...", options);
return false; return false;
} }
@ -236,12 +236,12 @@ private void updateConfiguration(Configuration conf2, String[] keyvalues) {
conf2.set(keyval[0], keyval[1]); conf2.set(keyval[0], keyval[1]);
num_confs_updated++; num_confs_updated++;
} else { } else {
LOG.warn("Ignoring -D option " + prop); LOG.warn("Ignoring -D option {}", prop);
} }
} }
} }
LOG.info("Updated " + num_confs_updated + LOG.info("Updated {} configuration settings from command line.",
" configuration settings from command line."); num_confs_updated);
} }
/** /**
@ -254,8 +254,8 @@ private int intArgument(CommandLine cli, String argName, int defaultValue) {
return Integer.parseInt(o); return Integer.parseInt(o);
} }
} catch (NumberFormatException ex) { } catch (NumberFormatException ex) {
LOG.error("Couldn't parse value (" + o + ") for option " LOG.error("Couldn't parse value ({}) for option {}. " +
+ argName + ". Using default: " + defaultValue); "Using default: {}", o, argName, defaultValue);
} }
return defaultValue; return defaultValue;

View File

@ -45,8 +45,8 @@
import org.apache.hadoop.util.ToolRunner; import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.tools.rumen.JobStoryProducer; import org.apache.hadoop.tools.rumen.JobStoryProducer;
import org.apache.hadoop.tools.rumen.ZombieJobProducer; import org.apache.hadoop.tools.rumen.ZombieJobProducer;
import org.apache.commons.logging.Log; import org.slf4j.Logger;
import org.apache.commons.logging.LogFactory; import org.slf4j.LoggerFactory;
/** /**
* Driver class for the Gridmix3 benchmark. Gridmix accepts a timestamped * Driver class for the Gridmix3 benchmark. Gridmix accepts a timestamped
@ -58,7 +58,7 @@
*/ */
public class Gridmix extends Configured implements Tool { public class Gridmix extends Configured implements Tool {
public static final Log LOG = LogFactory.getLog(Gridmix.class); public static final Logger LOG = LoggerFactory.getLogger(Gridmix.class);
/** /**
* Output (scratch) directory for submitted jobs. Relative paths are * Output (scratch) directory for submitted jobs. Relative paths are
@ -184,8 +184,8 @@ protected int writeInputData(long genbytes, Path inputDir)
final Configuration conf = getConf(); final Configuration conf = getConf();
if (inputDir.getFileSystem(conf).exists(inputDir)) { if (inputDir.getFileSystem(conf).exists(inputDir)) {
LOG.error("Gridmix input data directory " + inputDir LOG.error("Gridmix input data directory {} already exists " +
+ " already exists when -generate option is used.\n"); "when -generate option is used.", inputDir);
return STARTUP_FAILED_ERROR; return STARTUP_FAILED_ERROR;
} }
@ -193,13 +193,13 @@ protected int writeInputData(long genbytes, Path inputDir)
CompressionEmulationUtil.setupDataGeneratorConfig(conf); CompressionEmulationUtil.setupDataGeneratorConfig(conf);
final GenerateData genData = new GenerateData(conf, inputDir, genbytes); final GenerateData genData = new GenerateData(conf, inputDir, genbytes);
LOG.info("Generating " + StringUtils.humanReadableInt(genbytes) + LOG.info("Generating {} of test data...",
" of test data..."); StringUtils.TraditionalBinaryPrefix.long2String(genbytes, "", 1));
launchGridmixJob(genData); launchGridmixJob(genData);
FsShell shell = new FsShell(conf); FsShell shell = new FsShell(conf);
try { try {
LOG.info("Changing the permissions for inputPath " + inputDir.toString()); LOG.info("Changing the permissions for inputPath {}", inputDir);
shell.run(new String[] {"-chmod","-R","777", inputDir.toString()}); shell.run(new String[] {"-chmod","-R","777", inputDir.toString()});
} catch (Exception e) { } catch (Exception e) {
LOG.error("Couldnt change the file permissions " , e); LOG.error("Couldnt change the file permissions " , e);
@ -528,9 +528,7 @@ int start(Configuration conf, String traceIn, Path ioPath, long genbytes,
statistics.start(); statistics.start();
} catch (Throwable e) { } catch (Throwable e) {
LOG.error("Startup failed. " + e.toString() + "\n"); LOG.error("Startup failed. " + e.toString() + "\n");
if (LOG.isDebugEnabled()) { LOG.debug("Startup failed", e);
e.printStackTrace();
}
if (factory != null) factory.abort(); // abort pipeline if (factory != null) factory.abort(); // abort pipeline
exitCode = STARTUP_FAILED_ERROR; exitCode = STARTUP_FAILED_ERROR;
} finally { } finally {
@ -561,7 +559,7 @@ int start(Configuration conf, String traceIn, Path ioPath, long genbytes,
summarizer.finalize(factory, traceIn, genbytes, userResolver, stats, summarizer.finalize(factory, traceIn, genbytes, userResolver, stats,
conf); conf);
} }
IOUtils.cleanup(LOG, trace); IOUtils.cleanupWithLogger(LOG, trace);
} }
return exitCode; return exitCode;
} }

View File

@ -18,12 +18,12 @@
package org.apache.hadoop.fs.swift.http; package org.apache.hadoop.fs.swift.http;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.swift.exceptions.SwiftConnectionClosedException; import org.apache.hadoop.fs.swift.exceptions.SwiftConnectionClosedException;
import org.apache.hadoop.fs.swift.util.SwiftUtils; import org.apache.hadoop.fs.swift.util.SwiftUtils;
import org.apache.http.HttpResponse; import org.apache.http.HttpResponse;
import org.apache.http.client.methods.HttpRequestBase; import org.apache.http.client.methods.HttpRequestBase;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.EOFException; import java.io.EOFException;
@ -44,8 +44,8 @@
*/ */
public class HttpInputStreamWithRelease extends InputStream { public class HttpInputStreamWithRelease extends InputStream {
private static final Log LOG = private static final Logger LOG =
LogFactory.getLog(HttpInputStreamWithRelease.class); LoggerFactory.getLogger(HttpInputStreamWithRelease.class);
private final URI uri; private final URI uri;
private HttpRequestBase req; private HttpRequestBase req;
private HttpResponse resp; private HttpResponse resp;
@ -100,9 +100,7 @@ private synchronized boolean release(String reason, Exception ex) throws
if (!released) { if (!released) {
reasonClosed = reason; reasonClosed = reason;
try { try {
if (LOG.isDebugEnabled()) { LOG.debug("Releasing connection to {}: {}", uri, reason, ex);
LOG.debug("Releasing connection to " + uri + ": " + reason, ex);
}
if (req != null) { if (req != null) {
if (!dataConsumed) { if (!dataConsumed) {
req.abort(); req.abort();
@ -137,7 +135,7 @@ private IOException releaseAndRethrow(String operation, IOException ex) {
try { try {
release(operation, ex); release(operation, ex);
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.debug("Exception during release: " + operation + " - " + ioe, ioe); LOG.debug("Exception during release: {}", operation, ioe);
//make this the exception if there was none before //make this the exception if there was none before
if (ex == null) { if (ex == null) {
ex = ioe; ex = ioe;
@ -173,9 +171,7 @@ public int read() throws IOException {
try { try {
read = inStream.read(); read = inStream.read();
} catch (EOFException e) { } catch (EOFException e) {
if (LOG.isDebugEnabled()) { LOG.debug("EOF exception", e);
LOG.debug("EOF exception " + e, e);
}
read = -1; read = -1;
} catch (IOException e) { } catch (IOException e) {
throw releaseAndRethrow("read()", e); throw releaseAndRethrow("read()", e);
@ -200,9 +196,7 @@ public int read(byte[] b, int off, int len) throws IOException {
try { try {
read = inStream.read(b, off, len); read = inStream.read(b, off, len);
} catch (EOFException e) { } catch (EOFException e) {
if (LOG.isDebugEnabled()) { LOG.debug("EOF exception", e);
LOG.debug("EOF exception " + e, e);
}
read = -1; read = -1;
} catch (IOException e) { } catch (IOException e) {
throw releaseAndRethrow("read(b, off, " + len + ")", e); throw releaseAndRethrow("read(b, off, " + len + ")", e);
@ -222,13 +216,12 @@ public int read(byte[] b, int off, int len) throws IOException {
protected void finalize() { protected void finalize() {
try { try {
if (release("finalize()", constructionStack)) { if (release("finalize()", constructionStack)) {
LOG.warn("input stream of " + uri LOG.warn("input stream of {}" +
+ " not closed properly -cleaned up in finalize()"); " not closed properly -cleaned up in finalize()", uri);
} }
} catch (Exception e) { } catch (Exception e) {
//swallow anything that failed here //swallow anything that failed here
LOG.warn("Exception while releasing " + uri + "in finalizer", LOG.warn("Exception while releasing {} in finalizer", uri, e);
e);
} }
} }