HBASE-3229 Table creation, though using async call to master, can actually run for a while and cause RPC timeout
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1162209 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2bcc605a3f
commit
cc93cb0298
|
@ -217,6 +217,8 @@ Release 0.91.0 - Unreleased
|
|||
HTable(tablename) (Ramkrishna)
|
||||
HBASE-4217 HRS.closeRegion should be able to close regions with only
|
||||
the encoded name (ramkrishna.s.vasudevan)
|
||||
HBASE-3229 HBASE-3229 Table creation, though using "async" call to master,
|
||||
can actually run for a while and cause RPC timeout (Ming Ma)
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
|
||||
|
|
|
@ -352,7 +352,8 @@ public class HBaseAdmin implements Abortable, Closeable {
|
|||
}
|
||||
int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
|
||||
int prevRegCount = 0;
|
||||
for (int tries = 0; tries < numRetries; ++tries) {
|
||||
for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
|
||||
++tries) {
|
||||
// Wait for new table to come on-line
|
||||
final AtomicInteger actualRegCount = new AtomicInteger(0);
|
||||
MetaScannerVisitor visitor = new MetaScannerVisitor() {
|
||||
|
@ -383,7 +384,7 @@ public class HBaseAdmin implements Abortable, Closeable {
|
|||
};
|
||||
MetaScanner.metaScan(conf, visitor, desc.getName());
|
||||
if (actualRegCount.get() != numRegs) {
|
||||
if (tries == numRetries - 1) {
|
||||
if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
|
||||
throw new RegionOfflineException("Only " + actualRegCount.get() +
|
||||
" of " + numRegs + " regions are online; retries exhausted.");
|
||||
}
|
||||
|
|
|
@ -126,6 +126,7 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
|
|||
C_M_ADD_FAMILY (44), // Client asking Master to add family to table
|
||||
C_M_DELETE_FAMILY (45), // Client asking Master to delete family of table
|
||||
C_M_MODIFY_FAMILY (46), // Client asking Master to modify family of table
|
||||
C_M_CREATE_TABLE (47), // Client asking Master to create a table
|
||||
|
||||
// Updates from master to ZK. This is done by the master and there is
|
||||
// nothing to process by either Master or RS
|
||||
|
|
|
@ -124,6 +124,7 @@ public class ExecutorService {
|
|||
case C_M_DISABLE_TABLE:
|
||||
case C_M_ENABLE_TABLE:
|
||||
case C_M_MODIFY_TABLE:
|
||||
case C_M_CREATE_TABLE:
|
||||
return ExecutorType.MASTER_TABLE_OPERATIONS;
|
||||
|
||||
// RegionServer executor services
|
||||
|
|
|
@ -54,9 +54,9 @@ public interface HMasterInterface extends VersionedProtocol {
|
|||
// Admin tools would use these cmds
|
||||
|
||||
/**
|
||||
* Creates a new table. If splitKeys are specified, then the table will be
|
||||
* created with an initial set of multiple regions. If splitKeys is null,
|
||||
* the table will be created with a single region.
|
||||
* Creates a new table asynchronously. If splitKeys are specified, then the
|
||||
* table will be created with an initial set of multiple regions.
|
||||
* If splitKeys is null, the table will be created with a single region.
|
||||
* @param desc table descriptor
|
||||
* @param splitKeys
|
||||
* @throws IOException
|
||||
|
|
|
@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
|
|||
import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
|
||||
import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
|
@ -914,14 +915,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
}
|
||||
}
|
||||
|
||||
public void createTable(HTableDescriptor desc, byte [][] splitKeys)
|
||||
throws IOException {
|
||||
createTable(desc, splitKeys, false);
|
||||
}
|
||||
|
||||
public void createTable(HTableDescriptor hTableDescriptor,
|
||||
byte [][] splitKeys,
|
||||
boolean sync)
|
||||
byte [][] splitKeys)
|
||||
throws IOException {
|
||||
if (!isMasterRunning()) {
|
||||
throw new MasterNotRunningException();
|
||||
|
@ -930,17 +925,15 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
cpHost.preCreateTable(hTableDescriptor, splitKeys);
|
||||
}
|
||||
HRegionInfo [] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
|
||||
int timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
|
||||
// Need META availability to create a table
|
||||
try {
|
||||
if(catalogTracker.waitForMeta(timeout) == null) {
|
||||
throw new NotAllMetaRegionsOnlineException();
|
||||
|
||||
this.executorService.submit(new CreateTableHandler(this,
|
||||
this.fileSystemManager, this.serverManager, hTableDescriptor, conf,
|
||||
newRegions, catalogTracker, assignmentManager));
|
||||
|
||||
if (cpHost != null) {
|
||||
// TODO, remove sync parameter from postCreateTable method
|
||||
cpHost.postCreateTable(newRegions, false);
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("Interrupted waiting for meta availability", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
createTable(hTableDescriptor ,newRegions, sync);
|
||||
}
|
||||
|
||||
private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
|
||||
|
@ -962,85 +955,8 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
}
|
||||
}
|
||||
return hRegionInfos;
|
||||
}
|
||||
|
||||
private synchronized void createTable(final HTableDescriptor hTableDescriptor,
|
||||
final HRegionInfo [] newRegions,
|
||||
final boolean sync)
|
||||
throws IOException {
|
||||
String tableName = newRegions[0].getTableNameAsString();
|
||||
if (MetaReader.tableExists(catalogTracker, tableName)) {
|
||||
throw new TableExistsException(tableName);
|
||||
}
|
||||
// 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);
|
||||
|
||||
// 1. Set table enabling flag up in zk.
|
||||
try {
|
||||
assignmentManager.getZKTable().setEnabledTable(tableName);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Unable to ensure that the table will be" +
|
||||
" enabled because of a ZooKeeper issue", e);
|
||||
}
|
||||
|
||||
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
|
||||
final int batchSize = this.conf.getInt("hbase.master.createtable.batchsize", 100);
|
||||
HLog hlog = null;
|
||||
for (int regionIdx = 0; regionIdx < newRegions.length; regionIdx++) {
|
||||
HRegionInfo newRegion = newRegions[regionIdx];
|
||||
// 2. Create HRegion
|
||||
HRegion region = HRegion.createHRegion(newRegion,
|
||||
fileSystemManager.getRootDir(), conf, hTableDescriptor, hlog);
|
||||
if (hlog == null) {
|
||||
hlog = region.getLog();
|
||||
}
|
||||
|
||||
regionInfos.add(region.getRegionInfo());
|
||||
if (regionIdx % batchSize == 0) {
|
||||
// 3. Insert into META
|
||||
MetaEditor.addRegionsToMeta(catalogTracker, regionInfos);
|
||||
regionInfos.clear();
|
||||
}
|
||||
|
||||
// 4. Close the new region to flush to disk. Close log file too.
|
||||
region.close();
|
||||
}
|
||||
hlog.closeAndDelete();
|
||||
if (regionInfos.size() > 0) {
|
||||
MetaEditor.addRegionsToMeta(catalogTracker, regionInfos);
|
||||
}
|
||||
|
||||
// 5. Trigger immediate assignment of the regions in round-robin fashion
|
||||
List<ServerName> servers = serverManager.getOnlineServersList();
|
||||
try {
|
||||
this.assignmentManager.assignUserRegions(Arrays.asList(newRegions), servers);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Caught " + ie + " during round-robin assignment");
|
||||
throw new IOException(ie);
|
||||
}
|
||||
|
||||
// 6. If sync, wait for assignment of regions
|
||||
if (sync) {
|
||||
LOG.debug("Waiting for " + newRegions.length + " region(s) to be assigned");
|
||||
for (HRegionInfo regionInfo : newRegions) {
|
||||
try {
|
||||
this.assignmentManager.waitForAssignment(regionInfo);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info("Interrupted waiting for region to be assigned during " +
|
||||
"create table call", e);
|
||||
Thread.currentThread().interrupt();
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (cpHost != null) {
|
||||
cpHost.postCreateTable(newRegions, sync);
|
||||
}
|
||||
}
|
||||
|
||||
private static boolean isCatalogTable(final byte [] tableName) {
|
||||
return Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
|
||||
|
|
|
@ -0,0 +1,190 @@
|
|||
/**
|
||||
* Copyright 2011 The Apache Software Foundation
|
||||
*
|
||||
* 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.master.handler;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.master.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.ServerManager;
|
||||
import org.apache.hadoop.hbase.master.MasterFileSystem;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||
import org.apache.hadoop.hbase.TableExistsException;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
|
||||
/**
|
||||
* Handler to create a table.
|
||||
*/
|
||||
public class CreateTableHandler extends EventHandler {
|
||||
private static final Log LOG = LogFactory.getLog(CreateTableHandler.class);
|
||||
private MasterFileSystem fileSystemManager;
|
||||
private final HTableDescriptor hTableDescriptor;
|
||||
private Configuration conf;
|
||||
private final AssignmentManager assignmentManager;
|
||||
private final CatalogTracker catalogTracker;
|
||||
private final ServerManager serverManager;
|
||||
private final HRegionInfo [] newRegions;
|
||||
|
||||
public CreateTableHandler(Server server, MasterFileSystem fileSystemManager,
|
||||
ServerManager serverManager, HTableDescriptor hTableDescriptor,
|
||||
Configuration conf, HRegionInfo [] newRegions,
|
||||
CatalogTracker catalogTracker, AssignmentManager assignmentManager)
|
||||
throws NotAllMetaRegionsOnlineException, TableExistsException,
|
||||
IOException {
|
||||
super(server, EventType.C_M_CREATE_TABLE);
|
||||
|
||||
this.fileSystemManager = fileSystemManager;
|
||||
this.serverManager = serverManager;
|
||||
this.hTableDescriptor = hTableDescriptor;
|
||||
this.conf = conf;
|
||||
this.newRegions = newRegions;
|
||||
this.catalogTracker = catalogTracker;
|
||||
this.assignmentManager = assignmentManager;
|
||||
|
||||
int timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
|
||||
// Need META availability to create a table
|
||||
try {
|
||||
if(catalogTracker.waitForMeta(timeout) == null) {
|
||||
throw new NotAllMetaRegionsOnlineException();
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("Interrupted waiting for meta availability", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
String tableName = this.hTableDescriptor.getNameAsString();
|
||||
if (MetaReader.tableExists(catalogTracker, tableName)) {
|
||||
throw new TableExistsException(tableName);
|
||||
}
|
||||
|
||||
// If we have multiple client threads trying to create the table at the
|
||||
// same time, given the async nature of the operation, the table
|
||||
// could be in a state where .META. table hasn't been updated yet in
|
||||
// the process() function.
|
||||
// Use enabling state to tell if there is already a request for the same
|
||||
// table in progress. This will introduce a new zookeeper call. Given
|
||||
// createTable isn't a frequent operation, that should be ok.
|
||||
try {
|
||||
if (!this.assignmentManager.getZKTable().checkAndSetEnablingTable(
|
||||
tableName))
|
||||
throw new TableExistsException(tableName);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Unable to ensure that the table will be" +
|
||||
" enabling because of a ZooKeeper issue", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
String name = "UnknownServerName";
|
||||
if(server != null && server.getServerName() != null) {
|
||||
name = server.getServerName().toString();
|
||||
}
|
||||
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
|
||||
this.hTableDescriptor.getNameAsString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() {
|
||||
String tableName = this.hTableDescriptor.getNameAsString();
|
||||
try {
|
||||
LOG.info("Attemping to create the table " + tableName);
|
||||
handleCreateTable();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Error trying to create the table " + tableName, e);
|
||||
} catch (KeeperException e) {
|
||||
LOG.error("Error trying to create the table " + tableName, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void handleCreateTable() throws IOException, KeeperException {
|
||||
|
||||
// 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(this.hTableDescriptor, this.conf);
|
||||
|
||||
List<HRegionInfo> regionInfos = new ArrayList<HRegionInfo>();
|
||||
final int batchSize =
|
||||
this.conf.getInt("hbase.master.createtable.batchsize", 100);
|
||||
HLog hlog = null;
|
||||
for (int regionIdx = 0; regionIdx < this.newRegions.length; regionIdx++) {
|
||||
HRegionInfo newRegion = this.newRegions[regionIdx];
|
||||
// 1. Create HRegion
|
||||
HRegion region = HRegion.createHRegion(newRegion,
|
||||
this.fileSystemManager.getRootDir(), this.conf,
|
||||
this.hTableDescriptor, hlog);
|
||||
if (hlog == null) {
|
||||
hlog = region.getLog();
|
||||
}
|
||||
|
||||
regionInfos.add(region.getRegionInfo());
|
||||
if (regionIdx % batchSize == 0) {
|
||||
// 2. Insert into META
|
||||
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
|
||||
regionInfos.clear();
|
||||
}
|
||||
|
||||
// 3. Close the new region to flush to disk. Close log file too.
|
||||
region.close();
|
||||
}
|
||||
hlog.closeAndDelete();
|
||||
if (regionInfos.size() > 0) {
|
||||
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
|
||||
}
|
||||
|
||||
// 4. Trigger immediate assignment of the regions in round-robin fashion
|
||||
List<ServerName> servers = serverManager.getOnlineServersList();
|
||||
try {
|
||||
this.assignmentManager.assignUserRegions(Arrays.asList(newRegions),
|
||||
servers);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.error("Caught " + ie + " during round-robin assignment");
|
||||
throw new IOException(ie);
|
||||
}
|
||||
|
||||
// 5. Set table enabled flag up in zk.
|
||||
try {
|
||||
assignmentManager.getZKTable().
|
||||
setEnabledTable(this.hTableDescriptor.getNameAsString());
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Unable to ensure that the table will be" +
|
||||
" enabled because of a ZooKeeper issue", e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,5 +1,5 @@
|
|||
/**
|
||||
* Copyright 2010 The Apache Software Foundation
|
||||
* Copyright 2011 The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
|
@ -66,7 +66,8 @@ public class EnableTableHandler extends EventHandler {
|
|||
if(server != null && server.getServerName() != null) {
|
||||
name = server.getServerName().toString();
|
||||
}
|
||||
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableNameStr;
|
||||
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
|
||||
tableNameStr;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -119,7 +120,7 @@ public class EnableTableHandler extends EventHandler {
|
|||
break;
|
||||
}
|
||||
}
|
||||
// Flip the table to disabled.
|
||||
// Flip the table to enabled.
|
||||
if (done) this.assignmentManager.getZKTable().setEnabledTable(this.tableNameStr);
|
||||
LOG.info("Enabled table is done=" + done);
|
||||
}
|
||||
|
|
|
@ -164,13 +164,31 @@ public class ZKTable {
|
|||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (!isDisabledOrEnablingTable(tableName)) {
|
||||
LOG.warn("Moving table " + tableName + " state to disabling but was " +
|
||||
"not first in enabled state: " + this.cache.get(tableName));
|
||||
LOG.warn("Moving table " + tableName + " state to enabling but was " +
|
||||
"not first in disabled state: " + this.cache.get(tableName));
|
||||
}
|
||||
setTableState(tableName, TableState.ENABLING);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the specified table as ENABLING in zookeeper atomically
|
||||
* If the table is already in ENABLING state, no operation is performed
|
||||
* @param tableName
|
||||
* @return if the operation succeeds or not
|
||||
* @throws KeeperException unexpected zookeeper exception
|
||||
*/
|
||||
public boolean checkAndSetEnablingTable(final String tableName)
|
||||
throws KeeperException {
|
||||
synchronized (this.cache) {
|
||||
if (isEnablingTable(tableName)) {
|
||||
return false;
|
||||
}
|
||||
setTableState(tableName, TableState.ENABLING);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private void setTableState(final String tableName, final TableState state)
|
||||
throws KeeperException {
|
||||
String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName);
|
||||
|
|
|
@ -843,6 +843,27 @@ public class TestAdmin {
|
|||
new HTable(TEST_UTIL.getConfiguration(), name);
|
||||
}
|
||||
|
||||
/***
|
||||
* HMaster.createTable used to be kind of synchronous call
|
||||
* Thus creating of table with lots of regions can cause RPC timeout
|
||||
* After the fix to make createTable truly async, RPC timeout shouldn't be an
|
||||
* issue anymore
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testCreateTableRPCTimeOut() throws Exception {
|
||||
String name = "testCreateTableRPCTimeOut";
|
||||
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 1500);
|
||||
|
||||
int expectedRegions = 100;
|
||||
// Use 80 bit numbers to make sure we aren't limited
|
||||
byte [] startKey = { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1 };
|
||||
byte [] endKey = { 9, 9, 9, 9, 9, 9, 9, 9, 9, 9 };
|
||||
HBaseAdmin hbaseadmin = new HBaseAdmin(TEST_UTIL.getConfiguration());
|
||||
hbaseadmin.createTable(new HTableDescriptor(name), startKey, endKey,
|
||||
expectedRegions);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test read only tables
|
||||
* @throws Exception
|
||||
|
|
Loading…
Reference in New Issue