org.apache.hadoop.hbase.master.HMaster Maven / Gradle / Ivy
Show all versions of hbase-server Show documentation
/**
* 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;
import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Service;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import javax.servlet.ServletException;
import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
import org.apache.hadoop.hbase.ServerMetrics;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
import org.apache.hadoop.hbase.favored.FavoredNodesPromoter;
import org.apache.hadoop.hbase.http.InfoServer;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.log.HBaseMarkers;
import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
import org.apache.hadoop.hbase.master.balancer.BalancerChore;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.master.cleaner.DirScanPool;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
import org.apache.hadoop.hbase.master.locking.LockManager;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.InitMetaProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;
import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;
import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
import org.apache.hadoop.hbase.procedure2.LockedResource;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureStoreListener;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.quotas.MasterQuotasObserver;
import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
import org.apache.hadoop.hbase.quotas.QuotaUtil;
import org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;
import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.replication.ReplicationUtils;
import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader;
import org.apache.hadoop.hbase.replication.regionserver.ReplicationStatus;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.trace.TraceUtil;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EncryptionTest;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.IdLock;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.eclipse.jetty.server.Server;
import org.eclipse.jetty.server.ServerConnector;
import org.eclipse.jetty.servlet.ServletHolder;
import org.eclipse.jetty.webapp.WebAppContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
/**
* HMaster is the "master server" for HBase. An HBase cluster has one active
* master. If many masters are started, all compete. Whichever wins goes on to
* run the cluster. All others park themselves in their constructor until
* master or cluster shutdown or until the active master loses its lease in
* zookeeper. Thereafter, all running master jostle to take over master role.
*
* The Master can be asked shutdown the cluster. See {@link #shutdown()}. In
* this case it will tell all regionservers to go down and then wait on them
* all reporting in that they are down. This master will then shut itself down.
*
*
You can also shutdown just this master. Call {@link #stopMaster()}.
*
* @see org.apache.zookeeper.Watcher
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
@SuppressWarnings("deprecation")
public class HMaster extends HRegionServer implements MasterServices {
private static Logger LOG = LoggerFactory.getLogger(HMaster.class);
/**
* Protection against zombie master. Started once Master accepts active responsibility and
* starts taking over responsibilities. Allows a finite time window before giving up ownership.
*/
private static class InitializationMonitor extends HasThread {
/** The amount of time in milliseconds to sleep before checking initialization status. */
public static final String TIMEOUT_KEY = "hbase.master.initializationmonitor.timeout";
public static final long TIMEOUT_DEFAULT = TimeUnit.MILLISECONDS.convert(15, TimeUnit.MINUTES);
/**
* When timeout expired and initialization has not complete, call {@link System#exit(int)} when
* true, do nothing otherwise.
*/
public static final String HALT_KEY = "hbase.master.initializationmonitor.haltontimeout";
public static final boolean HALT_DEFAULT = false;
private final HMaster master;
private final long timeout;
private final boolean haltOnTimeout;
/** Creates a Thread that monitors the {@link #isInitialized()} state. */
InitializationMonitor(HMaster master) {
super("MasterInitializationMonitor");
this.master = master;
this.timeout = master.getConfiguration().getLong(TIMEOUT_KEY, TIMEOUT_DEFAULT);
this.haltOnTimeout = master.getConfiguration().getBoolean(HALT_KEY, HALT_DEFAULT);
this.setDaemon(true);
}
@Override
public void run() {
try {
while (!master.isStopped() && master.isActiveMaster()) {
Thread.sleep(timeout);
if (master.isInitialized()) {
LOG.debug("Initialization completed within allotted tolerance. Monitor exiting.");
} else {
LOG.error("Master failed to complete initialization after " + timeout + "ms. Please"
+ " consider submitting a bug report including a thread dump of this process.");
if (haltOnTimeout) {
LOG.error("Zombie Master exiting. Thread dump to stdout");
Threads.printThreadInfo(System.out, "Zombie HMaster");
System.exit(-1);
}
}
}
} catch (InterruptedException ie) {
LOG.trace("InitMonitor thread interrupted. Existing.");
}
}
}
// MASTER is name of the webapp and the attribute name used stuffing this
//instance into web context.
public static final String MASTER = "master";
// Manager and zk listener for master election
private final ActiveMasterManager activeMasterManager;
// Region server tracker
private RegionServerTracker regionServerTracker;
// Draining region server tracker
private DrainingServerTracker drainingServerTracker;
// Tracker for load balancer state
LoadBalancerTracker loadBalancerTracker;
// Tracker for meta location, if any client ZK quorum specified
MetaLocationSyncer metaLocationSyncer;
// Tracker for active master location, if any client ZK quorum specified
MasterAddressSyncer masterAddressSyncer;
// Tracker for split and merge state
private SplitOrMergeTracker splitOrMergeTracker;
// Tracker for region normalizer state
private RegionNormalizerTracker regionNormalizerTracker;
private ClusterSchemaService clusterSchemaService;
public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS =
"hbase.master.wait.on.service.seconds";
public static final int DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS = 5 * 60;
// Metrics for the HMaster
final MetricsMaster metricsMaster;
// file system manager for the master FS operations
private MasterFileSystem fileSystemManager;
private MasterWalManager walManager;
// server manager to deal with region server info
private volatile ServerManager serverManager;
// manager of assignment nodes in zookeeper
private AssignmentManager assignmentManager;
// manager of replication
private ReplicationPeerManager replicationPeerManager;
// buffer for "fatal error" notices from region servers
// in the cluster. This is only used for assisting
// operations/debugging.
MemoryBoundedLogMessageBuffer rsFatals;
// flag set after we become the active master (used for testing)
private volatile boolean activeMaster = false;
// flag set after we complete initialization once active
private final ProcedureEvent> initialized = new ProcedureEvent<>("master initialized");
// flag set after master services are started,
// initialization may have not completed yet.
volatile boolean serviceStarted = false;
// Maximum time we should run balancer for
private final int maxBlancingTime;
// Maximum percent of regions in transition when balancing
private final double maxRitPercent;
private final LockManager lockManager = new LockManager(this);
private LoadBalancer balancer;
private RegionNormalizer normalizer;
private BalancerChore balancerChore;
private RegionNormalizerChore normalizerChore;
private ClusterStatusChore clusterStatusChore;
private ClusterStatusPublisher clusterStatusPublisherChore = null;
private HbckChore hbckChore;
CatalogJanitor catalogJanitorChore;
private DirScanPool cleanerPool;
private LogCleaner logCleaner;
private HFileCleaner hfileCleaner;
private ReplicationBarrierCleaner replicationBarrierCleaner;
private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
private MobCompactionChore mobCompactChore;
private MasterMobCompactionThread mobCompactThread;
// used to synchronize the mobCompactionStates
private final IdLock mobCompactionLock = new IdLock();
// save the information of mob compactions in tables.
// the key is table name, the value is the number of compactions in that table.
private Map mobCompactionStates = Maps.newConcurrentMap();
MasterCoprocessorHost cpHost;
private final boolean preLoadTableDescriptors;
// Time stamps for when a hmaster became active
private long masterActiveTime;
// Time stamp for when HMaster finishes becoming Active Master
private long masterFinishedInitializationTime;
//should we check the compression codec type at master side, default true, HBASE-6370
private final boolean masterCheckCompression;
//should we check encryption settings at master side, default true
private final boolean masterCheckEncryption;
Map coprocessorServiceHandlers = Maps.newHashMap();
// monitor for snapshot of hbase tables
SnapshotManager snapshotManager;
// monitor for distributed procedures
private MasterProcedureManagerHost mpmHost;
// it is assigned after 'initialized' guard set to true, so should be volatile
private volatile MasterQuotaManager quotaManager;
private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier;
private QuotaObserverChore quotaObserverChore;
private SnapshotQuotaObserverChore snapshotQuotaChore;
private ProcedureExecutor procedureExecutor;
private WALProcedureStore procedureStore;
// handle table states
private TableStateManager tableStateManager;
private long splitPlanCount;
private long mergePlanCount;
/* Handle favored nodes information */
private FavoredNodesManager favoredNodesManager;
/** jetty server for master to redirect requests to regionserver infoServer */
private Server masterJettyServer;
// Determine if we should do normal startup or minimal "single-user" mode with no region
// servers and no user tables. Useful for repair and recovery of hbase:meta
private final boolean maintenanceMode;
static final String MAINTENANCE_MODE = "hbase.master.maintenance_mode";
public static class RedirectServlet extends HttpServlet {
private static final long serialVersionUID = 2894774810058302473L;
private final int regionServerInfoPort;
private final String regionServerHostname;
/**
* @param infoServer that we're trying to send all requests to
* @param hostname may be null. if given, will be used for redirects instead of host from client.
*/
public RedirectServlet(InfoServer infoServer, String hostname) {
regionServerInfoPort = infoServer.getPort();
regionServerHostname = hostname;
}
@Override
public void doGet(HttpServletRequest request,
HttpServletResponse response) throws ServletException, IOException {
String redirectHost = regionServerHostname;
if(redirectHost == null) {
redirectHost = request.getServerName();
if(!Addressing.isLocalAddress(InetAddress.getByName(redirectHost))) {
LOG.warn("Couldn't resolve '" + redirectHost + "' as an address local to this node and '" +
MASTER_HOSTNAME_KEY + "' is not set; client will get a HTTP 400 response. If " +
"your HBase deployment relies on client accessible names that the region server process " +
"can't resolve locally, then you should set the previously mentioned configuration variable " +
"to an appropriate hostname.");
// no sending client provided input back to the client, so the goal host is just in the logs.
response.sendError(400, "Request was to a host that I can't resolve for any of the network interfaces on " +
"this node. If this is due to an intermediary such as an HTTP load balancer or other proxy, your HBase " +
"administrator can set '" + MASTER_HOSTNAME_KEY + "' to point to the correct hostname.");
return;
}
}
// TODO this scheme should come from looking at the scheme registered in the infoserver's http server for the
// host and port we're using, but it's buried way too deep to do that ATM.
String redirectUrl = request.getScheme() + "://"
+ redirectHost + ":" + regionServerInfoPort
+ request.getRequestURI();
response.sendRedirect(redirectUrl);
}
}
/**
* Initializes the HMaster. The steps are as follows:
*
*
* - Initialize the local HRegionServer
*
- Start the ActiveMasterManager.
*
*
* Remaining steps of initialization occur in
* #finishActiveMasterInitialization(MonitoredTask) after
* the master becomes the active one.
*/
public HMaster(final Configuration conf)
throws IOException, KeeperException {
super(conf);
TraceUtil.initTracer(conf);
try {
if (conf.getBoolean(MAINTENANCE_MODE, false)) {
LOG.info("Detected {}=true via configuration.", MAINTENANCE_MODE);
maintenanceMode = true;
} else if (Boolean.getBoolean(MAINTENANCE_MODE)) {
LOG.info("Detected {}=true via environment variables.", MAINTENANCE_MODE);
maintenanceMode = true;
} else {
maintenanceMode = false;
}
this.rsFatals = new MemoryBoundedLogMessageBuffer(
conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
LOG.info("hbase.rootdir=" + getRootDir() +
", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
// Disable usage of meta replicas in the master
this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
decorateMasterConfiguration(this.conf);
// Hack! Maps DFSClient => Master for logs. HDFS made this
// config param for task trackers, but we can piggyback off of it.
if (this.conf.get("mapreduce.task.attempt.id") == null) {
this.conf.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName.toString());
}
// should we check the compression codec type at master side, default true, HBASE-6370
this.masterCheckCompression = conf.getBoolean("hbase.master.check.compression", true);
// should we check encryption settings at master side, default true
this.masterCheckEncryption = conf.getBoolean("hbase.master.check.encryption", true);
this.metricsMaster = new MetricsMaster(new MetricsMasterWrapperImpl(this));
// preload table descriptor at startup
this.preLoadTableDescriptors = conf.getBoolean("hbase.master.preload.tabledescriptors", true);
this.maxBlancingTime = getMaxBalancingTime();
this.maxRitPercent = conf.getDouble(HConstants.HBASE_MASTER_BALANCER_MAX_RIT_PERCENT,
HConstants.DEFAULT_HBASE_MASTER_BALANCER_MAX_RIT_PERCENT);
// Do we publish the status?
boolean shouldPublish = conf.getBoolean(HConstants.STATUS_PUBLISHED,
HConstants.STATUS_PUBLISHED_DEFAULT);
Class extends ClusterStatusPublisher.Publisher> publisherClass =
conf.getClass(ClusterStatusPublisher.STATUS_PUBLISHER_CLASS,
ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS,
ClusterStatusPublisher.Publisher.class);
if (shouldPublish) {
if (publisherClass == null) {
LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " +
ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS +
" is not set - not publishing status");
} else {
clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
getChoreService().scheduleChore(clusterStatusPublisherChore);
}
}
// Some unit tests don't need a cluster, so no zookeeper at all
if (!conf.getBoolean("hbase.testing.nocluster", false)) {
this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
} else {
this.activeMasterManager = null;
}
} catch (Throwable t) {
// Make sure we log the exception. HMaster is often started via reflection and the
// cause of failed startup is lost.
LOG.error("Failed construction of Master", t);
throw t;
}
}
@Override
protected String getUseThisHostnameInstead(Configuration conf) {
return conf.get(MASTER_HOSTNAME_KEY);
}
// Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will
// block in here until then.
@Override
public void run() {
try {
if (!conf.getBoolean("hbase.testing.nocluster", false)) {
Threads.setDaemonThreadRunning(new Thread(() -> {
try {
int infoPort = putUpJettyServer();
startActiveMasterManager(infoPort);
} catch (Throwable t) {
// Make sure we log the exception.
String error = "Failed to become Active Master";
LOG.error(error, t);
// Abort should have been called already.
if (!isAborted()) {
abort(error, t);
}
}
}), getName() + ":becomeActiveMaster");
}
// Fall in here even if we have been aborted. Need to run the shutdown services and
// the super run call will do this for us.
super.run();
} finally {
if (this.clusterSchemaService != null) {
// If on way out, then we are no longer active master.
this.clusterSchemaService.stopAsync();
try {
this.clusterSchemaService.awaitTerminated(
getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);
} catch (TimeoutException te) {
LOG.warn("Failed shutdown of clusterSchemaService", te);
}
}
this.activeMaster = false;
}
}
// return the actual infoPort, -1 means disable info server.
private int putUpJettyServer() throws IOException {
if (!conf.getBoolean("hbase.master.infoserver.redirect", true)) {
return -1;
}
final int infoPort = conf.getInt("hbase.master.info.port.orig",
HConstants.DEFAULT_MASTER_INFOPORT);
// -1 is for disabling info server, so no redirecting
if (infoPort < 0 || infoServer == null) {
return -1;
}
if(infoPort == infoServer.getPort()) {
return infoPort;
}
final String addr = conf.get("hbase.master.info.bindAddress", "0.0.0.0");
if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
String msg =
"Failed to start redirecting jetty server. Address " + addr
+ " does not belong to this host. Correct configuration parameter: "
+ "hbase.master.info.bindAddress";
LOG.error(msg);
throw new IOException(msg);
}
// TODO I'm pretty sure we could just add another binding to the InfoServer run by
// the RegionServer and have it run the RedirectServlet instead of standing up
// a second entire stack here.
masterJettyServer = new Server();
final ServerConnector connector = new ServerConnector(masterJettyServer);
connector.setHost(addr);
connector.setPort(infoPort);
masterJettyServer.addConnector(connector);
masterJettyServer.setStopAtShutdown(true);
final String redirectHostname =
StringUtils.isBlank(useThisHostnameInstead) ? null : useThisHostnameInstead;
final RedirectServlet redirect = new RedirectServlet(infoServer, redirectHostname);
final WebAppContext context = new WebAppContext(null, "/", null, null, null, null, WebAppContext.NO_SESSIONS);
context.addServlet(new ServletHolder(redirect), "/*");
context.setServer(masterJettyServer);
try {
masterJettyServer.start();
} catch (Exception e) {
throw new IOException("Failed to start redirecting jetty server", e);
}
return connector.getLocalPort();
}
@Override
protected Function getMetaTableObserver() {
return builder -> builder.setRegionReplication(conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
}
/**
* For compatibility, if failed with regionserver credentials, try the master one
*/
@Override
protected void login(UserProvider user, String host) throws IOException {
try {
super.login(user, host);
} catch (IOException ie) {
user.login("hbase.master.keytab.file",
"hbase.master.kerberos.principal", host);
}
}
/**
* If configured to put regions on active master,
* wait till a backup master becomes active.
* Otherwise, loop till the server is stopped or aborted.
*/
@Override
protected void waitForMasterActive(){
if (maintenanceMode) {
return;
}
boolean tablesOnMaster = LoadBalancer.isTablesOnMaster(conf);
while (!(tablesOnMaster && activeMaster) && !isStopped() && !isAborted()) {
sleeper.sleep();
}
}
@VisibleForTesting
public MasterRpcServices getMasterRpcServices() {
return (MasterRpcServices)rpcServices;
}
public boolean balanceSwitch(final boolean b) throws IOException {
return getMasterRpcServices().switchBalancer(b, BalanceSwitchMode.ASYNC);
}
@Override
protected String getProcessName() {
return MASTER;
}
@Override
protected boolean canCreateBaseZNode() {
return true;
}
@Override
protected boolean canUpdateTableDescriptor() {
return true;
}
@Override
protected RSRpcServices createRpcServices() throws IOException {
return new MasterRpcServices(this);
}
@Override
protected void configureInfoServer() {
infoServer.addServlet("master-status", "/master-status", MasterStatusServlet.class);
infoServer.setAttribute(MASTER, this);
if (LoadBalancer.isTablesOnMaster(conf)) {
super.configureInfoServer();
}
}
@Override
protected Class extends HttpServlet> getDumpServlet() {
return MasterDumpServlet.class;
}
@Override
public MetricsMaster getMasterMetrics() {
return metricsMaster;
}
/**
*
* Initialize all ZK based system trackers. But do not include {@link RegionServerTracker}, it
* should have already been initialized along with {@link ServerManager}.
*
*
* Will be overridden in tests.
*
*/
@VisibleForTesting
protected void initializeZKBasedSystemTrackers()
throws IOException, InterruptedException, KeeperException, ReplicationException {
this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
this.normalizer.setMasterServices(this);
this.normalizer.setMasterRpcServices((MasterRpcServices)rpcServices);
this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
this.loadBalancerTracker.start();
this.regionNormalizerTracker = new RegionNormalizerTracker(zooKeeper, this);
this.regionNormalizerTracker.start();
this.splitOrMergeTracker = new SplitOrMergeTracker(zooKeeper, conf, this);
this.splitOrMergeTracker.start();
this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf);
this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager);
this.drainingServerTracker.start();
String clientQuorumServers = conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM);
boolean clientZkObserverMode = conf.getBoolean(HConstants.CLIENT_ZOOKEEPER_OBSERVER_MODE,
HConstants.DEFAULT_CLIENT_ZOOKEEPER_OBSERVER_MODE);
if (clientQuorumServers != null && !clientZkObserverMode) {
// we need to take care of the ZK information synchronization
// if given client ZK are not observer nodes
ZKWatcher clientZkWatcher = new ZKWatcher(conf,
getProcessName() + ":" + rpcServices.getSocketAddress().getPort() + "-clientZK", this,
false, true);
this.metaLocationSyncer = new MetaLocationSyncer(zooKeeper, clientZkWatcher, this);
this.metaLocationSyncer.start();
this.masterAddressSyncer = new MasterAddressSyncer(zooKeeper, clientZkWatcher, this);
this.masterAddressSyncer.start();
// set cluster id is a one-go effort
ZKClusterId.setClusterId(clientZkWatcher, fileSystemManager.getClusterId());
}
// Set the cluster as up. If new RSs, they'll be waiting on this before
// going ahead with their startup.
boolean wasUp = this.clusterStatusTracker.isClusterUp();
if (!wasUp) this.clusterStatusTracker.setClusterUp();
LOG.info("Active/primary master=" + this.serverName +
", sessionid=0x" +
Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
", setting cluster-up flag (Was=" + wasUp + ")");
// create/initialize the snapshot manager and other procedure managers
this.snapshotManager = new SnapshotManager();
this.mpmHost = new MasterProcedureManagerHost();
this.mpmHost.register(this.snapshotManager);
this.mpmHost.register(new MasterFlushTableProcedureManager());
this.mpmHost.loadProcedures(conf);
this.mpmHost.initialize(this, this.metricsMaster);
}
/**
* Finish initialization of HMaster after becoming the primary master.
*
* The startup order is a bit complicated but very important, do not change it unless you know
* what you are doing.
*
* - Initialize file system based components - file system manager, wal manager, table
* descriptors, etc
* - Publish cluster id
* - Here comes the most complicated part - initialize server manager, assignment manager and
* region server tracker
*
* - Create server manager
* - Create procedure executor, load the procedures, but do not start workers. We will start it
* later after we finish scheduling SCPs to avoid scheduling duplicated SCPs for the same
* server
* - Create assignment manager and start it, load the meta region state, but do not load data
* from meta region
* - Start region server tracker, construct the online servers set and find out dead servers and
* schedule SCP for them. The online servers will be constructed by scanning zk, and we will also
* scan the wal directory to find out possible live region servers, and the differences between
* these two sets are the dead servers
*
*
* - If this is a new deploy, schedule a InitMetaProcedure to initialize meta
* - Start necessary service threads - balancer, catalog janior, executor services, and also the
* procedure executor, etc. Notice that the balancer must be created first as assignment manager
* may use it when assigning regions.
* - Wait for meta to be initialized if necesssary, start table state manager.
* - Wait for enough region servers to check-in
* - Let assignment manager load data from meta and construct region states
* - Start all other things such as chore services, etc
*
*
* Notice that now we will not schedule a special procedure to make meta online(unless the first
* time where meta has not been created yet), we will rely on SCP to bring meta online.
*/
private void finishActiveMasterInitialization(MonitoredTask status)
throws IOException, InterruptedException, KeeperException, ReplicationException {
/*
* We are active master now... go initialize components we need to run.
*/
status.setStatus("Initializing Master file system");
this.masterActiveTime = System.currentTimeMillis();
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
// Only initialize the MemStoreLAB when master carry table
if (LoadBalancer.isTablesOnMaster(conf)) {
initializeMemStoreChunkCreator();
}
this.fileSystemManager = new MasterFileSystem(conf);
this.walManager = new MasterWalManager(this);
// enable table descriptors cache
this.tableDescriptors.setCacheOn();
// warm-up HTDs cache on master initialization
if (preLoadTableDescriptors) {
status.setStatus("Pre-loading table descriptors");
this.tableDescriptors.getAll();
}
// Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
// only after it has checked in with the Master. At least a few tests ask Master for clusterId
// before it has called its run method and before RegionServer has done the reportForDuty.
ClusterId clusterId = fileSystemManager.getClusterId();
status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
this.clusterId = clusterId.toString();
// Precaution. Put in place the old hbck1 lock file to fence out old hbase1s running their
// hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set
// hbase.write.hbck1.lock.file to false.
if (this.conf.getBoolean("hbase.write.hbck1.lock.file", true)) {
HBaseFsck.checkAndMarkRunningHbck(this.conf,
HBaseFsck.createLockRetryCounterFactory(this.conf).create());
}
status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
this.serverManager = createServerManager(this);
createProcedureExecutor();
// Create Assignment Manager
this.assignmentManager = new AssignmentManager(this);
this.assignmentManager.start();
// Start RegionServerTracker with listing of servers found with exiting SCPs -- these should
// be registered in the deadServers set -- and with the list of servernames out on the
// filesystem that COULD BE 'alive' (we'll schedule SCPs for each and let SCP figure it out).
// We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.
// TODO: Generate the splitting and live Set in one pass instead of two as we currently do.
this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
this.regionServerTracker.start(
procedureExecutor.getProcedures().stream().filter(p -> p instanceof ServerCrashProcedure)
.map(p -> ((ServerCrashProcedure) p)).collect(Collectors.toSet()),
walManager.getLiveServersFromWALDir(), walManager.getSplittingServersFromWALDir());
// This manager will be started AFTER hbase:meta is confirmed on line.
// hbase.mirror.table.state.to.zookeeper is so hbase1 clients can connect. They read table
// state from zookeeper while hbase2 reads it from hbase:meta. Disable if no hbase1 clients.
this.tableStateManager =
this.conf.getBoolean(MirroringTableStateManager.MIRROR_TABLE_STATE_TO_ZK_KEY, true)?
new MirroringTableStateManager(this):
new TableStateManager(this);
status.setStatus("Initializing ZK system trackers");
initializeZKBasedSystemTrackers();
// Set ourselves as active Master now our claim has succeeded up in zk.
this.activeMaster = true;
// Start the Zombie master detector after setting master as active, see HBASE-21535
Thread zombieDetector = new Thread(new InitializationMonitor(this),
"ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
zombieDetector.setDaemon(true);
zombieDetector.start();
// This is for backwards compatibility
// See HBASE-11393
status.setStatus("Update TableCFs node in ZNode");
ReplicationPeerConfigUpgrader tableCFsUpdater =
new ReplicationPeerConfigUpgrader(zooKeeper, conf);
tableCFsUpdater.copyTableCFs();
if (!maintenanceMode) {
// Add the Observer to delete quotas on table deletion before starting all CPs by
// default with quota support, avoiding if user specifically asks to not load this Observer.
if (QuotaUtil.isQuotaEnabled(conf)) {
updateConfigurationForQuotasObserver(conf);
}
// initialize master side coprocessors before we start handling requests
status.setStatus("Initializing master coprocessors");
this.cpHost = new MasterCoprocessorHost(this, this.conf);
}
// Checking if meta needs initializing.
status.setStatus("Initializing meta table if this is a new deploy");
InitMetaProcedure initMetaProc = null;
// Print out state of hbase:meta on startup; helps debugging.
RegionState rs = this.assignmentManager.getRegionStates().
getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
LOG.info("hbase:meta {}", rs);
if (rs.isOffline()) {
Optional optProc = procedureExecutor.getProcedures().stream()
.filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
initMetaProc = optProc.orElseGet(() -> {
// schedule an init meta procedure if meta has not been deployed yet
InitMetaProcedure temp = new InitMetaProcedure();
procedureExecutor.submitProcedure(temp);
return temp;
});
}
if (this.balancer instanceof FavoredNodesPromoter) {
favoredNodesManager = new FavoredNodesManager(this);
}
// initialize load balancer
this.balancer.setMasterServices(this);
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
this.balancer.initialize();
// start up all service threads.
status.setStatus("Initializing master service threads");
startServiceThreads();
// wait meta to be initialized after we start procedure executor
if (initMetaProc != null) {
initMetaProc.await();
}
// Wake up this server to check in
sleeper.skipSleepCycle();
// Wait for region servers to report in.
// With this as part of master initialization, it precludes our being able to start a single
// server that is both Master and RegionServer. Needs more thought. TODO.
String statusStr = "Wait for region servers to report in";
status.setStatus(statusStr);
LOG.info(Objects.toString(status));
waitForRegionServers(status);
// Check if master is shutting down because issue initializing regionservers or balancer.
if (isStopped()) {
return;
}
status.setStatus("Starting assignment manager");
// FIRST HBASE:META READ!!!!
// The below cannot make progress w/o hbase:meta being online.
// This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
// as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
// if it is down. It may take a while to come online. So, wait here until meta if for sure
// available. That's what waitForMetaOnline does.
if (!waitForMetaOnline()) {
return;
}
this.assignmentManager.joinCluster();
// The below depends on hbase:meta being online.
this.tableStateManager.start();
// Below has to happen after tablestatemanager has started in the case where this hbase-2.x
// is being started over an hbase-1.x dataset. tablestatemanager runs a migration as part
// of its 'start' moving table state from zookeeper to hbase:meta. This migration needs to
// complete before we do this next step processing offline regions else it fails reading
// table states messing up master launch (namespace table, etc., are not assigned).
this.assignmentManager.processOfflineRegions();
// Initialize after meta is up as below scans meta
if (favoredNodesManager != null && !maintenanceMode) {
SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
new SnapshotOfRegionAssignmentFromMeta(getConnection());
snapshotOfRegionAssignment.initialize();
favoredNodesManager.initialize(snapshotOfRegionAssignment);
}
// set cluster status again after user regions are assigned
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
// Start balancer and meta catalog janitor after meta and regions have been assigned.
status.setStatus("Starting balancer and catalog janitor");
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
getChoreService().scheduleChore(clusterStatusChore);
this.balancerChore = new BalancerChore(this);
getChoreService().scheduleChore(balancerChore);
this.normalizerChore = new RegionNormalizerChore(this);
getChoreService().scheduleChore(normalizerChore);
this.catalogJanitorChore = new CatalogJanitor(this);
getChoreService().scheduleChore(catalogJanitorChore);
this.hbckChore = new HbckChore(this);
getChoreService().scheduleChore(hbckChore);
// NAMESPACE READ!!!!
// Here we expect hbase:namespace to be online. See inside initClusterSchemaService.
// TODO: Fix this. Namespace is a pain being a sort-of system table. Fold it in to hbase:meta.
// isNamespace does like isMeta and waits until namespace is onlined before allowing progress.
if (!waitForNamespaceOnline()) {
return;
}
status.setStatus("Starting cluster schema service");
initClusterSchemaService();
if (this.cpHost != null) {
try {
this.cpHost.preMasterInitialization();
} catch (IOException e) {
LOG.error("Coprocessor preMasterInitialization() hook failed", e);
}
}
status.markComplete("Initialization successful");
LOG.info(String.format("Master has completed initialization %.3fsec",
(System.currentTimeMillis() - masterActiveTime) / 1000.0f));
this.masterFinishedInitializationTime = System.currentTimeMillis();
configurationManager.registerObserver(this.balancer);
configurationManager.registerObserver(this.cleanerPool);
configurationManager.registerObserver(this.hfileCleaner);
configurationManager.registerObserver(this.logCleaner);
// Set master as 'initialized'.
setInitialized(true);
if (maintenanceMode) {
LOG.info("Detected repair mode, skipping final initialization steps.");
return;
}
assignmentManager.checkIfShouldMoveSystemRegionAsync();
status.setStatus("Assign meta replicas");
MasterMetaBootstrap metaBootstrap = createMetaBootstrap();
metaBootstrap.assignMetaReplicas();
status.setStatus("Starting quota manager");
initQuotaManager();
if (QuotaUtil.isQuotaEnabled(conf)) {
// Create the quota snapshot notifier
spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
spaceQuotaSnapshotNotifier.initialize(getClusterConnection());
this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());
// Start the chore to read the region FS space reports and act on them
getChoreService().scheduleChore(quotaObserverChore);
this.snapshotQuotaChore = new SnapshotQuotaObserverChore(this, getMasterMetrics());
// Start the chore to read snapshots and add their usage to table/NS quotas
getChoreService().scheduleChore(snapshotQuotaChore);
}
// clear the dead servers with same host name and port of online server because we are not
// removing dead server with same hostname and port of rs which is trying to check in before
// master initialization. See HBASE-5916.
this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
// Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
status.setStatus("Checking ZNode ACLs");
zooKeeper.checkAndSetZNodeAcls();
status.setStatus("Initializing MOB Cleaner");
initMobCleaner();
status.setStatus("Calling postStartMaster coprocessors");
if (this.cpHost != null) {
// don't let cp initialization errors kill the master
try {
this.cpHost.postStartMaster();
} catch (IOException ioe) {
LOG.error("Coprocessor postStartMaster() hook failed", ioe);
}
}
zombieDetector.interrupt();
/*
* After master has started up, lets do balancer post startup initialization. Since this runs
* in activeMasterManager thread, it should be fine.
*/
long start = System.currentTimeMillis();
this.balancer.postMasterStartupInitialize();
if (LOG.isDebugEnabled()) {
LOG.debug("Balancer post startup initialization complete, took " + (
(System.currentTimeMillis() - start) / 1000) + " seconds");
}
}
/**
* Check hbase:meta is up and ready for reading. For use during Master startup only.
* @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
* and we will hold here until operator intervention.
*/
@VisibleForTesting
public boolean waitForMetaOnline() throws InterruptedException {
return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
}
/**
* @return True if region is online and scannable else false if an error or shutdown (Otherwise
* we just block in here holding up all forward-progess).
*/
private boolean isRegionOnline(RegionInfo ri) throws InterruptedException {
RetryCounter rc = null;
while (!isStopped()) {
RegionState rs = this.assignmentManager.getRegionStates().getRegionState(ri);
if (rs.isOpened()) {
if (this.getServerManager().isServerOnline(rs.getServerName())) {
return true;
}
}
// Region is not OPEN.
Optional> optProc = this.procedureExecutor.getProcedures().
stream().filter(p -> p instanceof ServerCrashProcedure).findAny();
// TODO: Add a page to refguide on how to do repair. Have this log message point to it.
// Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
// then how to assign including how to break region lock if one held.
LOG.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
"progress, in holding-pattern until region onlined.",
ri.getRegionNameAsString(), rs, optProc.isPresent());
// Check once-a-minute.
if (rc == null) {
rc = new RetryCounterFactory(1000).create();
}
Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());
}
return false;
}
/**
* Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
* (TODO: Fix this! NS should not hold-up startup).
* @return True if namespace table is up/online.
*/
@VisibleForTesting
public boolean waitForNamespaceOnline() throws InterruptedException {
List ris = this.assignmentManager.getRegionStates().
getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
if (ris.isEmpty()) {
// If empty, means we've not assigned the namespace table yet... Just return true so startup
// continues and the namespace table gets created.
return true;
}
// Else there are namespace regions up in meta. Ensure they are assigned before we go on.
for (RegionInfo ri: ris) {
isRegionOnline(ri);
}
return true;
}
/**
* Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
* automatically remove quotas for a table when that table is deleted.
*/
@VisibleForTesting
public void updateConfigurationForQuotasObserver(Configuration conf) {
// We're configured to not delete quotas on table deletion, so we don't need to add the obs.
if (!conf.getBoolean(
MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE,
MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT)) {
return;
}
String[] masterCoprocs = conf.getStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
final int length = null == masterCoprocs ? 0 : masterCoprocs.length;
String[] updatedCoprocs = new String[length + 1];
if (length > 0) {
System.arraycopy(masterCoprocs, 0, updatedCoprocs, 0, masterCoprocs.length);
}
updatedCoprocs[length] = MasterQuotasObserver.class.getName();
conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, updatedCoprocs);
}
private void initMobCleaner() {
this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
getChoreService().scheduleChore(expiredMobFileCleanerChore);
int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
if (mobCompactionPeriod > 0) {
this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
getChoreService().scheduleChore(mobCompactChore);
} else {
LOG
.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled");
}
this.mobCompactThread = new MasterMobCompactionThread(this);
}
/**
*
* Create a {@link MasterMetaBootstrap} instance.
*
*
* Will be overridden in tests.
*
*/
@VisibleForTesting
protected MasterMetaBootstrap createMetaBootstrap() {
// We put this out here in a method so can do a Mockito.spy and stub it out
// w/ a mocked up MasterMetaBootstrap.
return new MasterMetaBootstrap(this);
}
/**
*
* Create a {@link ServerManager} instance.
*
*
* Will be overridden in tests.
*
*/
@VisibleForTesting
protected ServerManager createServerManager(final MasterServices master) throws IOException {
// We put this out here in a method so can do a Mockito.spy and stub it out
// w/ a mocked up ServerManager.
setupClusterConnection();
return new ServerManager(master);
}
private void waitForRegionServers(final MonitoredTask status)
throws IOException, InterruptedException {
this.serverManager.waitForRegionServers(status);
}
// Will be overridden in tests
@VisibleForTesting
protected void initClusterSchemaService() throws IOException, InterruptedException {
this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
this.clusterSchemaService.startAsync();
try {
this.clusterSchemaService.awaitRunning(getConfiguration().getInt(
HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);
} catch (TimeoutException toe) {
throw new IOException("Timedout starting ClusterSchemaService", toe);
}
}
private void initQuotaManager() throws IOException {
MasterQuotaManager quotaManager = new MasterQuotaManager(this);
quotaManager.start();
this.quotaManager = quotaManager;
}
private SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier() {
SpaceQuotaSnapshotNotifier notifier =
SpaceQuotaSnapshotNotifierFactory.getInstance().create(getConfiguration());
return notifier;
}
boolean isCatalogJanitorEnabled() {
return catalogJanitorChore != null ?
catalogJanitorChore.getEnabled() : false;
}
boolean isCleanerChoreEnabled() {
boolean hfileCleanerFlag = true, logCleanerFlag = true;
if (hfileCleaner != null) {
hfileCleanerFlag = hfileCleaner.getEnabled();
}
if (logCleaner != null) {
logCleanerFlag = logCleaner.getEnabled();
}
return (hfileCleanerFlag && logCleanerFlag);
}
@Override
public TableDescriptors getTableDescriptors() {
return this.tableDescriptors;
}
@Override
public ServerManager getServerManager() {
return this.serverManager;
}
@Override
public MasterFileSystem getMasterFileSystem() {
return this.fileSystemManager;
}
@Override
public MasterWalManager getMasterWalManager() {
return this.walManager;
}
@Override
public TableStateManager getTableStateManager() {
return tableStateManager;
}
/*
* Start up all services. If any of these threads gets an unhandled exception
* then they just die with a logged message. This should be fine because
* in general, we do not expect the master to get such unhandled exceptions
* as OOMEs; it should be lightly loaded. See what HRegionServer does if
* need to install an unexpected exception handler.
*/
private void startServiceThreads() throws IOException {
// Start the executor service pools
this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION, conf.getInt(
HConstants.MASTER_OPEN_REGION_THREADS, HConstants.MASTER_OPEN_REGION_THREADS_DEFAULT));
this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, conf.getInt(
HConstants.MASTER_CLOSE_REGION_THREADS, HConstants.MASTER_CLOSE_REGION_THREADS_DEFAULT));
this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
conf.getInt(HConstants.MASTER_SERVER_OPERATIONS_THREADS,
HConstants.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT));
this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
conf.getInt(HConstants.MASTER_META_SERVER_OPERATIONS_THREADS,
HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT));
this.executorService.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS, conf.getInt(
HConstants.MASTER_LOG_REPLAY_OPS_THREADS, HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT));
this.executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, conf.getInt(
SnapshotManager.SNAPSHOT_POOL_THREADS_KEY, SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT));
// We depend on there being only one instance of this executor running
// at a time. To do concurrency, would need fencing of enable/disable of
// tables.
// Any time changing this maxThreads to > 1, pls see the comment at
// AccessController#postCompletedCreateTableAction
this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
startProcedureExecutor();
// Create cleaner thread pool
cleanerPool = new DirScanPool(conf);
// Start log cleaner thread
int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 600 * 1000);
this.logCleaner = new LogCleaner(cleanerInterval, this, conf,
getMasterWalManager().getFileSystem(), getMasterWalManager().getOldLogDir(), cleanerPool);
getChoreService().scheduleChore(logCleaner);
// start the hfile archive cleaner thread
Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
Map params = new HashMap<>();
params.put(MASTER, this);
this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf,
getMasterFileSystem().getFileSystem(), archiveDir, cleanerPool, params);
getChoreService().scheduleChore(hfileCleaner);
replicationBarrierCleaner = new ReplicationBarrierCleaner(conf, this, getConnection(),
replicationPeerManager);
getChoreService().scheduleChore(replicationBarrierCleaner);
serviceStarted = true;
if (LOG.isTraceEnabled()) {
LOG.trace("Started service threads");
}
}
@Override
protected void stopServiceThreads() {
if (masterJettyServer != null) {
LOG.info("Stopping master jetty server");
try {
masterJettyServer.stop();
} catch (Exception e) {
LOG.error("Failed to stop master jetty server", e);
}
}
stopChores();
if (this.mobCompactThread != null) {
this.mobCompactThread.close();
}
super.stopServiceThreads();
if (cleanerPool != null) {
cleanerPool.shutdownNow();
cleanerPool = null;
}
LOG.debug("Stopping service threads");
if (this.quotaManager != null) {
this.quotaManager.stop();
}
if (this.activeMasterManager != null) {
this.activeMasterManager.stop();
}
if (this.serverManager != null) {
this.serverManager.stop();
}
if (this.assignmentManager != null) {
this.assignmentManager.stop();
}
stopProcedureExecutor();
if (this.walManager != null) {
this.walManager.stop();
}
if (this.fileSystemManager != null) {
this.fileSystemManager.stop();
}
if (this.mpmHost != null) {
this.mpmHost.stop("server shutting down.");
}
if (this.regionServerTracker != null) {
this.regionServerTracker.stop();
}
}
private void createProcedureExecutor() throws IOException {
MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
procedureStore =
new WALProcedureStore(conf, new MasterProcedureEnv.WALStoreLeaseRecovery(this));
procedureStore.registerListener(new ProcedureStoreListener() {
@Override
public void abortProcess() {
abort("The Procedure Store lost the lease", null);
}
});
MasterProcedureScheduler procedureScheduler = procEnv.getProcedureScheduler();
procedureExecutor = new ProcedureExecutor<>(conf, procEnv, procedureStore, procedureScheduler);
configurationManager.registerObserver(procEnv);
int cpus = Runtime.getRuntime().availableProcessors();
final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, Math.max(
(cpus > 0 ? cpus / 4 : 0), MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
final int urgentWorkers = conf
.getInt(MasterProcedureConstants.MASTER_URGENT_PROCEDURE_THREADS,
MasterProcedureConstants.DEFAULT_MASTER_URGENT_PROCEDURE_THREADS);
final boolean abortOnCorruption =
conf.getBoolean(MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
procedureStore.start(numThreads);
// Just initialize it but do not start the workers, we will start the workers later by calling
// startProcedureExecutor. See the javadoc for finishActiveMasterInitialization for more
// details.
procedureExecutor.init(numThreads, urgentWorkers, abortOnCorruption);
procEnv.getRemoteDispatcher().start();
}
private void startProcedureExecutor() throws IOException {
procedureExecutor.startWorkers();
}
private void stopProcedureExecutor() {
if (procedureExecutor != null) {
configurationManager.deregisterObserver(procedureExecutor.getEnvironment());
procedureExecutor.getEnvironment().getRemoteDispatcher().stop();
procedureExecutor.stop();
procedureExecutor.join();
procedureExecutor = null;
}
if (procedureStore != null) {
procedureStore.stop(isAborted());
procedureStore = null;
}
}
private void stopChores() {
ChoreService choreService = getChoreService();
if (choreService != null) {
choreService.cancelChore(this.expiredMobFileCleanerChore);
choreService.cancelChore(this.mobCompactChore);
choreService.cancelChore(this.balancerChore);
choreService.cancelChore(this.normalizerChore);
choreService.cancelChore(this.clusterStatusChore);
choreService.cancelChore(this.catalogJanitorChore);
choreService.cancelChore(this.clusterStatusPublisherChore);
choreService.cancelChore(this.snapshotQuotaChore);
choreService.cancelChore(this.logCleaner);
choreService.cancelChore(this.hfileCleaner);
choreService.cancelChore(this.replicationBarrierCleaner);
choreService.cancelChore(this.hbckChore);
}
}
/**
* @return Get remote side's InetAddress
*/
InetAddress getRemoteInetAddress(final int port,
final long serverStartCode) throws UnknownHostException {
// Do it out here in its own little method so can fake an address when
// mocking up in tests.
InetAddress ia = RpcServer.getRemoteIp();
// The call could be from the local regionserver,
// in which case, there is no remote address.
if (ia == null && serverStartCode == startcode) {
InetSocketAddress isa = rpcServices.getSocketAddress();
if (isa != null && isa.getPort() == port) {
ia = isa.getAddress();
}
}
return ia;
}
/**
* @return Maximum time we should run balancer for
*/
private int getMaxBalancingTime() {
int maxBalancingTime = getConfiguration().getInt(HConstants.HBASE_BALANCER_MAX_BALANCING, -1);
if (maxBalancingTime == -1) {
// if max balancing time isn't set, defaulting it to period time
maxBalancingTime = getConfiguration().getInt(HConstants.HBASE_BALANCER_PERIOD,
HConstants.DEFAULT_HBASE_BALANCER_PERIOD);
}
return maxBalancingTime;
}
/**
* @return Maximum number of regions in transition
*/
private int getMaxRegionsInTransition() {
int numRegions = this.assignmentManager.getRegionStates().getRegionAssignments().size();
return Math.max((int) Math.floor(numRegions * this.maxRitPercent), 1);
}
/**
* It first sleep to the next balance plan start time. Meanwhile, throttling by the max
* number regions in transition to protect availability.
* @param nextBalanceStartTime The next balance plan start time
* @param maxRegionsInTransition max number of regions in transition
* @param cutoffTime when to exit balancer
*/
private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransition,
long cutoffTime) {
boolean interrupted = false;
// Sleep to next balance plan start time
// But if there are zero regions in transition, it can skip sleep to speed up.
while (!interrupted && System.currentTimeMillis() < nextBalanceStartTime
&& this.assignmentManager.getRegionStates().hasRegionsInTransition()) {
try {
Thread.sleep(100);
} catch (InterruptedException ie) {
interrupted = true;
}
}
// Throttling by max number regions in transition
while (!interrupted
&& maxRegionsInTransition > 0
&& this.assignmentManager.getRegionStates().getRegionsInTransitionCount()
>= maxRegionsInTransition && System.currentTimeMillis() <= cutoffTime) {
try {
// sleep if the number of regions in transition exceeds the limit
Thread.sleep(100);
} catch (InterruptedException ie) {
interrupted = true;
}
}
if (interrupted) Thread.currentThread().interrupt();
}
public boolean balance() throws IOException {
return balance(false);
}
public boolean balance(boolean force) throws IOException {
// if master not initialized, don't run balancer.
if (!isInitialized()) {
LOG.debug("Master has not been initialized, don't run balancer.");
return false;
}
if (isInMaintenanceMode()) {
LOG.info("Master is in maintenanceMode mode, don't run balancer.");
return false;
}
int maxRegionsInTransition = getMaxRegionsInTransition();
synchronized (this.balancer) {
// If balance not true, don't run balancer.
if (!this.loadBalancerTracker.isBalancerOn()) return false;
// Only allow one balance run at at time.
if (this.assignmentManager.hasRegionsInTransition()) {
List regionsInTransition = assignmentManager.getRegionsInTransition();
// if hbase:meta region is in transition, result of assignment cannot be recorded
// ignore the force flag in that case
boolean metaInTransition = assignmentManager.isMetaRegionInTransition();
String prefix = force && !metaInTransition ? "R" : "Not r";
List toPrint = regionsInTransition;
int max = 5;
boolean truncated = false;
if (regionsInTransition.size() > max) {
toPrint = regionsInTransition.subList(0, max);
truncated = true;
}
LOG.info(prefix + "unning balancer because " + regionsInTransition.size() +
" region(s) in transition: " + toPrint + (truncated? "(truncated list)": ""));
if (!force || metaInTransition) return false;
}
if (this.serverManager.areDeadServersInProgress()) {
LOG.info("Not running balancer because processing dead regionserver(s): " +
this.serverManager.getDeadServers());
return false;
}
Map onlineServers = serverManager.getOnlineServers();
int regionNotOnOnlineServer = 0;
for (RegionState regionState : assignmentManager.getRegionStates().getRegionStates()) {
if (regionState.isOpened() && !onlineServers
.containsKey(regionState.getServerName())) {
LOG.warn("{} 's server is not in the online server list.", regionState);
regionNotOnOnlineServer++;
}
}
if (regionNotOnOnlineServer > 0) {
LOG.info("Not running balancer because {} regions found not on an online server",
regionNotOnOnlineServer);
return false;
}
if (this.cpHost != null) {
try {
if (this.cpHost.preBalance()) {
LOG.debug("Coprocessor bypassing balancer request");
return false;
}
} catch (IOException ioe) {
LOG.error("Error invoking master coprocessor preBalance()", ioe);
return false;
}
}
boolean isByTable = getConfiguration().getBoolean("hbase.master.loadbalance.bytable", false);
Map>> assignments =
this.assignmentManager.getRegionStates().getAssignmentsForBalancer(isByTable);
for (Map> serverMap : assignments.values()) {
serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());
}
//Give the balancer the current cluster state.
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
this.balancer.setClusterLoad(assignments);
List plans = new ArrayList<>();
for (Entry>> e : assignments.entrySet()) {
List partialPlans = this.balancer.balanceCluster(e.getKey(), e.getValue());
if (partialPlans != null) {
plans.addAll(partialPlans);
}
}
long balanceStartTime = System.currentTimeMillis();
long cutoffTime = balanceStartTime + this.maxBlancingTime;
int rpCount = 0; // number of RegionPlans balanced so far
if (plans != null && !plans.isEmpty()) {
int balanceInterval = this.maxBlancingTime / plans.size();
LOG.info("Balancer plans size is " + plans.size() + ", the balance interval is "
+ balanceInterval + " ms, and the max number regions in transition is "
+ maxRegionsInTransition);
for (RegionPlan plan: plans) {
LOG.info("balance " + plan);
//TODO: bulk assign
try {
this.assignmentManager.moveAsync(plan);
} catch (HBaseIOException hioe) {
//should ignore failed plans here, avoiding the whole balance plans be aborted
//later calls of balance() can fetch up the failed and skipped plans
LOG.warn("Failed balance plan: {}, just skip it", plan, hioe);
}
//rpCount records balance plans processed, does not care if a plan succeeds
rpCount++;
balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
cutoffTime);
// if performing next balance exceeds cutoff time, exit the loop
if (rpCount < plans.size() && System.currentTimeMillis() > cutoffTime) {
// TODO: After balance, there should not be a cutoff time (keeping it as
// a security net for now)
LOG.debug("No more balancing till next balance run; maxBalanceTime="
+ this.maxBlancingTime);
break;
}
}
}
if (this.cpHost != null) {
try {
this.cpHost.postBalance(rpCount < plans.size() ? plans.subList(0, rpCount) : plans);
} catch (IOException ioe) {
// balancing already succeeded so don't change the result
LOG.error("Error invoking master coprocessor postBalance()", ioe);
}
}
}
// If LoadBalancer did not generate any plans, it means the cluster is already balanced.
// Return true indicating a success.
return true;
}
@Override
@VisibleForTesting
public RegionNormalizer getRegionNormalizer() {
return this.normalizer;
}
/**
* Perform normalization of cluster (invoked by {@link RegionNormalizerChore}).
*
* @return true if normalization step was performed successfully, false otherwise
* (specifically, if HMaster hasn't been initialized properly or normalization
* is globally disabled)
*/
public boolean normalizeRegions() throws IOException {
if (!isInitialized()) {
LOG.debug("Master has not been initialized, don't run region normalizer.");
return false;
}
if (this.getServerManager().isClusterShutdown()) {
LOG.info("Cluster is shutting down, don't run region normalizer.");
return false;
}
if (isInMaintenanceMode()) {
LOG.info("Master is in maintenance mode, don't run region normalizer.");
return false;
}
if (!this.regionNormalizerTracker.isNormalizerOn()) {
LOG.debug("Region normalization is disabled, don't run region normalizer.");
return false;
}
synchronized (this.normalizer) {
// Don't run the normalizer concurrently
List allEnabledTables = new ArrayList<>(
this.tableStateManager.getTablesInStates(TableState.State.ENABLED));
Collections.shuffle(allEnabledTables);
for (TableName table : allEnabledTables) {
if (isInMaintenanceMode()) {
LOG.debug("Master is in maintenance mode, stop running region normalizer.");
return false;
}
TableDescriptor tblDesc = getTableDescriptors().get(table);
if (table.isSystemTable() || (tblDesc != null &&
!tblDesc.isNormalizationEnabled())) {
LOG.trace("Skipping normalization for {}, as it's either system"
+ " table or doesn't have auto normalization turned on", table);
continue;
}
List plans = this.normalizer.computePlanForTable(table);
if (plans != null) {
for (NormalizationPlan plan : plans) {
plan.execute(clusterConnection.getAdmin());
if (plan.getType() == PlanType.SPLIT) {
splitPlanCount++;
} else if (plan.getType() == PlanType.MERGE) {
mergePlanCount++;
}
}
}
}
}
// If Region did not generate any plans, it means the cluster is already balanced.
// Return true indicating a success.
return true;
}
/**
* @return Client info for use as prefix on an audit log string; who did an action
*/
@Override
public String getClientIdAuditPrefix() {
return "Client=" + RpcServer.getRequestUserName().orElse(null)
+ "/" + RpcServer.getRemoteAddress().orElse(null);
}
/**
* Switch for the background CatalogJanitor thread.
* Used for testing. The thread will continue to run. It will just be a noop
* if disabled.
* @param b If false, the catalog janitor won't do anything.
*/
public void setCatalogJanitorEnabled(final boolean b) {
this.catalogJanitorChore.setEnabled(b);
}
@Override
public long mergeRegions(
final RegionInfo[] regionsToMerge,
final boolean forcible,
final long ng,
final long nonce) throws IOException {
checkInitialized();
final String mergeRegionsStr = Arrays.stream(regionsToMerge).
map(r -> RegionInfo.getShortNameToLog(r)).collect(Collectors.joining(", "));
return MasterProcedureUtil.submitProcedure(new NonceProcedureRunnable(this, ng, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge);
String aid = getClientIdAuditPrefix();
LOG.info("{} merge regions {}", aid, mergeRegionsStr);
submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(),
regionsToMerge, forcible));
getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge);
}
@Override
protected String getDescription() {
return "MergeTableProcedure";
}
});
}
@Override
public long splitRegion(final RegionInfo regionInfo, final byte[] splitRow,
final long nonceGroup, final long nonce)
throws IOException {
checkInitialized();
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo.getTable(), splitRow);
LOG.info(getClientIdAuditPrefix() + " split " + regionInfo.getRegionNameAsString());
// Execute the operation asynchronously
submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo, splitRow));
}
@Override
protected String getDescription() {
return "SplitTableProcedure";
}
});
}
// Public so can be accessed by tests. Blocks until move is done.
// Replace with an async implementation from which you can get
// a success/failure result.
@VisibleForTesting
public void move(final byte[] encodedRegionName, byte[] destServerName) throws HBaseIOException {
RegionState regionState = assignmentManager.getRegionStates().
getRegionState(Bytes.toString(encodedRegionName));
RegionInfo hri;
if (regionState != null) {
hri = regionState.getRegion();
} else {
throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
}
ServerName dest;
List exclude = hri.getTable().isSystemTable() ? assignmentManager.getExcludedServersForSystemTable()
: new ArrayList<>(1);
if (destServerName != null && exclude.contains(ServerName.valueOf(Bytes.toString(destServerName)))) {
LOG.info(
Bytes.toString(encodedRegionName) + " can not move to " + Bytes.toString(destServerName)
+ " because the server is in exclude list");
destServerName = null;
}
if (destServerName == null || destServerName.length == 0) {
LOG.info("Passed destination servername is null/empty so " +
"choosing a server at random");
exclude.add(regionState.getServerName());
final List destServers = this.serverManager.createDestinationServersList(exclude);
dest = balancer.randomAssignment(hri, destServers);
if (dest == null) {
LOG.debug("Unable to determine a plan to assign " + hri);
return;
}
} else {
ServerName candidate = ServerName.valueOf(Bytes.toString(destServerName));
dest = balancer.randomAssignment(hri, Lists.newArrayList(candidate));
if (dest == null) {
LOG.debug("Unable to determine a plan to assign " + hri);
return;
}
// TODO: What is this? I don't get it.
if (dest.equals(serverName) && balancer instanceof BaseLoadBalancer
&& !((BaseLoadBalancer)balancer).shouldBeOnMaster(hri)) {
// To avoid unnecessary region moving later by balancer. Don't put user
// regions on master.
LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
+ " to avoid unnecessary region moving later by load balancer,"
+ " because it should not be on master");
return;
}
}
if (dest.equals(regionState.getServerName())) {
LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
+ " because region already assigned to the same server " + dest + ".");
return;
}
// Now we can do the move
RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
assert rp.getDestination() != null: rp.toString() + " " + dest;
try {
checkInitialized();
if (this.cpHost != null) {
this.cpHost.preMove(hri, rp.getSource(), rp.getDestination());
}
// Warmup the region on the destination before initiating the move. this call
// is synchronous and takes some time. doing it before the source region gets
// closed
serverManager.sendRegionWarmup(rp.getDestination(), hri);
LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
Future future = this.assignmentManager.moveAsync(rp);
try {
// Is this going to work? Will we throw exception on error?
// TODO: CompletableFuture rather than this stunted Future.
future.get();
} catch (InterruptedException | ExecutionException e) {
throw new HBaseIOException(e);
}
if (this.cpHost != null) {
this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
}
} catch (IOException ioe) {
if (ioe instanceof HBaseIOException) {
throw (HBaseIOException)ioe;
}
throw new HBaseIOException(ioe);
}
}
@Override
public long createTable(
final TableDescriptor tableDescriptor,
final byte [][] splitKeys,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
String namespace = tableDescriptor.getTableName().getNamespaceAsString();
this.clusterSchemaService.getNamespace(namespace);
RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, splitKeys);
sanityCheckTableDescriptor(tableDescriptor);
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preCreateTable(tableDescriptor, newRegions);
LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
// TODO: We can handle/merge duplicate requests, and differentiate the case of
// TableExistsException by saying if the schema is the same or not.
//
// We need to wait for the procedure to potentially fail due to "prepare" sanity
// checks. This will block only the beginning of the procedure. See HBASE-19953.
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
submitProcedure(new CreateTableProcedure(
procedureExecutor.getEnvironment(), tableDescriptor, newRegions, latch));
latch.await();
getMaster().getMasterCoprocessorHost().postCreateTable(tableDescriptor, newRegions);
}
@Override
protected String getDescription() {
return "CreateTableProcedure";
}
});
}
@Override
public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {
if (isStopped()) {
throw new MasterNotRunningException();
}
TableName tableName = tableDescriptor.getTableName();
if (!(tableName.isSystemTable())) {
throw new IllegalArgumentException(
"Only system table creation can use this createSystemTable API");
}
RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, null);
LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
// This special create table is called locally to master. Therefore, no RPC means no need
// to use nonce to detect duplicated RPC call.
long procId = this.procedureExecutor.submitProcedure(
new CreateTableProcedure(procedureExecutor.getEnvironment(), tableDescriptor, newRegions));
return procId;
}
/**
* Checks whether the table conforms to some sane limits, and configured
* values (compression, etc) work. Throws an exception if something is wrong.
* @throws IOException
*/
private void sanityCheckTableDescriptor(final TableDescriptor htd) throws IOException {
final String CONF_KEY = "hbase.table.sanity.checks";
boolean logWarn = false;
if (!conf.getBoolean(CONF_KEY, true)) {
logWarn = true;
}
String tableVal = htd.getValue(CONF_KEY);
if (tableVal != null && !Boolean.valueOf(tableVal)) {
logWarn = true;
}
// check max file size
long maxFileSizeLowerLimit = 2 * 1024 * 1024L; // 2M is the default lower limit
long maxFileSize = htd.getMaxFileSize();
if (maxFileSize < 0) {
maxFileSize = conf.getLong(HConstants.HREGION_MAX_FILESIZE, maxFileSizeLowerLimit);
}
if (maxFileSize < conf.getLong("hbase.hregion.max.filesize.limit", maxFileSizeLowerLimit)) {
String message = "MAX_FILESIZE for table descriptor or "
+ "\"hbase.hregion.max.filesize\" (" + maxFileSize
+ ") is too small, which might cause over splitting into unmanageable "
+ "number of regions.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// check flush size
long flushSizeLowerLimit = 1024 * 1024L; // 1M is the default lower limit
long flushSize = htd.getMemStoreFlushSize();
if (flushSize < 0) {
flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeLowerLimit);
}
if (flushSize < conf.getLong("hbase.hregion.memstore.flush.size.limit", flushSizeLowerLimit)) {
String message = "MEMSTORE_FLUSHSIZE for table descriptor or "
+ "\"hbase.hregion.memstore.flush.size\" ("+flushSize+") is too small, which might cause"
+ " very frequent flushing.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// check that coprocessors and other specified plugin classes can be loaded
try {
checkClassLoading(conf, htd);
} catch (Exception ex) {
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, ex.getMessage(), null);
}
// check compression can be loaded
try {
checkCompression(htd);
} catch (IOException e) {
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, e.getMessage(), e);
}
// check encryption can be loaded
try {
checkEncryption(conf, htd);
} catch (IOException e) {
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, e.getMessage(), e);
}
// Verify compaction policy
try{
checkCompactionPolicy(conf, htd);
} catch(IOException e){
warnOrThrowExceptionForFailure(false, CONF_KEY, e.getMessage(), e);
}
// check that we have at least 1 CF
if (htd.getColumnFamilyCount() == 0) {
String message = "Table should have at least one column family.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// check that we have minimum 1 region replicas
int regionReplicas = htd.getRegionReplication();
if (regionReplicas < 1) {
String message = "Table region replication should be at least one.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
if (hcd.getTimeToLive() <= 0) {
String message = "TTL for column family " + hcd.getNameAsString() + " must be positive.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// check blockSize
if (hcd.getBlocksize() < 1024 || hcd.getBlocksize() > 16 * 1024 * 1024) {
String message = "Block size for column family " + hcd.getNameAsString()
+ " must be between 1K and 16MB.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// check versions
if (hcd.getMinVersions() < 0) {
String message = "Min versions for column family " + hcd.getNameAsString()
+ " must be positive.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// max versions already being checked
// HBASE-13776 Setting illegal versions for ColumnFamilyDescriptor
// does not throw IllegalArgumentException
// check minVersions <= maxVerions
if (hcd.getMinVersions() > hcd.getMaxVersions()) {
String message = "Min versions for column family " + hcd.getNameAsString()
+ " must be less than the Max versions.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// check replication scope
checkReplicationScope(hcd);
// check data replication factor, it can be 0(default value) when user has not explicitly
// set the value, in this case we use default replication factor set in the file system.
if (hcd.getDFSReplication() < 0) {
String message = "HFile Replication for column family " + hcd.getNameAsString()
+ " must be greater than zero.";
warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
}
// TODO: should we check coprocessors and encryption ?
}
}
private void checkReplicationScope(ColumnFamilyDescriptor hcd) throws IOException{
// check replication scope
WALProtos.ScopeType scop = WALProtos.ScopeType.valueOf(hcd.getScope());
if (scop == null) {
String message = "Replication scope for column family "
+ hcd.getNameAsString() + " is " + hcd.getScope() + " which is invalid.";
LOG.error(message);
throw new DoNotRetryIOException(message);
}
}
private void checkCompactionPolicy(Configuration conf, TableDescriptor htd)
throws IOException {
// FIFO compaction has some requirements
// Actually FCP ignores periodic major compactions
String className = htd.getValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY);
if (className == null) {
className =
conf.get(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
ExploringCompactionPolicy.class.getName());
}
int blockingFileCount = HStore.DEFAULT_BLOCKING_STOREFILE_COUNT;
String sv = htd.getValue(HStore.BLOCKING_STOREFILES_KEY);
if (sv != null) {
blockingFileCount = Integer.parseInt(sv);
} else {
blockingFileCount = conf.getInt(HStore.BLOCKING_STOREFILES_KEY, blockingFileCount);
}
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
String compactionPolicy =
hcd.getConfigurationValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY);
if (compactionPolicy == null) {
compactionPolicy = className;
}
if (!compactionPolicy.equals(FIFOCompactionPolicy.class.getName())) {
continue;
}
// FIFOCompaction
String message = null;
// 1. Check TTL
if (hcd.getTimeToLive() == ColumnFamilyDescriptorBuilder.DEFAULT_TTL) {
message = "Default TTL is not supported for FIFO compaction";
throw new IOException(message);
}
// 2. Check min versions
if (hcd.getMinVersions() > 0) {
message = "MIN_VERSION > 0 is not supported for FIFO compaction";
throw new IOException(message);
}
// 3. blocking file count
sv = hcd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY);
if (sv != null) {
blockingFileCount = Integer.parseInt(sv);
}
if (blockingFileCount < 1000) {
message =
"Blocking file count '" + HStore.BLOCKING_STOREFILES_KEY + "' " + blockingFileCount
+ " is below recommended minimum of 1000 for column family "+ hcd.getNameAsString();
throw new IOException(message);
}
}
}
// HBASE-13350 - Helper method to log warning on sanity check failures if checks disabled.
private static void warnOrThrowExceptionForFailure(boolean logWarn, String confKey,
String message, Exception cause) throws IOException {
if (!logWarn) {
throw new DoNotRetryIOException(message + " Set " + confKey +
" to false at conf or table descriptor if you want to bypass sanity checks", cause);
}
LOG.warn(message);
}
private void startActiveMasterManager(int infoPort) throws KeeperException {
String backupZNode = ZNodePaths.joinZNode(
zooKeeper.getZNodePaths().backupMasterAddressesZNode, serverName.toString());
/*
* Add a ZNode for ourselves in the backup master directory since we
* may not become the active master. If so, we want the actual active
* master to know we are backup masters, so that it won't assign
* regions to us if so configured.
*
* If we become the active master later, ActiveMasterManager will delete
* this node explicitly. If we crash before then, ZooKeeper will delete
* this node for us since it is ephemeral.
*/
LOG.info("Adding backup master ZNode " + backupZNode);
if (!MasterAddressTracker.setMasterAddress(zooKeeper, backupZNode, serverName, infoPort)) {
LOG.warn("Failed create of " + backupZNode + " by " + serverName);
}
this.activeMasterManager.setInfoPort(infoPort);
int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
// If we're a backup master, stall until a primary to write this address
if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP, HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
LOG.debug("HMaster started in backup mode. Stalling until master znode is written.");
// This will only be a minute or so while the cluster starts up,
// so don't worry about setting watches on the parent znode
while (!activeMasterManager.hasActiveMaster()) {
LOG.debug("Waiting for master address and cluster state znode to be written.");
Threads.sleep(timeout);
}
}
MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
status.setDescription("Master startup");
try {
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
finishActiveMasterInitialization(status);
}
} catch (Throwable t) {
status.setStatus("Failed to become active: " + t.getMessage());
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
if (t instanceof NoClassDefFoundError && t.getMessage().
contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")) {
// improved error message for this special case
abort("HBase is having a problem with its Hadoop jars. You may need to recompile " +
"HBase against Hadoop version " + org.apache.hadoop.util.VersionInfo.getVersion() +
" or change your hadoop jars to start properly", t);
} else {
abort("Unhandled exception. Starting shutdown.", t);
}
} finally {
status.cleanup();
}
}
private void checkCompression(final TableDescriptor htd)
throws IOException {
if (!this.masterCheckCompression) return;
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
checkCompression(hcd);
}
}
private void checkCompression(final ColumnFamilyDescriptor hcd)
throws IOException {
if (!this.masterCheckCompression) return;
CompressionTest.testCompression(hcd.getCompressionType());
CompressionTest.testCompression(hcd.getCompactionCompressionType());
}
private void checkEncryption(final Configuration conf, final TableDescriptor htd)
throws IOException {
if (!this.masterCheckEncryption) return;
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
checkEncryption(conf, hcd);
}
}
private void checkEncryption(final Configuration conf, final ColumnFamilyDescriptor hcd)
throws IOException {
if (!this.masterCheckEncryption) return;
EncryptionTest.testEncryption(conf, hcd.getEncryptionType(), hcd.getEncryptionKey());
}
private void checkClassLoading(final Configuration conf, final TableDescriptor htd)
throws IOException {
RegionSplitPolicy.getSplitPolicyClass(htd, conf);
RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd);
}
private static boolean isCatalogTable(final TableName tableName) {
return tableName.equals(TableName.META_TABLE_NAME);
}
@Override
public long deleteTable(
final TableName tableName,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preDeleteTable(tableName);
LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
// TODO: We can handle/merge duplicate request
//
// We need to wait for the procedure to potentially fail due to "prepare" sanity
// checks. This will block only the beginning of the procedure. See HBASE-19953.
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
submitProcedure(new DeleteTableProcedure(procedureExecutor.getEnvironment(),
tableName, latch));
latch.await();
getMaster().getMasterCoprocessorHost().postDeleteTable(tableName);
}
@Override
protected String getDescription() {
return "DeleteTableProcedure";
}
});
}
@Override
public long truncateTable(
final TableName tableName,
final boolean preserveSplits,
final long nonceGroup,
final long nonce) throws IOException {
checkInitialized();
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preTruncateTable(tableName);
LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
submitProcedure(new TruncateTableProcedure(procedureExecutor.getEnvironment(),
tableName, preserveSplits, latch));
latch.await();
getMaster().getMasterCoprocessorHost().postTruncateTable(tableName);
}
@Override
protected String getDescription() {
return "TruncateTableProcedure";
}
});
}
@Override
public long addColumn(final TableName tableName, final ColumnFamilyDescriptor column,
final long nonceGroup, final long nonce) throws IOException {
checkInitialized();
checkTableExists(tableName);
return modifyTable(tableName, new TableDescriptorGetter() {
@Override
public TableDescriptor get() throws IOException {
TableDescriptor old = getTableDescriptors().get(tableName);
if (old.hasColumnFamily(column.getName())) {
throw new InvalidFamilyOperationException("Column family '" + column.getNameAsString()
+ "' in table '" + tableName + "' already exists so cannot be added");
}
return TableDescriptorBuilder.newBuilder(old).setColumnFamily(column).build();
}
}, nonceGroup, nonce);
}
/**
* Implement to return TableDescriptor after pre-checks
*/
protected interface TableDescriptorGetter {
TableDescriptor get() throws IOException;
}
@Override
public long modifyColumn(final TableName tableName, final ColumnFamilyDescriptor descriptor,
final long nonceGroup, final long nonce) throws IOException {
checkInitialized();
checkTableExists(tableName);
return modifyTable(tableName, new TableDescriptorGetter() {
@Override
public TableDescriptor get() throws IOException {
TableDescriptor old = getTableDescriptors().get(tableName);
if (!old.hasColumnFamily(descriptor.getName())) {
throw new InvalidFamilyOperationException("Family '" + descriptor.getNameAsString()
+ "' does not exist, so it cannot be modified");
}
return TableDescriptorBuilder.newBuilder(old).modifyColumnFamily(descriptor).build();
}
}, nonceGroup, nonce);
}
@Override
public long deleteColumn(final TableName tableName, final byte[] columnName,
final long nonceGroup, final long nonce) throws IOException {
checkInitialized();
checkTableExists(tableName);
return modifyTable(tableName, new TableDescriptorGetter() {
@Override
public TableDescriptor get() throws IOException {
TableDescriptor old = getTableDescriptors().get(tableName);
if (!old.hasColumnFamily(columnName)) {
throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
+ "' does not exist, so it cannot be deleted");
}
if (old.getColumnFamilyCount() == 1) {
throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
+ "' is the only column family in the table, so it cannot be deleted");
}
return TableDescriptorBuilder.newBuilder(old).removeColumnFamily(columnName).build();
}
}, nonceGroup, nonce);
}
@Override
public long enableTable(final TableName tableName, final long nonceGroup, final long nonce)
throws IOException {
checkInitialized();
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preEnableTable(tableName);
// Normally, it would make sense for this authorization check to exist inside
// AccessController, but because the authorization check is done based on internal state
// (rather than explicit permissions) we'll do the check here instead of in the
// coprocessor.
MasterQuotaManager quotaManager = getMasterQuotaManager();
if (quotaManager != null) {
if (quotaManager.isQuotaInitialized()) {
SpaceQuotaSnapshot currSnapshotOfTable =
QuotaTableUtil.getCurrentSnapshotFromQuotaTable(getConnection(), tableName);
if (currSnapshotOfTable != null) {
SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
if (quotaStatus.isInViolation()
&& SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy()) {
throw new AccessDeniedException("Enabling the table '" + tableName
+ "' is disallowed due to a violated space quota.");
}
}
} else if (LOG.isTraceEnabled()) {
LOG.trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");
}
}
LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
// Execute the operation asynchronously - client will check the progress of the operation
// In case the request is from a <1.1 client before returning,
// we want to make sure that the table is prepared to be
// enabled (the table is locked and the table state is set).
// Note: if the procedure throws exception, we will catch it and rethrow.
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
submitProcedure(new EnableTableProcedure(procedureExecutor.getEnvironment(),
tableName, false, prepareLatch));
prepareLatch.await();
getMaster().getMasterCoprocessorHost().postEnableTable(tableName);
}
@Override
protected String getDescription() {
return "EnableTableProcedure";
}
});
}
@Override
public long disableTable(final TableName tableName, final long nonceGroup, final long nonce)
throws IOException {
checkInitialized();
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
getMaster().getMasterCoprocessorHost().preDisableTable(tableName);
LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
// Execute the operation asynchronously - client will check the progress of the operation
// In case the request is from a <1.1 client before returning,
// we want to make sure that the table is prepared to be
// enabled (the table is locked and the table state is set).
// Note: if the procedure throws exception, we will catch it and rethrow.
//
// We need to wait for the procedure to potentially fail due to "prepare" sanity
// checks. This will block only the beginning of the procedure. See HBASE-19953.
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createBlockingLatch();
submitProcedure(new DisableTableProcedure(procedureExecutor.getEnvironment(),
tableName, false, prepareLatch));
prepareLatch.await();
getMaster().getMasterCoprocessorHost().postDisableTable(tableName);
}
@Override
protected String getDescription() {
return "DisableTableProcedure";
}
});
}
private long modifyTable(final TableName tableName,
final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, final long nonce)
throws IOException {
return MasterProcedureUtil
.submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
TableDescriptor newDescriptor = newDescriptorGetter.get();
sanityCheckTableDescriptor(newDescriptor);
TableDescriptor oldDescriptor = getMaster().getTableDescriptors().get(tableName);
getMaster().getMasterCoprocessorHost().preModifyTable(tableName, oldDescriptor,
newDescriptor);
LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
// Execute the operation synchronously - wait for the operation completes before
// continuing.
//
// We need to wait for the procedure to potentially fail due to "prepare" sanity
// checks. This will block only the beginning of the procedure. See HBASE-19953.
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
submitProcedure(
new ModifyTableProcedure(procedureExecutor.getEnvironment(), newDescriptor, latch));
latch.await();
getMaster().getMasterCoprocessorHost().postModifyTable(tableName, oldDescriptor,
newDescriptor);
}
@Override
protected String getDescription() {
return "ModifyTableProcedure";
}
});
}
@Override
public long modifyTable(final TableName tableName, final TableDescriptor newDescriptor,
final long nonceGroup, final long nonce) throws IOException {
checkInitialized();
return modifyTable(tableName, new TableDescriptorGetter() {
@Override
public TableDescriptor get() throws IOException {
return newDescriptor;
}
}, nonceGroup, nonce);
}
public long restoreSnapshot(final SnapshotDescription snapshotDesc,
final long nonceGroup, final long nonce, final boolean restoreAcl) throws IOException {
checkInitialized();
getSnapshotManager().checkSnapshotSupport();
// Ensure namespace exists. Will throw exception if non-known NS.
final TableName dstTable = TableName.valueOf(snapshotDesc.getTable());
getClusterSchema().getNamespace(dstTable.getNamespaceAsString());
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
setProcId(
getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(), restoreAcl));
}
@Override
protected String getDescription() {
return "RestoreSnapshotProcedure";
}
});
}
private void checkTableExists(final TableName tableName)
throws IOException, TableNotFoundException {
if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
throw new TableNotFoundException(tableName);
}
}
@Override
public void checkTableModifiable(final TableName tableName)
throws IOException, TableNotFoundException, TableNotDisabledException {
if (isCatalogTable(tableName)) {
throw new IOException("Can't modify catalog tables");
}
checkTableExists(tableName);
TableState ts = getTableStateManager().getTableState(tableName);
if (!ts.isDisabled()) {
throw new TableNotDisabledException("Not DISABLED; " + ts);
}
}
public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException {
return getClusterMetricsWithoutCoprocessor(EnumSet.allOf(Option.class));
}
public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet