HBASE-10796 Set default log level as INFO
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1582532 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2fa73bce2b
commit
325fcaf4e1
|
@ -74,7 +74,7 @@ log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%
|
|||
|
||||
log4j.logger.org.apache.zookeeper=INFO
|
||||
#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=INFO
|
||||
# Make these two classes INFO-level. Make them DEBUG to see more zk debug.
|
||||
log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
|
||||
log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
|
||||
|
|
|
@ -94,7 +94,7 @@ public class ConnectionUtils {
|
|||
int serversideMultiplier = c.getInt("hbase.client.serverside.retries.multiplier", 10);
|
||||
int retries = hcRetries * serversideMultiplier;
|
||||
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
|
||||
log.debug(sn + " HConnection server-to-server retries=" + retries);
|
||||
log.info(sn + " server-side HConnection retries=" + retries);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -163,7 +163,7 @@ public class RecoverableZooKeeper {
|
|||
"previous attempt succeeded.");
|
||||
return;
|
||||
}
|
||||
LOG.warn("Node " + path + " already deleted, retry=" + isRetry);
|
||||
LOG.info("Node " + path + " already deleted, retry=" + isRetry);
|
||||
throw e;
|
||||
|
||||
case CONNECTIONLOSS:
|
||||
|
@ -505,8 +505,7 @@ public class RecoverableZooKeeper {
|
|||
Bytes.toStringBinary(data));
|
||||
throw e;
|
||||
}
|
||||
LOG.info("Node " + path + " already exists and this is not a " +
|
||||
"retry");
|
||||
LOG.info("Node " + path + " already exists");
|
||||
throw e;
|
||||
|
||||
case CONNECTIONLOSS:
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.zookeeper.KeeperException;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
|
||||
|
||||
static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);
|
||||
/** Path of node being tracked */
|
||||
protected final String node;
|
||||
|
@ -144,7 +143,7 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
|
|||
|
||||
// It did not exists, and now it does.
|
||||
if (nodeExistsChecked){
|
||||
LOG.info("Node "+node+" now exists, resetting a watcher");
|
||||
LOG.debug("Node " + node + " now exists, resetting a watcher");
|
||||
try {
|
||||
// This does not create a watch if the node does not exists
|
||||
this.data = ZKUtil.getDataAndWatch(watcher, node);
|
||||
|
|
|
@ -65,7 +65,7 @@ public enum ChecksumType {
|
|||
// check if hadoop library is available
|
||||
try {
|
||||
ctor = ChecksumFactory.newConstructor(PURECRC32);
|
||||
LOG.info("Checksum using " + PURECRC32);
|
||||
LOG.debug("Checksum using " + PURECRC32);
|
||||
} catch (Exception e) {
|
||||
LOG.trace(PURECRC32 + " not available.");
|
||||
}
|
||||
|
@ -74,7 +74,7 @@ public enum ChecksumType {
|
|||
// This is available on all JVMs.
|
||||
if (ctor == null) {
|
||||
ctor = ChecksumFactory.newConstructor(JDKCRC);
|
||||
LOG.info("Checksum can use " + JDKCRC);
|
||||
LOG.debug("Checksum can use " + JDKCRC);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.trace(JDKCRC + " not available.");
|
||||
|
@ -108,7 +108,7 @@ public enum ChecksumType {
|
|||
LOG = LogFactory.getLog(ChecksumType.class);
|
||||
try {
|
||||
ctor = ChecksumFactory.newConstructor(PURECRC32C);
|
||||
LOG.info("Checksum can use " + PURECRC32C);
|
||||
LOG.debug("Checksum can use " + PURECRC32C);
|
||||
} catch (Exception e) {
|
||||
LOG.trace(PURECRC32C + " not available.");
|
||||
}
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
|
||||
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.DirectMemoryUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
|
|
|
@ -1046,7 +1046,7 @@ public class RpcServer implements RpcServerInterface {
|
|||
}
|
||||
} finally {
|
||||
if (error && call != null) {
|
||||
LOG.warn(getName() + call.toShortString() + ": output error");
|
||||
LOG.warn(getName() + call.toShortString() + ": output error -- closing");
|
||||
done = true; // error. no more data for this channel.
|
||||
closeConnection(call.connection);
|
||||
}
|
||||
|
|
|
@ -1460,7 +1460,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
if (regionCount == 0) {
|
||||
return true;
|
||||
}
|
||||
LOG.debug("Assigning " + regionCount + " region(s) to " + destination.toString());
|
||||
LOG.info("Assigning " + regionCount + " region(s) to " + destination.toString());
|
||||
Set<String> encodedNames = new HashSet<String>(regionCount);
|
||||
for (HRegionInfo region : regions) {
|
||||
encodedNames.add(region.getEncodedName());
|
||||
|
@ -1511,8 +1511,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
for (int oldCounter = 0; !server.isStopped();) {
|
||||
int count = counter.get();
|
||||
if (oldCounter != count) {
|
||||
LOG.info(destination.toString() + " unassigned znodes=" + count +
|
||||
" of total=" + total);
|
||||
LOG.debug(destination.toString() + " unassigned znodes=" + count +
|
||||
" of total=" + total + "; oldCounter=" + oldCounter);
|
||||
oldCounter = count;
|
||||
}
|
||||
if (count >= total) break;
|
||||
|
|
|
@ -115,7 +115,7 @@ public class DeadServer {
|
|||
}
|
||||
|
||||
public synchronized void finish(ServerName sn) {
|
||||
LOG.debug("Finished processing " + sn);
|
||||
LOG.info("Finished processing " + sn);
|
||||
this.numProcessing--;
|
||||
}
|
||||
|
||||
|
|
|
@ -174,7 +174,7 @@ public class MasterRpcServices extends RSRpcServices
|
|||
private RegionServerStartupResponse.Builder createConfigurationSubset() {
|
||||
RegionServerStartupResponse.Builder resp = addConfig(
|
||||
RegionServerStartupResponse.newBuilder(), HConstants.HBASE_DIR);
|
||||
return addConfig(resp, "fs.default.name");
|
||||
return addConfig(resp, "fs.defaultFS");
|
||||
}
|
||||
|
||||
private RegionServerStartupResponse.Builder addConfig(
|
||||
|
|
|
@ -40,17 +40,17 @@ public class RegionServerProcedureManagerHost extends
|
|||
|
||||
public void initialize(RegionServerServices rss) throws KeeperException {
|
||||
for (RegionServerProcedureManager proc : procedures) {
|
||||
LOG.info("Procedure " + proc.getProcedureSignature() + " is initializing");
|
||||
LOG.debug("Procedure " + proc.getProcedureSignature() + " is initializing");
|
||||
proc.initialize(rss);
|
||||
LOG.info("Procedure " + proc.getProcedureSignature() + " is initialized");
|
||||
LOG.debug("Procedure " + proc.getProcedureSignature() + " is initialized");
|
||||
}
|
||||
}
|
||||
|
||||
public void start() {
|
||||
for (RegionServerProcedureManager proc : procedures) {
|
||||
LOG.info("Procedure " + proc.getProcedureSignature() + " is starting");
|
||||
LOG.debug("Procedure " + proc.getProcedureSignature() + " is starting");
|
||||
proc.start();
|
||||
LOG.info("Procedure " + proc.getProcedureSignature() + " is started");
|
||||
LOG.debug("Procedure " + proc.getProcedureSignature() + " is started");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1084,7 +1084,7 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
String value = e.getValue();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Config from master: " + key + "=" + value);
|
||||
LOG.info("Config from master: " + key + "=" + value);
|
||||
}
|
||||
this.conf.set(key, value);
|
||||
}
|
||||
|
@ -1613,7 +1613,7 @@ public class HRegionServer extends HasThread implements
|
|||
public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct)
|
||||
throws KeeperException, IOException {
|
||||
rpcServices.checkOpen();
|
||||
LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString());
|
||||
LOG.info("Post open deploy tasks for " + r.getRegionNameAsString());
|
||||
// Do checks to see if we need to compact (references or too many files)
|
||||
for (Store s : r.getStores().values()) {
|
||||
if (s.hasReferences() || s.needsCompaction()) {
|
||||
|
@ -1637,7 +1637,7 @@ public class HRegionServer extends HasThread implements
|
|||
MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
|
||||
this.serverName, openSeqNum);
|
||||
}
|
||||
LOG.info("Finished post open deploy task for " + r.getRegionNameAsString());
|
||||
LOG.debug("Finished post open deploy task for " + r.getRegionNameAsString());
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -84,7 +84,7 @@ public class ShutdownHook {
|
|||
Runnable fsShutdownHook = suppressHdfsShutdownHook(fs);
|
||||
Thread t = new ShutdownHookThread(conf, stop, threadToJoin, fsShutdownHook);
|
||||
ShutdownHookManager.affixShutdownHook(t, 0);
|
||||
LOG.info("Installed shutdown hook thread: " + t.getName());
|
||||
LOG.debug("Installed shutdown hook thread: " + t.getName());
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -354,7 +354,7 @@ public class HLogSplitter {
|
|||
e = RemoteExceptionHandler.checkIOException(e);
|
||||
throw e;
|
||||
} finally {
|
||||
LOG.info("Finishing writing output logs and closing down.");
|
||||
LOG.debug("Finishing writing output logs and closing down.");
|
||||
if (outputSinkStarted) {
|
||||
progress_failed = outputSink.finishWritingAndClose() == null;
|
||||
}
|
||||
|
@ -450,7 +450,7 @@ public class HLogSplitter {
|
|||
if (!FSUtils.renameAndSetModifyTime(fs, p, newPath)) {
|
||||
LOG.warn("Unable to move " + p + " to " + newPath);
|
||||
} else {
|
||||
LOG.debug("Archived processed log " + p + " to " + newPath);
|
||||
LOG.info("Archived processed log " + p + " to " + newPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -944,7 +944,7 @@ public class HLogSplitter {
|
|||
* @throws IOException
|
||||
*/
|
||||
protected boolean finishWriting() throws IOException {
|
||||
LOG.info("Waiting for split writer threads to finish");
|
||||
LOG.debug("Waiting for split writer threads to finish");
|
||||
boolean progress_failed = false;
|
||||
for (WriterThread t : writerThreads) {
|
||||
t.finish();
|
||||
|
@ -1094,7 +1094,7 @@ public class HLogSplitter {
|
|||
if (!fs.rename(wap.p, dst)) {
|
||||
throw new IOException("Failed renaming " + wap.p + " to " + dst);
|
||||
}
|
||||
LOG.debug("Rename " + wap.p + " to " + dst);
|
||||
LOG.info("Rename " + wap.p + " to " + dst);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("Couldn't rename " + wap.p + " to " + dst, ioe);
|
||||
|
@ -1221,7 +1221,7 @@ public class HLogSplitter {
|
|||
}
|
||||
}
|
||||
Writer w = createWriter(fs, regionedits, conf);
|
||||
LOG.debug("Creating writer path=" + regionedits + " region=" + Bytes.toStringBinary(region));
|
||||
LOG.info("Creating writer path=" + regionedits + " region=" + Bytes.toStringBinary(region));
|
||||
return (new WriterAndPath(regionedits, w));
|
||||
}
|
||||
|
||||
|
|
|
@ -47,7 +47,6 @@ import org.apache.hadoop.hbase.replication.ReplicationListener;
|
|||
import org.apache.hadoop.hbase.replication.ReplicationPeers;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationQueues;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationTracker;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
|
|
Loading…
Reference in New Issue