HBASE-451 Remove HTableDescriptor from HRegionInfo -- part 2, some cleanup

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1138120 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-06-21 18:31:26 +00:00
parent cf25a779aa
commit b0439269fe
36 changed files with 798 additions and 543 deletions

View File

@ -0,0 +1,75 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.Map;
/**
* Get, remove and modify table descriptors.
* Used by servers to host descriptors.
*/
public interface TableDescriptors {
/**
* @param tablename
* @return HTableDescriptor for tablename
* @throws TableExistsException
* @throws FileNotFoundException
* @throws IOException
*/
public HTableDescriptor get(final String tablename)
throws TableExistsException, FileNotFoundException, IOException;
/**
* @param tablename
* @return HTableDescriptor for tablename
* @throws TableExistsException
* @throws FileNotFoundException
* @throws IOException
*/
public HTableDescriptor get(final byte[] tablename)
throws TableExistsException, FileNotFoundException, IOException;
/**
* Get Map of all HTableDescriptors. Populates the descriptor cache as a
* side effect.
* @param fs
* @param rootdir
* @return Map of all descriptors.
* @throws IOException
*/
public Map<String, HTableDescriptor> getAll()
throws IOException;
/**
* Add or update descriptor
* @param htd Descriptor to set into TableDescriptors
* @throws IOException
*/
public void add(final HTableDescriptor htd)
throws IOException;
/**
* @param tablename
* @return Instance of table descriptor or null if none found.
* @throws IOException
*/
public HTableDescriptor remove(final String tablename)
throws IOException;
}

View File

@ -25,7 +25,6 @@ import java.io.IOException;
import java.lang.Thread.UncaughtExceptionHandler; import java.lang.Thread.UncaughtExceptionHandler;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
@ -46,7 +45,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Chore; import org.apache.hadoop.hbase.Chore;
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.HTableDescriptor;
import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
@ -64,7 +62,6 @@ import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler; import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
import org.apache.hadoop.hbase.master.handler.SplitRegionHandler; import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.util.Writables;
@ -141,10 +138,6 @@ public class AssignmentManager extends ZooKeeperListener {
private final ExecutorService executorService; private final ExecutorService executorService;
private Map<String, HTableDescriptor> tableDescMap =
new HashMap<String, HTableDescriptor>();
/** /**
* Constructs a new assignment manager. * Constructs a new assignment manager.
* *
@ -153,10 +146,11 @@ public class AssignmentManager extends ZooKeeperListener {
* @param catalogTracker * @param catalogTracker
* @param service * @param service
* @throws KeeperException * @throws KeeperException
* @throws IOException
*/ */
public AssignmentManager(Server master, ServerManager serverManager, public AssignmentManager(Server master, ServerManager serverManager,
CatalogTracker catalogTracker, final ExecutorService service) CatalogTracker catalogTracker, final ExecutorService service)
throws KeeperException { throws KeeperException, IOException {
super(master.getZooKeeper()); super(master.getZooKeeper());
this.master = master; this.master = master;
this.serverManager = serverManager; this.serverManager = serverManager;
@ -172,7 +166,6 @@ public class AssignmentManager extends ZooKeeperListener {
this.zkTable = new ZKTable(this.master.getZooKeeper()); this.zkTable = new ZKTable(this.master.getZooKeeper());
this.maximumAssignmentAttempts = this.maximumAssignmentAttempts =
this.master.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10); this.master.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10);
initHTableDescriptorMap();
} }
/** /**
@ -1070,10 +1063,6 @@ public class AssignmentManager extends ZooKeeperListener {
} }
// Move on to open regions. // Move on to open regions.
try { try {
// Update the tableDesc map.
for (HRegionInfo region : regions) {
updateDescMap(region.getTableNameAsString());
}
// Send OPEN RPC. This can fail if the server on other end is is not up. // Send OPEN RPC. This can fail if the server on other end is is not up.
// If we fail, fail the startup by aborting the server. There is one // If we fail, fail the startup by aborting the server. There is one
// exception we will tolerate: ServerNotRunningException. This is thrown // exception we will tolerate: ServerNotRunningException. This is thrown
@ -2257,140 +2246,6 @@ public class AssignmentManager extends ZooKeeperListener {
LOG.info("Bulk assigning done"); LOG.info("Bulk assigning done");
} }
private void initHTableDescriptorMap() {
try {
synchronized (this.tableDescMap) {
this.tableDescMap =
FSUtils.getTableDescriptors(this.master.getConfiguration());
}
} catch (IOException e) {
LOG.info("IOException while initializing HTableDescriptor Map");
}
}
private HTableDescriptor readTableDescriptor(String tableName)
throws IOException {
return FSUtils.getHTableDescriptor(
this.master.getConfiguration(), tableName);
}
private boolean isRootOrMetaRegion(String tableName) {
return (
tableName.equals(
HRegionInfo.ROOT_REGIONINFO.getTableNameAsString())
||
tableName.equals(
HRegionInfo.FIRST_META_REGIONINFO.getTableNameAsString()));
}
private void updateDescMap(String tableName) throws IOException {
if (this.tableDescMap == null) {
LOG.error("Table Descriptor cache is null. " +
"Skipping desc map update for table = " + tableName);
return;
}
if (tableName == null || isRootOrMetaRegion(tableName))
return;
if (!this.tableDescMap.containsKey(tableName)) {
HTableDescriptor htd = readTableDescriptor(tableName);
if (htd != null) {
LOG.info("Updating TableDesc Map for tablename = " + tableName
+ "htd == " + htd);
synchronized (this.tableDescMap) {
this.tableDescMap.put(tableName, htd);
}
} else {
LOG.info("HTable Descriptor is NULL for table = " + tableName);
}
}
}
public void updateTableDesc(String tableName, HTableDescriptor htd) {
if (this.tableDescMap == null) {
LOG.error("Table Descriptor cache is null. " +
"Skipping desc map update for table = " + tableName);
return;
}
if (tableName == null || isRootOrMetaRegion(tableName))
return;
if (!this.tableDescMap.containsKey(tableName)) {
LOG.error("Table descriptor missing in DescMap. for tablename = " + tableName);
}
synchronized (this.tableDescMap) {
this.tableDescMap.put(tableName, htd);
}
LOG.info("TableDesc updated successfully for table = " + tableName);
}
public void deleteTableDesc(String tableName) {
if (this.tableDescMap == null) {
LOG.error("Table Descriptor cache is null. " +
"Skipping desc map update for table = " + tableName);
return;
}
if (tableName == null || isRootOrMetaRegion(tableName))
return;
if (!this.tableDescMap.containsKey(tableName)) {
LOG.error("Table descriptor missing in DescMap. for tablename = " + tableName);
}
synchronized (this.tableDescMap) {
this.tableDescMap.remove(tableName);
}
LOG.info("TableDesc removed successfully for table = " + tableName);
}
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
List htdList = null;
HTableDescriptor[] htd = null;
if (tableNames != null && tableNames.size() > 0) {
if (this.tableDescMap != null) {
htd = new HTableDescriptor[tableNames.size()];
htdList = new ArrayList();
synchronized (this.tableDescMap) {
int index = 0;
for (String tableName : tableNames) {
HTableDescriptor htdesc = this.tableDescMap.get(tableName);
htd[index++] = this.tableDescMap.get(tableName);
if (htdesc != null) {
htdList.add(htdesc);
}
}
}
}
}
if (htdList != null && htdList.size() > 0 ) {
return (HTableDescriptor[]) htdList.toArray(new HTableDescriptor[htdList.size()]);
}
return null;
}
public HTableDescriptor[] getHTableDescriptors() {
if (this.tableDescMap != null) {
synchronized (this.tableDescMap) {
Collection<HTableDescriptor> htdc = this.tableDescMap.values();
if (htdc != null) {
return htdc.toArray(new HTableDescriptor[htdc.size()]);
}
}
}
return null;
}
public HTableDescriptor getTableDescriptor(String tableName) {
HTableDescriptor htd = null;
if (tableName != null) {
synchronized (this.tableDescMap) {
htd = this.tableDescMap.get(tableName);
}
}
return htd;
}
/** /**
* State of a Region while undergoing transitions. * State of a Region while undergoing transitions.
*/ */

View File

@ -19,6 +19,7 @@
*/ */
package org.apache.hadoop.hbase.master; package org.apache.hadoop.hbase.master;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.TreeMap; import java.util.TreeMap;
@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
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.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.catalog.MetaEditor; import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader; import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -283,9 +285,8 @@ class CatalogJanitor extends Chore {
return result; return result;
} }
private HTableDescriptor getTableDescriptor(byte[] tableName) { private HTableDescriptor getTableDescriptor(byte[] tableName)
return this.services.getAssignmentManager().getTableDescriptor( throws TableExistsException, FileNotFoundException, IOException {
Bytes.toString(tableName)); return this.services.getTableDescriptors().get(Bytes.toString(tableName));
} }
}
}

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
@ -78,6 +79,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.replication.regionserver.Replication; import org.apache.hadoop.hbase.replication.regionserver.Replication;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.InfoServer; import org.apache.hadoop.hbase.util.InfoServer;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Sleeper; import org.apache.hadoop.hbase.util.Sleeper;
@ -178,6 +180,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
private MasterCoprocessorHost cpHost; private MasterCoprocessorHost cpHost;
private final ServerName serverName; private final ServerName serverName;
private TableDescriptors tableDescriptors;
/** /**
* Initializes the HMaster. The steps are as follows: * Initializes the HMaster. The steps are as follows:
* <p> * <p>
@ -410,7 +414,11 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
status.setStatus("Initializing Master file system"); status.setStatus("Initializing Master file system");
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring. // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
this.fileSystemManager = new MasterFileSystem(this, metrics); this.fileSystemManager = new MasterFileSystem(this, this, metrics);
this.tableDescriptors =
new FSTableDescriptors(this.fileSystemManager.getFileSystem(),
this.fileSystemManager.getRootDir());
// publish cluster ID // publish cluster ID
status.setStatus("Publishing Cluster ID in ZooKeeper"); status.setStatus("Publishing Cluster ID in ZooKeeper");
@ -579,6 +587,11 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
return -1; return -1;
} }
@Override
public TableDescriptors getTableDescriptors() {
return this.tableDescriptors;
}
/** @return InfoServer object. Maybe null.*/ /** @return InfoServer object. Maybe null.*/
public InfoServer getInfoServer() { public InfoServer getInfoServer() {
return this.infoServer; return this.infoServer;
@ -948,20 +961,19 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
return hRegionInfos; return hRegionInfos;
} }
private void storeTableDescriptor(HTableDescriptor hTableDescriptor)
throws IOException {
FSUtils.createTableDescriptor(hTableDescriptor, conf);
}
private synchronized void createTable(final HTableDescriptor hTableDescriptor, private synchronized void createTable(final HTableDescriptor hTableDescriptor,
final HRegionInfo [] newRegions, final HRegionInfo [] newRegions,
final boolean sync) final boolean sync)
throws IOException { throws IOException {
String tableName = newRegions[0].getTableNameAsString(); String tableName = newRegions[0].getTableNameAsString();
if(MetaReader.tableExists(catalogTracker, tableName)) { if (MetaReader.tableExists(catalogTracker, tableName)) {
throw new TableExistsException(tableName); throw new TableExistsException(tableName);
} }
storeTableDescriptor(hTableDescriptor); // TODO: Currently we make the table descriptor and as side-effect the
// tableDir is created. Should we change below method to be createTable
// where we create table in tmp dir with its table descriptor file and then
// do rename to move it into place?
FSUtils.createTableDescriptor(hTableDescriptor, conf);
for (HRegionInfo newRegion : newRegions) { for (HRegionInfo newRegion : newRegions) {
// 1. Set table enabling flag up in zk. // 1. Set table enabling flag up in zk.
@ -1373,37 +1385,40 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
} }
/** /**
* Get HTD array for given tables * Get HTD array for given tables
* @param tableNames * @param tableNames
* @return HTableDescriptor[] * @return HTableDescriptor[]
*/ */
public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) { public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
return this.assignmentManager.getHTableDescriptors(tableNames); List<HTableDescriptor> list =
new ArrayList<HTableDescriptor>(tableNames.size());
for (String s: tableNames) {
HTableDescriptor htd = null;
try {
htd = this.tableDescriptors.get(s);
} catch (IOException e) {
LOG.warn("Failed getting descriptor for " + s, e);
}
if (htd == null) continue;
list.add(htd);
}
return list.toArray(new HTableDescriptor [] {});
} }
/** /**
* Get all table descriptors * Get all table descriptors
* @return HTableDescriptor[] * @return All descriptors or null if none.
*/ */
public HTableDescriptor[] getHTableDescriptors() { public HTableDescriptor [] getHTableDescriptors() {
return this.assignmentManager.getHTableDescriptors(); Map<String, HTableDescriptor> descriptors = null;
} try {
descriptors = this.tableDescriptors.getAll();
/** } catch (IOException e) {
* Get a HTD for a given table name LOG.warn("Failed getting all descriptors", e);
* @param tableName
* @return HTableDescriptor
*/
/*
public HTableDescriptor getHTableDescriptor(byte[] tableName) {
if (tableName != null && tableName.length > 0) {
return this.assignmentManager.getTableDescriptor(
Bytes.toString(tableName));
} }
return null; return descriptors == null?
null: descriptors.values().toArray(new HTableDescriptor [] {});
} }
*/
/** /**
* Compute the average load across all region servers. * Compute the average load across all region servers.

View File

@ -33,19 +33,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.master.metrics.MasterMetrics; import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException; import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException;
@ -80,11 +77,14 @@ public class MasterFileSystem {
final Lock splitLogLock = new ReentrantLock(); final Lock splitLogLock = new ReentrantLock();
final boolean distributedLogSplitting; final boolean distributedLogSplitting;
final SplitLogManager splitLogManager; final SplitLogManager splitLogManager;
private final MasterServices services;
public MasterFileSystem(Server master, MasterMetrics metrics) public MasterFileSystem(Server master, MasterServices services,
MasterMetrics metrics)
throws IOException { throws IOException {
this.conf = master.getConfiguration(); this.conf = master.getConfiguration();
this.master = master; this.master = master;
this.services = services;
this.metrics = metrics; this.metrics = metrics;
// Set filesystem to be that of this.rootdir else we get complaints about // Set filesystem to be that of this.rootdir else we get complaints about
// mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
@ -410,28 +410,6 @@ public class MasterFileSystem {
return tableInfoPath; return tableInfoPath;
} }
/**
* Get table info path for a table.
* @param tableName
* @return Table info path
*/
private Path getTablePath(byte[] tableName) {
return new Path(this.rootdir, Bytes.toString(tableName));
}
/**
* Get a HTableDescriptor of a table.
* @param tableName
* @return HTableDescriptor
*/
public HTableDescriptor getTableDescriptor(byte[] tableName) {
try {
return FSUtils.getTableDescriptor(fs, this.rootdir, tableName);
} catch (IOException ioe) {
LOG.info("Exception during readTableDecriptor ", ioe);
}
return null;
}
/** /**
* Create new HTableDescriptor in HDFS. * Create new HTableDescriptor in HDFS.
* @param htableDescriptor * @param htableDescriptor
@ -440,19 +418,6 @@ public class MasterFileSystem {
FSUtils.createTableDescriptor(htableDescriptor, conf); FSUtils.createTableDescriptor(htableDescriptor, conf);
} }
/**
* Update a table descriptor.
* @param htableDescriptor
* @return updated HTableDescriptor
* @throws IOException
*/
public HTableDescriptor updateTableDescriptor(HTableDescriptor htableDescriptor)
throws IOException {
LOG.info("Update Table Descriptor. Current HTD = " + htableDescriptor);
FSUtils.updateHTableDescriptor(fs, conf, htableDescriptor);
return htableDescriptor;
}
/** /**
* Delete column of a table * Delete column of a table
* @param tableName * @param tableName
@ -464,9 +429,9 @@ public class MasterFileSystem {
throws IOException { throws IOException {
LOG.info("DeleteColumn. Table = " + Bytes.toString(tableName) LOG.info("DeleteColumn. Table = " + Bytes.toString(tableName)
+ " family = " + Bytes.toString(familyName)); + " family = " + Bytes.toString(familyName));
HTableDescriptor htd = getTableDescriptor(tableName); HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
htd.removeFamily(familyName); htd.removeFamily(familyName);
updateTableDescriptor(htd); this.services.getTableDescriptors().add(htd);
return htd; return htd;
} }
@ -482,14 +447,14 @@ public class MasterFileSystem {
LOG.info("AddModifyColumn. Table = " + Bytes.toString(tableName) LOG.info("AddModifyColumn. Table = " + Bytes.toString(tableName)
+ " HCD = " + hcd.toString()); + " HCD = " + hcd.toString());
HTableDescriptor htd = getTableDescriptor(tableName); HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
byte [] familyName = hcd.getName(); byte [] familyName = hcd.getName();
if(!htd.hasFamily(familyName)) { if(!htd.hasFamily(familyName)) {
throw new InvalidFamilyOperationException("Family '" + throw new InvalidFamilyOperationException("Family '" +
Bytes.toString(familyName) + "' doesn't exists so cannot be modified"); Bytes.toString(familyName) + "' doesn't exists so cannot be modified");
} }
htd.addFamily(hcd); htd.addFamily(hcd);
updateTableDescriptor(htd); this.services.getTableDescriptors().add(htd);
return htd; return htd;
} }
@ -502,17 +467,15 @@ public class MasterFileSystem {
*/ */
public HTableDescriptor addColumn(byte[] tableName, HColumnDescriptor hcd) public HTableDescriptor addColumn(byte[] tableName, HColumnDescriptor hcd)
throws IOException { throws IOException {
LOG.info("AddColumn. Table = " + Bytes.toString(tableName) LOG.info("AddColumn. Table = " + Bytes.toString(tableName) + " HCD = " +
+ " HCD = " + hcd.toString()); hcd.toString());
HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
HTableDescriptor htd = getTableDescriptor(tableName); if (htd == null) {
if(htd == null) {
throw new InvalidFamilyOperationException("Family '" + throw new InvalidFamilyOperationException("Family '" +
hcd.getNameAsString() + "' cannot be modified as HTD is null"); hcd.getNameAsString() + "' cannot be modified as HTD is null");
} }
htd.addFamily(hcd); htd.addFamily(hcd);
updateTableDescriptor(htd); this.services.getTableDescriptors().add(htd);
return htd; return htd;
} }
}
}

View File

@ -22,11 +22,10 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
/** /**
* Services Master supplies * Services Master supplies
@ -60,4 +59,8 @@ public interface MasterServices extends Server {
*/ */
public void checkTableModifiable(final byte [] tableName) throws IOException; public void checkTableModifiable(final byte [] tableName) throws IOException;
/**
* @return Return table descriptors implementation.
*/
public TableDescriptors getTableDescriptors();
} }

View File

