HBASE-9282 Minor logging cleanup; shorten logs, remove redundant info

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1516267 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-08-21 19:25:21 +00:00
parent 627493d222
commit 41db4e539f
8 changed files with 34 additions and 43 deletions

View File

@ -174,7 +174,7 @@ class AsyncProcess<CResult> {
}
public AsyncProcess(HConnection hc, TableName tableName, ExecutorService pool,
AsyncProcessCallback<CResult> callback, Configuration conf,
AsyncProcessCallback<CResult> callback, Configuration conf,
RpcRetryingCallerFactory rpcCaller) {
this.hConnection = hc;
this.tableName = tableName;
@ -534,9 +534,9 @@ class AsyncProcess<CResult> {
}
if (toReplay.isEmpty()) {
LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for all (" +
initialActions.size() + ") operations on server " + location.getServerName() +
" NOT resubmitting, tableName=" + tableName + ", location=" + location);
LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for all " +
initialActions.size() + "ops, NOT resubmitting,
tableName=" + tableName + ", location=" + location);
} else {
submit(initialActions, toReplay, numAttempt, true, errorsByServer);
}
@ -557,9 +557,8 @@ class AsyncProcess<CResult> {
HConnectionManager.ServerErrorTracker errorsByServer) {
if (responses == null) {
LOG.info("Attempt #" + numAttempt + "/" + numTries + " failed for all operations" +
" on server " + location.getServerName() + " , trying to resubmit," +
" tableName=" + tableName + ", location=" + location);
LOG.info("Attempt #" + numAttempt + "/" + numTries + " failed all ops, trying resubmit," +
" tableName=" + tableName + ", location=" + location);
resubmitAll(initialActions, rsActions, location, numAttempt + 1, null, errorsByServer);
return;
}
@ -618,11 +617,10 @@ class AsyncProcess<CResult> {
if (numAttempt > 3 && LOG.isDebugEnabled()) {
// We use this value to have some logs when we have multiple failures, but not too many
// logs as errors are to be expected wehn region moves, split and so on
LOG.debug("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
" operations on server " + location.getServerName() + ", resubmitting " +
toReplay.size() + ", tableName=" + tableName + ", location=" +
location + ", last exception was: " + throwable +
" - sleeping " + backOffTime + " ms.");
LOG.debug("Attempt #" + numAttempt + "/" + numTries + " failed " + failureCount +
" ops , resubmitting " + toReplay.size() + ", tableName=" + tableName + ", location=" +
location + ", last exception was: " + throwable.getMessage() +
" - sleeping " + backOffTime + " ms.");
}
try {
Thread.sleep(backOffTime);
@ -636,7 +634,7 @@ class AsyncProcess<CResult> {
submit(initialActions, toReplay, numAttempt + 1, true, errorsByServer);
} else if (failureCount != 0) {
LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
" operations on server " + location.getServerName() + " NOT resubmitting." +
" ops on " + location.getServerName() + " NOT resubmitting." +
", tableName=" + tableName + ", location=" + location);
}
}

View File

@ -202,7 +202,7 @@ public class HTable implements HTableInterface {
// we only create as many Runnables as there are region servers. It means
// it also scales when new region servers are added.
this.pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, TimeUnit.SECONDS,
new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("hbase-table"));
new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
((ThreadPoolExecutor) this.pool).allowCoreThreadTimeOut(true);
this.finishSetup();

View File

@ -147,7 +147,7 @@ public class ZKAssign {
public static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
ServerName serverName, final EventType event)
throws KeeperException, KeeperException.NodeExistsException {
LOG.debug(zkw.prefix("Creating unassigned node for " +
LOG.debug(zkw.prefix("Creating unassigned node " +
region.getEncodedName() + " in OFFLINE state"));
RegionTransition rt =
RegionTransition.createRegionTransition(event, region.getRegionName(), serverName);
@ -174,7 +174,7 @@ public class ZKAssign {
HRegionInfo region, ServerName serverName,
final AsyncCallback.StringCallback cb, final Object ctx)
throws KeeperException {
LOG.debug(zkw.prefix("Async create of unassigned node for " +
LOG.debug(zkw.prefix("Async create of unassigned node " +
region.getEncodedName() + " with OFFLINE state"));
RegionTransition rt =
RegionTransition.createRegionTransition(
@ -205,7 +205,7 @@ public class ZKAssign {
*/
public static int createOrForceNodeOffline(ZooKeeperWatcher zkw,
HRegionInfo region, ServerName serverName) throws KeeperException {
LOG.debug(zkw.prefix("Creating (or updating) unassigned node for " +
LOG.debug(zkw.prefix("Creating (or updating) unassigned node " +
region.getEncodedName() + " with OFFLINE state"));
RegionTransition rt = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_OFFLINE,
region.getRegionName(), serverName, HConstants.EMPTY_BYTE_ARRAY);
@ -403,7 +403,7 @@ public class ZKAssign {
EventType expectedState, int expectedVersion)
throws KeeperException, KeeperException.NoNodeException {
LOG.debug(zkw.prefix("Deleting existing unassigned " +
"node for " + encodedRegionName + " that is in expected state " + expectedState));
"node " + encodedRegionName + " in expected state " + expectedState));
String node = getNodeName(zkw, encodedRegionName);
zkw.sync(node);
Stat stat = new Stat();
@ -431,7 +431,7 @@ public class ZKAssign {
" state but after verifying state, we got a version mismatch"));
return false;
}
LOG.debug(zkw.prefix("Successfully deleted unassigned node for region " +
LOG.debug(zkw.prefix("Deleted unassigned node " +
encodedRegionName + " in expected state " + expectedState));
return true;
}
@ -478,7 +478,7 @@ public class ZKAssign {
public static int createNodeClosing(ZooKeeperWatcher zkw, HRegionInfo region,
ServerName serverName)
throws KeeperException, KeeperException.NodeExistsException {
LOG.debug(zkw.prefix("Creating unassigned node for " +
LOG.debug(zkw.prefix("Creating unassigned node " +
region.getEncodedName() + " in a CLOSING state"));
RegionTransition rt = RegionTransition.createRegionTransition(EventType.M_ZK_REGION_CLOSING,
region.getRegionName(), serverName, HConstants.EMPTY_BYTE_ARRAY);
@ -595,7 +595,7 @@ public class ZKAssign {
String encoded = region.getEncodedName();
if(LOG.isDebugEnabled()) {
LOG.debug(zkw.prefix("Attempting to retransition the opening state of node " +
LOG.debug(zkw.prefix("Attempting to retransition opening state of node " +
HRegionInfo.prettyPrint(encoded)));
}
@ -648,7 +648,7 @@ public class ZKAssign {
return -1;
}
if(LOG.isDebugEnabled()) {
LOG.debug(zkw.prefix("Successfully retransition the opening state of node " + encoded));
LOG.debug(zkw.prefix("Retransition opening state of node " + encoded));
}
return stat.getVersion() + 1;
} catch (KeeperException.NoNodeException nne) {
@ -858,7 +858,7 @@ public class ZKAssign {
return -1;
}
if(LOG.isDebugEnabled()) {
LOG.debug(zkw.prefix("Successfully transitioned node " + encoded +
LOG.debug(zkw.prefix("Transitioned node " + encoded +
" from " + beginState + " to " + endState));
}
return stat.getVersion() + 1;

View File

@ -203,8 +203,7 @@ public class Threads {
@Override
public Thread newThread(Runnable r) {
final String name = prefix + "-pool-" + poolNumber + "-thread-"
+ threadNumber.getAndIncrement();
final String name = prefix + "-pool" + poolNumber + "-t" + threadNumber.getAndIncrement();
return new Thread(group, r, name);
}
};

View File

@ -136,7 +136,7 @@ public class LruBlockCache implements BlockCache, HeapSize {
private final ScheduledExecutorService scheduleThreadPool =
Executors.newScheduledThreadPool(1,
new ThreadFactoryBuilder()
.setNameFormat("LRU Statistics #%d")
.setNameFormat("LruStats #%d")
.setDaemon(true)
.build());
@ -725,8 +725,7 @@ public class LruBlockCache implements BlockCache, HeapSize {
// Log size
long totalSize = heapSize();
long freeSize = maxSize - totalSize;
LruBlockCache.LOG.debug("Stats: " +
"total=" + StringUtils.byteDesc(totalSize) + ", " +
LruBlockCache.LOG.debug("Total=" + StringUtils.byteDesc(totalSize) + ", " +
"free=" + StringUtils.byteDesc(freeSize) + ", " +
"max=" + StringUtils.byteDesc(this.maxSize) + ", " +
"blocks=" + size() +", " +

View File

@ -1712,8 +1712,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV, openSeqNum);
}
LOG.info("Done with post open deploy task for region=" +
r.getRegionNameAsString());
LOG.info("Finished post open deploy task for " + r.getRegionNameAsString());
}
@ -3628,10 +3627,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
}
requestCount.increment();
LOG.info("Received close region: " + encodedRegionName +
"Transitioning in ZK: " + (zk ? "yes" : "no") +
". Version of ZK closing node:" + versionOfClosingNode +
". Destination server:" + sn);
LOG.info("Close " + encodedRegionName + ", via zk=" + (zk ? "yes" : "no") +
", znode version=" + versionOfClosingNode + ", on " + sn);
boolean closed = closeRegion(encodedRegionName, false, zk, versionOfClosingNode, sn);
CloseRegionResponse.Builder builder = CloseRegionResponse.newBuilder().setClosed(closed);

View File

@ -165,16 +165,15 @@ public class CloseRegionHandler extends EventHandler {
if (this.zk) {
if (setClosedState(this.expectedVersion, region)) {
LOG.debug("set region closed state in zk successfully for region " +
name + " sn name: " + this.server.getServerName());
LOG.debug("Set closed state in zk for " + name + " on " + this.server.getServerName());
} else {
LOG.debug("set region closed state in zk unsuccessfully for region " +
name + " sn name: " + this.server.getServerName());
LOG.debug("Set closed state in zk UNSUCCESSFUL for " + name + " on " +
this.server.getServerName());
}
}
// Done! Region is closed on this RS
LOG.debug("Closed region " + region.getRegionNameAsString());
LOG.debug("Closed " + region.getRegionNameAsString());
} finally {
this.rsServices.getRegionsInTransitionInRS().
remove(this.regionInfo.getEncodedNameAsBytes());

View File

@ -183,7 +183,7 @@ public class OpenRegionHandler extends EventHandler {
openSuccessful = true;
// Done! Successful region open
LOG.debug("Opened " + regionName + " on server:" +
LOG.debug("Opened " + regionName + " on " +
this.server.getServerName());
@ -212,7 +212,7 @@ public class OpenRegionHandler extends EventHandler {
+ " on region=" + regionName + ". It can be a critical error, as a region that"
+ " should be closed is now opened.");
}
}
}
}
}
@ -370,8 +370,7 @@ public class OpenRegionHandler extends EventHandler {
"so now unassigning -- closing region on server: " +
this.server.getServerName());
} else {
LOG.debug("region transitioned to opened in zookeeper: " +
r.getRegionInfo() + ", server: " + this.server.getServerName());
LOG.debug("Transitioned " + r.getEncodedName() + " to OPENED in zk on " + this.server.getServerName());
result = true;
}
} catch (KeeperException e) {