HBASE-8696 Fixup for logs that show when running hbase-it tests.

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1494453 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-06-19 04:48:04 +00:00
parent 051f7843f2
commit a71464bf84
19 changed files with 138 additions and 117 deletions

View File

@ -52,7 +52,7 @@ distMode=`$bin/hbase --config "$HBASE_CONF_DIR" org.apache.hadoop.hbase.util.HBa
if [ "$distMode" == 'false' ]
then
"$bin"/hbase-daemon.sh $commandToRun master
"$bin"/hbase-daemon.sh $commandToRun master $@
else
"$bin"/hbase-daemons.sh --config "${HBASE_CONF_DIR}" $commandToRun zookeeper
"$bin"/hbase-daemon.sh --config "${HBASE_CONF_DIR}" $commandToRun master

View File

@ -152,6 +152,13 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
return encodedName;
}
/**
* @return Return a short, printable name for this region (usually encoded name) for us logging.
*/
public String getShortNameToLog() {
return prettyPrint(this.getEncodedName());
}
/**
* Use logging.
* @param encodedRegionName The encoded regionname.
@ -648,14 +655,13 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
*/
@Override
public String toString() {
return "{" + HConstants.NAME + " => '" +
this.regionNameStr
return "{ENCODED => " + getEncodedName() + ", " +
HConstants.NAME + " => '" + this.regionNameStr
+ "', STARTKEY => '" +
Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
Bytes.toStringBinary(this.endKey) +
"', ENCODED => " + getEncodedName() + "," +
(isOffline()? " OFFLINE => true,": "") +
(isSplit()? " SPLIT => true,": "") + "}";
Bytes.toStringBinary(this.endKey) + "'" +
(isOffline()? ", OFFLINE => true": "") +
(isSplit()? ", SPLIT => true": "") + "}";
}
/**

View File

@ -51,7 +51,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
// Many threads can update the state at the stamp at the same time
private final AtomicLong stamp;
private HRegionInfo region;
private HRegionInfo hri;
private volatile ServerName serverName;
private volatile State state;
@ -66,7 +66,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
public RegionState(HRegionInfo region,
State state, long stamp, ServerName serverName) {
this.region = region;
this.hri = region;
this.state = state;
this.stamp = new AtomicLong(stamp);
this.serverName = serverName;
@ -85,7 +85,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
}
public HRegionInfo getRegion() {
return region;
return hri;
}
public ServerName getServerName() {
@ -150,7 +150,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
@Override
public String toString() {
return "{" + region.getRegionNameAsString()
return "{" + hri.getShortNameToLog()
+ " state=" + state
+ ", ts=" + stamp
+ ", server=" + serverName + "}";
@ -163,7 +163,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
long lstamp = stamp.get();
long relTime = System.currentTimeMillis() - lstamp;
return region.getRegionNameAsString()
return hri.getRegionNameAsString()
+ " state=" + state
+ ", ts=" + new Date(lstamp) + " (" + (relTime/1000) + "s ago)"
+ ", server=" + serverName;
@ -214,7 +214,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
default:
throw new IllegalStateException("");
}
regionState.setRegionInfo(HRegionInfo.convert(region));
regionState.setRegionInfo(HRegionInfo.convert(hri));
regionState.setState(rs);
regionState.setStamp(getStamp());
return regionState.build();
@ -278,8 +278,8 @@ public class RegionState implements org.apache.hadoop.io.Writable {
@Deprecated
@Override
public void readFields(DataInput in) throws IOException {
region = new HRegionInfo();
region.readFields(in);
hri = new HRegionInfo();
hri.readFields(in);
state = State.valueOf(in.readUTF());
stamp.set(in.readLong());
}
@ -290,7 +290,7 @@ public class RegionState implements org.apache.hadoop.io.Writable {
@Deprecated
@Override
public void write(DataOutput out) throws IOException {
region.write(out);
hri.write(out);
out.writeUTF(state.name());
out.writeLong(stamp.get());
}

View File

@ -223,12 +223,12 @@ public class ClassFinder {
if (!proceedOnExceptions) {
throw classNotFoundEx;
}
LOG.error("Failed to instantiate or check " + className + ": " + classNotFoundEx);
LOG.debug("Failed to instantiate or check " + className + ": " + classNotFoundEx);
} catch (LinkageError linkageEx) {
if (!proceedOnExceptions) {
throw linkageEx;
}
LOG.error("Failed to instantiate or check " + className + ": " + linkageEx);
LOG.debug("Failed to instantiate or check " + className + ": " + linkageEx);
}
return null;
}

View File

@ -63,7 +63,11 @@ public class HBaseClusterManager extends ClusterManager {
}
sshOptions = (sshOptions == null) ? "" : sshOptions;
tunnelCmd = conf.get("hbase.it.clustermanager.ssh.cmd", DEFAULT_TUNNEL_CMD);
LOG.info("Running with SSH user [" + sshUserName + "] and options [" + sshOptions + "]");
// Print out ssh special config if any.
if ((sshUserName != null && sshUserName.length() > 0) ||
(sshOptions != null && sshOptions.length() > 0)) {
LOG.info("Running with SSH user [" + sshUserName + "] and options [" + sshOptions + "]");
}
}
/**

View File

@ -46,9 +46,9 @@ public abstract class IngestIntegrationTestBase {
protected void setUp(int numSlavesBase, Configuration conf) throws Exception {
tableName = this.getClass().getSimpleName();
util = getTestingUtil(conf);
LOG.info("Initializing cluster with " + numSlavesBase + " servers");
LOG.debug("Initializing/checking cluster has " + numSlavesBase + " servers");
util.initializeCluster(numSlavesBase);
LOG.info("Done initializing cluster");
LOG.debug("Done initializing/checking cluster");
cluster = util.getHBaseClusterInterface();
deleteTableIfNecessary();
loadTool = new LoadTestTool();
@ -75,9 +75,9 @@ public abstract class IngestIntegrationTestBase {
}
protected void tearDown() throws Exception {
LOG.info("Restoring the cluster");
LOG.debug("Restoring the cluster");
util.restoreCluster();
LOG.info("Done restoring the cluster");
LOG.debug("Done restoring the cluster");
}
private void deleteTableIfNecessary() throws IOException {

View File

@ -109,7 +109,6 @@ public class HFileArchiver {
Preconditions.checkArgument(regionDir.toString().startsWith(tableDir.toString()));
Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(rootdir, tableDir, regionDir);
LOG.debug("Have an archive directory, preparing to move files");
FileStatusConverter getAsFile = new FileStatusConverter(fs);
// otherwise, we attempt to archive the store files
@ -142,7 +141,6 @@ public class HFileArchiver {
// if that was successful, then we delete the region
if (success) {
LOG.debug("Successfully resolved and archived, now can just delete region.");
return deleteRegionWithoutArchiving(fs, regionDir);
}
@ -222,7 +220,7 @@ public class HFileArchiver {
}
// otherwise we attempt to archive the store files
LOG.debug("Archiving compacted store files.");
if (LOG.isTraceEnabled()) LOG.trace("Archiving compacted store files.");
// wrap the storefile into a File
StoreToFile getStorePath = new StoreToFile(fs);
@ -280,7 +278,7 @@ public class HFileArchiver {
*/
private static boolean resolveAndArchive(FileSystem fs, Path baseArchiveDir,
Collection<File> toArchive) throws IOException {
LOG.debug("Starting to archive files:" + toArchive);
if (LOG.isTraceEnabled()) LOG.trace("Starting to archive files:" + toArchive);
long start = EnvironmentEdgeManager.currentTimeMillis();
List<File> failures = resolveAndArchive(fs, baseArchiveDir, toArchive, start);
@ -314,7 +312,7 @@ public class HFileArchiver {
// short circuit if no files to move
if (toArchive.size() == 0) return Collections.emptyList();
LOG.debug("moving files to the archive directory: " + baseArchiveDir);
if (LOG.isTraceEnabled()) LOG.trace("moving files to the archive directory: " + baseArchiveDir);
// make sure the archive directory exists
if (!fs.exists(baseArchiveDir)) {
@ -322,7 +320,7 @@ public class HFileArchiver {
throw new IOException("Failed to create the archive directory:" + baseArchiveDir
+ ", quitting archive attempt.");
}
LOG.debug("Created archive directory:" + baseArchiveDir);
if (LOG.isTraceEnabled()) LOG.trace("Created archive directory:" + baseArchiveDir);
}
List<File> failures = new ArrayList<File>();
@ -330,7 +328,7 @@ public class HFileArchiver {
for (File file : toArchive) {
// if its a file archive it
try {
LOG.debug("Archiving:" + file);
if (LOG.isTraceEnabled()) LOG.trace("Archiving: " + file);
if (file.isFile()) {
// attempt to archive the file
if (!resolveAndArchiveFile(baseArchiveDir, file, startTime)) {
@ -339,7 +337,7 @@ public class HFileArchiver {
}
} else {
// otherwise its a directory and we need to archive all files
LOG.debug(file + " is a directory, archiving children files");
if (LOG.isTraceEnabled()) LOG.trace(file + " is a directory, archiving children files");
// so we add the directory name to the one base archive
Path parentArchiveDir = new Path(baseArchiveDir, file.getName());
// and then get all the files from that directory and attempt to
@ -398,8 +396,10 @@ public class HFileArchiver {
LOG.debug("Backed up archive file from: " + archiveFile);
}
LOG.debug("No existing file in archive for:" + archiveFile +
if (LOG.isTraceEnabled()) {
LOG.trace("No existing file in archive for: " + archiveFile +
", free to archive original file.");
}
// at this point, we should have a free spot for the archive file
boolean success = false;

View File

@ -156,7 +156,7 @@ public class RpcServer implements RpcServerInterface {
private static final int DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER = 10;
/**
* The maximum size that we can hold in the IPC queue
* The maximum size that we can hold in the RPC queue
*/
private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;
@ -497,7 +497,7 @@ public class RpcServer implements RpcServerInterface {
readers = new Reader[readThreads];
readPool = Executors.newFixedThreadPool(readThreads,
new ThreadFactoryBuilder().setNameFormat(
"IPC Reader %d on port " + port).setDaemon(true).build());
"RpcServer.reader=%d,port=" + port).setDaemon(true).build());
for (int i = 0; i < readThreads; ++i) {
Reader reader = new Reader();
readers[i] = reader;
@ -507,7 +507,7 @@ public class RpcServer implements RpcServerInterface {
// Register accepts on the server socket with the selector.
acceptChannel.register(selector, SelectionKey.OP_ACCEPT);
this.setName("IPC Server listener on " + port);
this.setName("RpcServer.listener,port=" + port);
this.setDaemon(true);
}
@ -802,7 +802,7 @@ public class RpcServer implements RpcServerInterface {
private int pending; // connections waiting to register
Responder() throws IOException {
this.setName("IPC Server Responder");
this.setName("RpcServer.responder");
this.setDaemon(true);
writeSelector = Selector.open(); // create a selector
pending = 0;
@ -1782,12 +1782,12 @@ public class RpcServer implements RpcServerInterface {
this.myCallQueue = cq;
this.setDaemon(true);
String threadName = "IPC Server handler " + instanceNumber + " on " + port;
String threadName = "RpcServer.handler=" + instanceNumber + ",port=" + port;
if (cq == priorityCallQueue) {
// this is just an amazing hack, but it works.
threadName = "PRI " + threadName;
threadName = "Priority." + threadName;
} else if (cq == replicationQueue) {
threadName = "REPL " + threadName;
threadName = "Replication." + threadName;
}
this.setName(threadName);
this.status = TaskMonitor.get().createRPCStatus(threadName);
@ -1995,7 +1995,7 @@ public class RpcServer implements RpcServerInterface {
}
/**
* Setup response for the IPC Call.
* Setup response for the RPC Call.
*
* @param response buffer to serialize the response into
* @param call {@link Call} to which we are setting up the response

View File

@ -70,7 +70,7 @@ import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
* attribute on created hfiles. Calling write(null,null) will forceably roll
* all HFiles being written.
* <p>
* Using this class as part of a MapReduce job is best done
* Using this class as part of a MapReduce job is best done
* using {@link #configureIncrementalLoad(Job, HTable)}.
* @see KeyValueSortReducer
*/
@ -211,7 +211,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
.withChecksumType(HStore.getChecksumType(conf))
.withBytesPerChecksum(HStore.getBytesPerChecksum(conf))
.build();
this.writers.put(family, wl);
return wl;
}
@ -340,7 +340,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
KeyValueSerialization.class.getName());
// Use table's region boundaries for TOP split points.
LOG.info("Looking up current regions for table " + table);
LOG.info("Looking up current regions for table " + Bytes.toString(table.getTableName()));
List<ImmutableBytesWritable> startKeys = getRegionStartKeys(table);
LOG.info("Configuring " + startKeys.size() + " reduce partitions " +
"to match current region count");
@ -350,10 +350,10 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
// Set compression algorithms based on column families
configureCompression(table, conf);
configureBloomType(table, conf);
TableMapReduceUtil.addDependencyJars(job);
TableMapReduceUtil.initCredentials(job);
LOG.info("Incremental table output configured.");
LOG.info("Incremental table " + Bytes.toString(table.getTableName()) + " output configured.");
}
/**
@ -373,10 +373,10 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
private static Map<byte[], String> createFamilyBloomMap(Configuration conf) {
return createFamilyConfValueMap(conf, BLOOM_TYPE_CONF_KEY);
}
/**
* Run inside the task to deserialize column family to given conf value map.
*
*
* @param conf
* @param confName
* @return a map of column family to the given configuration value
@ -450,7 +450,7 @@ public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable,
// Get rid of the last ampersand
conf.set(COMPRESSION_CONF_KEY, compressionConfigValue.toString());
}
/**
* Serialize column family to bloom type map to configuration.
* Invoked while configuring the MR job for incremental load.

View File

@ -1465,15 +1465,12 @@ public class AssignmentManager extends ZooKeeperListener {
* @param regions Regions to assign.
* @return true if successful
*/
boolean assign(final ServerName destination,
final List<HRegionInfo> regions) {
boolean assign(final ServerName destination, final List<HRegionInfo> regions) {
int regionCount = regions.size();
if (regionCount == 0) {
return true;
}
LOG.debug("Bulk assigning " + regionCount + " region(s) to " +
destination.toString());
LOG.debug("Assigning " + regionCount + " region(s) to " + destination.toString());
Set<String> encodedNames = new HashSet<String>(regionCount);
for (HRegionInfo region : regions) {
encodedNames.add(region.getEncodedName());
@ -2408,9 +2405,10 @@ public class AssignmentManager extends ZooKeeperListener {
// Not use bulk assignment. This could be more efficient in small
// cluster, especially mini cluster for testing, so that tests won't time out
LOG.info("Not use bulk assigning since we are assigning only "
+ regions + " region(s) to " + servers + " server(s)");
if (LOG.isTraceEnabled()) {
LOG.trace("Not using bulk assignment since we are assigning only " + regions +
" region(s) to " + servers + " server(s)");
}
for (Map.Entry<ServerName, List<HRegionInfo>> plan: bulkPlan.entrySet()) {
assign(plan.getKey(), plan.getValue());
}

View File

@ -63,7 +63,7 @@ public class CatalogJanitor extends Chore {
private AtomicBoolean alreadyRunning = new AtomicBoolean(false);
CatalogJanitor(final Server server, final MasterServices services) {
super(server.getServerName() + "-CatalogJanitor",
super("CatalogJanitor-" + server.getServerName().toShortString(),
server.getConfiguration().getInt("hbase.catalogjanitor.interval", 300000),
server);
this.server = server;
@ -338,7 +338,7 @@ public class CatalogJanitor extends Chore {
this.services.getAssignmentManager().regionOffline(parent);
}
FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
LOG.debug("Archiving parent region:" + parent);
if (LOG.isTraceEnabled()) LOG.trace("Archiving parent region: " + parent);
HFileArchiver.archiveRegion(this.services.getConfiguration(), fs, parent);
MetaEditor.deleteRegion(this.server.getCatalogTracker(), parent);
result = true;

View File

@ -69,7 +69,6 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.exceptions.MasterNotRunningException;
import org.apache.hadoop.hbase.exceptions.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.exceptions.PleaseHoldException;
import org.apache.hadoop.hbase.exceptions.TableNotDisabledException;
import org.apache.hadoop.hbase.exceptions.TableNotFoundException;
@ -736,7 +735,7 @@ MasterServices, Server {
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
if (!masterRecovery) {
this.executorService = new ExecutorService(getServerName().toString());
this.executorService = new ExecutorService(getServerName().toShortString());
this.serverManager = createServerManager(this, this);
}

View File

@ -53,8 +53,11 @@ public class HMasterCommandLine extends ServerCommandLine {
" stop Start cluster shutdown; Master signals RegionServer shutdown\n" +
" clear Delete the master znode in ZooKeeper after a master crashes\n "+
" where [opts] are:\n" +
" --minServers=<servers> Minimum RegionServers needed to host user tables.\n" +
" --backup Master should start in backup mode";
" --minRegionServers=<servers> Minimum RegionServers needed to host user tables.\n" +
" --localRegionServers=<servers> " +
"RegionServers to start in master process when in standalone mode.\n" +
" --masters=<servers> Masters to start in this process.\n" +
" --backup Master should start in backup mode";
private final Class<? extends HMaster> masterClass;
@ -69,10 +72,12 @@ public class HMasterCommandLine extends ServerCommandLine {
public int run(String args[]) throws Exception {
Options opt = new Options();
opt.addOption("minServers", true, "Minimum RegionServers needed to host user tables");
opt.addOption("localRegionServers", true,
"RegionServers to start in master process when running standalone");
opt.addOption("masters", true, "Masters to start in this process");
opt.addOption("minRegionServers", true, "Minimum RegionServers needed to host user tables");
opt.addOption("backup", false, "Do not try to become HMaster until the primary fails");
CommandLine cmd;
try {
cmd = new GnuParser().parse(opt, args);
@ -83,6 +88,14 @@ public class HMasterCommandLine extends ServerCommandLine {
}
if (cmd.hasOption("minRegionServers")) {
String val = cmd.getOptionValue("minRegionServers");
getConf().setInt("hbase.regions.server.count.min",
Integer.valueOf(val));
LOG.debug("minRegionServers set to " + val);
}
// minRegionServers used to be minServers. Support it too.
if (cmd.hasOption("minServers")) {
String val = cmd.getOptionValue("minServers");
getConf().setInt("hbase.regions.server.count.min",
@ -95,6 +108,20 @@ public class HMasterCommandLine extends ServerCommandLine {
getConf().setBoolean(HConstants.MASTER_TYPE_BACKUP, true);
}
// How many regionservers to startup in this process (we run regionservers in same process as
// master when we are in local/standalone mode. Useful testing)
if (cmd.hasOption("localRegionServers")) {
String val = cmd.getOptionValue("localRegionServers");
getConf().setInt("hbase.regionservers", Integer.valueOf(val));
LOG.debug("localRegionServers set to " + val);
}
// How many masters to startup inside this process; useful testing
if (cmd.hasOption("masters")) {
String val = cmd.getOptionValue("masters");
getConf().setInt("hbase.masters", Integer.valueOf(val));
LOG.debug("masters set to " + val);
}
List<String> remainingArgs = cmd.getArgList();
if (remainingArgs.size() != 1) {
usage(null);
@ -147,8 +174,8 @@ public class HMasterCommandLine extends ServerCommandLine {
Integer.toString(clientPort));
// Need to have the zk cluster shutdown when master is shutdown.
// Run a subclass that does the zk cluster shutdown on its way out.
LocalHBaseCluster cluster = new LocalHBaseCluster(conf, 1, 1,
LocalHMaster.class, HRegionServer.class);
LocalHBaseCluster cluster = new LocalHBaseCluster(conf, conf.getInt("hbase.masters", 1),
conf.getInt("hbase.regionservers", 1), LocalHMaster.class, HRegionServer.class);
((LocalHMaster)cluster.getMaster(0)).setZKCluster(zooKeeperCluster);
cluster.startup();
waitOnMasterThreads(cluster);
@ -199,8 +226,8 @@ public class HMasterCommandLine extends ServerCommandLine {
private void waitOnMasterThreads(LocalHBaseCluster cluster) throws InterruptedException{
List<JVMClusterUtil.MasterThread> masters = cluster.getMasters();
List<JVMClusterUtil.RegionServerThread> regionservers = cluster.getRegionServers();
if (masters != null) {
if (masters != null) {
for (JVMClusterUtil.MasterThread t : masters) {
t.join();
if(t.getMaster().isAborted()) {
@ -216,7 +243,7 @@ public class HMasterCommandLine extends ServerCommandLine {
t.getRegionServer().stop("HMaster Aborted; Bringing down regions servers");
}
}
/*
* Version of master that will shutdown the passed zk cluster on its way out.
*/

View File

@ -194,9 +194,8 @@ public class RegionStates {
String regionName = hri.getEncodedName();
RegionState regionState = regionStates.get(regionName);
if (regionState != null) {
LOG.warn("Tried to create a state of a region already in RegionStates "
+ hri + ", used existing state: " + regionState
+ ", ignored new state: state=OFFLINE, server=null");
LOG.warn("Tried to create a state of a region already in RegionStates, " +
"used existing state: " + regionState + ", ignored new state: state=OFFLINE, server=null");
} else {
regionState = new RegionState(hri, State.OFFLINE);
regionStates.put(regionName, regionState);
@ -225,7 +224,7 @@ public class RegionStates {
*/
public synchronized RegionState updateRegionState(
final RegionTransition transition, final State state) {
byte[] regionName = transition.getRegionName();
byte [] regionName = transition.getRegionName();
HRegionInfo regionInfo = getRegionInfo(regionName);
if (regionInfo == null) {
String prettyRegionName = HRegionInfo.prettyPrint(
@ -248,13 +247,14 @@ public class RegionStates {
ServerName newServerName = serverName;
if (serverName != null &&
(state == State.CLOSED || state == State.OFFLINE)) {
LOG.warn("Closed region " + hri + " still on "
LOG.warn("Closed region " + hri.getShortNameToLog() + " still on "
+ serverName + "? Ignored, reset it to null");
newServerName = null;
}
if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) {
LOG.warn("Failed to transition " + hri + " on " + serverName + ": " + state);
LOG.warn("Failed to transition " + hri.getShortNameToLog() + " on " + serverName + ": " +
state);
}
String regionName = hri.getEncodedName();
@ -262,7 +262,7 @@ public class RegionStates {
hri, state, System.currentTimeMillis(), newServerName);
RegionState oldState = regionStates.put(regionName, regionState);
if (oldState == null || oldState.getState() != regionState.getState()) {
LOG.info("Region " + hri + " transitioned from " + oldState + " to " + regionState);
LOG.info("Region transitioned from " + oldState + " to " + regionState);
}
if (state != State.SPLITTING && (newServerName != null
|| (state != State.PENDING_CLOSE && state != State.CLOSING))) {
@ -284,13 +284,13 @@ public class RegionStates {
String regionName = hri.getEncodedName();
RegionState oldState = regionStates.get(regionName);
if (oldState == null) {
LOG.warn("Online a region not in RegionStates: " + hri);
LOG.warn("Online a region not in RegionStates: " + hri.getShortNameToLog());
} else {
State state = oldState.getState();
ServerName sn = oldState.getServerName();
if (state != State.OPEN || sn == null || !sn.equals(serverName)) {
LOG.debug("Online a region with current state=" + state + ", expected state=OPEN"
+ ", assigned to server: " + sn + " expected " + serverName);
LOG.debug("Online a region " + hri.getShortNameToLog() + " with current state=" + state +
", expected state=OPEN" + ", assigned to server: " + sn + " expected " + serverName);
}
}
updateRegionState(hri, State.OPEN, serverName);
@ -298,7 +298,7 @@ public class RegionStates {
ServerName oldServerName = regionAssignments.put(hri, serverName);
if (!serverName.equals(oldServerName)) {
LOG.info("Onlined region " + hri + " on " + serverName);
LOG.info("Onlined region " + hri.getShortNameToLog() + " on " + serverName);
Set<HRegionInfo> regions = serverHoldings.get(serverName);
if (regions == null) {
regions = new HashSet<HRegionInfo>();
@ -306,7 +306,7 @@ public class RegionStates {
}
regions.add(hri);
if (oldServerName != null) {
LOG.info("Offlined region " + hri + " from " + oldServerName);
LOG.info("Offlined region " + hri.getShortNameToLog() + " from " + oldServerName);
serverHoldings.get(oldServerName).remove(hri);
}
}
@ -319,13 +319,13 @@ public class RegionStates {
String regionName = hri.getEncodedName();
RegionState oldState = regionStates.get(regionName);
if (oldState == null) {
LOG.warn("Offline a region not in RegionStates: " + hri);
LOG.warn("Offline a region not in RegionStates: " + hri.getShortNameToLog());
} else {
State state = oldState.getState();
ServerName sn = oldState.getServerName();
if (state != State.OFFLINE || sn != null) {
LOG.debug("Offline a region with current state=" + state + ", expected state=OFFLINE"
+ ", assigned to server: " + sn + ", expected null");
LOG.debug("Offline a region " + hri.getShortNameToLog() + " with current state=" + state +
", expected state=OFFLINE" + ", assigned to server: " + sn + ", expected null");
}
}
updateRegionState(hri, State.OFFLINE);
@ -333,7 +333,7 @@ public class RegionStates {
ServerName oldServerName = regionAssignments.remove(hri);
if (oldServerName != null) {
LOG.info("Offlined region " + hri + " from " + oldServerName);
LOG.info("Offlined region " + hri.getShortNameToLog() + " from " + oldServerName);
serverHoldings.get(oldServerName).remove(hri);
}
}
@ -360,16 +360,14 @@ public class RegionStates {
// Region is open on this region server, but in transition.
// This region must be moving away from this server.
// SSH will handle it, either skip assigning, or re-assign.
LOG.info("Transitioning region "
+ state + " will be handled by SSH for " + sn);
LOG.info("Transitioning region " + state + " will be handled by SSH for " + sn);
} else if (sn.equals(state.getServerName())) {
// Region is in transition on this region server, and this
// region is not open on this server. So the region must be
// moving to this server from another one (i.e. opening or
// pending open on this server, was open on another one
if (state.isPendingOpen() || state.isOpening()) {
LOG.info("Found opening region "
+ state + " to be reassigned by SSH for " + sn);
LOG.info("Found opening region " + state + " to be reassigned by SSH for " + sn);
rits.add(hri);
} else {
LOG.warn("THIS SHOULD NOT HAPPEN: unexpected state "

View File

@ -101,18 +101,17 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
if (regionState != null && regionState.isOpened()) {
openedNodeDeleted = deleteOpenedNode(expectedVersion);
if (!openedNodeDeleted) {
LOG.error("The znode of region " + regionInfo.getRegionNameAsString()
+ " could not be deleted.");
LOG.error("Znode of region " + regionInfo.getShortNameToLog() + " could not be deleted.");
}
} else {
LOG.warn("Skipping the onlining of " + regionInfo.getRegionNameAsString() +
LOG.warn("Skipping the onlining of " + regionInfo.getShortNameToLog() +
" because regions is NOT in RIT -- presuming this is because it SPLIT");
}
if (!openedNodeDeleted) {
if (this.assignmentManager.getZKTable().isDisablingOrDisabledTable(
regionInfo.getTableNameAsString())) {
debugLog(regionInfo, "Opened region "
+ regionInfo.getRegionNameAsString() + " but "
+ regionInfo.getShortNameToLog() + " but "
+ "this table is disabled, triggering close of region");
assignmentManager.unassign(regionInfo);
}
@ -121,7 +120,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
private boolean deleteOpenedNode(int expectedVersion) {
debugLog(regionInfo, "Handling OPENED event for " +
this.regionInfo.getRegionNameAsString() + " from " + this.sn.toString() +
this.regionInfo.getShortNameToLog() + " from " + this.sn.toString() +
"; deleting unassigned node");
try {
// delete the opened znode only if the version matches.
@ -129,7 +128,7 @@ public class OpenedRegionHandler extends EventHandler implements TotesHRegionInf
regionInfo.getEncodedName(), EventType.RS_ZK_REGION_OPENED, expectedVersion);
} catch(KeeperException.NoNodeException e){
// Getting no node exception here means that already the region has been opened.
LOG.warn("The znode of the region " + regionInfo.getRegionNameAsString() +
LOG.warn("The znode of the region " + regionInfo.getShortNameToLog() +
" would have already been deleted");
return false;
} catch (KeeperException e) {

View File

@ -605,7 +605,8 @@ public class HRegion implements HeapSize { // , Writable{
// Use maximum of log sequenceid or that which was found in stores
// (particularly if no recovered edits, seqid will be -1).
long nextSeqid = maxSeqId + 1;
LOG.info("Onlined " + this.toString() + "; next sequenceid=" + nextSeqid);
LOG.info("Onlined " + this.getRegionInfo().getShortNameToLog() +
"; next sequenceid=" + nextSeqid);
// A region can be reopened if failed a split; reset flags
this.closing.set(false);
@ -631,8 +632,7 @@ public class HRegion implements HeapSize { // , Writable{
if (!htableDescriptor.getFamilies().isEmpty()) {
// initialize the thread pool for opening stores in parallel.
ThreadPoolExecutor storeOpenerThreadPool =
getStoreOpenAndCloseThreadPool(
"StoreOpenerThread-" + this.getRegionNameAsString());
getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog());
CompletionService<HStore> completionService =
new ExecutorCompletionService<HStore>(storeOpenerThreadPool);
@ -4133,10 +4133,7 @@ public class HRegion implements HeapSize { // , Writable{
final RegionServerServices rsServices, final CancelableProgressable reporter)
throws IOException {
if (info == null) throw new NullPointerException("Passed region info is null");
LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
if (LOG.isDebugEnabled()) {
LOG.debug("Opening region: " + info);
}
LOG.info("Open " + info);
Path dir = HTableDescriptor.getTableDir(rootDir, info.getTableName());
HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info, htd, rsServices);
return r.openHRegion(reporter);

View File

@ -1145,10 +1145,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
String hostnameFromMasterPOV = e.getValue();
this.serverNameFromMasterPOV = new ServerName(hostnameFromMasterPOV,
this.isa.getPort(), this.startcode);
if (!this.serverNameFromMasterPOV.equals(this.isa.getHostName())) {
if (!hostnameFromMasterPOV.equals(this.isa.getHostName())) {
LOG.info("Master passed us a different hostname to use; was=" +
this.isa.getHostName() + ", but now=" +
this.serverNameFromMasterPOV.getHostname());
this.isa.getHostName() + ", but now=" + hostnameFromMasterPOV);
}
continue;
}
@ -1186,7 +1185,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this));
startServiceThreads();
LOG.info("Serving as " + this.serverNameFromMasterPOV +
", RPC listening on " + this.isa +
", RpcServer on " + this.isa +
", sessionid=0x" +
Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
isOnline = true;
@ -1308,7 +1307,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
final Stoppable stopper) {
super("CompactionChecker", sleepTime, h);
this.instance = h;
LOG.info("Runs every " + StringUtils.formatTime(sleepTime));
LOG.info(this.getName() + " runs every " + StringUtils.formatTime(sleepTime));
/* MajorCompactPriority is configurable.
* If not set, the compaction will use default priority.
@ -1864,9 +1863,6 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
}
new InetSocketAddress(sn.getHostname(), sn.getPort());
LOG.info("Attempting connect to Master server at " +
this.masterAddressManager.getMasterAddress());
try {
BlockingRpcChannel channel = this.rpcClient.createBlockingRpcChannel(sn,
User.getCurrent(), this.rpcTimeout);
@ -1919,8 +1915,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
if (masterServerName == null) return result;
try {
this.requestCount.set(0);
LOG.info("Telling master at " + masterServerName + " that we are up " +
"with port=" + this.isa.getPort() + ", startcode=" + this.startcode);
LOG.info("reportForDuty to master=" + masterServerName + " with port=" + this.isa.getPort() +
", startcode=" + this.startcode);
long now = EnvironmentEdgeManager.currentTimeMillis();
int port = this.isa.getPort();
RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
@ -3418,8 +3414,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
removeFromOnlineRegions(onlineRegion, null);
}
}
LOG.info("Received request to open region: " + region.getRegionNameAsString() + " on "
+ this.serverNameFromMasterPOV);
LOG.info("Open " + region.getRegionNameAsString());
htd = htds.get(region.getTableNameAsString());
if (htd == null) {
htd = this.tableDescriptors.get(region.getTableName());

View File

@ -82,7 +82,7 @@ public class CompactionConfiguration {
// Make it 0.5 so jitter has us fall evenly either side of when the compaction should run
majorCompactionJitter = conf.getFloat("hbase.hregion.majorcompaction.jitter", 0.50F);
LOG.info("Compaction configuration " + this.toString());
LOG.info(this);
}
@Override

View File

@ -21,13 +21,11 @@ package org.apache.hadoop.hbase.util;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.lang.reflect.Method;
import java.net.URL;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.http.HttpServer;
import org.mortbay.jetty.handler.ContextHandlerCollection;
import org.mortbay.jetty.servlet.Context;