@ -231,7 +231,7 @@ public class SplitLogManager extends ZooKeeperListener {
for (FileStatus lf : logfiles) { for (FileStatus lf : logfiles) {
// TODO If the log file is still being written to - which is most likely // TODO If the log file is still being written to - which is most likely
// the case for the last log file - then its length will show up here // the case for the last log file - then its length will show up here
// as zero. The size of such a file can only be retrieved after after // as zero. The size of such a file can only be retrieved after
// recover-lease is done. totalSize will be under in most cases and the // recover-lease is done. totalSize will be under in most cases and the
// metrics that it drives will also be under-reported. // metrics that it drives will also be under-reported.
totalSize += lf.getLen(); totalSize += lf.getLen();

View File

@ -71,7 +71,7 @@ public class DeleteTableHandler extends TableEventHandler {
// Delete table from FS // Delete table from FS
this.masterServices.getMasterFileSystem().deleteTable(tableName); this.masterServices.getMasterFileSystem().deleteTable(tableName);
// Update table descriptor cache // Update table descriptor cache
am.deleteTableDesc(Bytes.toString(tableName)); this.masterServices.getTableDescriptors().remove(Bytes.toString(tableName));
// If entry for this table in zk, and up in AssignmentManager, remove it. // If entry for this table in zk, and up in AssignmentManager, remove it.
// Call to undisableTable does this. TODO: Make a more formal purge table. // Call to undisableTable does this. TODO: Make a more formal purge table.

View File

@ -25,10 +25,7 @@ import java.util.List;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
public class ModifyTableHandler extends TableEventHandler { public class ModifyTableHandler extends TableEventHandler {
private final HTableDescriptor htd; private final HTableDescriptor htd;
@ -43,20 +40,10 @@ public class ModifyTableHandler extends TableEventHandler {
@Override @Override
protected void handleTableOperation(List<HRegionInfo> hris) protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException { throws IOException {
AssignmentManager am = this.masterServices.getAssignmentManager(); // Update descriptor
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName)); this.masterServices.getTableDescriptors().add(this.htd);
if (htd == null) {
throw new IOException("Modify Table operation could not be completed as " +
"HTableDescritor is missing for table = "
+ Bytes.toString(tableName));
}
// Update table descriptor in HDFS
HTableDescriptor updatedHTD = this.masterServices.getMasterFileSystem()
.updateTableDescriptor(this.htd);
// Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), updatedHTD);
} }
@Override @Override
public String toString() { public String toString() {
String name = "UnknownServerName"; String name = "UnknownServerName";

View File

@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -49,7 +48,7 @@ public class TableAddFamilyHandler extends TableEventHandler {
protected void handleTableOperation(List<HRegionInfo> hris) protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException { throws IOException {
AssignmentManager am = this.masterServices.getAssignmentManager(); AssignmentManager am = this.masterServices.getAssignmentManager();
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName)); HTableDescriptor htd = this.masterServices.getTableDescriptors().get(Bytes.toString(tableName));
byte [] familyName = familyDesc.getName(); byte [] familyName = familyDesc.getName();
if (htd == null) { if (htd == null) {
throw new IOException("Add Family operation could not be completed as " + throw new IOException("Add Family operation could not be completed as " +
@ -65,7 +64,7 @@ public class TableAddFamilyHandler extends TableEventHandler {
htd = this.masterServices.getMasterFileSystem() htd = this.masterServices.getMasterFileSystem()
.addColumn(tableName, familyDesc); .addColumn(tableName, familyDesc);
// Update in-memory descriptor cache // Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), htd); this.masterServices.getTableDescriptors().add(htd);
} }
@Override @Override
public String toString() { public String toString() {

View File

@ -26,9 +26,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -48,7 +46,7 @@ public class TableDeleteFamilyHandler extends TableEventHandler {
@Override @Override
protected void handleTableOperation(List<HRegionInfo> hris) throws IOException { protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
AssignmentManager am = this.masterServices.getAssignmentManager(); AssignmentManager am = this.masterServices.getAssignmentManager();
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName)); HTableDescriptor htd = this.masterServices.getTableDescriptors().get(Bytes.toString(tableName));
if (htd == null) { if (htd == null) {
throw new IOException("Add Family operation could not be completed as " + throw new IOException("Add Family operation could not be completed as " +
"HTableDescritor is missing for table = " "HTableDescritor is missing for table = "
@ -63,7 +61,7 @@ public class TableDeleteFamilyHandler extends TableEventHandler {
htd = this.masterServices.getMasterFileSystem() htd = this.masterServices.getMasterFileSystem()
.deleteColumn(tableName, familyName); .deleteColumn(tableName, familyName);
// Update in-memory descriptor cache // Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), htd); this.masterServices.getTableDescriptors().add(htd);
} }
@Override @Override

View File

@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -49,7 +48,7 @@ public class TableModifyFamilyHandler extends TableEventHandler {
@Override @Override
protected void handleTableOperation(List<HRegionInfo> regions) throws IOException { protected void handleTableOperation(List<HRegionInfo> regions) throws IOException {
AssignmentManager am = this.masterServices.getAssignmentManager(); AssignmentManager am = this.masterServices.getAssignmentManager();
HTableDescriptor htd = am.getTableDescriptor(Bytes.toString(tableName)); HTableDescriptor htd = this.masterServices.getTableDescriptors().get(Bytes.toString(tableName));
byte [] familyName = familyDesc.getName(); byte [] familyName = familyDesc.getName();
if (htd == null) { if (htd == null) {
throw new IOException("Modify Family operation could not be completed as " + throw new IOException("Modify Family operation could not be completed as " +
@ -61,10 +60,9 @@ public class TableModifyFamilyHandler extends TableEventHandler {
Bytes.toString(familyName) + "' doesn't exists so cannot be modified"); Bytes.toString(familyName) + "' doesn't exists so cannot be modified");
} }
// Update table descriptor in HDFS // Update table descriptor in HDFS
htd = this.masterServices.getMasterFileSystem() htd = this.masterServices.getMasterFileSystem().modifyColumn(tableName, familyDesc);
.modifyColumn(tableName, familyDesc);
// Update in-memory descriptor cache // Update in-memory descriptor cache
am.updateTableDesc(Bytes.toString(tableName), htd); this.masterServices.getTableDescriptors().add(htd);
} }
@Override @Override
public String toString() { public String toString() {

View File

@ -199,8 +199,6 @@ public class HRegion implements HeapSize { // , Writable{
final Path regiondir; final Path regiondir;
KeyValue.KVComparator comparator; KeyValue.KVComparator comparator;
private Pair<Long,Long> lastCompactInfo = null;
/* /*
* Data structure of write state flags used coordinating flushes, * Data structure of write state flags used coordinating flushes,
* compactions and closes. * compactions and closes.
@ -282,6 +280,7 @@ public class HRegion implements HeapSize { // , Writable{
this.log = null; this.log = null;
this.regiondir = null; this.regiondir = null;
this.regionInfo = null; this.regionInfo = null;
this.htableDescriptor = null;
this.threadWakeFrequency = 0L; this.threadWakeFrequency = 0L;
this.coprocessorHost = null; this.coprocessorHost = null;
} }
@ -310,26 +309,22 @@ public class HRegion implements HeapSize { // , Writable{
* @see HRegion#newHRegion(Path, HLog, FileSystem, Configuration, org.apache.hadoop.hbase.HRegionInfo, FlushRequester) * @see HRegion#newHRegion(Path, HLog, FileSystem, Configuration, org.apache.hadoop.hbase.HRegionInfo, FlushRequester)
*/ */
public HRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf, public HRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf,
HRegionInfo regionInfo, RegionServerServices rsServices) { HRegionInfo regionInfo, final HTableDescriptor htd,
RegionServerServices rsServices) {
this.tableDir = tableDir; this.tableDir = tableDir;
this.comparator = regionInfo.getComparator(); this.comparator = regionInfo.getComparator();
this.log = log; this.log = log;
this.fs = fs; this.fs = fs;
this.conf = conf; this.conf = conf;
this.regionInfo = regionInfo; this.regionInfo = regionInfo;
this.htableDescriptor = htd;
this.rsServices = rsServices; this.rsServices = rsServices;
this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY,
10 * 1000); 10 * 1000);
String encodedNameStr = this.regionInfo.getEncodedName(); String encodedNameStr = this.regionInfo.getEncodedName();
setHTableSpecificConf();
this.regiondir = getRegionDir(this.tableDir, encodedNameStr); this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
try {
LOG.info("Setting table desc from HDFS. Region = "
+ this.regionInfo.getTableNameAsString());
loadHTableDescriptor(tableDir);
LOG.info(" This HTD from HDFS == " + this.htableDescriptor);
} catch (IOException ioe) {
LOG.error("Could not instantiate region as error loading HTableDescriptor");
}
// don't initialize coprocessors if not running within a regionserver // don't initialize coprocessors if not running within a regionserver
// TODO: revisit if coprocessors should load in other cases // TODO: revisit if coprocessors should load in other cases
if (rsServices != null) { if (rsServices != null) {
@ -341,38 +336,17 @@ public class HRegion implements HeapSize { // , Writable{
} }
} }
private void loadHTableDescriptor(Path tableDir) throws IOException { void setHTableSpecificConf() {
LOG.debug("Assigning tabledesc from .tableinfo for region = " if (this.htableDescriptor == null) return;
+ this.regionInfo.getRegionNameAsString()); LOG.info("Setting up tabledescriptor config now ...");
// load HTableDescriptor long flushSize = this.htableDescriptor.getMemStoreFlushSize();
this.htableDescriptor = FSUtils.getTableDescriptor(tableDir, fs); if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
if (this.htableDescriptor != null) { HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
setHTableSpecificConf();
} else {
throw new IOException("Table description missing in " +
".tableinfo. Cannot create new region."
+ " current region is == " + this.regionInfo.toString());
} }
this.memstoreFlushSize = flushSize;
} this.blockingMemStoreSize = this.memstoreFlushSize *
conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
private void setHTableSpecificConf() {
if (this.htableDescriptor != null) {
LOG.info("Setting up tabledescriptor config now ...");
long flushSize = this.htableDescriptor.getMemStoreFlushSize();
if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
}
this.memstoreFlushSize = flushSize;
this.blockingMemStoreSize = this.memstoreFlushSize *
conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
}
}
public void setHtableDescriptor(HTableDescriptor htableDescriptor) {
this.htableDescriptor = htableDescriptor;
} }
/** /**
@ -2763,11 +2737,12 @@ public class HRegion implements HeapSize { // , Writable{
* @param conf is global configuration settings. * @param conf is global configuration settings.
* @param regionInfo - HRegionInfo that describes the region * @param regionInfo - HRegionInfo that describes the region
* is new), then read them from the supplied path. * is new), then read them from the supplied path.
* @param htd
* @param rsServices * @param rsServices
* @return the new instance * @return the new instance
*/ */
public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs, public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
Configuration conf, HRegionInfo regionInfo, Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
RegionServerServices rsServices) { RegionServerServices rsServices) {
try { try {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@ -2776,9 +2751,10 @@ public class HRegion implements HeapSize { // , Writable{
Constructor<? extends HRegion> c = Constructor<? extends HRegion> c =
regionClass.getConstructor(Path.class, HLog.class, FileSystem.class, regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
Configuration.class, HRegionInfo.class, RegionServerServices.class); Configuration.class, HRegionInfo.class, HTableDescriptor.class,
RegionServerServices.class);
return c.newInstance(tableDir, log, fs, conf, regionInfo, rsServices); return c.newInstance(tableDir, log, fs, conf, regionInfo, htd, rsServices);
} catch (Throwable e) { } catch (Throwable e) {
// todo: what should I throw here? // todo: what should I throw here?
throw new IllegalStateException("Could not instantiate a region instance.", e); throw new IllegalStateException("Could not instantiate a region instance.", e);
@ -2800,9 +2776,8 @@ public class HRegion implements HeapSize { // , Writable{
* @throws IOException * @throws IOException
*/ */
public static HRegion createHRegion(final HRegionInfo info, final Path rootDir, public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
final Configuration conf, final Configuration conf, final HTableDescriptor hTableDescriptor)
final HTableDescriptor hTableDescriptor) throws IOException {
throws IOException {
LOG.info("creating HRegion " + info.getTableNameAsString() LOG.info("creating HRegion " + info.getTableNameAsString()
+ " HTD == " + hTableDescriptor + " RootDir = " + rootDir + + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
" Table name == " + info.getTableNameAsString()); " Table name == " + info.getTableNameAsString());
@ -2812,11 +2787,10 @@ public class HRegion implements HeapSize { // , Writable{
Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName()); Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
FileSystem fs = FileSystem.get(conf); FileSystem fs = FileSystem.get(conf);
fs.mkdirs(regionDir); fs.mkdirs(regionDir);
FSUtils.createTableDescriptor(fs, hTableDescriptor, tableDir);
HRegion region = HRegion.newHRegion(tableDir, HRegion region = HRegion.newHRegion(tableDir,
new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME), new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf), new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf),
fs, conf, info, null); fs, conf, info, hTableDescriptor, null);
region.initialize(); region.initialize();
return region; return region;
} }
@ -2833,10 +2807,11 @@ public class HRegion implements HeapSize { // , Writable{
* *
* @throws IOException * @throws IOException
*/ */
public static HRegion openHRegion(final HRegionInfo info, final HLog wal, public static HRegion openHRegion(final HRegionInfo info,
final HTableDescriptor htd, final HLog wal,
final Configuration conf) final Configuration conf)
throws IOException { throws IOException {
return openHRegion(info, wal, conf, null, null); return openHRegion(info, htd, wal, conf, null, null);
} }
/** /**
@ -2853,8 +2828,9 @@ public class HRegion implements HeapSize { // , Writable{
* *
* @throws IOException * @throws IOException
*/ */
public static HRegion openHRegion(final HRegionInfo info, final HLog wal, public static HRegion openHRegion(final HRegionInfo info,
final Configuration conf, final RegionServerServices rsServices, final HTableDescriptor htd, final HLog wal, final Configuration conf,
final RegionServerServices rsServices,
final CancelableProgressable reporter) final CancelableProgressable reporter)
throws IOException { throws IOException {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -2866,14 +2842,14 @@ public class HRegion implements HeapSize { // , Writable{
Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf), Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
info.getTableName()); info.getTableName());
HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info, HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
rsServices); htd, rsServices);
return r.openHRegion(reporter); return r.openHRegion(reporter);
} }
public static HRegion openHRegion(Path tableDir, final HRegionInfo info, public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
final HLog wal, final Configuration conf) final HTableDescriptor htd, final HLog wal, final Configuration conf)
throws IOException { throws IOException {
return openHRegion(tableDir, info, wal, conf, null, null); return openHRegion(tableDir, info, htd, wal, conf, null, null);
} }
/** /**
@ -2891,21 +2867,19 @@ public class HRegion implements HeapSize { // , Writable{
* @throws IOException * @throws IOException
*/ */
public static HRegion openHRegion(final Path tableDir, final HRegionInfo info, public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
final HLog wal, final Configuration conf, final HTableDescriptor htd, final HLog wal, final Configuration conf,
final RegionServerServices rsServices, final RegionServerServices rsServices,
final CancelableProgressable reporter) final CancelableProgressable reporter)
throws IOException { throws IOException {
if (info == null) throw new NullPointerException("Passed region info is null");
LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString()); LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Opening region: " + info); LOG.debug("Opening region: " + info);
} }
if (info == null) {
throw new NullPointerException("Passed region info is null");
}
Path dir = HTableDescriptor.getTableDir(tableDir, Path dir = HTableDescriptor.getTableDir(tableDir,
info.getTableName()); info.getTableName());
HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info, HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
rsServices); htd, rsServices);
return r.openHRegion(reporter); return r.openHRegion(reporter);
} }
@ -3077,7 +3051,8 @@ public class HRegion implements HeapSize { // , Writable{
* @return new merged region * @return new merged region
* @throws IOException * @throws IOException
*/ */
public static HRegion merge(HRegion a, HRegion b) throws IOException { public static HRegion merge(HRegion a, HRegion b)
throws IOException {
if (!a.getRegionInfo().getTableNameAsString().equals( if (!a.getRegionInfo().getTableNameAsString().equals(
b.getRegionInfo().getTableNameAsString())) { b.getRegionInfo().getTableNameAsString())) {
throw new IOException("Regions do not belong to the same table"); throw new IOException("Regions do not belong to the same table");
@ -3179,7 +3154,8 @@ public class HRegion implements HeapSize { // , Writable{
LOG.debug("Files for new region"); LOG.debug("Files for new region");
listPaths(fs, newRegionDir); listPaths(fs, newRegionDir);
} }
HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf, newRegionInfo, null); HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf,
newRegionInfo, a.getTableDesc(), null);
dstRegion.readRequestsCount.set(a.readRequestsCount.get() + b.readRequestsCount.get()); dstRegion.readRequestsCount.set(a.readRequestsCount.get() + b.readRequestsCount.get());
dstRegion.writeRequestsCount.set(a.writeRequestsCount.get() + b.writeRequestsCount.get()); dstRegion.writeRequestsCount.set(a.writeRequestsCount.get() + b.writeRequestsCount.get());
dstRegion.initialize(); dstRegion.initialize();
@ -3592,7 +3568,7 @@ public class HRegion implements HeapSize { // , Writable{
public static final long FIXED_OVERHEAD = ClassSize.align( public static final long FIXED_OVERHEAD = ClassSize.align(
(4 * Bytes.SIZEOF_LONG) + ClassSize.ARRAY + (4 * Bytes.SIZEOF_LONG) + ClassSize.ARRAY +
ClassSize.align(28 * ClassSize.REFERENCE) + ClassSize.OBJECT + ClassSize.align(27 * ClassSize.REFERENCE) + ClassSize.OBJECT +
ClassSize.align(Bytes.SIZEOF_INT)); ClassSize.align(Bytes.SIZEOF_INT));
public static final long DEEP_OVERHEAD = FIXED_OVERHEAD + public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
@ -3745,10 +3721,11 @@ public class HRegion implements HeapSize { // , Writable{
String metaStr = Bytes.toString(HConstants.META_TABLE_NAME); String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
// Currently expects tables have one region only. // Currently expects tables have one region only.
if (p.getName().startsWith(rootStr)) { if (p.getName().startsWith(rootStr)) {
region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO, null); region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
HTableDescriptor.ROOT_TABLEDESC, null);
} else if (p.getName().startsWith(metaStr)) { } else if (p.getName().startsWith(metaStr)) {
region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.FIRST_META_REGIONINFO, region = HRegion.newHRegion(p, log, fs, c,
null); HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
} else { } else {
throw new IOException("Not a known catalog table: " + p.toString()); throw new IOException("Not a known catalog table: " + p.toString());
} }

View File

@ -63,12 +63,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress; import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HServerInfo; import org.apache.hadoop.hbase.HServerInfo;
import org.apache.hadoop.hbase.HServerLoad; import org.apache.hadoop.hbase.HServerLoad;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterAddressTracker; import org.apache.hadoop.hbase.MasterAddressTracker;
import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.RemoteExceptionHandler;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.UnknownRowLockException; import org.apache.hadoop.hbase.UnknownRowLockException;
import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.YouAreDeadException;
@ -120,6 +122,7 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest; import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.InfoServer; import org.apache.hadoop.hbase.util.InfoServer;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
@ -293,6 +296,11 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
*/ */
private final long startcode; private final long startcode;
/**
* Go here to get table descriptors.
*/
private TableDescriptors tableDescriptors;
/** /**
* Starts a HRegionServer at the default location * Starts a HRegionServer at the default location
* *
@ -863,6 +871,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
// Get fs instance used by this RS // Get fs instance used by this RS
this.fs = FileSystem.get(this.conf); this.fs = FileSystem.get(this.conf);
this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR)); this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
this.tableDescriptors = new FSTableDescriptors(this.fs, this.rootDir, true);
this.hlog = setupWALAndReplication(); this.hlog = setupWALAndReplication();
// Init in here rather than in constructor after thread name has been set // Init in here rather than in constructor after thread name has been set
this.metrics = new RegionServerMetrics(); this.metrics = new RegionServerMetrics();
@ -2268,12 +2277,13 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
LOG.info("Received request to open region: " + LOG.info("Received request to open region: " +
region.getRegionNameAsString()); region.getRegionNameAsString());
if (this.stopped) throw new RegionServerStoppedException(); if (this.stopped) throw new RegionServerStoppedException();
HTableDescriptor htd = this.tableDescriptors.get(region.getTableName());
if (region.isRootRegion()) { if (region.isRootRegion()) {
this.service.submit(new OpenRootHandler(this, this, region)); this.service.submit(new OpenRootHandler(this, this, region, htd));
} else if(region.isMetaRegion()) { } else if(region.isMetaRegion()) {
this.service.submit(new OpenMetaHandler(this, this, region)); this.service.submit(new OpenMetaHandler(this, this, region, htd));
} else { } else {
this.service.submit(new OpenRegionHandler(this, this, region)); this.service.submit(new OpenRegionHandler(this, this, region, htd));
} }
} }

View File

@ -567,7 +567,7 @@ public class SplitTransaction {
this.splitdir, hri); this.splitdir, hri);
HRegion r = HRegion.newHRegion(this.parent.getTableDir(), HRegion r = HRegion.newHRegion(this.parent.getTableDir(),
this.parent.getLog(), fs, this.parent.getConf(), this.parent.getLog(), fs, this.parent.getConf(),
hri, rsServices); hri, this.parent.getTableDesc(), rsServices);
r.readRequestsCount.set(this.parent.getReadRequestsCount() / 2); r.readRequestsCount.set(this.parent.getReadRequestsCount() / 2);
r.writeRequestsCount.set(this.parent.getWriteRequestsCount() / 2); r.writeRequestsCount.set(this.parent.getWriteRequestsCount() / 2);
HRegion.moveInitialFilesIntoPlace(fs, regionDir, r.getRegionDir()); HRegion.moveInitialFilesIntoPlace(fs, regionDir, r.getRegionDir());

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.regionserver.handler; package org.apache.hadoop.hbase.regionserver.handler;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@ -30,7 +31,8 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
*/ */
public class OpenMetaHandler extends OpenRegionHandler { public class OpenMetaHandler extends OpenRegionHandler {
public OpenMetaHandler(final Server server, public OpenMetaHandler(final Server server,
final RegionServerServices rsServices, HRegionInfo regionInfo) { final RegionServerServices rsServices, HRegionInfo regionInfo,
super(server,rsServices, regionInfo, EventType.M_RS_OPEN_META); final HTableDescriptor htd) {
super(server,rsServices, regionInfo, htd, EventType.M_RS_OPEN_META);
} }
} }

View File

@ -26,6 +26,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
@ -45,6 +46,7 @@ public class OpenRegionHandler extends EventHandler {
private final RegionServerServices rsServices; private final RegionServerServices rsServices;
private final HRegionInfo regionInfo; private final HRegionInfo regionInfo;
private final HTableDescriptor htd;
// We get version of our znode at start of open process and monitor it across // We get version of our znode at start of open process and monitor it across
// the total open. We'll fail the open if someone hijacks our znode; we can // the total open. We'll fail the open if someone hijacks our znode; we can
@ -52,16 +54,18 @@ public class OpenRegionHandler extends EventHandler {
private volatile int version = -1; private volatile int version = -1;
public OpenRegionHandler(final Server server, public OpenRegionHandler(final Server server,
final RegionServerServices rsServices, HRegionInfo regionInfo) { final RegionServerServices rsServices, HRegionInfo regionInfo,
this(server, rsServices, regionInfo, EventType.M_RS_OPEN_REGION); HTableDescriptor htd) {
this (server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION);
} }
protected OpenRegionHandler(final Server server, protected OpenRegionHandler(final Server server,
final RegionServerServices rsServices, final HRegionInfo regionInfo, final RegionServerServices rsServices, final HRegionInfo regionInfo,
EventType eventType) { final HTableDescriptor htd, EventType eventType) {
super(server, eventType); super(server, eventType);
this.rsServices = rsServices; this.rsServices = rsServices;
this.regionInfo = regionInfo; this.regionInfo = regionInfo;
this.htd = htd;
} }
public HRegionInfo getRegionInfo() { public HRegionInfo getRegionInfo() {
@ -184,7 +188,7 @@ public class OpenRegionHandler extends EventHandler {
// Was there an exception opening the region? This should trigger on // Was there an exception opening the region? This should trigger on
// InterruptedException too. If so, we failed. // InterruptedException too. If so, we failed.
return !t.interrupted() && t.getException() == null; return !Thread.interrupted() && t.getException() == null;
} }
/** /**
@ -269,8 +273,9 @@ public class OpenRegionHandler extends EventHandler {
try { try {
// Instantiate the region. This also periodically tickles our zk OPENING // Instantiate the region. This also periodically tickles our zk OPENING
// state so master doesn't timeout this region in transition. // state so master doesn't timeout this region in transition.
region = HRegion.openHRegion(tableDir, this.regionInfo, this.rsServices.getWAL(), region = HRegion.openHRegion(tableDir, this.regionInfo, this.htd,
this.server.getConfiguration(), this.rsServices, this.rsServices.getWAL(), this.server.getConfiguration(),
this.rsServices,
new CancelableProgressable() { new CancelableProgressable() {
public boolean progress() { public boolean progress() {
// We may lose the znode ownership during the open. Currently its // We may lose the znode ownership during the open. Currently its
@ -296,8 +301,9 @@ public class OpenRegionHandler extends EventHandler {
try { try {
// Instantiate the region. This also periodically tickles our zk OPENING // Instantiate the region. This also periodically tickles our zk OPENING
// state so master doesn't timeout this region in transition. // state so master doesn't timeout this region in transition.
region = HRegion.openHRegion(this.regionInfo, this.rsServices.getWAL(), region = HRegion.openHRegion(this.regionInfo, this.htd,
this.server.getConfiguration(), this.rsServices, this.rsServices.getWAL(), this.server.getConfiguration(),
this.rsServices,
new CancelableProgressable() { new CancelableProgressable() {
public boolean progress() { public boolean progress() {
// We may lose the znode ownership during the open. Currently its // We may lose the znode ownership during the open. Currently its
@ -375,4 +381,4 @@ public class OpenRegionHandler extends EventHandler {
private boolean isGoodVersion() { private boolean isGoodVersion() {
return this.version != -1; return this.version != -1;
} }
} }

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.regionserver.handler; package org.apache.hadoop.hbase.regionserver.handler;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@ -30,7 +31,8 @@ import org.apache.hadoop.hbase.regionserver.RegionServerServices;
*/ */
public class OpenRootHandler extends OpenRegionHandler { public class OpenRootHandler extends OpenRegionHandler {
public OpenRootHandler(final Server server, public OpenRootHandler(final Server server,
final RegionServerServices rsServices, HRegionInfo regionInfo) { final RegionServerServices rsServices, HRegionInfo regionInfo,
super(server, rsServices, regionInfo, EventType.M_RS_OPEN_ROOT); final HTableDescriptor htd) {
super(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_ROOT);
} }
} }

View File

@ -173,7 +173,10 @@ public class SequenceFileLogWriter implements HLog.Writer {
@Override @Override
public void close() throws IOException { public void close() throws IOException {
this.writer.close(); if (this.writer != null) {
this.writer.close();
this.writer = null;
}
} }
@Override @Override
@ -205,4 +208,4 @@ public class SequenceFileLogWriter implements HLog.Writer {
public FSDataOutputStream getWriterFSDataOutputStream() { public FSDataOutputStream getWriterFSDataOutputStream() {
return this.writer_out; return this.writer_out;
} }
} }

View File

@ -0,0 +1,184 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableExistsException;
/**
* Implementation of {@link TableDescriptors} that reads descriptors from the
* passed filesystem. It expects descriptors to be in a file under the
* table's directory in FS. Can be read-only -- i.e. does not modify
* the filesystem or can be read and write.
*/
public class FSTableDescriptors implements TableDescriptors {
private final FileSystem fs;
private final Path rootdir;
private final boolean fsreadonly;
long cachehits = 0;
long invocations = 0;
// This cache does not age out the old stuff. Thinking is that the amount
// of data we keep up in here is so small, no need to do occasional purge.
// TODO.
private final Map<String, TableDescriptorModtime> cache =
new ConcurrentHashMap<String, TableDescriptorModtime>();
/**
* Data structure to hold modification time and table descriptor.
*/
static class TableDescriptorModtime {
private final HTableDescriptor descriptor;
private final long modtime;
TableDescriptorModtime(final long modtime, final HTableDescriptor htd) {
this.descriptor = htd;
this.modtime = modtime;
}
long getModtime() {
return this.modtime;
}
HTableDescriptor getTableDescriptor() {
return this.descriptor;
}
}
public FSTableDescriptors(final FileSystem fs, final Path rootdir) {
this(fs, rootdir, false);
}
/**
* @param fs
* @param rootdir
* @param fsreadOnly True if we are read-only when it comes to filesystem
* operations; i.e. on remove, we do not do delete in fs.
*/
public FSTableDescriptors(final FileSystem fs, final Path rootdir,
final boolean fsreadOnly) {
super();
this.fs = fs;
this.rootdir = rootdir;
this.fsreadonly = fsreadOnly;
}
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.TableDescriptors#getHTableDescriptor(java.lang.String)
*/
@Override
public HTableDescriptor get(final byte [] tablename)
throws TableExistsException, FileNotFoundException, IOException {
return get(Bytes.toString(tablename));
}
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.TableDescriptors#getTableDescriptor(byte[])
*/
@Override
public HTableDescriptor get(final String tablename)
throws TableExistsException, FileNotFoundException, IOException {
invocations++;
if (HTableDescriptor.ROOT_TABLEDESC.getNameAsString().equals(tablename)) {
cachehits++;
return HTableDescriptor.ROOT_TABLEDESC;
}
if (HTableDescriptor.META_TABLEDESC.getNameAsString().equals(tablename)) {
cachehits++;
return HTableDescriptor.META_TABLEDESC;
}
// Look in cache of descriptors.
TableDescriptorModtime tdm = this.cache.get(tablename);
// Check mod time has not changed (this is trip to NN).
long modtime =
FSUtils.getTableInfoModtime(this.fs, this.rootdir, tablename);
if (tdm != null) {
if (modtime <= tdm.getModtime()) {
cachehits++;
return tdm.getTableDescriptor();
}
}
HTableDescriptor htd =
FSUtils.getTableDescriptor(this.fs, this.rootdir, tablename);
if (htd == null) {
// More likely is above will throw a FileNotFoundException
throw new TableExistsException("No descriptor for " + tablename);
}
this.cache.put(tablename, new TableDescriptorModtime(modtime, htd));
return htd;
}
/* (non-Javadoc)
* @see org.apache.hadoop.hbase.TableDescriptors#getTableDescriptors(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path)
*/
@Override
public Map<String, HTableDescriptor> getAll()
throws IOException {
Map<String, HTableDescriptor> htds = new TreeMap<String, HTableDescriptor>();
List<Path> tableDirs = FSUtils.getTableDirs(fs, rootdir);
for (Path d: tableDirs) {
HTableDescriptor htd = get(d.getName());
if (htd == null) continue;
htds.put(d.getName(), htd);
}
return htds;
}
@Override
public void add(HTableDescriptor htd) throws IOException {
if (Bytes.equals(HConstants.ROOT_TABLE_NAME, htd.getName())) {
throw new NotImplementedException();
}
if (Bytes.equals(HConstants.META_TABLE_NAME, htd.getName())) {
throw new NotImplementedException();
}
if (!this.fsreadonly) FSUtils.updateHTableDescriptor(this.fs, this.rootdir, htd);
long modtime =
FSUtils.getTableInfoModtime(this.fs, this.rootdir, htd.getNameAsString());
this.cache.put(htd.getNameAsString(), new TableDescriptorModtime(modtime, htd));
}
@Override
public HTableDescriptor remove(final String tablename)
throws IOException {
if (!this.fsreadonly) {
Path tabledir = FSUtils.getTablePath(this.rootdir, tablename);
if (this.fs.exists(tabledir)) {
if (!this.fs.delete(tabledir, true)) {
throw new IOException("Failed delete of " + tabledir.toString());
}
}
}
TableDescriptorModtime tdm = this.cache.remove(tablename);
return tdm == null? null: tdm.getTableDescriptor();
}
}

View File

@ -49,7 +49,9 @@ import java.io.InterruptedIOException;
import java.lang.reflect.InvocationTargetException; import java.lang.reflect.InvocationTargetException;
import java.net.URI; import java.net.URI;
import java.net.URISyntaxException; import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
/** /**
@ -835,68 +837,68 @@ public class FSUtils {
LOG.info("Finished lease recover attempt for " + p); LOG.info("Finished lease recover attempt for " + p);
} }
/**
public static Map<String, HTableDescriptor> getTableDescriptors( * @param fs
final Configuration config) * @param rootdir
* @return All the table directories under <code>rootdir</code>
* @throws IOException
*/
public static List<Path> getTableDirs(final FileSystem fs, final Path rootdir)
throws IOException { throws IOException {
Path path = getRootDir(config);
// since HMaster.getFileSystem() is package private
FileSystem fs = path.getFileSystem(config);
return getTableDescriptors(fs, path);
}
public static Map<String, HTableDescriptor> getTableDescriptors(
final FileSystem fs, final Path hbaseRootDir)
throws IOException {
Map<String, HTableDescriptor> desc =
new HashMap<String, HTableDescriptor>();
DirFilter df = new DirFilter(fs);
// presumes any directory under hbase.rootdir is a table // presumes any directory under hbase.rootdir is a table
FileStatus [] tableDirs = fs.listStatus(hbaseRootDir, df); FileStatus [] dirs = fs.listStatus(rootdir, new DirFilter(fs));
for (FileStatus tableDir : tableDirs) { List<Path> tabledirs = new ArrayList<Path>(dirs.length);
Path d = tableDir.getPath(); for (FileStatus dir: dirs) {
String tableName = d.getName(); Path p = dir.getPath();
String tableName = p.getName();
if (tableName.equals(HConstants.HREGION_LOGDIR_NAME) if (tableName.equals(HConstants.HREGION_LOGDIR_NAME) ||
|| tableName.equals(Bytes.toString(HConstants.ROOT_TABLE_NAME)) tableName.equals(Bytes.toString(HConstants.ROOT_TABLE_NAME)) ||
|| tableName.equals(Bytes.toString(HConstants.META_TABLE_NAME)) tableName.equals(Bytes.toString(HConstants.META_TABLE_NAME)) ||
|| tableName.equals(HConstants.HREGION_OLDLOGDIR_NAME) tableName.equals(HConstants.HREGION_OLDLOGDIR_NAME) ) {
) {
continue; continue;
} }
LOG.info("Adding tabledescriptor for table = " + tableName); tabledirs.add(p);
HTableDescriptor htd = readTableDescriptor(fs, hbaseRootDir,
tableName);
if (htd != null) {
if (!desc.containsKey(tableName)) {
desc.put(tableName, htd);
}
}
} }
return desc; return tabledirs;
}
private static Path getTableInfoPath(Path hbaseRootDir, String tableName) {
Path tablePath = new Path(hbaseRootDir, tableName);
return new Path(tablePath, HConstants.TABLEINFO_NAME);
} }
/** /**
* Get table info path for a table. * Get table info path for a table.
* @param rootdir
* @param tableName * @param tableName
* @return Table info path * @return Table info path
*/ */
private static Path getTableInfoPath(byte[] tableName, Configuration conf) throws IOException { private static Path getTableInfoPath(Path rootdir, String tablename) {
Path tablePath = new Path(getRootDir(conf), Bytes.toString(tableName)); Path tablePath = getTablePath(rootdir, tablename);
Path tableInfoPath = new Path(tablePath, HConstants.TABLEINFO_NAME); return new Path(tablePath, HConstants.TABLEINFO_NAME);
return tableInfoPath;
} }
private static Path getTablePath(byte[] tableName, Configuration conf) throws IOException { /**
return new Path(getRootDir(conf), Bytes.toString(tableName)); * @param fs
* @param rootdir
* @param tablename
* @return Modification time for the table {@link HConstants#TABLEINFO_NAME} file.
* @throws IOException
*/
public static long getTableInfoModtime(final FileSystem fs, final Path rootdir,
final String tablename)
throws IOException {
Path p = getTablePath(rootdir, tablename);
FileStatus [] status = fs.listStatus(p);
if (status.length < 1) throw new FileNotFoundException("No status for " + p.toString());
return status[0].getModificationTime();
} }
private static FileSystem getCurrentFileSystem(Configuration conf) throws IOException { public static Path getTablePath(Path rootdir, byte [] tableName) {
return getTablePath(rootdir, Bytes.toString(tableName));
}
public static Path getTablePath(Path rootdir, final String tableName) {
return new Path(rootdir, tableName);
}
private static FileSystem getCurrentFileSystem(Configuration conf)
throws IOException {
return getRootDir(conf).getFileSystem(conf); return getRootDir(conf).getFileSystem(conf);
} }
@ -908,27 +910,11 @@ public class FSUtils {
* @throws IOException * @throws IOException
*/ */
public static HTableDescriptor getHTableDescriptor(Configuration config, public static HTableDescriptor getHTableDescriptor(Configuration config,
String tableName) String tableName)
throws IOException { throws IOException {
Path path = getRootDir(config); Path path = getRootDir(config);
FileSystem fs = path.getFileSystem(config); FileSystem fs = path.getFileSystem(config);
return readTableDescriptor(fs, path, tableName); return getTableDescriptor(fs, path, tableName);
}
private static HTableDescriptor readTableDescriptor(FileSystem fs,
Path hbaseRootDir,
String tableName) {
try {
FSDataInputStream fsDataInputStream =
fs.open(getTableInfoPath(hbaseRootDir, tableName));
HTableDescriptor hTableDescriptor = new HTableDescriptor();
hTableDescriptor.readFields(fsDataInputStream);
fsDataInputStream.close();
return hTableDescriptor;
} catch (IOException ioe) {
LOG.info("Exception during readTableDecriptor. Current table name = " + tableName , ioe);
}
return null;
} }
/** /**
@ -936,92 +922,99 @@ public class FSUtils {
* @param fs * @param fs
* @param hbaseRootDir * @param hbaseRootDir
* @param tableName * @param tableName
* @return * @return Descriptor or null if none found.
* @throws IOException * @throws IOException
*/ */
public static HTableDescriptor getTableDescriptor(FileSystem fs, public static HTableDescriptor getTableDescriptor(FileSystem fs,
Path hbaseRootDir, Path hbaseRootDir, byte[] tableName)
byte[] tableName) throws IOException {
throws IOException { return getTableDescriptor(fs, hbaseRootDir, Bytes.toString(tableName));
return readTableDescriptor(fs, hbaseRootDir, Bytes.toString(tableName));
} }
public static HTableDescriptor getTableDescriptor(FileSystem fs,
public static HTableDescriptor getTableDescriptor(Path tableDir, FileSystem fs) { Path hbaseRootDir, String tableName) {
HTableDescriptor htd = null;
try { try {
LOG.info("Reading table descriptor from .tableinfo. current path = " htd = getTableDescriptor(fs, getTablePath(hbaseRootDir, tableName));
+ tableDir);
if (tableDir == null) {
LOG.info("Reading table descriptor from .tableinfo current tablename is NULL ");
return null;
}
FSDataInputStream fsDataInputStream =
fs.open(new Path(tableDir, HConstants.TABLEINFO_NAME));
HTableDescriptor hTableDescriptor = new HTableDescriptor();
hTableDescriptor.readFields(fsDataInputStream);
LOG.info("Current tabledescriptor from .tableinfo is " + hTableDescriptor.toString());
fsDataInputStream.close();
return hTableDescriptor;
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.info("Exception during getTableDescriptor ", ioe); LOG.debug("Exception during readTableDecriptor. Current table name = " +
tableName , ioe);
} }
return null; return htd;
} }
/** public static HTableDescriptor getTableDescriptor(FileSystem fs, Path tableDir)
* Create new HTableDescriptor in HDFS. throws IOException {
if (tableDir == null) throw new NullPointerException();
FSDataInputStream fsDataInputStream =
fs.open(new Path(tableDir, HConstants.TABLEINFO_NAME));
HTableDescriptor hTableDescriptor = null;
try {
hTableDescriptor = new HTableDescriptor();
hTableDescriptor.readFields(fsDataInputStream);
} finally {
fsDataInputStream.close();
}
return hTableDescriptor;
}
/**
* Create new HTableDescriptor in HDFS. Happens when we are creating table.
/**
* @param htableDescriptor * @param htableDescriptor
* @param conf
*/ */
public static void createTableDescriptor(HTableDescriptor htableDescriptor, public static void createTableDescriptor(HTableDescriptor htableDescriptor,
Configuration conf) { Configuration conf) {
try { try {
Path tableDir = getTablePath(htableDescriptor.getName(), conf);
FileSystem fs = getCurrentFileSystem(conf); FileSystem fs = getCurrentFileSystem(conf);
createTableDescriptor(fs, htableDescriptor, tableDir); createTableDescriptor(fs, getRootDir(conf), htableDescriptor);
} catch(IOException ioe) { } catch(IOException ioe) {
LOG.info("IOException while trying to create tableInfo in HDFS", ioe); LOG.info("IOException while trying to create tableInfo in HDFS", ioe);
} }
} }
/**
* @param fs
* @param htableDescriptor
* @param rootdir
*/
public static void createTableDescriptor(FileSystem fs, public static void createTableDescriptor(FileSystem fs,
HTableDescriptor htableDescriptor, Path rootdir, HTableDescriptor htableDescriptor) {
Path tableDir) {
try { try {
Path tableInfoPath = new Path(tableDir, HConstants.TABLEINFO_NAME); Path tableInfoPath =
LOG.info("Current tableInfoPath = " + tableInfoPath getTableInfoPath(rootdir, htableDescriptor.getNameAsString());
+ " tableDir = " + tableDir) ; LOG.info("Current tableInfoPath = " + tableInfoPath) ;
if (fs.exists(tableInfoPath) && if (fs.exists(tableInfoPath) &&
fs.getFileStatus(tableInfoPath).getLen() > 0) { fs.getFileStatus(tableInfoPath).getLen() > 0) {
LOG.info("TableInfo already exists.. Skipping creation"); LOG.info("TableInfo already exists.. Skipping creation");
return; return;
} }
writeTableDescriptor(fs, htableDescriptor, tableDir); writeTableDescriptor(fs, htableDescriptor,
getTablePath(rootdir, htableDescriptor.getNameAsString()));
} catch(IOException ioe) { } catch(IOException ioe) {
LOG.info("IOException while trying to create tableInfo in HDFS", ioe); LOG.info("IOException while trying to create tableInfo in HDFS", ioe);
} }
} }
/**
* Called when we are creating a table to write out the tables' descriptor.
* @param fs
* @param hTableDescriptor
* @param tableDir
* @throws IOException
*/
private static void writeTableDescriptor(FileSystem fs, private static void writeTableDescriptor(FileSystem fs,
HTableDescriptor hTableDescriptor, HTableDescriptor hTableDescriptor, Path tableDir)
Path tableDir) throws IOException { throws IOException {
// Create in tmpdir and then move into place in case we crash after // Create in tmpdir and then move into place in case we crash after
// create but before close. If we don't successfully close the file, // create but before close. If we don't successfully close the file,
// subsequent region reopens will fail the below because create is // subsequent region reopens will fail the below because create is
// registered in NN. // registered in NN.
Path tableInfoPath = new Path(tableDir, HConstants.TABLEINFO_NAME); Path tableInfoPath = new Path(tableDir, HConstants.TABLEINFO_NAME);
Path tmpPath = new Path(new Path(tableDir,".tmp"), Path tmpPath = new Path(new Path(tableDir,".tmp"), HConstants.TABLEINFO_NAME);
HConstants.TABLEINFO_NAME);
LOG.info("TableInfoPath = " + tableInfoPath + " tmpPath = " + tmpPath); LOG.info("TableInfoPath = " + tableInfoPath + " tmpPath = " + tmpPath);
FSDataOutputStream out = fs.create(tmpPath, true); writeHTD(fs, tmpPath, hTableDescriptor);
try {
hTableDescriptor.write(out);
out.write('\n');
out.write('\n');
out.write(Bytes.toBytes(hTableDescriptor.toString()));
} finally {
out.close();
}
if (!fs.rename(tmpPath, tableInfoPath)) { if (!fs.rename(tmpPath, tableInfoPath)) {
throw new IOException("Unable to rename " + tmpPath + " to " + throw new IOException("Unable to rename " + tmpPath + " to " +
tableInfoPath); tableInfoPath);
@ -1030,29 +1023,34 @@ public class FSUtils {
} }
} }
/**
* Update table descriptor
* @param fs
* @param conf
* @param hTableDescriptor
* @throws IOException
*/
public static void updateHTableDescriptor(FileSystem fs, Path rootdir,
HTableDescriptor hTableDescriptor)
throws IOException {
Path tableInfoPath =
getTableInfoPath(rootdir, hTableDescriptor.getNameAsString());
writeHTD(fs, tableInfoPath, hTableDescriptor);
LOG.info("updateHTableDescriptor. Updated tableinfo in HDFS under " +
tableInfoPath + " For HTD => " + hTableDescriptor.toString());
}
public static void updateHTableDescriptor(FileSystem fs, private static void writeHTD(final FileSystem fs, final Path p,
Configuration conf, final HTableDescriptor htd)
HTableDescriptor hTableDescriptor) throws IOException throws IOException {
{ FSDataOutputStream out = fs.create(p, true);
Path tableInfoPath = getTableInfoPath(hTableDescriptor.getName(), conf);
FSDataOutputStream out = fs.create(tableInfoPath, true);
try { try {
hTableDescriptor.write(out); htd.write(out);
out.write('\n'); out.write('\n');
out.write('\n'); out.write('\n');
out.write(Bytes.toBytes(hTableDescriptor.toString())); out.write(Bytes.toBytes(htd.toString()));
LOG.info("updateHTableDescriptor. Updated tableinfo in HDFS under "
+ tableInfoPath + " For HTD => "
+ hTableDescriptor.toString());
} finally { } finally {
out.close(); out.close();
} }
} }
}
private static Path getTmpDir(HTableDescriptor htableDescriptor, Configuration configuration)
throws IOException {
return new Path(getTablePath(htableDescriptor.getName(), configuration), ".tmp");
}
}

View File

@ -55,6 +55,7 @@ import java.util.Random;
* a table by merging adjacent regions. * a table by merging adjacent regions.
*/ */
class HMerge { class HMerge {
// TODO: Where is this class used? How does it relate to Merge in same package?
static final Log LOG = LogFactory.getLog(HMerge.class); static final Log LOG = LogFactory.getLog(HMerge.class);
static final Random rand = new Random(); static final Random rand = new Random();
@ -135,12 +136,12 @@ class HMerge {
protected final Configuration conf; protected final Configuration conf;
protected final FileSystem fs; protected final FileSystem fs;
protected final Path tabledir; protected final Path tabledir;
protected final HTableDescriptor htd;
protected final HLog hlog; protected final HLog hlog;
private final long maxFilesize; private final long maxFilesize;
protected Merger(Configuration conf, FileSystem fs, protected Merger(Configuration conf, FileSystem fs, final byte [] tableName)
final byte [] tableName)
throws IOException { throws IOException {
this.conf = conf; this.conf = conf;
this.fs = fs; this.fs = fs;
@ -151,6 +152,7 @@ class HMerge {
fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR))), fs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR))),
Bytes.toString(tableName) Bytes.toString(tableName)
); );
this.htd = FSUtils.getTableDescriptor(this.fs, this.tabledir);
Path logdir = new Path(tabledir, "merge_" + System.currentTimeMillis() + Path logdir = new Path(tabledir, "merge_" + System.currentTimeMillis() +
HConstants.HREGION_LOGDIR_NAME); HConstants.HREGION_LOGDIR_NAME);
Path oldLogDir = new Path(tabledir, HConstants.HREGION_OLDLOGDIR_NAME); Path oldLogDir = new Path(tabledir, HConstants.HREGION_OLDLOGDIR_NAME);
@ -188,13 +190,13 @@ class HMerge {
long nextSize = 0; long nextSize = 0;
for (int i = 0; i < info.length - 1; i++) { for (int i = 0; i < info.length - 1; i++) {
if (currentRegion == null) { if (currentRegion == null) {
currentRegion = currentRegion = HRegion.newHRegion(tabledir, hlog, fs, conf, info[i],
HRegion.newHRegion(tabledir, hlog, fs, conf, info[i], null); this.htd, null);
currentRegion.initialize(); currentRegion.initialize();
currentSize = currentRegion.getLargestHStoreSize(); currentSize = currentRegion.getLargestHStoreSize();
} }
nextRegion = nextRegion = HRegion.newHRegion(tabledir, hlog, fs, conf, info[i + 1],
HRegion.newHRegion(tabledir, hlog, fs, conf, info[i + 1], null); this.htd, null);
nextRegion.initialize(); nextRegion.initialize();
nextSize = nextRegion.getLargestHStoreSize(); nextSize = nextRegion.getLargestHStoreSize();
@ -357,7 +359,7 @@ class HMerge {
// Scan root region to find all the meta regions // Scan root region to find all the meta regions
root = HRegion.newHRegion(rootTableDir, hlog, fs, conf, root = HRegion.newHRegion(rootTableDir, hlog, fs, conf,
HRegionInfo.ROOT_REGIONINFO, null); HRegionInfo.ROOT_REGIONINFO, HTableDescriptor.ROOT_TABLEDESC, null);
root.initialize(); root.initialize();
Scan scan = new Scan(); Scan scan = new Scan();
@ -431,4 +433,4 @@ class HMerge {
} }
} }
} }
} }

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
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.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@ -153,7 +154,7 @@ public class Merge extends Configured implements Tool {
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
List<KeyValue> cells2 = rootRegion.get(get, null).list(); List<KeyValue> cells2 = rootRegion.get(get, null).list();
HRegionInfo info2 = Writables.getHRegionInfo((cells2 == null)? null: cells2.get(0).getValue()); HRegionInfo info2 = Writables.getHRegionInfo((cells2 == null)? null: cells2.get(0).getValue());
HRegion merged = merge(info1, rootRegion, info2, rootRegion); HRegion merged = merge(HTableDescriptor.META_TABLEDESC, info1, rootRegion, info2, rootRegion);
LOG.info("Adding " + merged.getRegionInfo() + " to " + LOG.info("Adding " + merged.getRegionInfo() + " to " +
rootRegion.getRegionInfo()); rootRegion.getRegionInfo());
HRegion.addRegionToMETA(rootRegion, merged); HRegion.addRegionToMETA(rootRegion, merged);
@ -216,8 +217,9 @@ public class Merge extends Configured implements Tool {
Get get = new Get(region1); Get get = new Get(region1);
get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
List<KeyValue> cells1 = metaRegion1.get(get, null).list(); List<KeyValue> cells1 = metaRegion1.get(get, null).list();
HRegionInfo info1 = Writables.getHRegionInfo((cells1 == null)? null: cells1.get(0).getValue()); HRegionInfo info1 =
if (info1== null) { Writables.getHRegionInfo((cells1 == null)? null: cells1.get(0).getValue());
if (info1 == null) {
throw new NullPointerException("info1 is null using key " + throw new NullPointerException("info1 is null using key " +
Bytes.toStringBinary(region1) + " in " + meta1); Bytes.toStringBinary(region1) + " in " + meta1);
} }
@ -235,7 +237,9 @@ public class Merge extends Configured implements Tool {
if (info2 == null) { if (info2 == null) {
throw new NullPointerException("info2 is null using key " + meta2); throw new NullPointerException("info2 is null using key " + meta2);
} }
HRegion merged = merge(info1, metaRegion1, info2, metaRegion2); HTableDescriptor htd = FSUtils.getTableDescriptor(FileSystem.get(getConf()),
this.rootdir, this.tableName);
HRegion merged = merge(htd, info1, metaRegion1, info2, metaRegion2);
// Now find the meta region which will contain the newly merged region // Now find the meta region which will contain the newly merged region
@ -267,8 +271,8 @@ public class Merge extends Configured implements Tool {
* to scan the meta if the resulting merged region does not go in either) * to scan the meta if the resulting merged region does not go in either)
* Returns HRegion object for newly merged region * Returns HRegion object for newly merged region
*/ */
private HRegion merge(HRegionInfo info1, HRegion meta1, HRegionInfo info2, private HRegion merge(final HTableDescriptor htd, HRegionInfo info1,
HRegion meta2) HRegion meta1, HRegionInfo info2, HRegion meta2)
throws IOException { throws IOException {
if (info1 == null) { if (info1 == null) {
throw new IOException("Could not find " + Bytes.toStringBinary(region1) + " in " + throw new IOException("Could not find " + Bytes.toStringBinary(region1) + " in " +
@ -280,9 +284,9 @@ public class Merge extends Configured implements Tool {
} }
HRegion merged = null; HRegion merged = null;
HLog log = utils.getLog(); HLog log = utils.getLog();
HRegion r1 = HRegion.openHRegion(info1, log, getConf()); HRegion r1 = HRegion.openHRegion(info1, htd, log, getConf());
try { try {
HRegion r2 = HRegion.openHRegion(info2, log, getConf()); HRegion r2 = HRegion.openHRegion(info2, htd, log, getConf());
try { try {
merged = HRegion.merge(r1, r2); merged = HRegion.merge(r1, r2);
} finally { } finally {

View File

@ -20,14 +20,23 @@
package org.apache.hadoop.hbase.util; package org.apache.hadoop.hbase.util;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
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.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
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.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.HTable;
@ -36,17 +45,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
/** /**
* Contains utility methods for manipulating HBase meta tables. * Contains utility methods for manipulating HBase meta tables.
@ -59,7 +58,6 @@ public class MetaUtils {
private static final Log LOG = LogFactory.getLog(MetaUtils.class); private static final Log LOG = LogFactory.getLog(MetaUtils.class);
private final Configuration conf; private final Configuration conf;
private FileSystem fs; private FileSystem fs;
private Path rootdir;
private HLog log; private HLog log;
private HRegion rootRegion; private HRegion rootRegion;
private Map<byte [], HRegion> metaRegions = Collections.synchronizedSortedMap( private Map<byte [], HRegion> metaRegions = Collections.synchronizedSortedMap(
@ -89,8 +87,6 @@ public class MetaUtils {
*/ */
private void initialize() throws IOException { private void initialize() throws IOException {
this.fs = FileSystem.get(this.conf); this.fs = FileSystem.get(this.conf);
// Get root directory of HBase installation
this.rootdir = FSUtils.getRootDir(this.conf);
} }
/** /**
@ -266,14 +262,16 @@ public class MetaUtils {
if (this.rootRegion != null) { if (this.rootRegion != null) {
return this.rootRegion; return this.rootRegion;
} }
this.rootRegion = HRegion.openHRegion(HRegionInfo.ROOT_REGIONINFO, getLog(), this.rootRegion = HRegion.openHRegion(HRegionInfo.ROOT_REGIONINFO,
HTableDescriptor.ROOT_TABLEDESC, getLog(),
this.conf); this.conf);
this.rootRegion.compactStores(); this.rootRegion.compactStores();
return this.rootRegion; return this.rootRegion;
} }
private HRegion openMetaRegion(HRegionInfo metaInfo) throws IOException { private HRegion openMetaRegion(HRegionInfo metaInfo) throws IOException {
HRegion meta = HRegion.openHRegion(metaInfo, getLog(), this.conf); HRegion meta = HRegion.openHRegion(metaInfo, HTableDescriptor.META_TABLEDESC,
getLog(), this.conf);
meta.compactStores(); meta.compactStores();
return meta; return meta;
} }

View File

@ -173,7 +173,7 @@ public abstract class HBaseTestCase extends TestCase {
throws IOException { throws IOException {
HRegion r = new HRegion(closedRegion.getTableDir(), closedRegion.getLog(), HRegion r = new HRegion(closedRegion.getTableDir(), closedRegion.getLog(),
closedRegion.getFilesystem(), closedRegion.getConf(), closedRegion.getFilesystem(), closedRegion.getConf(),
closedRegion.getRegionInfo(), null); closedRegion.getRegionInfo(), closedRegion.getTableDesc(), null);
r.initialize(); r.initialize();
return r; return r;
} }

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.client.Put;
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;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.JVMClusterUtil;
/** /**
@ -91,6 +92,9 @@ public class TestRegionRebalancing extends HBaseClusterTestCase {
HRegion.addRegionToMETA(meta, region); HRegion.addRegionToMETA(meta, region);
} }
closeRootAndMeta(); closeRootAndMeta();
// Add new table descriptor file
FSUtils.createTableDescriptor(this.desc, this.conf);
} }
/** /**

View File

@ -183,7 +183,7 @@ public class TestCoprocessorInterface extends HBaseTestCase {
//HRegionInfo info = new HRegionInfo(tableName, null, null, false); //HRegionInfo info = new HRegionInfo(tableName, null, null, false);
HRegion r = new HRegion(closedRegion.getTableDir(), closedRegion.getLog(), HRegion r = new HRegion(closedRegion.getTableDir(), closedRegion.getLog(),
closedRegion.getFilesystem(), closedRegion.getConf(), closedRegion.getFilesystem(), closedRegion.getConf(),
closedRegion.getRegionInfo(), null); closedRegion.getRegionInfo(), closedRegion.getTableDesc(), null);
r.initialize(); r.initialize();
// this following piece is a hack. currently a coprocessorHost // this following piece is a hack. currently a coprocessorHost

View File

@ -273,9 +273,9 @@ public class TestWALObserver {
// Make a new wal for new region open. // Make a new wal for new region open.
HLog wal2 = createWAL(newConf); HLog wal2 = createWAL(newConf);
Path tableDir = Path tableDir =
HTableDescriptor.getTableDir(hbaseRootDir, hri.getTableName()); HTableDescriptor.getTableDir(hbaseRootDir, hri.getTableName());
HRegion region = new HRegion(tableDir, wal2, FileSystem.get(newConf), HRegion region = new HRegion(tableDir, wal2, FileSystem.get(newConf),
newConf, hri, TEST_UTIL.getHBaseCluster().getRegionServer(0)); newConf, hri, htd, TEST_UTIL.getHBaseCluster().getRegionServer(0));
long seqid2 = region.initialize(); long seqid2 = region.initialize();
SampleRegionWALObserver cp2 = SampleRegionWALObserver cp2 =

View File

@ -24,9 +24,11 @@ import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -40,6 +42,8 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
@ -119,11 +123,8 @@ public class TestCatalogJanitor {
private final AssignmentManager asm; private final AssignmentManager asm;
MockMasterServices(final Server server) throws IOException { MockMasterServices(final Server server) throws IOException {
this.mfs = new MasterFileSystem(server, null); this.mfs = new MasterFileSystem(server, this, null);
HTableDescriptor htd = new HTableDescriptor("table");
htd.addFamily(new HColumnDescriptor("family"));
this.asm = Mockito.mock(AssignmentManager.class); this.asm = Mockito.mock(AssignmentManager.class);
Mockito.when(asm.getTableDescriptor("table")).thenReturn(htd);
} }
@Override @Override
@ -185,6 +186,43 @@ public class TestCatalogJanitor {
public boolean isStopped() { public boolean isStopped() {
return false; return false;
} }
@Override
public TableDescriptors getTableDescriptors() {
return new TableDescriptors() {
@Override
public HTableDescriptor remove(String tablename) throws IOException {
// TODO Auto-generated method stub
return null;
}
@Override
public Map<String, HTableDescriptor> getAll() throws IOException {
// TODO Auto-generated method stub
return null;
}
@Override
public HTableDescriptor get(byte[] tablename)
throws TableExistsException, FileNotFoundException, IOException {
return get(Bytes.toString(tablename));
}
@Override
public HTableDescriptor get(String tablename)
throws TableExistsException, FileNotFoundException, IOException {
HTableDescriptor htd = new HTableDescriptor("table");
htd.addFamily(new HColumnDescriptor("family"));
return htd;
}
@Override
public void add(HTableDescriptor htd) throws IOException {
// TODO Auto-generated method stub
}
};
}
} }
@Test @Test

View File

@ -87,7 +87,7 @@ public class TestCompactSelection extends TestCase {
HLog hlog = new HLog(fs, logdir, oldLogDir, conf); HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
HRegion.createHRegion(info, basedir, conf, htd); HRegion.createHRegion(info, basedir, conf, htd);
Path tableDir = new Path(basedir, Bytes.toString(htd.getName())); Path tableDir = new Path(basedir, Bytes.toString(htd.getName()));
HRegion region = new HRegion(tableDir, hlog, fs, conf, info, null); HRegion region = new HRegion(tableDir, hlog, fs, conf, info, htd, null);
store = new Store(basedir, region, hcd, fs, conf); store = new Store(basedir, region, hcd, fs, conf);
TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir()); TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir());

View File

@ -154,7 +154,7 @@ public class TestSplitTransaction {
for (HRegion r: daughters) { for (HRegion r: daughters) {
// Open so can count its content. // Open so can count its content.
HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(), HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(),
r.getLog(), r.getConf()); r.getTableDesc(), r.getLog(), r.getConf());
try { try {
int count = countRows(openRegion); int count = countRows(openRegion);
assertTrue(count > 0 && count != rowcount); assertTrue(count > 0 && count != rowcount);
@ -209,7 +209,7 @@ public class TestSplitTransaction {
for (HRegion r: daughters) { for (HRegion r: daughters) {
// Open so can count its content. // Open so can count its content.
HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(), HRegion openRegion = HRegion.openHRegion(this.testdir, r.getRegionInfo(),
r.getLog(), r.getConf()); r.getTableDesc(), r.getLog(), r.getConf());
try { try {
int count = countRows(openRegion); int count = countRows(openRegion);
assertTrue(count > 0 && count != rowcount); assertTrue(count > 0 && count != rowcount);
@ -254,6 +254,7 @@ public class TestSplitTransaction {
htd.addFamily(hcd); htd.addFamily(hcd);
HRegionInfo hri = new HRegionInfo(htd.getName(), STARTROW, ENDROW); HRegionInfo hri = new HRegionInfo(htd.getName(), STARTROW, ENDROW);
HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd); HRegion.createHRegion(hri, testdir, TEST_UTIL.getConfiguration(), htd);
return HRegion.openHRegion(testdir, hri, wal, TEST_UTIL.getConfiguration()); return HRegion.openHRegion(testdir, hri, htd, wal,
TEST_UTIL.getConfiguration());
} }
} }

View File

@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
import org.mockito.Mockito; import org.mockito.Mockito;
@ -129,7 +128,7 @@ public class TestStore extends TestCase {
htd.addFamily(hcd); htd.addFamily(hcd);
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false); HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
HLog hlog = new HLog(fs, logdir, oldLogDir, conf); HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
HRegion region = new HRegion(basedir, hlog, fs, conf, info, null); HRegion region = new HRegion(basedir, hlog, fs, conf, info, htd, null);
store = new Store(basedir, region, hcd, fs, conf); store = new Store(basedir, region, hcd, fs, conf);
} }

View File

@ -238,7 +238,7 @@ public class TestOpenRegionHandler {
HRegion region = HRegion region =
HRegion.createHRegion(hri, HBaseTestingUtility.getTestDir(), HTU HRegion.createHRegion(hri, HBaseTestingUtility.getTestDir(), HTU
.getConfiguration(), htd); .getConfiguration(), htd);
OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri) { OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri, htd) {
HRegion openRegion() { HRegion openRegion() {
// Open region first, then remove znode as though it'd been hijacked. // Open region first, then remove znode as though it'd been hijacked.
//HRegion region = super.openRegion(); //HRegion region = super.openRegion();

View File

@ -165,7 +165,7 @@ public class TestWALReplay {
wal3.setSequenceNumber(wal2.getSequenceNumber()); wal3.setSequenceNumber(wal2.getSequenceNumber());
try { try {
final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri, final HRegion region = new HRegion(basedir, wal3, this.fs, this.conf, hri,
null); htd, null);
long seqid = region.initialize(); long seqid = region.initialize();
assertTrue(seqid > wal3.getSequenceNumber()); assertTrue(seqid > wal3.getSequenceNumber());
@ -193,11 +193,11 @@ public class TestWALReplay {
final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr); final HRegionInfo hri = createBasic3FamilyHRegionInfo(tableNameStr);
final Path basedir = new Path(this.hbaseRootDir, tableNameStr); final Path basedir = new Path(this.hbaseRootDir, tableNameStr);
deleteDir(basedir); deleteDir(basedir);
HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr); final HTableDescriptor htd = createBasic3FamilyHTD(tableNameStr);
HRegion region2 = HRegion.createHRegion(hri, HRegion region2 = HRegion.createHRegion(hri,
hbaseRootDir, this.conf, htd); hbaseRootDir, this.conf, htd);
HLog wal = createWAL(this.conf); HLog wal = createWAL(this.conf);
HRegion region = HRegion.openHRegion(hri, wal, this.conf); HRegion region = HRegion.openHRegion(hri, htd, wal, this.conf);
Path f = new Path(basedir, "hfile"); Path f = new Path(basedir, "hfile");
HFile.Writer writer = new HFile.Writer(this.fs, f); HFile.Writer writer = new HFile.Writer(this.fs, f);
byte [] family = htd.getFamilies().iterator().next().getName(); byte [] family = htd.getFamilies().iterator().next().getName();
@ -218,7 +218,7 @@ public class TestWALReplay {
runWALSplit(newConf); runWALSplit(newConf);
HLog wal2 = createWAL(newConf); HLog wal2 = createWAL(newConf);
HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf), HRegion region2 = new HRegion(basedir, wal2, FileSystem.get(newConf),
newConf, hri, null); newConf, hri, htd, null);
long seqid2 = region2.initialize(); long seqid2 = region2.initialize();
assertTrue(seqid2 > -1); assertTrue(seqid2 > -1);
@ -257,7 +257,7 @@ public class TestWALReplay {
// of the families during the load of edits so its seqid is not same as // of the families during the load of edits so its seqid is not same as
// others to test we do right thing when different seqids. // others to test we do right thing when different seqids.
HLog wal = createWAL(this.conf); HLog wal = createWAL(this.conf);
HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, null); HRegion region = new HRegion(basedir, wal, this.fs, this.conf, hri, htd, null);
long seqid = region.initialize(); long seqid = region.initialize();
// HRegionServer usually does this. It knows the largest seqid across all regions. // HRegionServer usually does this. It knows the largest seqid across all regions.
wal.setSequenceNumber(seqid); wal.setSequenceNumber(seqid);
@ -282,7 +282,7 @@ public class TestWALReplay {
wal.close(); wal.close();
runWALSplit(this.conf); runWALSplit(this.conf);
HLog wal2 = createWAL(this.conf); HLog wal2 = createWAL(this.conf);
HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, null) { HRegion region2 = new HRegion(basedir, wal2, this.fs, this.conf, hri, htd, null) {
@Override @Override
protected boolean restoreEdit(Store s, KeyValue kv) { protected boolean restoreEdit(Store s, KeyValue kv) {
super.restoreEdit(s, kv); super.restoreEdit(s, kv);
@ -317,7 +317,7 @@ public class TestWALReplay {
// Make a new wal for new region open. // Make a new wal for new region open.
HLog wal3 = createWAL(newConf); HLog wal3 = createWAL(newConf);
final AtomicInteger countOfRestoredEdits = new AtomicInteger(0); final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, null) { HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) {
@Override @Override
protected boolean restoreEdit(Store s, KeyValue kv) { protected boolean restoreEdit(Store s, KeyValue kv) {
boolean b = super.restoreEdit(s, kv); boolean b = super.restoreEdit(s, kv);
@ -409,7 +409,7 @@ public class TestWALReplay {
final AtomicInteger flushcount = new AtomicInteger(0); final AtomicInteger flushcount = new AtomicInteger(0);
try { try {
final HRegion region = final HRegion region =
new HRegion(basedir, newWal, newFS, newConf, hri, null) { new HRegion(basedir, newWal, newFS, newConf, hri, htd, null) {
protected boolean internalFlushcache( protected boolean internalFlushcache(
final HLog wal, final long myseqid, MonitoredTask status) final HLog wal, final long myseqid, MonitoredTask status)
throws IOException { throws IOException {

View File

@ -0,0 +1,133 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import static org.junit.Assert.*;
import java.io.IOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.junit.Test;
/**
* Tests for {@link FSTableDescriptors}.
*/
public class TestFSTableDescriptors {
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
@Test
public void testRemoves() throws IOException {
final String name = "testRemoves";
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
// Cleanup old tests if any detrius laying around.
Path rootdir = new Path(HBaseTestingUtility.getTestDir(), name);
TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
HTableDescriptor htd = new HTableDescriptor(name);
htds.add(htd);
assertNotNull(htds.remove(htd.getNameAsString()));
assertNull(htds.remove(htd.getNameAsString()));
}
@Test public void testReadingHTDFromFS() throws IOException {
final String name = "testReadingHTDFromFS";
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
HTableDescriptor htd = new HTableDescriptor(name);
Path rootdir = HBaseTestingUtility.getTestDir(name);
createHTDInFS(fs, rootdir, htd);
HTableDescriptor htd2 =
FSUtils.getTableDescriptor(fs, rootdir, htd.getNameAsString());
assertTrue(htd.equals(htd2));
}
private void createHTDInFS(final FileSystem fs, Path rootdir,
final HTableDescriptor htd)
throws IOException {
FSUtils.createTableDescriptor(fs, rootdir, htd);
}
@Test public void testHTableDescriptors()
throws IOException, InterruptedException {
final String name = "testHTableDescriptors";
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
// Cleanup old tests if any detrius laying around.
Path rootdir = new Path(HBaseTestingUtility.getTestDir(), name);
final int count = 10;
// Write out table infos.
for (int i = 0; i < count; i++) {
HTableDescriptor htd = new HTableDescriptor(name + i);
createHTDInFS(fs, rootdir, htd);
}
FSTableDescriptors htds = new FSTableDescriptors(fs, rootdir);
for (int i = 0; i < count; i++) {
assertTrue(htds.get(Bytes.toBytes(name + i)) != null);
}
for (int i = 0; i < count; i++) {
assertTrue(htds.get(Bytes.toBytes(name + i)) != null);
}
// Update the table infos
for (int i = 0; i < count; i++) {
HTableDescriptor htd = new HTableDescriptor(name + i);
htd.addFamily(new HColumnDescriptor("" + i));
FSUtils.updateHTableDescriptor(fs, rootdir, htd);
}
for (int i = 0; i < count; i++) {
assertTrue(htds.get(Bytes.toBytes(name + i)) != null);
}
for (int i = 0; i < count; i++) {
assertTrue(htds.get(Bytes.toBytes(name + i)) != null);
}
assertEquals(htds.invocations, count * 4);
assertEquals(htds.cachehits, count * 2);
assertTrue(htds.get(HConstants.ROOT_TABLE_NAME) != null);
assertEquals(htds.invocations, count * 4 + 1);
assertEquals(htds.cachehits, count * 2 + 1);
}
@Test (expected=java.io.FileNotFoundException.class)
public void testNoSuchTable() throws IOException {
final String name = "testNoSuchTable";
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
// Cleanup old tests if any detrius laying around.
Path rootdir = new Path(HBaseTestingUtility.getTestDir(), name);
TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
htds.get("NoSuchTable");
}
@Test
public void testUpdates() throws IOException {
final String name = "testUpdates";
FileSystem fs = FileSystem.get(UTIL.getConfiguration());
// Cleanup old tests if any detrius laying around.
Path rootdir = new Path(HBaseTestingUtility.getTestDir(), name);
TableDescriptors htds = new FSTableDescriptors(fs, rootdir);
HTableDescriptor htd = new HTableDescriptor(name);
htds.add(htd);
htds.add(htd);
htds.add(htd);
}
}

View File

@ -190,7 +190,7 @@ public class TestMergeTool extends HBaseTestCase {
// Now verify that we can read all the rows from regions 0, 1 // Now verify that we can read all the rows from regions 0, 1
// in the new merged region. // in the new merged region.
HRegion merged = HRegion.openHRegion(mergedInfo, log, this.conf); HRegion merged = HRegion.openHRegion(mergedInfo, this.desc, log, this.conf);
verifyMerge(merged, upperbound); verifyMerge(merged, upperbound);
merged.close(); merged.close();
LOG.info("Verified " + msg); LOG.info("Verified " + msg);