HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)

This commit is contained in:
Uma Mahesh 2015-12-11 17:57:35 -08:00
parent 576b569b6c
commit 796a676d18
8 changed files with 89 additions and 134 deletions

View File

@ -1737,6 +1737,8 @@ Release 2.8.0 - UNRELEASED
HDFS-9472. concat() API does not give proper exception messages on ./reserved
relative path (Rakesh R via umamahesh)
HDFS-9528. Cleanup namenode audit/log/exception messages. (szetszwo via umamahesh)
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

View File

@ -1072,27 +1072,17 @@ public class BlockManager implements BlockStatsMXBean {
public void verifyReplication(String src,
short replication,
String clientName) throws IOException {
String err = null;
if (replication > maxReplication) {
err = " exceeds maximum of " + maxReplication;
} else if (replication < minReplication) {
err = " is less than the required minimum of " + minReplication;
}
if (replication < minReplication || replication > maxReplication) {
StringBuilder msg = new StringBuilder("Requested replication factor of ");
msg.append(replication);
if (replication > maxReplication) {
msg.append(" exceeds maximum of ");
msg.append(maxReplication);
} else {
msg.append(" is less than the required minimum of ");
msg.append(minReplication);
}
msg.append(" for ").append(src);
if (clientName != null) {
msg.append(" from ").append(clientName);
}
throw new IOException(msg.toString());
if (err != null) {
throw new IOException("Requested replication factor of " + replication
+ err + " for " + src
+ (clientName == null? "": ", clientName=" + clientName));
}
}
@ -1228,8 +1218,7 @@ public class BlockManager implements BlockStatsMXBean {
}
}
if (datanodes.length() != 0) {
blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock,
datanodes.toString());
blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock, datanodes);
}
}
@ -2850,7 +2839,8 @@ public class BlockManager implements BlockStatsMXBean {
if (result == AddBlockResult.ADDED) {
curReplicaDelta = 1;
if (logEveryBlock) {
logAddStoredBlock(storedBlock, node);
blockLog.debug("BLOCK* addStoredBlock: {} is added to {} (size={})",
node, storedBlock, storedBlock.getNumBytes());
}
} else if (result == AddBlockResult.REPLACED) {
curReplicaDelta = 0;
@ -2923,21 +2913,6 @@ public class BlockManager implements BlockStatsMXBean {
return storedBlock;
}
private void logAddStoredBlock(BlockInfo storedBlock,
DatanodeDescriptor node) {
if (!blockLog.isDebugEnabled()) {
return;
}
StringBuilder sb = new StringBuilder(500);
sb.append("BLOCK* addStoredBlock: blockMap updated: ")
.append(node)
.append(" is added to ");
storedBlock.appendStringTo(sb);
sb.append(" size " )
.append(storedBlock.getNumBytes());
blockLog.debug(sb.toString());
}
/**
* Invalidate corrupt replicas.
* <p>

View File

@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.util.Time.monotonicNow;
import java.util.AbstractList;
import java.util.Iterator;
import java.util.LinkedList;
@ -26,13 +29,9 @@ import java.util.Queue;
import java.util.TreeMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -44,8 +43,9 @@ import org.apache.hadoop.util.ChunkedArrayList;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.util.Time.monotonicNow;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* Manages datanode decommissioning. A background monitor thread
@ -441,17 +441,10 @@ public class DecommissionManager {
LOG.debug("Node {} is sufficiently replicated and healthy, "
+ "marked as decommissioned.", dn);
} else {
if (LOG.isDebugEnabled()) {
StringBuilder b = new StringBuilder("Node {} ");
if (isHealthy) {
b.append("is ");
} else {
b.append("isn't ");
}
b.append("healthy and still needs to replicate {} more blocks," +
" decommissioning is still in progress.");
LOG.debug(b.toString(), dn, blocks.size());
}
LOG.debug("Node {} {} healthy."
+ " It needs to replicate {} more blocks."
+ " Decommissioning is still in progress.",
dn, isHealthy? "is": "isn't", blocks.size());
}
} else {
LOG.debug("Node {} still has {} blocks to replicate "
@ -587,8 +580,7 @@ public class DecommissionManager {
}
@VisibleForTesting
void runMonitor() throws ExecutionException, InterruptedException {
Future f = executor.submit(monitor);
f.get();
void runMonitorForTest() throws ExecutionException, InterruptedException {
executor.submit(monitor).get();
}
}

View File

@ -317,19 +317,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
NameNodeMXBean {
public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
private static final ThreadLocal<StringBuilder> auditBuffer =
new ThreadLocal<StringBuilder>() {
@Override
protected StringBuilder initialValue() {
return new StringBuilder();
}
};
private final BlockIdManager blockIdManager;
@VisibleForTesting
public boolean isAuditEnabled() {
return !isDefaultAuditLogger || auditLog.isInfoEnabled();
boolean isAuditEnabled() {
return (!isDefaultAuditLogger || auditLog.isInfoEnabled())
&& !auditLoggers.isEmpty();
}
private void logAuditEvent(boolean succeeded, String cmd, String src)
@ -358,14 +350,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
stat.getAccessTime(), stat.getPermission(), stat.getOwner(),
stat.getGroup(), symlink, path);
}
final String ugiStr = ugi.toString();
for (AuditLogger logger : auditLoggers) {
if (logger instanceof HdfsAuditLogger) {
HdfsAuditLogger hdfsLogger = (HdfsAuditLogger) logger;
hdfsLogger.logAuditEvent(succeeded, ugi.toString(), addr, cmd, src, dst,
hdfsLogger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst,
status, CallerContext.getCurrent(), ugi, dtSecretManager);
} else {
logger.logAuditEvent(succeeded, ugi.toString(), addr,
cmd, src, dst, status);
logger.logAuditEvent(succeeded, ugiStr, addr, cmd, src, dst, status);
}
}
}
@ -616,15 +608,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
// sharedEditsDirs.
if (!editsDirs.contains(u) &&
!sharedEditsDirs.contains(u)) {
throw new IllegalArgumentException(
"Required edits directory " + u.toString() + " not present in " +
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + ". " +
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" +
editsDirs.toString() + "; " +
DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY + "=" +
requiredEditsDirs.toString() + ". " +
DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY + "=" +
sharedEditsDirs.toString() + ".");
throw new IllegalArgumentException("Required edits directory " + u
+ " not found: "
+ DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY + "=" + editsDirs + "; "
+ DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY
+ "=" + requiredEditsDirs + "; "
+ DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY
+ "=" + sharedEditsDirs);
}
}
@ -698,11 +688,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
FSNamesystem(Configuration conf, FSImage fsImage, boolean ignoreRetryCache)
throws IOException {
provider = DFSUtil.createKeyProviderCryptoExtension(conf);
if (provider == null) {
LOG.info("No KeyProvider found.");
} else {
LOG.info("Found KeyProvider: " + provider.toString());
}
LOG.info("KeyProvider: " + provider);
if (conf.getBoolean(DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY,
DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT)) {
LOG.info("Enabling async auditlog");
@ -2077,11 +2063,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
.append(", clientMachine=").append(clientMachine)
.append(", createParent=").append(createParent)
.append(", replication=").append(replication)
.append(", createFlag=").append(flag.toString())
.append(", createFlag=").append(flag)
.append(", blockSize=").append(blockSize)
.append(", supportedVersions=")
.append(supportedVersions == null ? null : Arrays.toString
(supportedVersions));
.append(Arrays.toString(supportedVersions));
NameNode.stateChangeLog.debug(builder.toString());
}
if (!DFSUtil.isValidName(src)) {
@ -2513,43 +2498,40 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
getEditLog().logSync();
}
INodeFile checkLease(
String src, String holder, INode inode, long fileId)
private String leaseExceptionString(String src, long fileId, String holder) {
final Lease lease = leaseManager.getLease(holder);
return src + " (inode " + fileId + ") " + (lease != null? lease.toString()
: "Holder " + holder + " does not have any open files.");
}
INodeFile checkLease(String src, String holder, INode inode, long fileId)
throws LeaseExpiredException, FileNotFoundException {
assert hasReadLock();
final String ident = src + " (inode " + fileId + ")";
if (inode == null) {
Lease lease = leaseManager.getLease(holder);
throw new FileNotFoundException(
"No lease on " + ident + ": File does not exist. "
+ (lease != null ? lease.toString()
: "Holder " + holder + " does not have any open files."));
throw new FileNotFoundException("File does not exist: "
+ leaseExceptionString(src, fileId, holder));
}
if (!inode.isFile()) {
Lease lease = leaseManager.getLease(holder);
throw new LeaseExpiredException(
"No lease on " + ident + ": INode is not a regular file. "
+ (lease != null ? lease.toString()
: "Holder " + holder + " does not have any open files."));
throw new LeaseExpiredException("INode is not a regular file: "
+ leaseExceptionString(src, fileId, holder));
}
final INodeFile file = inode.asFile();
if (!file.isUnderConstruction()) {
Lease lease = leaseManager.getLease(holder);
throw new LeaseExpiredException(
"No lease on " + ident + ": File is not open for writing. "
+ (lease != null ? lease.toString()
: "Holder " + holder + " does not have any open files."));
throw new LeaseExpiredException("File is not open for writing: "
+ leaseExceptionString(src, fileId, holder));
}
// No further modification is allowed on a deleted file.
// A file is considered deleted, if it is not in the inodeMap or is marked
// as deleted in the snapshot feature.
if (isFileDeleted(file)) {
throw new FileNotFoundException(src);
throw new FileNotFoundException("File is deleted: "
+ leaseExceptionString(src, fileId, holder));
}
String clientName = file.getFileUnderConstructionFeature().getClientName();
if (holder != null && !clientName.equals(holder)) {
throw new LeaseExpiredException("Lease mismatch on " + ident +
" owned by " + clientName + " but is accessed by " + holder);
final String owner = file.getFileUnderConstructionFeature().getClientName();
if (holder != null && !owner.equals(holder)) {
throw new LeaseExpiredException("Client (=" + holder
+ ") is not the lease owner (=" + owner + ": "
+ leaseExceptionString(src, fileId, holder));
}
return file;
}
@ -6188,7 +6170,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
if (success) {
getEditLog().logSync();
}
String idStr = "{id: " + directive.getId().toString() + "}";
final String idStr = "{id: " + directive.getId() + "}";
logAuditEvent(success, "modifyCacheDirective", idStr,
directive.toString(), null);
}
@ -6662,6 +6644,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
*/
@VisibleForTesting
static class DefaultAuditLogger extends HdfsAuditLogger {
private static final ThreadLocal<StringBuilder> STRING_BUILDER =
new ThreadLocal<StringBuilder>() {
@Override
protected StringBuilder initialValue() {
return new StringBuilder();
}
};
private boolean isCallerContextEnabled;
private int callerContextMaxLen;
private int callerSignatureMaxLen;
@ -6696,7 +6686,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
if (auditLog.isDebugEnabled() ||
(auditLog.isInfoEnabled() && !debugCmdSet.contains(cmd))) {
final StringBuilder sb = auditBuffer.get();
final StringBuilder sb = STRING_BUILDER.get();
sb.setLength(0);
sb.append("allowed=").append(succeeded).append("\t");
sb.append("ugi=").append(userName).append("\t");

View File

@ -320,7 +320,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
sb.append("FSCK started by " +
UserGroupInformation.getCurrentUser() + " from " +
remoteAddress + " at " + new Date());
out.println(sb.toString());
out.println(sb);
sb.append(" for blockIds: \n");
for (String blk: blocks) {
if(blk == null || !blk.contains(Block.BLOCK_FILE_PREFIX)) {
@ -331,7 +331,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
blockIdCK(blk);
sb.append(blk + "\n");
}
LOG.info(sb.toString());
LOG.info(sb);
namenode.getNamesystem().logFsckEvent("/", remoteAddress);
out.flush();
return;
@ -383,7 +383,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
out.println(ecRes);
if (this.showStoragePolcies) {
out.print(storageTypeSummary.toString());
out.print(storageTypeSummary);
}
out.println("FSCK ended at " + new Date() + " in "
@ -725,7 +725,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
// might still be accessible as the block might be incorrectly marked as
// corrupted by client machines.
report.append(" MISSING!");
res.addMissing(block.toString(), block.getNumBytes());
res.addMissing(blkName, block.getNumBytes());
missing++;
missize += block.getNumBytes();
} else {
@ -793,7 +793,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
out.print(" OK\n");
}
if (showBlocks) {
out.print(report.toString() + "\n");
out.print(report + "\n");
}
}
}

View File

@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
@ -198,14 +199,12 @@ public class TestFileAppend4 {
t.join();
LOG.info("Close finished.");
// We expect that close will get a "File is not open"
// error.
// We expect that close will get a "File is not open" error.
Throwable thrownByClose = err.get();
assertNotNull(thrownByClose);
assertTrue(thrownByClose instanceof IOException);
if (!thrownByClose.getMessage().contains(
"No lease on /testRecoverFinalized"))
throw thrownByClose;
assertTrue(thrownByClose instanceof LeaseExpiredException);
GenericTestUtils.assertExceptionContains("File is not open for writing",
thrownByClose);
} finally {
cluster.shutdown();
}
@ -281,10 +280,9 @@ public class TestFileAppend4 {
// error.
Throwable thrownByClose = err.get();
assertNotNull(thrownByClose);
assertTrue(thrownByClose instanceof IOException);
if (!thrownByClose.getMessage().contains(
"Lease mismatch"))
throw thrownByClose;
assertTrue(thrownByClose instanceof LeaseExpiredException);
GenericTestUtils.assertExceptionContains("not the lease owner",
thrownByClose);
// The appender should be able to close properly
appenderStream.close();

View File

@ -165,7 +165,7 @@ public class TestFileCreation {
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
try {
FsServerDefaults serverDefaults = fs.getServerDefaults();
FsServerDefaults serverDefaults = fs.getServerDefaults(new Path("/"));
assertEquals(DFS_BLOCK_SIZE_DEFAULT, serverDefaults.getBlockSize());
assertEquals(DFS_BYTES_PER_CHECKSUM_DEFAULT, serverDefaults.getBytesPerChecksum());
assertEquals(DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT, serverDefaults.getWritePacketSize());
@ -413,8 +413,7 @@ public class TestFileCreation {
stm1.close();
fail("Should have exception closing stm1 since it was deleted");
} catch (IOException ioe) {
GenericTestUtils.assertExceptionContains("No lease on /testfile", ioe);
GenericTestUtils.assertExceptionContains("File does not exist.", ioe);
GenericTestUtils.assertExceptionContains("File does not exist", ioe);
}
} finally {
@ -805,7 +804,6 @@ public class TestFileCreation {
public static void testFileCreationNonRecursive(FileSystem fs) throws IOException {
final Path path = new Path("/" + Time.now()
+ "-testFileCreationNonRecursive");
FSDataOutputStream out = null;
IOException expectedException = null;
final String nonExistDir = "/non-exist-" + Time.now();

View File

@ -306,7 +306,7 @@ public class BlockManagerTestUtil {
*/
public static void recheckDecommissionState(DatanodeManager dm)
throws ExecutionException, InterruptedException {
dm.getDecomManager().runMonitor();
dm.getDecomManager().runMonitorForTest();
}
/**