Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
/**
/*
* 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
Expand Down Expand Up @@ -80,6 +80,7 @@
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLoadStats;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RegionStatesCount;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
Expand All @@ -93,6 +94,7 @@
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
import org.apache.hadoop.hbase.protobuf.ProtobufMessageConverter;
import org.apache.hadoop.hbase.quotas.QuotaScope;
Expand Down Expand Up @@ -3071,6 +3073,44 @@ public static ProcedureDescription buildProcedureDescription(String signature, S
return builder.build();
}

/**
* Get the Meta region state from the passed data bytes. Can handle both old and new style
* server names.
* @param data protobuf serialized data with meta server name.
* @param replicaId replica ID for this region
* @return RegionState instance corresponding to the serialized data.
* @throws DeserializationException if the data is invalid.
*/
public static RegionState parseMetaRegionStateFrom(final byte[] data, int replicaId)
throws DeserializationException {
RegionState.State state = RegionState.State.OPEN;
ServerName serverName;
if (data != null && data.length > 0 && ProtobufUtil.isPBMagicPrefix(data)) {
try {
int prefixLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.MetaRegionServer rl =
ZooKeeperProtos.MetaRegionServer.parser().parseFrom(data, prefixLen,
data.length - prefixLen);
if (rl.hasState()) {
state = RegionState.State.convert(rl.getState());
}
HBaseProtos.ServerName sn = rl.getServer();
serverName = ServerName.valueOf(
sn.getHostName(), sn.getPort(), sn.getStartCode());
} catch (InvalidProtocolBufferException e) {
throw new DeserializationException("Unable to parse meta region location");
}
} else {
// old style of meta region location?
serverName = parseServerNameFrom(data);
}
if (serverName == null) {
state = RegionState.State.OFFLINE;
}
return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
}

/**
* Get a ServerName from the passed in data bytes.
* @param data Data with a serialize server name in it; can handle the old style
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,8 @@ public class ZNodePaths {
// TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
public static final char ZNODE_PATH_SEPARATOR = '/';

private static final String META_ZNODE_PREFIX = "meta-region-server";
public static final String META_ZNODE_PREFIX_CONF_KEY = "zookeeper.znode.metaserver";
public static final String META_ZNODE_PREFIX = "meta-region-server";
private static final String DEFAULT_SNAPSHOT_CLEANUP_ZNODE = "snapshot-cleanup";

// base znode for this cluster
Expand Down Expand Up @@ -104,7 +105,7 @@ public class ZNodePaths {
public ZNodePaths(Configuration conf) {
baseZNode = conf.get(ZOOKEEPER_ZNODE_PARENT, DEFAULT_ZOOKEEPER_ZNODE_PARENT);
ImmutableMap.Builder<Integer, String> builder = ImmutableMap.builder();
metaZNodePrefix = conf.get("zookeeper.znode.metaserver", META_ZNODE_PREFIX);
metaZNodePrefix = conf.get(META_ZNODE_PREFIX_CONF_KEY, META_ZNODE_PREFIX);
String defaultMetaReplicaZNode = ZNodePaths.joinZNode(baseZNode, metaZNodePrefix);
builder.put(DEFAULT_REPLICA_ID, defaultMetaReplicaZNode);
int numMetaReplicas = conf.getInt(META_REPLICAS_NUM, DEFAULT_META_REPLICA_NUM);
Expand Down Expand Up @@ -189,7 +190,19 @@ public String getZNodeForReplica(int replicaId) {
}

/**
* Parse the meta replicaId from the passed znode name.
* Parses the meta replicaId from the passed path.
* @param path the name of the full path which includes baseZNode.
* @return replicaId
*/
public int getMetaReplicaIdFromPath(String path) {
// Extract the znode from path. The prefix is of the following format.
// baseZNode + PATH_SEPARATOR.
int prefixLen = baseZNode.length() + 1;
return getMetaReplicaIdFromZnode(path.substring(prefixLen));
}

/**
* Parse the meta replicaId from the passed znode
* @param znode the name of the znode, does not include baseZNode
* @return replicaId
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
/**
/*
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
Expand All @@ -17,25 +17,24 @@
* limitations under the License.
*/
package org.apache.hadoop.hbase.master;

import java.io.IOException;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicBoolean;

import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
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.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKListener;
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.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

/**
* Handles everything on master-side related to master election.
Expand All @@ -57,12 +56,18 @@ public class ActiveMasterManager extends ZKListener {
final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
final AtomicBoolean clusterShutDown = new AtomicBoolean(false);

// This server's information.
private final ServerName sn;
private int infoPort;
private final Server master;

// Active master's server name. Invalidated anytime active master changes (based on ZK
// notifications) and lazily fetched on-demand.
// ServerName is immutable, so we don't need heavy synchronization around it.
private volatile ServerName activeMasterServerName;

/**
* @param watcher
* @param watcher ZK watcher
* @param sn ServerName
* @param master In an instance of a Master.
*/
Expand Down Expand Up @@ -106,6 +111,30 @@ void handle(final String path) {
}
}

/**
* Fetches the active master's ServerName from zookeeper.
*/
private void fetchAndSetActiveMasterServerName() {
LOG.debug("Attempting to fetch active master sn from zk");
try {
activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher);
} catch (IOException | KeeperException e) {
// Log and ignore for now and re-fetch later if needed.
LOG.error("Error fetching active master information", e);
}
}

public Optional<ServerName> getActiveMasterServerName() {
if (!clusterHasActiveMaster.get()) {
return Optional.empty();
}
if (activeMasterServerName == null) {
fetchAndSetActiveMasterServerName();
}
// It could still be null, but return whatever we have.
return Optional.ofNullable(activeMasterServerName);
}

/**
* Handle a change in the master node. Doesn't matter whether this was called
* from a nodeCreated or nodeDeleted event because there are no guarantees
Expand Down Expand Up @@ -134,6 +163,9 @@ private void handleMasterNodeChange() {
// Notify any thread waiting to become the active master
clusterHasActiveMaster.notifyAll();
}
// Reset the active master sn. Will be re-fetched later if needed.
// We don't want to make a synchronous RPC under a monitor.
activeMasterServerName = null;
}
} catch (KeeperException ke) {
master.abort("Received an unexpected KeeperException, aborting", ke);
Expand All @@ -151,8 +183,8 @@ private void handleMasterNodeChange() {
* @param checkInterval the interval to check if the master is stopped
* @param startupStatus the monitor status to track the progress
* @return True if no issue becoming active master else false if another
* master was running or if some other problem (zookeeper, stop flag has been
* set on this Master)
* master was running or if some other problem (zookeeper, stop flag has been
* set on this Master)
*/
boolean blockUntilBecomingActiveMaster(
int checkInterval, MonitoredTask startupStatus) {
Expand All @@ -178,10 +210,14 @@ boolean blockUntilBecomingActiveMaster(
// We are the master, return
startupStatus.setStatus("Successfully registered as active master.");
this.clusterHasActiveMaster.set(true);
activeMasterServerName = sn;
LOG.info("Registered as active master=" + this.sn);
return true;
}

// Invalidate the active master name so that subsequent requests do not get any stale
// master information. Will be re-fetched if needed.
activeMasterServerName = null;
// There is another active master running elsewhere or this is a restart
// and the master ephemeral node has not expired yet.
this.clusterHasActiveMaster.set(true);
Expand All @@ -208,7 +244,8 @@ boolean blockUntilBecomingActiveMaster(
ZKUtil.deleteNode(this.watcher, this.watcher.getZNodePaths().masterAddressZNode);

// We may have failed to delete the znode at the previous step, but
// we delete the file anyway: a second attempt to delete the znode is likely to fail again.
// we delete the file anyway: a second attempt to delete the znode is likely to fail
// again.
ZNodeClearer.deleteMyEphemeralNodeOnDisk();
} else {
msg = "Another master is the active master, " + currentMaster +
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,155 @@
/*
* 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 java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
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.base.Preconditions;

/**
* Caches the cluster ID of the cluster. For standby masters, this is used to serve the client
* RPCs that fetch the cluster ID. ClusterID is only created by an active master if one does not
* already exist. Standby masters just read the information from the file system. This class is
* thread-safe.
*
* TODO: Make it a singleton without affecting concurrent junit tests.
*/
@InterfaceAudience.Private
public class CachedClusterId {

public static final Logger LOG = LoggerFactory.getLogger(CachedClusterId.class);
private static final int MAX_FETCH_TIMEOUT_MS = 10000;

private Path rootDir;
private FileSystem fs;

// When true, indicates that a FileSystem fetch of ClusterID is in progress. This is used to
// avoid multiple fetches from FS and let only one thread fetch the information.
AtomicBoolean fetchInProgress = new AtomicBoolean(false);

// When true, it means that the cluster ID has been fetched successfully from fs.
private AtomicBoolean isClusterIdSet = new AtomicBoolean(false);
// Immutable once set and read multiple times.
private ClusterId clusterId;

// cache stats for testing.
private AtomicInteger cacheMisses = new AtomicInteger(0);

public CachedClusterId(Configuration conf) throws IOException {
rootDir = FSUtils.getRootDir(conf);
fs = rootDir.getFileSystem(conf);
}

/**
* Succeeds only once, when setting to a non-null value. Overwrites are not allowed.
*/
private void setClusterId(ClusterId id) {
if (id == null || isClusterIdSet.get()) {
return;
}
clusterId = id;
isClusterIdSet.set(true);
}

/**
* Returns a cached copy of the cluster ID. null if the cache is not populated.
*/
private String getClusterId() {
if (!isClusterIdSet.get()) {
return null;
}
// It is ok to read without a lock since clusterId is immutable once set.
return clusterId.toString();
}

/**
* Attempts to fetch the cluster ID from the file system. If no attempt is already in progress,
* synchronously fetches the cluster ID and sets it. If an attempt is already in progress,
* returns right away and the caller is expected to wait for the fetch to finish.
* @return true if the attempt is done, false if another thread is already fetching it.
*/
private boolean attemptFetch() {
if (fetchInProgress.compareAndSet(false, true)) {
// A fetch is not in progress, so try fetching the cluster ID synchronously and then notify
// the waiting threads.
try {
cacheMisses.incrementAndGet();
setClusterId(FSUtils.getClusterId(fs, rootDir));
} catch (IOException e) {
LOG.warn("Error fetching cluster ID", e);
} finally {
Preconditions.checkState(fetchInProgress.compareAndSet(true, false));
synchronized (fetchInProgress) {
fetchInProgress.notifyAll();
}
}
return true;
}
return false;
}

private void waitForFetchToFinish() throws InterruptedException {
synchronized (fetchInProgress) {
while (fetchInProgress.get()) {
// We don't want the fetches to block forever, for example if there are bugs
// of missing notifications.
fetchInProgress.wait(MAX_FETCH_TIMEOUT_MS);
}
}
}

/**
* Fetches the ClusterId from FS if it is not cached locally. Atomically updates the cached
* copy and is thread-safe. Optimized to do a single fetch when there are multiple threads are
* trying get from a clean cache.
*
* @return ClusterId by reading from FileSystem or null in any error case or cluster ID does
* not exist on the file system.
*/
public String getFromCacheOrFetch() {
String id = getClusterId();
if (id != null) {
return id;
}
if (!attemptFetch()) {
// A fetch is in progress.
try {
waitForFetchToFinish();
} catch (InterruptedException e) {
// pass and return whatever is in the cache.
}
}
return getClusterId();
}

@VisibleForTesting
public int getCacheStats() {
return cacheMisses.get();
}
}
Loading