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:
parent
627493d222
commit
41db4e539f
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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() +", " +
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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) {
|
||||
|
|
Loading…
Reference in New Issue