HBASE-6876 Clean up WARNs and log messages around startup; REAPPLY

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1390848 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-09-27 05:40:09 +00:00
parent c8375a55e9
commit b5cf874819
7 changed files with 18 additions and 17 deletions

View File

@ -2450,7 +2450,7 @@ public class HConnectionManager {
c.getInt("hbase.client.serverside.retries.multiplier", 10); c.getInt("hbase.client.serverside.retries.multiplier", 10);
int retries = hcRetries * serversideMultiplier; int retries = hcRetries * serversideMultiplier;
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries); c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, retries);
log.debug("Set serverside HConnection retries=" + retries); log.debug("HConnection retries=" + retries);
} }
} }

View File

@ -62,8 +62,8 @@ public class HBaseRpcMetrics implements Updater {
metricsRecord.setTag("port", port); metricsRecord.setTag("port", port);
LOG.info("Initializing RPC Metrics with hostName=" LOG.info("Initializing RPC Metrics for className="
+ hostName + ", port=" + port); + hostName + " on port=" + port);
context.registerUpdater(this); context.registerUpdater(this);

View File

@ -525,6 +525,7 @@ public abstract class HBaseServer implements RpcServer {
readers[i] = reader; readers[i] = reader;
readPool.execute(reader); readPool.execute(reader);
} }
LOG.info("Started " + readThreads + " reader(s) in Listener.");
// Register accepts on the server socket with the selector. // Register accepts on the server socket with the selector.
acceptChannel.register(selector, SelectionKey.OP_ACCEPT); acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
@ -541,7 +542,6 @@ public abstract class HBaseServer implements RpcServer {
this.readSelector = Selector.open(); this.readSelector = Selector.open();
} }
public void run() { public void run() {
LOG.info("Starting " + getName());
try { try {
doRunLoop(); doRunLoop();
} finally { } finally {

View File

@ -20,14 +20,12 @@ package org.apache.hadoop.hbase.ipc;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import javax.net.SocketFactory; import javax.net.SocketFactory;
import org.apache.hadoop.hbase.ipc.VersionedProtocol;
import org.apache.hadoop.hbase.security.User;
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.hbase.security.User;
import com.google.protobuf.ServiceException;
/** An RPC implementation. */ /** An RPC implementation. */
@InterfaceAudience.Private @InterfaceAudience.Private

View File

@ -145,10 +145,11 @@ class ActiveMasterManager extends ZooKeeperListener {
this.watcher.getMasterAddressZNode(), this.sn)) { this.watcher.getMasterAddressZNode(), this.sn)) {
// If we were a backup master before, delete our ZNode from the backup // If we were a backup master before, delete our ZNode from the backup
// master directory since we are the active now // master directory since we are the active now)
if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) {
LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory"); LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory");
ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode); ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
}
// Save the znode in a file, this will allow to check if we crash in the launch scripts // Save the znode in a file, this will allow to check if we crash in the launch scripts
ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString()); ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());

View File

@ -185,6 +185,7 @@ import org.apache.hadoop.net.DNS;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.Watcher; import org.apache.zookeeper.Watcher;
import org.apache.hadoop.hbase.trace.SpanReceiverHost; import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.util.FSUtils;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
@ -324,6 +325,8 @@ Server {
public HMaster(final Configuration conf) public HMaster(final Configuration conf)
throws IOException, KeeperException, InterruptedException { throws IOException, KeeperException, InterruptedException {
this.conf = new Configuration(conf); this.conf = new Configuration(conf);
LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) +
", hbase.cluster.distributed=" + this.conf.getBoolean("hbase.cluster.distributed", false));
// Disable the block cache on the master // Disable the block cache on the master
this.conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f); this.conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
// Set how many times to retry talking to another server over HConnection. // Set how many times to retry talking to another server over HConnection.
@ -333,7 +336,7 @@ Server {
conf.get("hbase.master.dns.interface", "default"), conf.get("hbase.master.dns.interface", "default"),
conf.get("hbase.master.dns.nameserver", "default"))); conf.get("hbase.master.dns.nameserver", "default")));
int port = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT); int port = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT);
// Creation of a HSA will force a resolve. // Creation of a ISA will force a resolve.
InetSocketAddress initialIsa = new InetSocketAddress(hostname, port); InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
if (initialIsa.getAddress() == null) { if (initialIsa.getAddress() == null) {
throw new IllegalArgumentException("Failed resolve of " + initialIsa); throw new IllegalArgumentException("Failed resolve of " + initialIsa);

View File

@ -134,12 +134,11 @@ public class RecoverableZooKeeper {
switch (e.code()) { switch (e.code()) {
case NONODE: case NONODE:
if (isRetry) { if (isRetry) {
LOG.info("Node " + path + " already deleted. Assuming that a " + LOG.info("Node " + path + " already deleted. Assuming a " +
"previous attempt succeeded."); "previous attempt succeeded.");
return; return;
} }
LOG.warn("Node " + path + " already deleted, and this is not a " + LOG.warn("Node " + path + " already deleted, retry=" + isRetry);
"retry");
throw e; throw e;
case CONNECTIONLOSS: case CONNECTIONLOSS: