Adds rewrite of store files. Removed HRS.getServerInfo. Use getHServerInfo instead

M  src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java
  Removed getServerInfo.  Use getHServerInfo instead.
M  src/java/org/apache/hadoop/hbase/migration/package-info.java
  Fixed package.
M src/java/org/apache/hadoop/hbase/migration/nineteen/io/HBaseMapFile.java
  Use the migration HSK explicitly.  Disable getting midky.  Unnecessary
  in a migration.
M src/java/org/apache/hadoop/hbase/migration/nineteen/regionserver/HStoreFile.java
  Pass encoded name of region rather than a HRI. I don't know HRI
  when migrating and all that is wanted anyways is the encoded name.
  Added in static isReference.
M src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
  Minor javadoc and formatting.  Improved DEBUG messaging.
  Removed getServerInfo. Use getHServerInfo instead.
M src/java/org/apache/hadoop/hbase/regionserver/HLog.java
  Added logging of writer closes on splits.
M src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java
  Minor formatting.
M src/java/org/apache/hadoop/hbase/HServerInfo.java
  Removed useless qualifiers.
M src/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
  Use getHServerInfo instead of getServerInfo.
M src/java/org/apache/hadoop/hbase/master/ServerManager.java
  Formatting.  More detailed log message.
M src/java/org/apache/hadoop/hbase/master/HMaster.java
  Minor formatting.
M src/java/org/apache/hadoop/hbase/master/RegionManager.java
  Minor formatting.
M src/java/org/apache/hadoop/hbase/HServerAddress.java
  Minor formatting.
M src/java/org/apache/hadoop/hbase/util/MetaUtils.java
  Minor formatting.
M src/java/org/apache/hadoop/hbase/util/Migrate.java
  Added rewrite of store file from old to new format.
M src/webapps/regionserver/regionserver.jsp
  Use getHServerInfo instead of getServerInfo.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@796326 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-07-21 14:40:34 +00:00
parent dc82f5d093
commit c66bdea9bb
15 changed files with 175 additions and 100 deletions

View File

@ -163,7 +163,7 @@ public class HServerAddress implements WritableComparable<HServerAddress> {
} }
public void write(DataOutput out) throws IOException { public void write(DataOutput out) throws IOException {
if(address == null) { if (address == null) {
out.writeUTF(""); out.writeUTF("");
out.writeInt(0); out.writeInt(0);

View File

@ -38,7 +38,7 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
private long startCode; private long startCode;
private HServerLoad load; private HServerLoad load;
private int infoPort; private int infoPort;
private transient volatile String serverName = null; private String serverName = null;
private String name; private String name;
/** default constructor - used by Writable */ /** default constructor - used by Writable */

View File

@ -195,10 +195,12 @@ public class LocalHBaseCluster implements HConstants {
while (regionServerThread.isAlive()) { while (regionServerThread.isAlive()) {
try { try {
LOG.info("Waiting on " + LOG.info("Waiting on " +
regionServerThread.getRegionServer().getServerInfo().toString()); regionServerThread.getRegionServer().getHServerInfo().toString());
regionServerThread.join(); regionServerThread.join();
} catch (InterruptedException e) { } catch (InterruptedException e) {
e.printStackTrace(); e.printStackTrace();
} catch (IOException e) {
e.printStackTrace();
} }
} }
return regionServerThread.getName(); return regionServerThread.getName();

View File

@ -1369,8 +1369,8 @@ class RegionManager implements HConstants {
int numSrvRegs = srvLoad.getNumberOfRegions(); int numSrvRegs = srvLoad.getNumberOfRegions();
if (numSrvRegs > avgLoadPlusSlop) { if (numSrvRegs > avgLoadPlusSlop) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Server is overloaded. Server load: " + numSrvRegs + LOG.debug("Server is overloaded: load=" + numSrvRegs +
" avg: " + avgLoad + ", slop: " + this.slop); ", avg=" + avgLoad + ", slop=" + this.slop);
} }
return numSrvRegs - (int)Math.ceil(avgLoad); return numSrvRegs - (int)Math.ceil(avgLoad);
} }

View File

@ -183,10 +183,9 @@ class ServerManager implements HConstants {
if (storedInfo != null && !master.closed.get()) { if (storedInfo != null && !master.closed.get()) {
// The startup message was from a known server with the same name. // The startup message was from a known server with the same name.
// Timeout the old one right away. // Timeout the old one right away.
HServerAddress root = master.getRootRegionLocation(); master.getRootRegionLocation();
try { try {
master.toDoQueue.put( master.toDoQueue.put(new ProcessServerShutdown(master, storedInfo));
new ProcessServerShutdown(master, storedInfo));
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.error("Insertion into toDoQueue was interrupted", e); LOG.error("Insertion into toDoQueue was interrupted", e);
} }
@ -261,7 +260,7 @@ class ServerManager implements HConstants {
} }
if (master.shutdownRequested.get()) { if (master.shutdownRequested.get()) {
if(quiescedServers.get() >= serversToServerInfo.size()) { if (quiescedServers.get() >= serversToServerInfo.size()) {
// If the only servers we know about are meta servers, then we can // If the only servers we know about are meta servers, then we can
// proceed with shutdown // proceed with shutdown
LOG.info("All user tables quiesced. Proceeding with shutdown"); LOG.info("All user tables quiesced. Proceeding with shutdown");
@ -290,13 +289,13 @@ class ServerManager implements HConstants {
HServerInfo storedInfo = serversToServerInfo.get(info.getServerName()); HServerInfo storedInfo = serversToServerInfo.get(info.getServerName());
if (storedInfo == null) { if (storedInfo == null) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("received server report from unknown server: " + LOG.debug("Received report from unknown server -- telling it " +
info.getServerName()); "to " + CALL_SERVER_STARTUP + ": " + info.getServerName());
} }
// The HBaseMaster may have been restarted. // The HBaseMaster may have been restarted.
// Tell the RegionServer to start over and call regionServerStartup() // Tell the RegionServer to start over and call regionServerStartup()
return new HMsg[]{CALL_SERVER_STARTUP}; return new HMsg[] {CALL_SERVER_STARTUP};
} else if (storedInfo.getStartCode() != info.getStartCode()) { } else if (storedInfo.getStartCode() != info.getStartCode()) {
// This state is reachable if: // This state is reachable if:
// //
@ -317,7 +316,7 @@ class ServerManager implements HConstants {
serversToServerInfo.notifyAll(); serversToServerInfo.notifyAll();
} }
return new HMsg[]{REGIONSERVER_STOP}; return new HMsg[] {REGIONSERVER_STOP};
} else { } else {
return processRegionServerAllsWell(info, mostLoadedRegions, msgs); return processRegionServerAllsWell(info, mostLoadedRegions, msgs);
} }
@ -372,13 +371,12 @@ class ServerManager implements HConstants {
* @return * @return
* @throws IOException * @throws IOException
*/ */
private HMsg[] processRegionServerAllsWell(HServerInfo serverInfo, HRegionInfo[] mostLoadedRegions, HMsg[] msgs) private HMsg[] processRegionServerAllsWell(HServerInfo serverInfo,
final HRegionInfo[] mostLoadedRegions, HMsg[] msgs)
throws IOException { throws IOException {
// Refresh the info object and the load information // Refresh the info object and the load information
serverAddressToServerInfo.put(serverInfo.getServerAddress(), serverInfo); serverAddressToServerInfo.put(serverInfo.getServerAddress(), serverInfo);
serversToServerInfo.put(serverInfo.getServerName(), serverInfo); serversToServerInfo.put(serverInfo.getServerName(), serverInfo);
HServerLoad load = serversToLoad.get(serverInfo.getServerName()); HServerLoad load = serversToLoad.get(serverInfo.getServerName());
if (load != null) { if (load != null) {
this.master.getMetrics().incrementRequests(load.getNumberOfRequests()); this.master.getMetrics().incrementRequests(load.getNumberOfRequests());
@ -436,8 +434,9 @@ class ServerManager implements HConstants {
int openingCount = 0; int openingCount = 0;
for (int i = 0; i < incomingMsgs.length; i++) { for (int i = 0; i < incomingMsgs.length; i++) {
HRegionInfo region = incomingMsgs[i].getRegionInfo(); HRegionInfo region = incomingMsgs[i].getRegionInfo();
LOG.info("Received " + incomingMsgs[i] + " from " + LOG.info("Processing " + incomingMsgs[i] + " from " +
serverInfo.getServerName() + "; " + (i + 1) + " of " + incomingMsgs.length); serverInfo.getServerName() + "; " + (i + 1) + " of " +
incomingMsgs.length);
switch (incomingMsgs[i].getType()) { switch (incomingMsgs[i].getType()) {
case MSG_REPORT_PROCESS_OPEN: case MSG_REPORT_PROCESS_OPEN:
openingCount++; openingCount++;

View File

@ -24,8 +24,8 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.migration.nineteen.HStoreKey;
import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.MapFile;
import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.Writable;
@ -73,10 +73,10 @@ public class HBaseMapFile extends MapFile {
public HBaseReader(FileSystem fs, String dirName, Configuration conf, public HBaseReader(FileSystem fs, String dirName, Configuration conf,
boolean blockCacheEnabled, HRegionInfo hri) boolean blockCacheEnabled, HRegionInfo hri)
throws IOException { throws IOException {
super(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri), super(fs, dirName, new org.apache.hadoop.hbase.migration.nineteen.HStoreKey.HStoreKeyWritableComparator(hri),
conf, false); // defer opening streams conf, false); // defer opening streams
this.blockCacheEnabled = blockCacheEnabled; this.blockCacheEnabled = blockCacheEnabled;
open(fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri), conf); open(fs, dirName, new org.apache.hadoop.hbase.migration.nineteen.HStoreKey.HStoreKeyWritableComparator(hri), conf);
// Force reading of the mapfile index by calling midKey. Reading the // Force reading of the mapfile index by calling midKey. Reading the
// index will bring the index into memory over here on the client and // index will bring the index into memory over here on the client and
@ -85,7 +85,9 @@ public class HBaseMapFile extends MapFile {
// load the index force the issue in HStoreFile MapFiles because an // load the index force the issue in HStoreFile MapFiles because an
// access may not happen for some time; meantime we're using up datanode // access may not happen for some time; meantime we're using up datanode
// resources (See HADOOP-2341). midKey() goes to index. Does not seek. // resources (See HADOOP-2341). midKey() goes to index. Does not seek.
midKey();
// Disable for migration !!! midKey();
} }
} }
@ -101,7 +103,7 @@ public class HBaseMapFile extends MapFile {
public HBaseWriter(Configuration conf, FileSystem fs, String dirName, public HBaseWriter(Configuration conf, FileSystem fs, String dirName,
SequenceFile.CompressionType compression, final HRegionInfo hri) SequenceFile.CompressionType compression, final HRegionInfo hri)
throws IOException { throws IOException {
super(conf, fs, dirName, new HStoreKey.HStoreKeyWritableComparator(hri), super(conf, fs, dirName, new org.apache.hadoop.hbase.migration.nineteen.HStoreKey.HStoreKeyWritableComparator(hri),
VALUE_CLASS, compression); VALUE_CLASS, compression);
// Default for mapfiles is 128. Makes random reads faster if we // Default for mapfiles is 128. Makes random reads faster if we
// have more keys indexed and we're not 'next'-ing around in the // have more keys indexed and we're not 'next'-ing around in the

View File

@ -23,6 +23,8 @@ import java.io.DataInputStream;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.Random; import java.util.Random;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -44,6 +46,9 @@ import org.apache.hadoop.io.SequenceFile;
* A HStore data file. HStores usually have one or more of these files. They * A HStore data file. HStores usually have one or more of these files. They
* are produced by flushing the memcache to disk. * are produced by flushing the memcache to disk.
* *
* <p>This one has been doctored to be used in migrations. Private and
* protecteds have been made public, etc.
*
* <p>Each HStore maintains a bunch of different data files. The filename is a * <p>Each HStore maintains a bunch of different data files. The filename is a
* mix of the parent dir, the region name, the column name, and a file * mix of the parent dir, the region name, the column name, and a file
* identifier. The name may also be a reference to a store file located * identifier. The name may also be a reference to a store file located
@ -110,14 +115,14 @@ public class HStoreFile implements HConstants {
* @param colFamily name of the column family * @param colFamily name of the column family
* @param fileId file identifier * @param fileId file identifier
* @param ref Reference to another HStoreFile. * @param ref Reference to another HStoreFile.
* @param hri The region info for this file (HACK HBASE-868). TODO: Fix. * @param encodedName Encoded name.
* @throws IOException * @throws IOException
*/ */
HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir, public HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir,
final HRegionInfo hri, byte [] colFamily, long fileId, final int encodedName, byte [] colFamily, long fileId,
final Reference ref) final Reference ref)
throws IOException { throws IOException {
this(conf, fs, basedir, hri, colFamily, fileId, ref, false); this(conf, fs, basedir, encodedName, colFamily, fileId, ref, false);
} }
/** /**
@ -127,20 +132,21 @@ public class HStoreFile implements HConstants {
* @param colFamily name of the column family * @param colFamily name of the column family
* @param fileId file identifier * @param fileId file identifier
* @param ref Reference to another HStoreFile. * @param ref Reference to another HStoreFile.
* @param hri The region info for this file (HACK HBASE-868). TODO: Fix. * @param encodedName Encoded name.
* @param mc Try if this file was result of a major compression. * @param mc Try if this file was result of a major compression.
* @throws IOException * @throws IOException
*/ */
HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir, HStoreFile(HBaseConfiguration conf, FileSystem fs, Path basedir,
final HRegionInfo hri, byte [] colFamily, long fileId, final int encodedName, byte [] colFamily, long fileId,
final Reference ref, final boolean mc) final Reference ref, final boolean mc)
throws IOException { throws IOException {
this.conf = conf; this.conf = conf;
this.fs = fs; this.fs = fs;
this.basedir = basedir; this.basedir = basedir;
this.encodedRegionName = hri.getEncodedName(); this.encodedRegionName = encodedName;
this.colFamily = colFamily; this.colFamily = colFamily;
this.hri = hri; // NOT PASSED IN MIGRATIONS
this.hri = null;
long id = fileId; long id = fileId;
if (id == -1) { if (id == -1) {
@ -164,6 +170,27 @@ public class HStoreFile implements HConstants {
return reference != null; return reference != null;
} }
private static final Pattern REF_NAME_PARSER =
Pattern.compile("^(\\d+)(?:\\.(.+))?$");
/**
* @param p Path to check.
* @return True if the path has format of a HStoreFile reference.
*/
public static boolean isReference(final Path p) {
return isReference(p, REF_NAME_PARSER.matcher(p.getName()));
}
private static boolean isReference(final Path p, final Matcher m) {
if (m == null || !m.matches()) {
LOG.warn("Failed match of store file name " + p.toString());
throw new RuntimeException("Failed match of store file name " +
p.toString());
}
return m.groupCount() > 1 && m.group(2) != null;
}
Reference getReference() { Reference getReference() {
return reference; return reference;
} }
@ -316,7 +343,7 @@ public class HStoreFile implements HConstants {
* @return The sequence id contained in the info file * @return The sequence id contained in the info file
* @throws IOException * @throws IOException
*/ */
long loadInfo(final FileSystem filesystem) throws IOException { public long loadInfo(final FileSystem filesystem) throws IOException {
Path p = null; Path p = null;
if (isReference()) { if (isReference()) {
p = getInfoFilePath(reference.getEncodedRegionName(), p = getInfoFilePath(reference.getEncodedRegionName(),

View File

@ -1004,6 +1004,7 @@ public class HLog implements HConstants, Syncable {
splits = new ArrayList<Path>(logWriters.size()); splits = new ArrayList<Path>(logWriters.size());
for (WriterAndPath wap : logWriters.values()) { for (WriterAndPath wap : logWriters.values()) {
wap.w.close(); wap.w.close();
LOG.debug("Closed " + wap.p);
splits.add(wap.p); splits.add(wap.p);
} }
} }

View File

@ -280,19 +280,20 @@ public class HRegionServer implements HConstants, HRegionInterface,
/** /**
* Creates all of the state that needs to be reconstructed in case we are * Creates all of the state that needs to be reconstructed in case we are
* doing a restart. This is shared between the constructor and restart(). * doing a restart. This is shared between the constructor and restart().
* Both call it.
* @throws IOException * @throws IOException
*/ */
private void reinitialize() throws IOException { private void reinitialize() throws IOException {
abortRequested = false; this.abortRequested = false;
stopRequested.set(false); this.stopRequested.set(false);
shutdownHDFS.set(true); this.shutdownHDFS.set(true);
// Server to handle client requests // Server to handle client requests
this.server = HBaseRPC.getServer(this, address.getBindAddress(), this.server = HBaseRPC.getServer(this, address.getBindAddress(),
address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10), address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
false, conf); false, conf);
this.server.setErrorHandler(this); this.server.setErrorHandler(this);
// Address is givin a default IP for the moment. Will be changed after // Address is giving a default IP for the moment. Will be changed after
// calling the master. // calling the master.
this.serverInfo = new HServerInfo(new HServerAddress( this.serverInfo = new HServerInfo(new HServerAddress(
new InetSocketAddress(address.getBindAddress(), new InetSocketAddress(address.getBindAddress(),
@ -302,11 +303,8 @@ public class HRegionServer implements HConstants, HRegionInterface,
throw new NullPointerException("Server address cannot be null; " + throw new NullPointerException("Server address cannot be null; " +
"hbase-958 debugging"); "hbase-958 debugging");
} }
reinitializeThreads(); reinitializeThreads();
reinitializeZooKeeper(); reinitializeZooKeeper();
int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4); int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4);
for(int i = 0; i < nbBlocks; i++) { for(int i = 0; i < nbBlocks; i++) {
reservedSpace.add(new byte[DEFAULT_SIZE_RESERVATION_BLOCK]); reservedSpace.add(new byte[DEFAULT_SIZE_RESERVATION_BLOCK]);
@ -387,11 +385,9 @@ public class HRegionServer implements HConstants, HRegionInterface,
private void restart() { private void restart() {
LOG.info("Restarting Region Server"); LOG.info("Restarting Region Server");
shutdownHDFS.set(false); shutdownHDFS.set(false);
abort(); abort();
Threads.shutdown(regionServerThread); Threads.shutdown(regionServerThread);
boolean done = false; boolean done = false;
while (!done) { while (!done) {
try { try {
@ -401,7 +397,6 @@ public class HRegionServer implements HConstants, HRegionInterface,
LOG.debug("Error trying to reinitialize ZooKeeper", e); LOG.debug("Error trying to reinitialize ZooKeeper", e);
} }
} }
Thread t = new Thread(this); Thread t = new Thread(this);
String name = regionServerThread.getName(); String name = regionServerThread.getName();
t.setName(name); t.setName(name);
@ -717,10 +712,14 @@ public class HRegionServer implements HConstants, HRegionInterface,
} }
// Master may have sent us a new address with the other configs. // Master may have sent us a new address with the other configs.
// Update our address in this case. See HBASE-719 // Update our address in this case. See HBASE-719
if(conf.get("hbase.regionserver.address") != null) String hra = conf.get("hbase.regionserver.address");
serverInfo.setServerAddress(new HServerAddress if (address != null) {
(conf.get("hbase.regionserver.address"), HServerAddress hsa = new HServerAddress (hra,
serverInfo.getServerAddress().getPort())); this.serverInfo.getServerAddress().getPort());
LOG.info("Master passed us address to use. Was=" +
this.serverInfo.getServerAddress() + ", Now=" + hra);
this.serverInfo.setServerAddress(hsa);
}
// Master sent us hbase.rootdir to use. Should be fully qualified // Master sent us hbase.rootdir to use. Should be fully qualified
// path with file system specification included. Set 'fs.default.name' // path with file system specification included. Set 'fs.default.name'
// to match the filesystem on hbase.rootdir else underlying hadoop hdfs // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
@ -1043,7 +1042,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
private HLog setupHLog() throws RegionServerRunningException, private HLog setupHLog() throws RegionServerRunningException,
IOException { IOException {
Path logdir = new Path(rootDir, HLog.getHLogDirectoryName(serverInfo)); Path logdir = new Path(rootDir, HLog.getHLogDirectoryName(this.serverInfo));
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Log dir " + logdir); LOG.debug("Log dir " + logdir);
} }
@ -1184,7 +1183,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
LOG.info("Failed binding http info server to port: " + port); LOG.info("Failed binding http info server to port: " + port);
port++; port++;
// update HRS server info // update HRS server info
serverInfo.setInfoPort(port); this.serverInfo.setInfoPort(port);
} }
} }
} }
@ -1204,7 +1203,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
// a thread. // a thread.
this.server.start(); this.server.start();
LOG.info("HRegionServer started at: " + LOG.info("HRegionServer started at: " +
serverInfo.getServerAddress().toString()); this.serverInfo.getServerAddress().toString());
} }
/* /*
@ -1351,12 +1350,12 @@ public class HRegionServer implements HConstants, HRegionInterface,
boolean startCodeOk = false; boolean startCodeOk = false;
while(!startCodeOk) { while(!startCodeOk) {
serverInfo.setStartCode(System.currentTimeMillis()); serverInfo.setStartCode(System.currentTimeMillis());
startCodeOk = zooKeeperWrapper.writeRSLocation(serverInfo); startCodeOk = zooKeeperWrapper.writeRSLocation(this.serverInfo);
if(!startCodeOk) { if(!startCodeOk) {
LOG.debug("Start code already taken, trying another one"); LOG.debug("Start code already taken, trying another one");
} }
} }
result = this.hbaseMaster.regionServerStartup(serverInfo); result = this.hbaseMaster.regionServerStartup(this.serverInfo);
break; break;
} catch (Leases.LeaseStillHeldException e) { } catch (Leases.LeaseStillHeldException e) {
LOG.info("Lease " + e.getName() + " already held on master. Check " + LOG.info("Lease " + e.getName() + " already held on master. Check " +
@ -1898,8 +1897,8 @@ public class HRegionServer implements HConstants, HRegionInterface,
try { try {
checkOpen(); checkOpen();
} catch (IOException e) { } catch (IOException e) {
// If checkOpen failed, cancel this lease; filesystem is gone or we're // If checkOpen failed, server not running or filesystem gone,
// closing or something. // cancel this lease; filesystem is gone or we're closing or something.
this.leases.cancelLease(scannerName); this.leases.cancelLease(scannerName);
throw e; throw e;
} }
@ -1932,11 +1931,10 @@ public class HRegionServer implements HConstants, HRegionInterface,
synchronized(scanners) { synchronized(scanners) {
s = scanners.remove(scannerName); s = scanners.remove(scannerName);
} }
if(s == null) { if (s != null) {
throw new UnknownScannerException(scannerName);
}
s.close(); s.close();
this.leases.cancelLease(scannerName); this.leases.cancelLease(scannerName);
}
} catch (Throwable t) { } catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t)); throw convertThrowableToIOE(cleanup(t));
} }
@ -2115,13 +2113,6 @@ public class HRegionServer implements HConstants, HRegionInterface,
} }
} }
/**
* @return Info on this server.
*/
public HServerInfo getServerInfo() {
return this.serverInfo;
}
/** @return the info server */ /** @return the info server */
public InfoServer getInfoServer() { public InfoServer getInfoServer() {
return infoServer; return infoServer;
@ -2472,7 +2463,6 @@ public class HRegionServer implements HConstants, HRegionInterface,
return serverInfo; return serverInfo;
} }
/** /**
* @param args * @param args
*/ */

View File

@ -463,7 +463,7 @@ public class StoreFile implements HConstants {
* @param mc True if this file is product of a major compaction * @param mc True if this file is product of a major compaction
* @throws IOException * @throws IOException
*/ */
static void appendMetadata(final HFile.Writer w, final long maxSequenceId, public static void appendMetadata(final HFile.Writer w, final long maxSequenceId,
final boolean mc) final boolean mc)
throws IOException { throws IOException {
w.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); w.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));

View File

@ -195,8 +195,7 @@ public class MetaUtils {
Scan scan = new Scan(); Scan scan = new Scan();
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
InternalScanner rootScanner = InternalScanner rootScanner = this.rootRegion.getScanner(scan);
rootRegion.getScanner(scan);
try { try {
List<KeyValue> results = new ArrayList<KeyValue>(); List<KeyValue> results = new ArrayList<KeyValue>();

View File

@ -20,7 +20,6 @@
package org.apache.hadoop.hbase.util; package org.apache.hadoop.hbase.util;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import org.apache.commons.cli.Options; import org.apache.commons.cli.Options;
@ -34,10 +33,17 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.migration.nineteen.io.BloomFilterMapFile;
import org.apache.hadoop.hbase.migration.nineteen.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.FSUtils.DirFilter; import org.apache.hadoop.hbase.util.FSUtils.DirFilter;
import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.Tool;
@ -244,16 +250,7 @@ public class Migrate extends Configured implements Tool {
// TOOD: Verify all has been brought over from old to new layout. // TOOD: Verify all has been brought over from old to new layout.
final MetaUtils utils = new MetaUtils(this.conf); final MetaUtils utils = new MetaUtils(this.conf);
try { try {
// TODO: Set the .META. and -ROOT- to flush at 16k? 32k? rewriteHRegionInfo(utils.getRootRegion().getRegionInfo());
// TODO: Enable block cache on all tables
// TODO: Rewrite MEMCACHE_FLUSHSIZE as MEMSTORE_FLUSHSIZE name has changed.
// TODO: Remove tableindexer 'index' attribute index from TableDescriptor (See HBASE-1586)
// TODO: TODO: Move of in-memory parameter from table to column family (from HTD to HCD).
// TODO: Purge isInMemory, etc., methods from HTD as part of migration.
// TODO: Clean up old region log files (HBASE-698)
updateVersions(utils.getRootRegion().getRegionInfo());
enableBlockCache(utils.getRootRegion().getRegionInfo());
// Scan the root region // Scan the root region
utils.scanRootRegion(new MetaUtils.ScannerListener() { utils.scanRootRegion(new MetaUtils.ScannerListener() {
public boolean processRow(HRegionInfo info) public boolean processRow(HRegionInfo info)
@ -262,8 +259,7 @@ public class Migrate extends Configured implements Tool {
migrationNeeded = true; migrationNeeded = true;
return false; return false;
} }
updateVersions(utils.getRootRegion(), info); rewriteHRegionInfo(utils.getRootRegion(), info);
enableBlockCache(utils.getRootRegion(), info);
return true; return true;
} }
}); });
@ -316,9 +312,7 @@ set to control the master's address (not mandatory).
if (mfs.length > 1) { if (mfs.length > 1) {
throw new IOException("Should only be one directory in: " + mfdir); throw new IOException("Should only be one directory in: " + mfdir);
} }
Path mf = mfs[0].getPath(); rewrite(this.conf, this.fs, mfs[0].getPath());
Path infofile = new Path(new Path(family, "info"), mf.getName());
rewrite(this.fs, mf, infofile);
} }
} }
} }
@ -326,20 +320,60 @@ set to control the master's address (not mandatory).
/** /**
* Rewrite the passed mapfile * Rewrite the passed mapfile
* @param mapfiledir * @param fs
* @param infofile * @param mf
* @throws IOExcepion * @throws IOExcepion
*/ */
public static void rewrite (final FileSystem fs, final Path mapfiledir, public static void rewrite (final HBaseConfiguration conf, final FileSystem fs,
final Path infofile) final Path mf)
throws IOException { throws IOException {
if (!fs.exists(mapfiledir)) { Path familydir = mf.getParent().getParent();
throw new FileNotFoundException(mapfiledir.toString()); Path regiondir = familydir.getParent();
} Path basedir = regiondir.getParent();
if (!fs.exists(infofile)) { if (HStoreFile.isReference(mf)) {
throw new FileNotFoundException(infofile.toString()); throw new IOException(mf.toString() + " is Reference");
} }
HStoreFile hsf = new HStoreFile(conf, fs, basedir,
Integer.parseInt(regiondir.getName()),
Bytes.toBytes(familydir.getName()), Long.parseLong(mf.getName()), null);
BloomFilterMapFile.Reader src = hsf.getReader(fs, false, false);
HFile.Writer tgt = StoreFile.getWriter(fs, familydir);
// From old 0.19 HLogEdit.
ImmutableBytesWritable deleteBytes =
new ImmutableBytesWritable("HBASE::DELETEVAL".getBytes("UTF-8"));
try {
while (true) {
HStoreKey key = new HStoreKey();
ImmutableBytesWritable value = new ImmutableBytesWritable();
if (!src.next(key, value)) {
break;
}
byte [][] parts = KeyValue.parseColumn(key.getColumn());
KeyValue kv = deleteBytes.equals(value)?
new KeyValue(key.getRow(), parts[0], parts[1],
key.getTimestamp(), KeyValue.Type.Delete):
new KeyValue(key.getRow(), parts[0], parts[1],
key.getTimestamp(), value.get());
tgt.append(kv);
}
long seqid = hsf.loadInfo(fs);
StoreFile.appendMetadata(tgt, seqid,
hsf.isMajorCompaction());
// Success, delete src.
src.close();
tgt.close();
hsf.delete();
// If we rewrote src, delete mapfiles and info dir.
fs.delete(mf.getParent(), true);
fs.delete(new Path(familydir, "info"), true);
LOG.info("Rewrote " + mf.toString() + " as " + tgt.toString());
} catch (IOException e) {
// If error, delete tgt.
src.close();
tgt.close();
fs.delete(tgt.getPath(), true);
}
} }
/* /*
@ -347,9 +381,9 @@ set to control the master's address (not mandatory).
* @param mr * @param mr
* @param oldHri * @param oldHri
*/ */
void enableBlockCache(HRegion mr, HRegionInfo oldHri) void rewriteHRegionInfo(HRegion mr, HRegionInfo oldHri)
throws IOException { throws IOException {
if (!enableBlockCache(oldHri)) { if (!rewriteHRegionInfo(oldHri)) {
return; return;
} }
Put put = new Put(oldHri.getRegionName()); Put put = new Put(oldHri.getRegionName());
@ -363,7 +397,7 @@ set to control the master's address (not mandatory).
* @param hri Update versions. * @param hri Update versions.
* @param true if we changed value * @param true if we changed value
*/ */
private boolean enableBlockCache(final HRegionInfo hri) { private boolean rewriteHRegionInfo(final HRegionInfo hri) {
boolean result = false; boolean result = false;
HColumnDescriptor hcd = HColumnDescriptor hcd =
hri.getTableDesc().getFamily(HConstants.CATALOG_FAMILY); hri.getTableDesc().getFamily(HConstants.CATALOG_FAMILY);
@ -374,6 +408,17 @@ set to control the master's address (not mandatory).
// Set blockcache enabled. // Set blockcache enabled.
hcd.setBlockCacheEnabled(true); hcd.setBlockCacheEnabled(true);
return true; return true;
// TODO: Rewrite MEMCACHE_FLUSHSIZE as MEMSTORE_FLUSHSIZE name has changed.
// TODO: Remove tableindexer 'index' attribute index from TableDescriptor (See HBASE-1586)
// TODO: TODO: Move of in-memory parameter from table to column family (from HTD to HCD).
// TODO: Purge isInMemory, etc., methods from HTD as part of migration.
// TODO: Set the .META. and -ROOT- to flush at 16k? 32k?
// TODO: Enable block cache on all tables
// TODO: Clean up old region log files (HBASE-698)
} }

View File

@ -112,7 +112,11 @@ public class MiniHBaseCluster implements HConstants {
*/ */
public void abortRegionServer(int serverNumber) { public void abortRegionServer(int serverNumber) {
HRegionServer server = getRegionServer(serverNumber); HRegionServer server = getRegionServer(serverNumber);
LOG.info("Aborting " + server.getServerInfo().toString()); try {
LOG.info("Aborting " + server.getHServerInfo().toString());
} catch (IOException e) {
e.printStackTrace();
}
server.abort(); server.abort();
} }

View File

@ -1,5 +1,6 @@
<%@ page contentType="text/html;charset=UTF-8" <%@ page contentType="text/html;charset=UTF-8"
import="java.util.*" import="java.util.*"
import="java.io.IOException"
import="org.apache.hadoop.io.Text" import="org.apache.hadoop.io.Text"
import="org.apache.hadoop.hbase.regionserver.HRegionServer" import="org.apache.hadoop.hbase.regionserver.HRegionServer"
import="org.apache.hadoop.hbase.regionserver.HRegion" import="org.apache.hadoop.hbase.regionserver.HRegion"
@ -10,7 +11,12 @@
import="org.apache.hadoop.hbase.HServerLoad" import="org.apache.hadoop.hbase.HServerLoad"
import="org.apache.hadoop.hbase.HRegionInfo" %><% import="org.apache.hadoop.hbase.HRegionInfo" %><%
HRegionServer regionServer = (HRegionServer)getServletContext().getAttribute(HRegionServer.REGIONSERVER); HRegionServer regionServer = (HRegionServer)getServletContext().getAttribute(HRegionServer.REGIONSERVER);
HServerInfo serverInfo = regionServer.getServerInfo(); HServerInfo serverInfo = null;
try {
serverInfo = regionServer.getHServerInfo();
} catch (IOException e) {
e.printStackTrace();
}
RegionServerMetrics metrics = regionServer.getMetrics(); RegionServerMetrics metrics = regionServer.getMetrics();
Collection<HRegionInfo> onlineRegions = regionServer.getSortedOnlineRegionInfos(); Collection<HRegionInfo> onlineRegions = regionServer.getSortedOnlineRegionInfos();
int interval = regionServer.getConfiguration().getInt("hbase.regionserver.msginterval", 3000)/1000; int interval = regionServer.getConfiguration().getInt("hbase.regionserver.msginterval", 3000)/1000;