HBASE-11611 Clean up ZK-based region assignment

This commit is contained in:
Jimmy Xiang 2014-07-29 16:15:42 -07:00
parent e17a3ca091
commit 17dff6818e
78 changed files with 523 additions and 12604 deletions

View File

@ -1,139 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import org.apache.hadoop.hbase.util.ByteStringer;
import com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Current state of a region in transition. Holds state of a region as it moves through the
* steps that take it from offline to open, etc. Used by regionserver, master, and zk packages.
* Encapsulates protobuf serialization/deserialization so we don't leak generated pb outside this
* class. Create an instance using createRegionTransition(EventType, byte[], ServerName).
* <p>Immutable
*/
@InterfaceAudience.Private
public class RegionTransition {
private final ZooKeeperProtos.RegionTransition rt;
/**
* Shutdown constructor
*/
private RegionTransition() {
this(null);
}
private RegionTransition(final ZooKeeperProtos.RegionTransition rt) {
this.rt = rt;
}
public EventType getEventType() {
return EventType.get(this.rt.getEventTypeCode());
}
public ServerName getServerName() {
return ProtobufUtil.toServerName(this.rt.getServerName());
}
public long getCreateTime() {
return this.rt.getCreateTime();
}
/**
* @return Full region name
*/
public byte [] getRegionName() {
return this.rt.getRegionName().toByteArray();
}
public byte [] getPayload() {
return this.rt.getPayload().toByteArray();
}
@Override
public String toString() {
byte [] payload = getPayload();
return "region=" + Bytes.toStringBinary(getRegionName()) + ", state=" + getEventType() +
", servername=" + getServerName() + ", createTime=" + this.getCreateTime() +
", payload.length=" + (payload == null? 0: payload.length);
}
/**
* @param type
* @param regionName
* @param sn
* @return a serialized pb {@link RegionTransition}
*/
public static RegionTransition createRegionTransition(final EventType type,
final byte [] regionName, final ServerName sn) {
return createRegionTransition(type, regionName, sn, null);
}
/**
* @param type
* @param regionName
* @param sn
* @param payload May be null
* @return a serialized pb {@link RegionTransition}
*/
public static RegionTransition createRegionTransition(final EventType type,
final byte [] regionName, final ServerName sn, final byte [] payload) {
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName pbsn =
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName.newBuilder().
setHostName(sn.getHostname()).setPort(sn.getPort()).setStartCode(sn.getStartcode()).build();
ZooKeeperProtos.RegionTransition.Builder builder = ZooKeeperProtos.RegionTransition.newBuilder().
setEventTypeCode(type.getCode()).setRegionName(ByteStringer.wrap(regionName)).
setServerName(pbsn);
builder.setCreateTime(System.currentTimeMillis());
if (payload != null) builder.setPayload(ByteStringer.wrap(payload));
return new RegionTransition(builder.build());
}
/**
* @param data Serialized date to parse.
* @return A RegionTransition instance made of the passed <code>data</code>
* @throws DeserializationException
* @see #toByteArray()
*/
public static RegionTransition parseFrom(final byte [] data) throws DeserializationException {
ProtobufUtil.expectPBMagicPrefix(data);
try {
int prefixLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.RegionTransition rt = ZooKeeperProtos.RegionTransition.newBuilder().
mergeFrom(data, prefixLen, data.length - prefixLen).build();
return new RegionTransition(rt);
} catch (InvalidProtocolBufferException e) {
throw new DeserializationException(e);
}
}
/**
* @return This instance serialized into a byte array
* @see #parseFrom(byte[])
*/
public byte [] toByteArray() {
return ProtobufUtil.prependPBMagic(this.rt.toByteArray());
}
}

View File

@ -1427,7 +1427,7 @@ public class HBaseAdmin implements Admin {
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
// Close the region without updating zk state.
CloseRegionRequest request =
RequestConverter.buildCloseRegionRequest(sn, encodedRegionName, false);
RequestConverter.buildCloseRegionRequest(sn, encodedRegionName);
try {
CloseRegionResponse response = admin.closeRegion(null, request);
boolean isRegionClosed = response.getClosed();
@ -1452,7 +1452,7 @@ public class HBaseAdmin implements Admin {
throws IOException {
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
// Close the region without updating zk state.
ProtobufUtil.closeRegion(admin, sn, hri.getRegionName(), false);
ProtobufUtil.closeRegion(admin, sn, hri.getRegionName());
}
/**

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.master;
import java.util.Date;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -28,8 +27,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
/**
* State of a Region while undergoing transitions.
* Region state cannot be modified except the stamp field.
* So it is almost immutable.
* This class is immutable.
*/
@InterfaceAudience.Private
public class RegionState {
@ -58,16 +56,10 @@ public class RegionState {
// master doesn't know it's already created
}
// Many threads can update the state at the stamp at the same time
private final AtomicLong stamp;
private HRegionInfo hri;
private volatile ServerName serverName;
private volatile State state;
public RegionState() {
this.stamp = new AtomicLong(System.currentTimeMillis());
}
private final long stamp;
private final HRegionInfo hri;
private final ServerName serverName;
private final State state;
public RegionState(HRegionInfo region, State state) {
this(region, state, System.currentTimeMillis(), null);
@ -82,20 +74,16 @@ public class RegionState {
State state, long stamp, ServerName serverName) {
this.hri = region;
this.state = state;
this.stamp = new AtomicLong(stamp);
this.stamp = stamp;
this.serverName = serverName;
}
public void updateTimestampToNow() {
setTimestamp(System.currentTimeMillis());
}
public State getState() {
return state;
}
public long getStamp() {
return stamp.get();
return stamp;
}
public HRegionInfo getRegion() {
@ -248,12 +236,10 @@ public class RegionState {
* A slower (but more easy-to-read) stringification
*/
public String toDescriptiveString() {
long lstamp = stamp.get();
long relTime = System.currentTimeMillis() - lstamp;
long relTime = System.currentTimeMillis() - stamp;
return hri.getRegionNameAsString()
+ " state=" + state
+ ", ts=" + new Date(lstamp) + " (" + (relTime/1000) + "s ago)"
+ ", ts=" + new Date(stamp) + " (" + (relTime/1000) + "s ago)"
+ ", server=" + serverName;
}
@ -380,10 +366,6 @@ public class RegionState {
return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),state,proto.getStamp(),null);
}
protected void setTimestamp(final long timestamp) {
stamp.set(timestamp);
}
/**
* Check if two states are the same, except timestamp
*/

View File

@ -1142,6 +1142,7 @@ public final class ProtobufUtil {
return toMutation(type, mutation, builder, HConstants.NO_NONCE);
}
@SuppressWarnings("deprecation")
public static MutationProto toMutation(final MutationType type, final Mutation mutation,
MutationProto.Builder builder, long nonce)
throws IOException {
@ -1616,13 +1617,12 @@ public final class ProtobufUtil {
*
* @param admin
* @param regionName
* @param transitionInZK
* @throws IOException
*/
public static void closeRegion(final AdminService.BlockingInterface admin,
final ServerName server, final byte[] regionName, final boolean transitionInZK) throws IOException {
final ServerName server, final byte[] regionName) throws IOException {
CloseRegionRequest closeRegionRequest =
RequestConverter.buildCloseRegionRequest(server, regionName, transitionInZK);
RequestConverter.buildCloseRegionRequest(server, regionName);
try {
admin.closeRegion(null, closeRegionRequest);
} catch (ServiceException se) {
@ -1636,18 +1636,15 @@ public final class ProtobufUtil {
*
* @param admin
* @param regionName
* @param versionOfClosingNode
* @return true if the region is closed
* @throws IOException
*/
public static boolean closeRegion(final AdminService.BlockingInterface admin,
final ServerName server,
final byte[] regionName,
final int versionOfClosingNode, final ServerName destinationServer,
final boolean transitionInZK) throws IOException {
final ServerName server, final byte[] regionName,
final ServerName destinationServer) throws IOException {
CloseRegionRequest closeRegionRequest =
RequestConverter.buildCloseRegionRequest(server,
regionName, versionOfClosingNode, destinationServer, transitionInZK);
regionName, destinationServer);
try {
CloseRegionResponse response = admin.closeRegion(null, closeRegionRequest);
return ResponseConverter.isClosed(response);
@ -1666,7 +1663,7 @@ public final class ProtobufUtil {
public static void openRegion(final AdminService.BlockingInterface admin,
ServerName server, final HRegionInfo region) throws IOException {
OpenRegionRequest request =
RequestConverter.buildOpenRegionRequest(server, region, -1, null, null);
RequestConverter.buildOpenRegionRequest(server, region, null, null);
try {
admin.openRegion(null, request);
} catch (ServiceException se) {
@ -2488,6 +2485,7 @@ public final class ProtobufUtil {
}
}
@SuppressWarnings("deprecation")
public static CompactionDescriptor toCompactionDescriptor(HRegionInfo info, byte[] family,
List<Path> inputPaths, List<Path> outputPaths, Path storeDir) {
// compaction descriptor contains relative paths.

View File

@ -22,7 +22,6 @@ import java.util.List;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.configuration.Configuration;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -104,7 +103,6 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionReq
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Triple;
import com.google.protobuf.ByteString;
@ -708,14 +706,12 @@ public final class RequestConverter {
* @return a protocol buffer OpenRegionRequest
*/
public static OpenRegionRequest
buildOpenRegionRequest(final List<Triple<HRegionInfo, Integer,
buildOpenRegionRequest(final List<Pair<HRegionInfo,
List<ServerName>>> regionOpenInfos, Boolean openForReplay) {
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
for (Triple<HRegionInfo, Integer, List<ServerName>> regionOpenInfo: regionOpenInfos) {
Integer second = regionOpenInfo.getSecond();
int versionOfOfflineNode = second == null ? -1 : second.intValue();
builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(), versionOfOfflineNode,
regionOpenInfo.getThird(), openForReplay));
for (Pair<HRegionInfo, List<ServerName>> regionOpenInfo: regionOpenInfos) {
builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(),
regionOpenInfo.getSecond(), openForReplay));
}
return builder.build();
}
@ -725,16 +721,15 @@ public final class RequestConverter {
*
* @param server the serverName for the RPC
* @param region the region to open
* @param versionOfOfflineNode that needs to be present in the offline node
* @param favoredNodes
* @param openForReplay
* @return a protocol buffer OpenRegionRequest
*/
public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
final HRegionInfo region, final int versionOfOfflineNode, List<ServerName> favoredNodes,
final HRegionInfo region, List<ServerName> favoredNodes,
Boolean openForReplay) {
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
builder.addOpenInfo(buildRegionOpenInfo(region, versionOfOfflineNode, favoredNodes,
builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes,
openForReplay));
if (server != null) {
builder.setServerStartCode(server.getStartcode());
@ -765,33 +760,21 @@ public final class RequestConverter {
* Create a CloseRegionRequest for a given region name
*
* @param regionName the name of the region to close
* @param transitionInZK indicator if to transition in ZK
* @return a CloseRegionRequest
*/
public static CloseRegionRequest buildCloseRegionRequest(ServerName server,
final byte[] regionName, final boolean transitionInZK) {
CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setTransitionInZK(transitionInZK);
if (server != null) {
builder.setServerStartCode(server.getStartcode());
}
return builder.build();
final byte[] regionName) {
return buildCloseRegionRequest(server, regionName, null);
}
public static CloseRegionRequest buildCloseRegionRequest(ServerName server,
final byte[] regionName, final int versionOfClosingNode,
ServerName destinationServer, final boolean transitionInZK) {
final byte[] regionName, ServerName destinationServer) {
CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
builder.setVersionOfClosingNode(versionOfClosingNode);
builder.setTransitionInZK(transitionInZK);
if (destinationServer != null){
builder.setDestinationServer(ProtobufUtil.toServerName( destinationServer) );
builder.setDestinationServer(ProtobufUtil.toServerName(destinationServer));
}
if (server != null) {
builder.setServerStartCode(server.getStartcode());
@ -803,18 +786,15 @@ public final class RequestConverter {
* Create a CloseRegionRequest for a given encoded region name
*
* @param encodedRegionName the name of the region to close
* @param transitionInZK indicator if to transition in ZK
* @return a CloseRegionRequest
*/
public static CloseRegionRequest
buildCloseRegionRequest(ServerName server, final String encodedRegionName,
final boolean transitionInZK) {
buildCloseRegionRequest(ServerName server, final String encodedRegionName) {
CloseRegionRequest.Builder builder = CloseRegionRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.ENCODED_REGION_NAME,
Bytes.toBytes(encodedRegionName));
builder.setRegion(region);
builder.setTransitionInZK(transitionInZK);
if (server != null) {
builder.setServerStartCode(server.getStartcode());
}
@ -1498,13 +1478,10 @@ public final class RequestConverter {
* Create a RegionOpenInfo based on given region info and version of offline node
*/
private static RegionOpenInfo buildRegionOpenInfo(
final HRegionInfo region, final int versionOfOfflineNode,
final HRegionInfo region,
final List<ServerName> favoredNodes, Boolean openForReplay) {
RegionOpenInfo.Builder builder = RegionOpenInfo.newBuilder();
builder.setRegion(HRegionInfo.convert(region));
if (versionOfOfflineNode >= 0) {
builder.setVersionOfOfflineNode(versionOfOfflineNode);
}
if (favoredNodes != null) {
for (ServerName server : favoredNodes) {
builder.addFavoredNodes(ProtobufUtil.toServerName(server));

View File

@ -263,10 +263,6 @@ public class ZKUtil {
private final String keytabFile;
private final String principal;
public JaasConfiguration(String loginContextName, String principal) {
this(loginContextName, principal, null, true);
}
public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
}
@ -951,7 +947,7 @@ public class ZKUtil {
conf.get("hbase.zookeeper.client.keytab.file") != null);
}
private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
private static List<ACL> createACL(ZooKeeperWatcher zkw, String node) {
if (isSecureZooKeeper(zkw.getConfiguration())) {
// Certain znodes are accessed directly by the client,
// so they must be readable by non-authenticated clients
@ -961,7 +957,6 @@ public class ZKUtil {
(node.equals(zkw.clusterIdZNode) == true) ||
(node.equals(zkw.rsZNode) == true) ||
(node.equals(zkw.backupMasterAddressesZNode) == true) ||
(node.startsWith(zkw.assignmentZNode) == true) ||
(node.startsWith(zkw.tableZNode) == true)) {
return ZooKeeperWatcher.CREATOR_ALL_AND_WORLD_READABLE;
}
@ -1779,8 +1774,6 @@ public class ZKUtil {
" byte(s) of data from znode " + znode +
(watcherSet? " and set watcher; ": "; data=") +
(data == null? "null": data.length == 0? "empty": (
znode.startsWith(zkw.assignmentZNode)?
ZKAssign.toString(data): // We should not be doing this reaching into another class
znode.startsWith(zkw.metaServerZNode)?
getServerNameOrEmptyString(data):
znode.startsWith(zkw.backupMasterAddressesZNode)?

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.zookeeper;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
@ -92,8 +93,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
public String backupMasterAddressesZNode;
// znode containing the current cluster state
public String clusterStateZNode;
// znode used for region transitioning and assignment
public String assignmentZNode;
// znode used for table disabling/enabling
public String tableZNode;
// znode containing the unique cluster ID
@ -111,11 +110,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
// Certain ZooKeeper nodes need to be world-readable
public static final ArrayList<ACL> CREATOR_ALL_AND_WORLD_READABLE =
new ArrayList<ACL>() { {
add(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE));
add(new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
}};
public static final List<ACL> CREATOR_ALL_AND_WORLD_READABLE =
Arrays.asList(new ACL(ZooDefs.Perms.READ,ZooDefs.Ids.ANYONE_ID_UNSAFE),
new ACL(ZooDefs.Perms.ALL,ZooDefs.Ids.AUTH_IDS));
private final Configuration conf;
@ -171,9 +168,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
try {
// Create all the necessary "directories" of znodes
ZKUtil.createWithParents(this, baseZNode);
if (conf.getBoolean("hbase.assignment.usezk", false)) {
ZKUtil.createAndFailSilent(this, assignmentZNode);
}
ZKUtil.createAndFailSilent(this, rsZNode);
ZKUtil.createAndFailSilent(this, drainingZNode);
ZKUtil.createAndFailSilent(this, tableZNode);
@ -220,8 +214,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
conf.get("zookeeper.znode.backup.masters", "backup-masters"));
clusterStateZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.state", "running"));
assignmentZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.unassigned", "region-in-transition"));
tableZNode = ZKUtil.joinZNode(baseZNode,
conf.get("zookeeper.znode.tableEnableDisable", "table"));
clusterIdZNode = ZKUtil.joinZNode(baseZNode,

View File

@ -58,21 +58,6 @@ message ClusterUp {
required string start_date = 1;
}
/**
* What we write under unassigned up in zookeeper as a region moves through
* open/close, etc., regions. Details a region in transition.
*/
message RegionTransition {
// Code for EventType gotten by doing o.a.h.h.EventHandler.EventType.getCode()
required uint32 event_type_code = 1;
// Full regionname in bytes
required bytes region_name = 2;
required uint64 create_time = 3;
// The region server where the transition will happen or is happening
required ServerName server_name = 4;
optional bytes payload = 5;
}
/**
* WAL SplitLog directory znodes have this for content. Used doing distributed
* WAL splitting. Holds current state and name of server that originated split.

View File

@ -52,24 +52,4 @@ public abstract class BaseCoordinatedStateManager implements CoordinatedStateMan
@Override
public abstract TableStateManager getTableStateManager() throws InterruptedException,
CoordinatedStateException;
/**
* Method to retrieve coordination for split transaction.
*/
abstract public SplitTransactionCoordination getSplitTransactionCoordination();
/**
* Method to retrieve coordination for closing region operations.
*/
public abstract CloseRegionCoordination getCloseRegionCoordination();
/**
* Method to retrieve coordination for opening region operations.
*/
public abstract OpenRegionCoordination getOpenRegionCoordination();
/**
* Method to retrieve coordination for region merge transaction
*/
public abstract RegionMergeCoordination getRegionMergeCoordination();
}

View File

@ -1,69 +0,0 @@
/**
* 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.coordination;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
/**
* Coordinated operations for close region handlers.
*/
@InterfaceAudience.Private
public interface CloseRegionCoordination {
/**
* Called before actual region closing to check that we can do close operation
* on this region.
* @param regionInfo region being closed
* @param crd details about closing operation
* @return true if caller shall proceed and close, false if need to abort closing.
*/
boolean checkClosingState(HRegionInfo regionInfo, CloseRegionDetails crd);
/**
* Called after region is closed to notify all interesting parties / "register"
* region as finally closed.
* @param region region being closed
* @param sn ServerName on which task runs
* @param crd details about closing operation
*/
void setClosedState(HRegion region, ServerName sn, CloseRegionDetails crd);
/**
* Construct CloseRegionDetails instance from CloseRegionRequest.
* @return instance of CloseRegionDetails
*/
CloseRegionDetails parseFromProtoRequest(AdminProtos.CloseRegionRequest request);
/**
* Get details object with params for case when we're closing on
* regionserver side internally (not because of RPC call from master),
* so we don't parse details from protobuf request.
*/
CloseRegionDetails getDetaultDetails();
/**
* Marker interface for region closing tasks. Used to carry implementation details in
* encapsulated way through Handlers to the consensus API.
*/
static interface CloseRegionDetails {
}
}

View File

@ -1,129 +0,0 @@
/**
* 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.coordination;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import java.io.IOException;
/**
* Cocoordination operations for opening regions.
*/
@InterfaceAudience.Private
public interface OpenRegionCoordination {
//---------------------
// RS-side operations
//---------------------
/**
* Tries to move regions to OPENED state.
*
* @param r Region we're working on.
* @param ord details about region opening task
* @return whether transition was successful or not
* @throws java.io.IOException
*/
boolean transitionToOpened(HRegion r, OpenRegionDetails ord) throws IOException;
/**
* Transitions region from offline to opening state.
* @param regionInfo region we're working on.
* @param ord details about opening task.
* @return true if successful, false otherwise
*/
boolean transitionFromOfflineToOpening(HRegionInfo regionInfo,
OpenRegionDetails ord);
/**
* Heartbeats to prevent timeouts.
*
* @param ord details about opening task.
* @param regionInfo region we're working on.
* @param rsServices instance of RegionServerrServices
* @param context used for logging purposes only
* @return true if successful heartbeat, false otherwise.
*/
boolean tickleOpening(OpenRegionDetails ord, HRegionInfo regionInfo,
RegionServerServices rsServices, String context);
/**
* Tries transition region from offline to failed open.
* @param rsServices instance of RegionServerServices
* @param hri region we're working on
* @param ord details about region opening task
* @return true if successful, false otherwise
*/
boolean tryTransitionFromOfflineToFailedOpen(RegionServerServices rsServices,
HRegionInfo hri, OpenRegionDetails ord);
/**
* Tries transition from Opening to Failed open.
* @param hri region we're working on
* @param ord details about region opening task
* @return true if successfu. false otherwise.
*/
boolean tryTransitionFromOpeningToFailedOpen(HRegionInfo hri, OpenRegionDetails ord);
/**
* Construct OpenRegionDetails instance from part of protobuf request.
* @return instance of OpenRegionDetails.
*/
OpenRegionDetails parseFromProtoRequest(AdminProtos.OpenRegionRequest.RegionOpenInfo
regionOpenInfo);
/**
* Get details object with params for case when we're opening on
* regionserver side with all "default" properties.
*/
OpenRegionDetails getDetailsForNonCoordinatedOpening();
//-------------------------
// HMaster-side operations
//-------------------------
/**
* Commits opening operation on HM side (steps required for "commit"
* are determined by coordination implementation).
* @return true if committed successfully, false otherwise.
*/
public boolean commitOpenOnMasterSide(AssignmentManager assignmentManager,
HRegionInfo regionInfo,
OpenRegionDetails ord);
/**
* Interface for region opening tasks. Used to carry implementation details in
* encapsulated way through Handlers to the coordination API.
*/
static interface OpenRegionDetails {
/**
* Sets server name on which opening operation is running.
*/
void setServerName(ServerName serverName);
/**
* @return server name on which opening op is running.
*/
ServerName getServerName();
}
}

View File

@ -1,106 +0,0 @@
/**
* 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.coordination;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
/**
* Coordination operations for region merge transaction. The operation should be coordinated at the
* following stages:<br>
* 1. startRegionMergeTransaction - all preparation/initialization for merge region transaction<br>
* 2. waitForRegionMergeTransaction - wait until coordination complete all works related
* to merge<br>
* 3. confirmRegionMergeTransaction - confirm that the merge could be completed and none of merging
* regions moved somehow<br>
* 4. completeRegionMergeTransaction - all steps that are required to complete the transaction.
* Called after PONR (point of no return) <br>
*/
@InterfaceAudience.Private
public interface RegionMergeCoordination {
RegionMergeDetails getDefaultDetails();
/**
* Dummy interface for region merge transaction details.
*/
public static interface RegionMergeDetails {
}
/**
* Start the region merge transaction
* @param region region to be created as offline
* @param serverName server event originates from
* @throws IOException
*/
void startRegionMergeTransaction(HRegionInfo region, ServerName serverName, HRegionInfo a,
HRegionInfo b) throws IOException;
/**
* Get everything ready for region merge
* @throws IOException
*/
void waitForRegionMergeTransaction(RegionServerServices services, HRegionInfo mergedRegionInfo,
HRegion region_a, HRegion region_b, RegionMergeDetails details) throws IOException;
/**
* Confirm that the region merge can be performed
* @param merged region
* @param a merging region A
* @param b merging region B
* @param serverName server event originates from
* @param rmd region merge details
* @throws IOException If thrown, transaction failed.
*/
void confirmRegionMergeTransaction(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
ServerName serverName, RegionMergeDetails rmd) throws IOException;
/**
* @param merged region
* @param a merging region A
* @param b merging region B
* @param serverName server event originates from
* @param rmd region merge details
* @throws IOException
*/
void processRegionMergeRequest(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
ServerName serverName, RegionMergeDetails rmd) throws IOException;
/**
* Finish off merge transaction
* @param services Used to online/offline regions.
* @param merged region
* @param region_a merging region A
* @param region_b merging region B
* @param rmd region merge details
* @param mergedRegion
* @throws IOException If thrown, transaction failed. Call
* {@link RegionMergeTransaction#rollback(Server, RegionServerServices)}
*/
void completeRegionMergeTransaction(RegionServerServices services, HRegionInfo merged,
HRegion region_a, HRegion region_b, RegionMergeDetails rmd, HRegion mergedRegion)
throws IOException;
/**
* This method is used during rollback
* @param merged region to be rolled back
*/
void clean(HRegionInfo merged);
}

View File

@ -1,101 +0,0 @@
/**
*
* 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.coordination;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.SplitTransaction;
/**
* Coordination operations for split transaction. The split operation should be coordinated at the
* following stages:
* 1. start - all preparation/initialization for split transaction should be done there.
* 2. waitForSplitTransaction - the coordination should perform all logic related to split
* transaction and wait till it's finished
* 3. completeSplitTransaction - all steps that are required to complete the transaction.
* Called after PONR (point of no return)
*/
@InterfaceAudience.Private
public interface SplitTransactionCoordination {
/**
* Dummy interface for split transaction details.
*/
public static interface SplitTransactionDetails {
}
SplitTransactionDetails getDefaultDetails();
/**
* init coordination for split transaction
* @param parent region to be created as offline
* @param serverName server event originates from
* @param hri_a daughter region
* @param hri_b daughter region
* @throws IOException
*/
void startSplitTransaction(HRegion parent, ServerName serverName,
HRegionInfo hri_a, HRegionInfo hri_b) throws IOException;
/**
* Wait while coordination process the transaction
* @param services Used to online/offline regions.
* @param parent region
* @param hri_a daughter region
* @param hri_b daughter region
* @param std split transaction details
* @throws IOException
*/
void waitForSplitTransaction(final RegionServerServices services,
HRegion parent, HRegionInfo hri_a, HRegionInfo hri_b, SplitTransactionDetails std)
throws IOException;
/**
* Finish off split transaction
* @param services Used to online/offline regions.
* @param first daughter region
* @param second daughter region
* @param std split transaction details
* @param parent
* @throws IOException If thrown, transaction failed. Call
* {@link SplitTransaction#rollback(Server, RegionServerServices)}
*/
void completeSplitTransaction(RegionServerServices services, HRegion first,
HRegion second, SplitTransactionDetails std, HRegion parent) throws IOException;
/**
* clean the split transaction
* @param hri node to delete
*/
void clean(final HRegionInfo hri);
/**
* Required by AssignmentManager
*/
int processTransition(HRegionInfo p, HRegionInfo hri_a, HRegionInfo hri_b,
ServerName sn, SplitTransactionDetails std) throws IOException;
}

View File

@ -1,314 +0,0 @@
/**
* 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.coordination;
import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_SPLIT;
import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_SPLITTING;
import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REQUEST_REGION_SPLIT;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.SplitTransaction;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
public class ZKSplitTransactionCoordination implements SplitTransactionCoordination {
private CoordinatedStateManager coordinationManager;
private final ZooKeeperWatcher watcher;
private static final Log LOG = LogFactory.getLog(ZKSplitTransactionCoordination.class);
public ZKSplitTransactionCoordination(CoordinatedStateManager coordinationProvider,
ZooKeeperWatcher watcher) {
this.coordinationManager = coordinationProvider;
this.watcher = watcher;
}
/**
* Creates a new ephemeral node in the PENDING_SPLIT state for the specified region. Create it
* ephemeral in case regionserver dies mid-split.
* <p>
* Does not transition nodes from other states. If a node already exists for this region, an
* Exception will be thrown.
* @param parent region to be created as offline
* @param serverName server event originates from
* @param hri_a daughter region
* @param hri_b daughter region
* @throws IOException
*/
@Override
public void startSplitTransaction(HRegion parent, ServerName serverName, HRegionInfo hri_a,
HRegionInfo hri_b) throws IOException {
HRegionInfo region = parent.getRegionInfo();
try {
LOG.debug(watcher.prefix("Creating ephemeral node for " + region.getEncodedName()
+ " in PENDING_SPLIT state"));
byte[] payload = HRegionInfo.toDelimitedByteArray(hri_a, hri_b);
RegionTransition rt =
RegionTransition.createRegionTransition(RS_ZK_REQUEST_REGION_SPLIT,
region.getRegionName(), serverName, payload);
String node = ZKAssign.getNodeName(watcher, region.getEncodedName());
if (!ZKUtil.createEphemeralNodeAndWatch(watcher, node, rt.toByteArray())) {
throw new IOException("Failed create of ephemeral " + node);
}
} catch (KeeperException e) {
throw new IOException("Failed creating PENDING_SPLIT znode on "
+ parent.getRegionNameAsString(), e);
}
}
/**
* Transitions an existing ephemeral node for the specified region which is currently in the begin
* state to be in the end state. Master cleans up the final SPLIT znode when it reads it (or if we
* crash, zk will clean it up).
* <p>
* Does not transition nodes from other states. If for some reason the node could not be
* transitioned, the method returns -1. If the transition is successful, the version of the node
* after transition is returned.
* <p>
* This method can fail and return false for three different reasons:
* <ul>
* <li>Node for this region does not exist</li>
* <li>Node for this region is not in the begin state</li>
* <li>After verifying the begin state, update fails because of wrong version (this should never
* actually happen since an RS only does this transition following a transition to the begin
* state. If two RS are conflicting, one would fail the original transition to the begin state and
* not this transition)</li>
* </ul>
* <p>
* Does not set any watches.
* <p>
* This method should only be used by a RegionServer when splitting a region.
* @param parent region to be transitioned to opened
* @param a Daughter a of split
* @param b Daughter b of split
* @param serverName server event originates from
* @param std split transaction details
* @param beginState the expected current state the znode should be
* @param endState the state to be transition to
* @return version of node after transition, -1 if unsuccessful transition
* @throws IOException
*/
private int transitionSplittingNode(HRegionInfo parent, HRegionInfo a, HRegionInfo b,
ServerName serverName, SplitTransactionDetails std, final EventType beginState,
final EventType endState) throws IOException {
ZkSplitTransactionDetails zstd = (ZkSplitTransactionDetails) std;
byte[] payload = HRegionInfo.toDelimitedByteArray(a, b);
try {
return ZKAssign.transitionNode(watcher, parent, serverName, beginState, endState,
zstd.getZnodeVersion(), payload);
} catch (KeeperException e) {
throw new IOException(
"Failed transition of splitting node " + parent.getRegionNameAsString(), e);
}
}
/**
* Wait for the splitting node to be transitioned from pending_split to splitting by master.
* That's how we are sure master has processed the event and is good with us to move on. If we
* don't get any update, we periodically transition the node so that master gets the callback. If
* the node is removed or is not in pending_split state any more, we abort the split.
*/
@Override
public void waitForSplitTransaction(final RegionServerServices services, HRegion parent,
HRegionInfo hri_a, HRegionInfo hri_b, SplitTransactionDetails sptd) throws IOException {
ZkSplitTransactionDetails zstd = (ZkSplitTransactionDetails) sptd;
// After creating the split node, wait for master to transition it
// from PENDING_SPLIT to SPLITTING so that we can move on. We want master
// knows about it and won't transition any region which is splitting.
try {
int spins = 0;
Stat stat = new Stat();
ServerName expectedServer = coordinationManager.getServer().getServerName();
String node = parent.getRegionInfo().getEncodedName();
while (!(coordinationManager.getServer().isStopped() || services.isStopping())) {
if (spins % 5 == 0) {
LOG.debug("Still waiting for master to process " + "the pending_split for " + node);
SplitTransactionDetails temp = getDefaultDetails();
transitionSplittingNode(parent.getRegionInfo(), hri_a, hri_b, expectedServer, temp,
RS_ZK_REQUEST_REGION_SPLIT, RS_ZK_REQUEST_REGION_SPLIT);
}
Thread.sleep(100);
spins++;
byte[] data = ZKAssign.getDataNoWatch(watcher, node, stat);
if (data == null) {
throw new IOException("Data is null, splitting node " + node + " no longer exists");
}
RegionTransition rt = RegionTransition.parseFrom(data);
EventType et = rt.getEventType();
if (et == RS_ZK_REGION_SPLITTING) {
ServerName serverName = rt.getServerName();
if (!serverName.equals(expectedServer)) {
throw new IOException("Splitting node " + node + " is for " + serverName + ", not us "
+ expectedServer);
}
byte[] payloadOfSplitting = rt.getPayload();
List<HRegionInfo> splittingRegions =
HRegionInfo.parseDelimitedFrom(payloadOfSplitting, 0, payloadOfSplitting.length);
assert splittingRegions.size() == 2;
HRegionInfo a = splittingRegions.get(0);
HRegionInfo b = splittingRegions.get(1);
if (!(hri_a.equals(a) && hri_b.equals(b))) {
throw new IOException("Splitting node " + node + " is for " + a + ", " + b
+ ", not expected daughters: " + hri_a + ", " + hri_b);
}
// Master has processed it.
zstd.setZnodeVersion(stat.getVersion());
return;
}
if (et != RS_ZK_REQUEST_REGION_SPLIT) {
throw new IOException("Splitting node " + node + " moved out of splitting to " + et);
}
}
// Server is stopping/stopped
throw new IOException("Server is " + (services.isStopping() ? "stopping" : "stopped"));
} catch (Exception e) {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new IOException("Failed getting SPLITTING znode on " + parent.getRegionNameAsString(),
e);
}
}
/**
* Finish off split transaction, transition the zknode
* @param services Used to online/offline regions.
* @param a daughter region
* @param b daughter region
* @param std split transaction details
* @param parent
* @throws IOException If thrown, transaction failed. Call
* {@link SplitTransaction#rollback(Server, RegionServerServices)}
*/
@Override
public void completeSplitTransaction(final RegionServerServices services, HRegion a, HRegion b,
SplitTransactionDetails std, HRegion parent) throws IOException {
ZkSplitTransactionDetails zstd = (ZkSplitTransactionDetails) std;
// Tell master about split by updating zk. If we fail, abort.
if (coordinationManager.getServer() != null) {
try {
zstd.setZnodeVersion(transitionSplittingNode(parent.getRegionInfo(), a.getRegionInfo(),
b.getRegionInfo(), coordinationManager.getServer().getServerName(), zstd,
RS_ZK_REGION_SPLITTING, RS_ZK_REGION_SPLIT));
int spins = 0;
// Now wait for the master to process the split. We know it's done
// when the znode is deleted. The reason we keep tickling the znode is
// that it's possible for the master to miss an event.
do {
if (spins % 10 == 0) {
LOG.debug("Still waiting on the master to process the split for "
+ parent.getRegionInfo().getEncodedName());
}
Thread.sleep(100);
// When this returns -1 it means the znode doesn't exist
zstd.setZnodeVersion(transitionSplittingNode(parent.getRegionInfo(), a.getRegionInfo(),
b.getRegionInfo(), coordinationManager.getServer().getServerName(), zstd,
RS_ZK_REGION_SPLIT, RS_ZK_REGION_SPLIT));
spins++;
} while (zstd.getZnodeVersion() != -1 && !coordinationManager.getServer().isStopped()
&& !services.isStopping());
} catch (Exception e) {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new IOException("Failed telling master about split", e);
}
}
// Leaving here, the splitdir with its dross will be in place but since the
// split was successful, just leave it; it'll be cleaned when parent is
// deleted and cleaned up.
}
@Override
public void clean(final HRegionInfo hri) {
try {
// Only delete if its in expected state; could have been hijacked.
if (!ZKAssign.deleteNode(coordinationManager.getServer().getZooKeeper(),
hri.getEncodedName(), RS_ZK_REQUEST_REGION_SPLIT, coordinationManager.getServer()
.getServerName())) {
ZKAssign.deleteNode(coordinationManager.getServer().getZooKeeper(), hri.getEncodedName(),
RS_ZK_REGION_SPLITTING, coordinationManager.getServer().getServerName());
}
} catch (KeeperException.NoNodeException e) {
LOG.info("Failed cleanup zk node of " + hri.getRegionNameAsString(), e);
} catch (KeeperException e) {
coordinationManager.getServer().abort("Failed cleanup of " + hri.getRegionNameAsString(), e);
}
}
/**
* ZK-based implementation. Has details about whether the state transition should be reflected in
* ZK, as well as expected version of znode.
*/
public static class ZkSplitTransactionDetails implements
SplitTransactionCoordination.SplitTransactionDetails {
private int znodeVersion;
public ZkSplitTransactionDetails() {
}
/**
* @return znode current version
*/
public int getZnodeVersion() {
return znodeVersion;
}
/**
* @param znodeVersion znode new version
*/
public void setZnodeVersion(int znodeVersion) {
this.znodeVersion = znodeVersion;
}
}
@Override
public SplitTransactionDetails getDefaultDetails() {
ZkSplitTransactionDetails zstd = new ZkSplitTransactionDetails();
zstd.setZnodeVersion(-1);
return zstd;
}
@Override
public int processTransition(HRegionInfo p, HRegionInfo hri_a, HRegionInfo hri_b, ServerName sn,
SplitTransactionDetails std) throws IOException {
return transitionSplittingNode(p, hri_a, hri_b, sn, std, RS_ZK_REQUEST_REGION_SPLIT,
RS_ZK_REGION_SPLITTING);
}
}

View File

@ -1,197 +0,0 @@
/**
* 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.coordination;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import java.io.IOException;
/**
* ZK-based implementation of {@link CloseRegionCoordination}.
*/
@InterfaceAudience.Private
public class ZkCloseRegionCoordination implements CloseRegionCoordination {
private static final Log LOG = LogFactory.getLog(ZkCloseRegionCoordination.class);
private final static int FAILED_VERSION = -1;
private CoordinatedStateManager csm;
private final ZooKeeperWatcher watcher;
public ZkCloseRegionCoordination(CoordinatedStateManager csm, ZooKeeperWatcher watcher) {
this.csm = csm;
this.watcher = watcher;
}
/**
* In ZK-based version we're checking for bad znode state, e.g. if we're
* trying to delete the znode, and it's not ours (version doesn't match).
*/
@Override
public boolean checkClosingState(HRegionInfo regionInfo, CloseRegionDetails crd) {
ZkCloseRegionDetails zkCrd = (ZkCloseRegionDetails) crd;
try {
return zkCrd.isPublishStatusInZk() && !ZKAssign.checkClosingState(watcher,
regionInfo, ((ZkCloseRegionDetails) crd).getExpectedVersion());
} catch (KeeperException ke) {
csm.getServer().abort("Unrecoverable exception while checking state with zk " +
regionInfo.getRegionNameAsString() + ", still finishing close", ke);
throw new RuntimeException(ke);
}
}
/**
* In ZK-based version we do some znodes transitioning.
*/
@Override
public void setClosedState(HRegion region, ServerName sn, CloseRegionDetails crd) {
ZkCloseRegionDetails zkCrd = (ZkCloseRegionDetails) crd;
String name = region.getRegionInfo().getRegionNameAsString();
if (zkCrd.isPublishStatusInZk()) {
if (setClosedState(region,sn, zkCrd)) {
LOG.debug("Set closed state in zk for " + name + " on " + sn);
} else {
LOG.debug("Set closed state in zk UNSUCCESSFUL for " + name + " on " + sn);
}
}
}
/**
* Parse ZK-related fields from request.
*/
@Override
public CloseRegionDetails parseFromProtoRequest(AdminProtos.CloseRegionRequest request) {
ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
new ZkCloseRegionCoordination.ZkCloseRegionDetails();
zkCrd.setPublishStatusInZk(request.getTransitionInZK());
int versionOfClosingNode = -1;
if (request.hasVersionOfClosingNode()) {
versionOfClosingNode = request.getVersionOfClosingNode();
}
zkCrd.setExpectedVersion(versionOfClosingNode);
return zkCrd;
}
/**
* No ZK tracking will be performed for that case.
* This method should be used when we want to construct CloseRegionDetails,
* but don't want any coordination on that (when it's initiated by regionserver),
* so no znode state transitions will be performed.
*/
@Override
public CloseRegionDetails getDetaultDetails() {
ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
new ZkCloseRegionCoordination.ZkCloseRegionDetails();
zkCrd.setPublishStatusInZk(false);
zkCrd.setExpectedVersion(FAILED_VERSION);
return zkCrd;
}
/**
* Transition ZK node to CLOSED
* @param region HRegion instance being closed
* @param sn ServerName on which task runs
* @param zkCrd details about region closing operation.
* @return If the state is set successfully
*/
private boolean setClosedState(final HRegion region,
ServerName sn,
ZkCloseRegionDetails zkCrd) {
final int expectedVersion = zkCrd.getExpectedVersion();
try {
if (ZKAssign.transitionNodeClosed(watcher, region.getRegionInfo(),
sn, expectedVersion) == FAILED_VERSION) {
LOG.warn("Completed the CLOSE of a region but when transitioning from " +
" CLOSING to CLOSED got a version mismatch, someone else clashed " +
"so now unassigning");
region.close();
return false;
}
} catch (NullPointerException e) {
// I've seen NPE when table was deleted while close was running in unit tests.
LOG.warn("NPE during close -- catching and continuing...", e);
return false;
} catch (KeeperException e) {
LOG.error("Failed transitioning node from CLOSING to CLOSED", e);
return false;
} catch (IOException e) {
LOG.error("Failed to close region after failing to transition", e);
return false;
}
return true;
}
/**
* ZK-based implementation. Has details about whether the state transition should be
* reflected in ZK, as well as expected version of znode.
*/
public static class ZkCloseRegionDetails implements CloseRegionCoordination.CloseRegionDetails {
/**
* True if we are to update zk about the region close; if the close
* was orchestrated by master, then update zk. If the close is being run by
* the regionserver because its going down, don't update zk.
* */
private boolean publishStatusInZk;
/**
* The version of znode to compare when RS transitions the znode from
* CLOSING state.
*/
private int expectedVersion = FAILED_VERSION;
public ZkCloseRegionDetails() {
}
public ZkCloseRegionDetails(boolean publishStatusInZk, int expectedVersion) {
this.publishStatusInZk = publishStatusInZk;
this.expectedVersion = expectedVersion;
}
public boolean isPublishStatusInZk() {
return publishStatusInZk;
}
public void setPublishStatusInZk(boolean publishStatusInZk) {
this.publishStatusInZk = publishStatusInZk;
}
public int getExpectedVersion() {
return expectedVersion;
}
public void setExpectedVersion(int expectedVersion) {
this.expectedVersion = expectedVersion;
}
}
}

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase.coordination;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.Server;
@ -32,23 +30,13 @@ import org.apache.zookeeper.KeeperException;
*/
@InterfaceAudience.Private
public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
private static final Log LOG = LogFactory.getLog(ZkCoordinatedStateManager.class);
protected Server server;
protected ZooKeeperWatcher watcher;
protected SplitTransactionCoordination splitTransactionCoordination;
protected CloseRegionCoordination closeRegionCoordination;
protected OpenRegionCoordination openRegionCoordination;
protected RegionMergeCoordination regionMergeCoordination;
@Override
public void initialize(Server server) {
this.server = server;
this.watcher = server.getZooKeeper();
splitTransactionCoordination = new ZKSplitTransactionCoordination(this, watcher);
closeRegionCoordination = new ZkCloseRegionCoordination(this, watcher);
openRegionCoordination = new ZkOpenRegionCoordination(this, watcher);
regionMergeCoordination = new ZkRegionMergeCoordination(this, watcher);
}
@Override
@ -65,24 +53,4 @@ public class ZkCoordinatedStateManager extends BaseCoordinatedStateManager {
throw new CoordinatedStateException(e);
}
}
@Override
public SplitTransactionCoordination getSplitTransactionCoordination() {
return splitTransactionCoordination;
}
@Override
public CloseRegionCoordination getCloseRegionCoordination() {
return closeRegionCoordination;
}
@Override
public OpenRegionCoordination getOpenRegionCoordination() {
return openRegionCoordination;
}
@Override
public RegionMergeCoordination getRegionMergeCoordination() {
return regionMergeCoordination;
}
}

View File

@ -1,414 +0,0 @@
/**
* 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.coordination;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import java.io.IOException;
/**
* ZK-based implementation of {@link OpenRegionCoordination}.
*/
@InterfaceAudience.Private
public class ZkOpenRegionCoordination implements OpenRegionCoordination {
private static final Log LOG = LogFactory.getLog(ZkOpenRegionCoordination.class);
private CoordinatedStateManager coordination;
private final ZooKeeperWatcher watcher;
public ZkOpenRegionCoordination(CoordinatedStateManager coordination,
ZooKeeperWatcher watcher) {
this.coordination = coordination;
this.watcher = watcher;
}
//-------------------------------
// Region Server-side operations
//-------------------------------
/**
* @param r Region we're working on.
* @return whether znode is successfully transitioned to OPENED state.
* @throws java.io.IOException
*/
@Override
public boolean transitionToOpened(final HRegion r, OpenRegionDetails ord) throws IOException {
ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
boolean result = false;
HRegionInfo hri = r.getRegionInfo();
final String name = hri.getRegionNameAsString();
// Finally, Transition ZK node to OPENED
try {
if (ZKAssign.transitionNodeOpened(watcher, hri,
zkOrd.getServerName(), zkOrd.getVersion()) == -1) {
String warnMsg = "Completed the OPEN of region " + name +
" but when transitioning from " + " OPENING to OPENED ";
try {
String node = ZKAssign.getNodeName(watcher, hri.getEncodedName());
if (ZKUtil.checkExists(watcher, node) < 0) {
// if the znode
coordination.getServer().abort(warnMsg + "the znode disappeared", null);
} else {
LOG.warn(warnMsg + "got a version mismatch, someone else clashed; " +
"so now unassigning -- closing region on server: " + zkOrd.getServerName());
}
} catch (KeeperException ke) {
coordination.getServer().abort(warnMsg, ke);
}
} else {
LOG.debug("Transitioned " + r.getRegionInfo().getEncodedName() +
" to OPENED in zk on " + zkOrd.getServerName());
result = true;
}
} catch (KeeperException e) {
LOG.error("Failed transitioning node " + name +
" from OPENING to OPENED -- closing region", e);
}
return result;
}
/**
* Transition ZK node from OFFLINE to OPENING.
* @param regionInfo region info instance
* @param ord - instance of open region details, for ZK implementation
* will include version Of OfflineNode that needs to be compared
* before changing the node's state from OFFLINE
* @return True if successful transition.
*/
@Override
public boolean transitionFromOfflineToOpening(HRegionInfo regionInfo,
OpenRegionDetails ord) {
ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
// encoded name is used as znode encoded name in ZK
final String encodedName = regionInfo.getEncodedName();
// TODO: should also handle transition from CLOSED?
try {
// Initialize the znode version.
zkOrd.setVersion(ZKAssign.transitionNode(watcher, regionInfo,
zkOrd.getServerName(), EventType.M_ZK_REGION_OFFLINE,
EventType.RS_ZK_REGION_OPENING, zkOrd.getVersionOfOfflineNode()));
} catch (KeeperException e) {
LOG.error("Error transition from OFFLINE to OPENING for region=" +
encodedName, e);
zkOrd.setVersion(-1);
return false;
}
boolean b = isGoodVersion(zkOrd);
if (!b) {
LOG.warn("Failed transition from OFFLINE to OPENING for region=" +
encodedName);
}
return b;
}
/**
* Update our OPENING state in zookeeper.
* Do this so master doesn't timeout this region-in-transition.
* We may lose the znode ownership during the open. Currently its
* too hard interrupting ongoing region open. Just let it complete
* and check we still have the znode after region open.
*
* @param context Some context to add to logs if failure
* @return True if successful transition.
*/
@Override
public boolean tickleOpening(OpenRegionDetails ord, HRegionInfo regionInfo,
RegionServerServices rsServices, final String context) {
ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
if (!isRegionStillOpening(regionInfo, rsServices)) {
LOG.warn("Open region aborted since it isn't opening any more");
return false;
}
// If previous checks failed... do not try again.
if (!isGoodVersion(zkOrd)) return false;
String encodedName = regionInfo.getEncodedName();
try {
zkOrd.setVersion(ZKAssign.confirmNodeOpening(watcher,
regionInfo, zkOrd.getServerName(), zkOrd.getVersion()));
} catch (KeeperException e) {
coordination.getServer().abort("Exception refreshing OPENING; region=" + encodedName +
", context=" + context, e);
zkOrd.setVersion(-1);
return false;
}
boolean b = isGoodVersion(zkOrd);
if (!b) {
LOG.warn("Failed refreshing OPENING; region=" + encodedName +
", context=" + context);
}
return b;
}
/**
* Try to transition to open.
*
* This is not guaranteed to succeed, we just do our best.
*
* @param rsServices
* @param hri Region we're working on.
* @param ord Details about region open task
* @return whether znode is successfully transitioned to FAILED_OPEN state.
*/
@Override
public boolean tryTransitionFromOfflineToFailedOpen(RegionServerServices rsServices,
final HRegionInfo hri,
OpenRegionDetails ord) {
ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
boolean result = false;
final String name = hri.getRegionNameAsString();
try {
LOG.info("Opening of region " + hri + " failed, transitioning" +
" from OFFLINE to FAILED_OPEN in ZK, expecting version " +
zkOrd.getVersionOfOfflineNode());
if (ZKAssign.transitionNode(
rsServices.getZooKeeper(), hri,
rsServices.getServerName(),
EventType.M_ZK_REGION_OFFLINE,
EventType.RS_ZK_REGION_FAILED_OPEN,
zkOrd.getVersionOfOfflineNode()) == -1) {
LOG.warn("Unable to mark region " + hri + " as FAILED_OPEN. " +
"It's likely that the master already timed out this open " +
"attempt, and thus another RS already has the region.");
} else {
result = true;
}
} catch (KeeperException e) {
LOG.error("Failed transitioning node " + name + " from OFFLINE to FAILED_OPEN", e);
}
return result;
}
private boolean isGoodVersion(ZkOpenRegionDetails zkOrd) {
return zkOrd.getVersion() != -1;
}
/**
* This is not guaranteed to succeed, we just do our best.
* @param hri Region we're working on.
* @return whether znode is successfully transitioned to FAILED_OPEN state.
*/
@Override
public boolean tryTransitionFromOpeningToFailedOpen(final HRegionInfo hri,
OpenRegionDetails ord) {
ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
boolean result = false;
final String name = hri.getRegionNameAsString();
try {
LOG.info("Opening of region " + hri + " failed, transitioning" +
" from OPENING to FAILED_OPEN in ZK, expecting version " + zkOrd.getVersion());
if (ZKAssign.transitionNode(
watcher, hri,
zkOrd.getServerName(),
EventType.RS_ZK_REGION_OPENING,
EventType.RS_ZK_REGION_FAILED_OPEN,
zkOrd.getVersion()) == -1) {
LOG.warn("Unable to mark region " + hri + " as FAILED_OPEN. " +
"It's likely that the master already timed out this open " +
"attempt, and thus another RS already has the region.");
} else {
result = true;
}
} catch (KeeperException e) {
LOG.error("Failed transitioning node " + name +
" from OPENING to FAILED_OPEN", e);
}
return result;
}
/**
* Parse ZK-related fields from request.
*/
@Override
public OpenRegionCoordination.OpenRegionDetails parseFromProtoRequest(
AdminProtos.OpenRegionRequest.RegionOpenInfo regionOpenInfo) {
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
int versionOfOfflineNode = -1;
if (regionOpenInfo.hasVersionOfOfflineNode()) {
versionOfOfflineNode = regionOpenInfo.getVersionOfOfflineNode();
}
zkCrd.setVersionOfOfflineNode(versionOfOfflineNode);
zkCrd.setServerName(coordination.getServer().getServerName());
return zkCrd;
}
/**
* No ZK tracking will be performed for that case.
* This method should be used when we want to construct CloseRegionDetails,
* but don't want any coordination on that (when it's initiated by regionserver),
* so no znode state transitions will be performed.
*/
@Override
public OpenRegionCoordination.OpenRegionDetails getDetailsForNonCoordinatedOpening() {
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setVersionOfOfflineNode(-1);
zkCrd.setServerName(coordination.getServer().getServerName());
return zkCrd;
}
//--------------------------
// HMaster-side operations
//--------------------------
@Override
public boolean commitOpenOnMasterSide(AssignmentManager assignmentManager,
HRegionInfo regionInfo,
OpenRegionDetails ord) {
boolean committedSuccessfully = true;
// Code to defend against case where we get SPLIT before region open
// processing completes; temporary till we make SPLITs go via zk -- 0.92.
RegionState regionState = assignmentManager.getRegionStates()
.getRegionTransitionState(regionInfo.getEncodedName());
boolean openedNodeDeleted = false;
if (regionState != null && regionState.isOpened()) {
openedNodeDeleted = deleteOpenedNode(regionInfo, ord);
if (!openedNodeDeleted) {
LOG.error("Znode of region " + regionInfo.getShortNameToLog() + " could not be deleted.");
}
} else {
LOG.warn("Skipping the onlining of " + regionInfo.getShortNameToLog() +
" because regions is NOT in RIT -- presuming this is because it SPLIT");
}
if (!openedNodeDeleted) {
if (assignmentManager.getTableStateManager().isTableState(regionInfo.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
debugLog(regionInfo, "Opened region "
+ regionInfo.getShortNameToLog() + " but "
+ "this table is disabled, triggering close of region");
committedSuccessfully = false;
}
}
return committedSuccessfully;
}
private boolean deleteOpenedNode(HRegionInfo regionInfo, OpenRegionDetails ord) {
ZkOpenRegionDetails zkOrd = (ZkOpenRegionDetails) ord;
int expectedVersion = zkOrd.getVersion();
debugLog(regionInfo, "Handling OPENED of " +
regionInfo.getShortNameToLog() + " from " + zkOrd.getServerName().toString() +
"; deleting unassigned node");
try {
// delete the opened znode only if the version matches.
return ZKAssign.deleteNode(this.coordination.getServer().getZooKeeper(),
regionInfo.getEncodedName(), EventType.RS_ZK_REGION_OPENED, expectedVersion);
} catch(KeeperException.NoNodeException e){
// Getting no node exception here means that already the region has been opened.
LOG.warn("The znode of the region " + regionInfo.getShortNameToLog() +
" would have already been deleted");
return false;
} catch (KeeperException e) {
this.coordination.getServer().abort("Error deleting OPENED node in ZK (" +
regionInfo.getRegionNameAsString() + ")", e);
}
return false;
}
private void debugLog(HRegionInfo region, String string) {
if (region.isMetaTable()) {
LOG.info(string);
} else {
LOG.debug(string);
}
}
// Additional classes and helper methods
/**
* ZK-based implementation. Has details about whether the state transition should be
* reflected in ZK, as well as expected version of znode.
*/
public static class ZkOpenRegionDetails implements OpenRegionCoordination.OpenRegionDetails {
// We get version of our znode at start of open process and monitor it across
// the total open. We'll fail the open if someone hijacks our znode; we can
// tell this has happened if version is not as expected.
private volatile int version = -1;
//version of the offline node that was set by the master
private volatile int versionOfOfflineNode = -1;
/**
* Server name the handler is running on.
*/
private ServerName serverName;
public ZkOpenRegionDetails() {
}
public ZkOpenRegionDetails(int versionOfOfflineNode) {
this.versionOfOfflineNode = versionOfOfflineNode;
}
public int getVersionOfOfflineNode() {
return versionOfOfflineNode;
}
public void setVersionOfOfflineNode(int versionOfOfflineNode) {
this.versionOfOfflineNode = versionOfOfflineNode;
}
public int getVersion() {
return version;
}
public void setVersion(int version) {
this.version = version;
}
@Override
public ServerName getServerName() {
return serverName;
}
@Override
public void setServerName(ServerName serverName) {
this.serverName = serverName;
}
}
private boolean isRegionStillOpening(HRegionInfo regionInfo, RegionServerServices rsServices) {
byte[] encodedName = regionInfo.getEncodedNameAsBytes();
Boolean action = rsServices.getRegionsInTransitionInRS().get(encodedName);
return Boolean.TRUE.equals(action); // true means opening for RIT
}
}

View File

@ -1,326 +0,0 @@
/**
*
* 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.coordination;
import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_MERGED;
import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REGION_MERGING;
import static org.apache.hadoop.hbase.executor.EventType.RS_ZK_REQUEST_REGION_MERGE;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.data.Stat;
public class ZkRegionMergeCoordination implements RegionMergeCoordination {
private CoordinatedStateManager manager;
private final ZooKeeperWatcher watcher;
private static final Log LOG = LogFactory.getLog(ZkRegionMergeCoordination.class);
public ZkRegionMergeCoordination(CoordinatedStateManager manager,
ZooKeeperWatcher watcher) {
this.manager = manager;
this.watcher = watcher;
}
/**
* ZK-based implementation. Has details about whether the state transition should be reflected in
* ZK, as well as expected version of znode.
*/
public static class ZkRegionMergeDetails implements RegionMergeCoordination.RegionMergeDetails {
private int znodeVersion;
public ZkRegionMergeDetails() {
}
public int getZnodeVersion() {
return znodeVersion;
}
public void setZnodeVersion(int znodeVersion) {
this.znodeVersion = znodeVersion;
}
}
@Override
public RegionMergeDetails getDefaultDetails() {
ZkRegionMergeDetails zstd = new ZkRegionMergeDetails();
zstd.setZnodeVersion(-1);
return zstd;
}
/**
* Wait for the merging node to be transitioned from pending_merge
* to merging by master. That's how we are sure master has processed
* the event and is good with us to move on. If we don't get any update,
* we periodically transition the node so that master gets the callback.
* If the node is removed or is not in pending_merge state any more,
* we abort the merge.
* @throws IOException
*/
@Override
public void waitForRegionMergeTransaction(RegionServerServices services,
HRegionInfo mergedRegionInfo, HRegion region_a, HRegion region_b, RegionMergeDetails details)
throws IOException {
try {
int spins = 0;
Stat stat = new Stat();
ServerName expectedServer = manager.getServer().getServerName();
String node = mergedRegionInfo.getEncodedName();
ZkRegionMergeDetails zdetails = (ZkRegionMergeDetails) details;
while (!(manager.getServer().isStopped() || services.isStopping())) {
if (spins % 5 == 0) {
LOG.debug("Still waiting for master to process " + "the pending_merge for " + node);
ZkRegionMergeDetails zrmd = (ZkRegionMergeDetails) getDefaultDetails();
transitionMergingNode(mergedRegionInfo, region_a.getRegionInfo(),
region_b.getRegionInfo(), expectedServer, zrmd, RS_ZK_REQUEST_REGION_MERGE,
RS_ZK_REQUEST_REGION_MERGE);
}
Thread.sleep(100);
spins++;
byte[] data = ZKAssign.getDataNoWatch(watcher, node, stat);
if (data == null) {
throw new IOException("Data is null, merging node " + node + " no longer exists");
}
RegionTransition rt = RegionTransition.parseFrom(data);
EventType et = rt.getEventType();
if (et == RS_ZK_REGION_MERGING) {
ServerName serverName = rt.getServerName();
if (!serverName.equals(expectedServer)) {
throw new IOException("Merging node " + node + " is for " + serverName + ", not us "
+ expectedServer);
}
byte[] payloadOfMerging = rt.getPayload();
List<HRegionInfo> mergingRegions =
HRegionInfo.parseDelimitedFrom(payloadOfMerging, 0, payloadOfMerging.length);
assert mergingRegions.size() == 3;
HRegionInfo a = mergingRegions.get(1);
HRegionInfo b = mergingRegions.get(2);
HRegionInfo hri_a = region_a.getRegionInfo();
HRegionInfo hri_b = region_b.getRegionInfo();
if (!(hri_a.equals(a) && hri_b.equals(b))) {
throw new IOException("Merging node " + node + " is for " + a + ", " + b
+ ", not expected regions: " + hri_a + ", " + hri_b);
}
// Master has processed it.
zdetails.setZnodeVersion(stat.getVersion());
return;
}
if (et != RS_ZK_REQUEST_REGION_MERGE) {
throw new IOException("Merging node " + node + " moved out of merging to " + et);
}
}
// Server is stopping/stopped
throw new IOException("Server is " + (services.isStopping() ? "stopping" : "stopped"));
} catch (Exception e) {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new IOException("Failed getting MERGING znode on "
+ mergedRegionInfo.getRegionNameAsString(), e);
}
}
/**
* Creates a new ephemeral node in the PENDING_MERGE state for the merged region.
* Create it ephemeral in case regionserver dies mid-merge.
*
* <p>
* Does not transition nodes from other states. If a node already exists for
* this region, a {@link NodeExistsException} will be thrown.
*
* @param region region to be created as offline
* @param serverName server event originates from
* @throws IOException
*/
@Override
public void startRegionMergeTransaction(final HRegionInfo region, final ServerName serverName,
final HRegionInfo a, final HRegionInfo b) throws IOException {
LOG.debug(watcher.prefix("Creating ephemeral node for " + region.getEncodedName()
+ " in PENDING_MERGE state"));
byte[] payload = HRegionInfo.toDelimitedByteArray(region, a, b);
RegionTransition rt =
RegionTransition.createRegionTransition(RS_ZK_REQUEST_REGION_MERGE, region.getRegionName(),
serverName, payload);
String node = ZKAssign.getNodeName(watcher, region.getEncodedName());
try {
if (!ZKUtil.createEphemeralNodeAndWatch(watcher, node, rt.toByteArray())) {
throw new IOException("Failed create of ephemeral " + node);
}
} catch (KeeperException e) {
throw new IOException(e);
}
}
/*
* (non-Javadoc)
* @see
* org.apache.hadoop.hbase.regionserver.coordination.RegionMergeCoordination#clean(org.apache.hadoop
* .hbase.Server, org.apache.hadoop.hbase.HRegionInfo)
*/
@Override
public void clean(final HRegionInfo hri) {
try {
// Only delete if its in expected state; could have been hijacked.
if (!ZKAssign.deleteNode(watcher, hri.getEncodedName(), RS_ZK_REQUEST_REGION_MERGE, manager
.getServer().getServerName())) {
ZKAssign.deleteNode(watcher, hri.getEncodedName(), RS_ZK_REGION_MERGING, manager
.getServer().getServerName());
}
} catch (KeeperException.NoNodeException e) {
LOG.info("Failed cleanup zk node of " + hri.getRegionNameAsString(), e);
} catch (KeeperException e) {
manager.getServer().abort("Failed cleanup zk node of " + hri.getRegionNameAsString(), e);
}
}
/*
* ZooKeeper implementation of finishRegionMergeTransaction
*/
@Override
public void completeRegionMergeTransaction(final RegionServerServices services,
HRegionInfo mergedRegionInfo, HRegion region_a, HRegion region_b, RegionMergeDetails rmd,
HRegion mergedRegion) throws IOException {
ZkRegionMergeDetails zrmd = (ZkRegionMergeDetails) rmd;
if (manager.getServer() == null
|| manager.getServer().getCoordinatedStateManager() == null) {
return;
}
// Tell master about merge by updating zk. If we fail, abort.
try {
transitionMergingNode(mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo(),
manager.getServer().getServerName(), rmd, RS_ZK_REGION_MERGING, RS_ZK_REGION_MERGED);
long startTime = EnvironmentEdgeManager.currentTimeMillis();
int spins = 0;
// Now wait for the master to process the merge. We know it's done
// when the znode is deleted. The reason we keep tickling the znode is
// that it's possible for the master to miss an event.
do {
if (spins % 10 == 0) {
LOG.debug("Still waiting on the master to process the merge for "
+ mergedRegionInfo.getEncodedName() + ", waited "
+ (EnvironmentEdgeManager.currentTimeMillis() - startTime) + "ms");
}
Thread.sleep(100);
// When this returns -1 it means the znode doesn't exist
transitionMergingNode(mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo(),
manager.getServer().getServerName(), rmd, RS_ZK_REGION_MERGED, RS_ZK_REGION_MERGED);
spins++;
} while (zrmd.getZnodeVersion() != -1 && !manager.getServer().isStopped()
&& !services.isStopping());
} catch (Exception e) {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw new IOException("Failed telling master about merge "
+ mergedRegionInfo.getEncodedName(), e);
}
// Leaving here, the mergedir with its dross will be in place but since the
// merge was successful, just leave it; it'll be cleaned when region_a is
// cleaned up by CatalogJanitor on master
}
/*
* Zookeeper implementation of region merge confirmation
*/
@Override
public void confirmRegionMergeTransaction(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
ServerName serverName, RegionMergeDetails rmd) throws IOException {
transitionMergingNode(merged, a, b, serverName, rmd, RS_ZK_REGION_MERGING,
RS_ZK_REGION_MERGING);
}
/*
* Zookeeper implementation of region merge processing
*/
@Override
public void processRegionMergeRequest(HRegionInfo p, HRegionInfo hri_a, HRegionInfo hri_b,
ServerName sn, RegionMergeDetails rmd) throws IOException {
transitionMergingNode(p, hri_a, hri_b, sn, rmd, EventType.RS_ZK_REQUEST_REGION_MERGE,
EventType.RS_ZK_REGION_MERGING);
}
/**
* Transitions an existing ephemeral node for the specified region which is
* currently in the begin state to be in the end state. Master cleans up the
* final MERGE znode when it reads it (or if we crash, zk will clean it up).
*
* <p>
* Does not transition nodes from other states. If for some reason the node
* could not be transitioned, the method returns -1. If the transition is
* successful, the version of the node after transition is updated in details.
*
* <p>
* This method can fail and return false for three different reasons:
* <ul>
* <li>Node for this region does not exist</li>
* <li>Node for this region is not in the begin state</li>
* <li>After verifying the begin state, update fails because of wrong version
* (this should never actually happen since an RS only does this transition
* following a transition to the begin state. If two RS are conflicting, one would
* fail the original transition to the begin state and not this transition)</li>
* </ul>
*
* <p>
* Does not set any watches.
*
* <p>
* This method should only be used by a RegionServer when merging two regions.
*
* @param merged region to be transitioned to opened
* @param a merging region A
* @param b merging region B
* @param serverName server event originates from
* @param rmd region merge details
* @param beginState the expected current state the node should be
* @param endState the state to be transition to
* @throws IOException
*/
private void transitionMergingNode(HRegionInfo merged, HRegionInfo a, HRegionInfo b,
ServerName serverName, RegionMergeDetails rmd, final EventType beginState,
final EventType endState) throws IOException {
ZkRegionMergeDetails zrmd = (ZkRegionMergeDetails) rmd;
byte[] payload = HRegionInfo.toDelimitedByteArray(merged, a, b);
try {
zrmd.setZnodeVersion(ZKAssign.transitionNode(watcher, merged, serverName, beginState,
endState, zrmd.getZnodeVersion(), payload));
} catch (KeeperException e) {
throw new IOException(e);
}
}
}

View File

@ -45,7 +45,7 @@ public class AssignCallable implements Callable<Object> {
@Override
public Object call() throws Exception {
assignmentManager.assign(hri, true, newPlan);
assignmentManager.assign(hri, newPlan);
return null;
}
}

View File

@ -258,12 +258,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
* {@link #finishActiveMasterInitialization(MonitoredTask)} after
* the master becomes the active one.
*
* @throws InterruptedException
* @throws KeeperException
* @throws IOException
*/
public HMaster(final Configuration conf, CoordinatedStateManager csm)
throws IOException, KeeperException, InterruptedException {
throws IOException, KeeperException {
super(conf, csm);
this.rsFatals = new MemoryBoundedLogMessageBuffer(
conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
@ -413,7 +412,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
this.assignmentManager = new AssignmentManager(this, serverManager,
this.balancer, this.service, this.metricsMaster,
this.tableLockManager);
zooKeeper.registerListenerFirst(assignmentManager);
this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
this.serverManager);
@ -674,34 +672,29 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
RegionStates regionStates = assignmentManager.getRegionStates();
regionStates.createRegionState(HRegionInfo.FIRST_META_REGIONINFO);
boolean rit = this.assignmentManager
.processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
boolean metaRegionLocation = metaTableLocator.verifyMetaRegionLocation(
this.getShortCircuitConnection(), this.getZooKeeper(), timeout);
ServerName currentMetaServer = metaTableLocator.getMetaRegionLocation(this.getZooKeeper());
if (!metaRegionLocation) {
// Meta location is not verified. It should be in transition, or offline.
// We will wait for it to be assigned in enableSSHandWaitForMeta below.
assigned++;
if (!rit) {
// Assign meta since not already in transition
if (currentMetaServer != null) {
// If the meta server is not known to be dead or online,
// just split the meta log, and don't expire it since this
// could be a full cluster restart. Otherwise, we will think
// this is a failover and lose previous region locations.
// If it is really a failover case, AM will find out in rebuilding
// user regions. Otherwise, we are good since all logs are split
// or known to be replayed before user regions are assigned.
if (serverManager.isServerOnline(currentMetaServer)) {
LOG.info("Forcing expire of " + currentMetaServer);
serverManager.expireServer(currentMetaServer);
}
splitMetaLogBeforeAssignment(currentMetaServer);
previouslyFailedMetaRSs.add(currentMetaServer);
if (currentMetaServer != null) {
// If the meta server is not known to be dead or online,
// just split the meta log, and don't expire it since this
// could be a full cluster restart. Otherwise, we will think
// this is a failover and lose previous region locations.
// If it is really a failover case, AM will find out in rebuilding
// user regions. Otherwise, we are good since all logs are split
// or known to be replayed before user regions are assigned.
if (serverManager.isServerOnline(currentMetaServer)) {
LOG.info("Forcing expire of " + currentMetaServer);
serverManager.expireServer(currentMetaServer);
}
assignmentManager.assignMeta();
splitMetaLogBeforeAssignment(currentMetaServer);
previouslyFailedMetaRSs.add(currentMetaServer);
}
assignmentManager.assignMeta();
assigned++;
} else {
// Region already assigned. We didn't assign it. Add to in-memory state.
regionStates.updateRegionState(
@ -725,8 +718,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// No need to wait for meta is assigned = 0 when meta is just verified.
enableServerShutdownHandler(assigned != 0);
LOG.info("hbase:meta assigned=" + assigned + ", rit=" + rit +
", location=" + metaTableLocator.getMetaRegionLocation(this.getZooKeeper()));
LOG.info("hbase:meta assigned=" + assigned + ", location="
+ metaTableLocator.getMetaRegionLocation(this.getZooKeeper()));
status.setStatus("META assigned.");
}
@ -1736,7 +1729,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
public void assignRegion(HRegionInfo hri) {
assignmentManager.assign(hri, true);
assignmentManager.assign(hri);
}
/**

View File

@ -359,7 +359,7 @@ public class MasterRpcServices extends RSRpcServices
}
LOG.info(master.getClientIdAuditPrefix()
+ " assign " + regionInfo.getRegionNameAsString());
master.assignmentManager.assign(regionInfo, true, true);
master.assignmentManager.assign(regionInfo, true);
if (master.cpHost != null) {
master.cpHost.postAssign(regionInfo);
}
@ -1074,6 +1074,7 @@ public class MasterRpcServices extends RSRpcServices
*
*/
@Override
@SuppressWarnings("deprecation")
public OfflineRegionResponse offlineRegion(RpcController controller,
OfflineRegionRequest request) throws ServiceException {
final byte [] regionName = request.getRegion().getValue().toByteArray();
@ -1203,6 +1204,7 @@ public class MasterRpcServices extends RSRpcServices
}
@Override
@SuppressWarnings("deprecation")
public UnassignRegionResponse unassignRegion(RpcController controller,
UnassignRegionRequest req) throws ServiceException {
try {

View File

@ -1,114 +0,0 @@
/**
* 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.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.AsyncCallback.StringCallback;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.Stat;
/**
* Callback handler for creating unassigned offline znodes
* used during bulk assign, async setting region to offline.
*/
@InterfaceAudience.Private
public class OfflineCallback implements StringCallback {
private static final Log LOG = LogFactory.getLog(OfflineCallback.class);
private final ExistCallback callBack;
private final ZooKeeperWatcher zkw;
private final ServerName destination;
private final AtomicInteger counter;
OfflineCallback(final ZooKeeperWatcher zkw,
final ServerName destination, final AtomicInteger counter,
final Map<String, Integer> offlineNodesVersions) {
this.callBack = new ExistCallback(
destination, counter, offlineNodesVersions);
this.destination = destination;
this.counter = counter;
this.zkw = zkw;
}
@Override
public void processResult(int rc, String path, Object ctx, String name) {
if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
LOG.warn("Node for " + path + " already exists");
} else if (rc != 0) {
// This is result code. If non-zero, need to resubmit.
LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
"FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
this.counter.addAndGet(1);
return;
}
if (LOG.isDebugEnabled()) {
LOG.debug("rs=" + ctx + ", server=" + destination);
}
// Async exists to set a watcher so we'll get triggered when
// unassigned node changes.
ZooKeeper zk = this.zkw.getRecoverableZooKeeper().getZooKeeper();
zk.exists(path, this.zkw, callBack, ctx);
}
/**
* Callback handler for the exists call that sets watcher on unassigned znodes.
* Used during bulk assign on startup.
*/
static class ExistCallback implements StatCallback {
private static final Log LOG = LogFactory.getLog(ExistCallback.class);
private final Map<String, Integer> offlineNodesVersions;
private final AtomicInteger counter;
private ServerName destination;
ExistCallback(final ServerName destination,
final AtomicInteger counter,
final Map<String, Integer> offlineNodesVersions) {
this.offlineNodesVersions = offlineNodesVersions;
this.destination = destination;
this.counter = counter;
}
@Override
public void processResult(int rc, String path, Object ctx, Stat stat) {
if (rc != 0) {
// This is result code. If non-zero, need to resubmit.
LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
"FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
this.counter.addAndGet(1);
return;
}
if (LOG.isDebugEnabled()) {
LOG.debug("rs=" + ctx + ", server=" + destination);
}
HRegionInfo region = ((RegionState)ctx).getRegion();
offlineNodesVersions.put(
region.getEncodedName(), Integer.valueOf(stat.getVersion()));
this.counter.addAndGet(1);
}
}
}

View File

@ -22,16 +22,15 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
@ -40,7 +39,6 @@ import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConfigUtil;
import com.google.common.base.Preconditions;
@ -59,7 +57,6 @@ public class RegionStateStore {
private volatile HTableInterface metaTable;
private volatile boolean initialized;
private final boolean noPersistence;
private final Server server;
/**
@ -131,25 +128,19 @@ public class RegionStateStore {
}
RegionStateStore(final Server server) {
Configuration conf = server.getConfiguration();
// No need to persist if using ZK but not migrating
noPersistence = ConfigUtil.useZKForAssignment(conf)
&& !conf.getBoolean("hbase.assignment.usezk.migrating", false);
this.server = server;
initialized = false;
}
@SuppressWarnings("deprecation")
void start() throws IOException {
if (!noPersistence) {
if (server instanceof RegionServerServices) {
metaRegion = ((RegionServerServices)server).getFromOnlineRegions(
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
}
if (metaRegion == null) {
metaTable = new HTable(TableName.META_TABLE_NAME,
server.getShortCircuitConnection());
}
if (server instanceof RegionServerServices) {
metaRegion = ((RegionServerServices)server).getFromOnlineRegions(
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
}
if (metaRegion == null) {
metaTable = new HTable(TableName.META_TABLE_NAME,
server.getShortCircuitConnection());
}
initialized = true;
}
@ -170,7 +161,7 @@ public class RegionStateStore {
@SuppressWarnings("deprecation")
void updateRegionState(long openSeqNum,
RegionState newState, RegionState oldState) {
if (noPersistence || !initialized) {
if (!initialized) {
return;
}

View File

@ -34,21 +34,17 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableStateManager;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@ -349,28 +345,6 @@ public class RegionStates {
regionState == null ? null : regionState.getServerName());
}
/**
* Update a region state. It will be put in transition if not already there.
*
* If we can't find the region info based on the region name in
* the transition, log a warning and return null.
*/
public RegionState updateRegionState(
final RegionTransition transition, final State state) {
byte [] regionName = transition.getRegionName();
HRegionInfo regionInfo = getRegionInfo(regionName);
if (regionInfo == null) {
String prettyRegionName = HRegionInfo.prettyPrint(
HRegionInfo.encodeRegionName(regionName));
LOG.warn("Failed to find region " + prettyRegionName
+ " in updating its state to " + state
+ " based on region transition " + transition);
return null;
}
return updateRegionState(regionInfo, state,
transition.getServerName());
}
/**
* Update a region state. It will be put in transition if not already there.
*/
@ -548,8 +522,7 @@ public class RegionStates {
/**
* A server is offline, all regions on it are dead.
*/
public synchronized List<HRegionInfo> serverOffline(
final ZooKeeperWatcher watcher, final ServerName sn) {
public synchronized List<HRegionInfo> serverOffline(final ServerName sn) {
// Offline all regions on this server not already in transition.
List<HRegionInfo> rits = new ArrayList<HRegionInfo>();
Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
@ -565,13 +538,7 @@ public class RegionStates {
regionsToOffline.add(region);
} else if (isRegionInState(region, State.SPLITTING, State.MERGING)) {
LOG.debug("Offline splitting/merging region " + getRegionState(region));
try {
// Delete the ZNode if exists
ZKAssign.deleteNodeFailSilent(watcher, region);
regionsToOffline.add(region);
} catch (KeeperException ke) {
server.abort("Unexpected ZK exception deleting node " + region, ke);
}
regionsToOffline.add(region);
}
}

View File

@ -64,7 +64,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.R
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Triple;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@ -159,7 +159,7 @@ public class ServerManager {
* handler is not enabled, is queued up.
* <p>
* So this is a set of region servers known to be dead but not submitted to
* ServerShutdownHander for processing yet.
* ServerShutdownHandler for processing yet.
*/
private Set<ServerName> queuedDeadServers = new HashSet<ServerName>();
@ -310,7 +310,7 @@ public class ServerManager {
* Check is a server of same host and port already exists,
* if not, or the existed one got a smaller start code, record it.
*
* @param sn the server to check and record
* @param serverName the server to check and record
* @param sl the server load on the server
* @return true if the server is recorded, otherwise, false
*/
@ -717,12 +717,10 @@ public class ServerManager {
* <p>
* @param server server to open a region
* @param region region to open
* @param versionOfOfflineNode that needs to be present in the offline node
* when RS tries to change the state from OFFLINE to other states.
* @param favoredNodes
*/
public RegionOpeningState sendRegionOpen(final ServerName server,
HRegionInfo region, int versionOfOfflineNode, List<ServerName> favoredNodes)
HRegionInfo region, List<ServerName> favoredNodes)
throws IOException {
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
@ -730,8 +728,8 @@ public class ServerManager {
" failed because no RPC connection found to this server");
return RegionOpeningState.FAILED_OPENING;
}
OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server,
region, versionOfOfflineNode, favoredNodes,
OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server,
region, favoredNodes,
(RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
try {
OpenRegionResponse response = admin.openRegion(null, request);
@ -751,7 +749,7 @@ public class ServerManager {
* @return a list of region opening states
*/
public List<RegionOpeningState> sendRegionOpen(ServerName server,
List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos)
List<Pair<HRegionInfo, List<ServerName>>> regionOpenInfos)
throws IOException {
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
@ -760,7 +758,7 @@ public class ServerManager {
return null;
}
OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(regionOpenInfos,
OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(regionOpenInfos,
(RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
try {
OpenRegionResponse response = admin.openRegion(null, request);
@ -777,15 +775,11 @@ public class ServerManager {
* have the specified region or the region is being split.
* @param server server to open a region
* @param region region to open
* @param versionOfClosingNode
* the version of znode to compare when RS transitions the znode from
* CLOSING state.
* @param dest - if the region is moved to another server, the destination server. null otherwise.
* @return true if server acknowledged close, false if not
* @throws IOException
*/
public boolean sendRegionClose(ServerName server, HRegionInfo region,
int versionOfClosingNode, ServerName dest, boolean transitionInZK) throws IOException {
ServerName dest) throws IOException {
if (server == null) throw new NullPointerException("Passed server is null");
AdminService.BlockingInterface admin = getRsAdmin(server);
if (admin == null) {
@ -795,12 +789,12 @@ public class ServerManager {
" failed because no RPC connection found to this server");
}
return ProtobufUtil.closeRegion(admin, server, region.getRegionName(),
versionOfClosingNode, dest, transitionInZK);
dest);
}
public boolean sendRegionClose(ServerName server,
HRegionInfo region, int versionOfClosingNode) throws IOException {
return sendRegionClose(server, region, versionOfClosingNode, null, true);
HRegionInfo region) throws IOException {
return sendRegionClose(server, region, null);
}
/**

View File

@ -1,108 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
/**
* Handles CLOSED region event on Master.
* <p>
* If table is being disabled, deletes ZK unassigned node and removes from
* regions in transition.
* <p>
* Otherwise, assigns the region to another server.
*/
@InterfaceAudience.Private
public class ClosedRegionHandler extends EventHandler implements TotesHRegionInfo {
private static final Log LOG = LogFactory.getLog(ClosedRegionHandler.class);
private final AssignmentManager assignmentManager;
private final HRegionInfo regionInfo;
private final ClosedPriority priority;
private enum ClosedPriority {
META (1),
USER (2);
private final int value;
ClosedPriority(int value) {
this.value = value;
}
public int getValue() {
return value;
}
};
public ClosedRegionHandler(Server server, AssignmentManager assignmentManager,
HRegionInfo regionInfo) {
super(server, EventType.RS_ZK_REGION_CLOSED);
this.assignmentManager = assignmentManager;
this.regionInfo = regionInfo;
if(regionInfo.isMetaRegion()) {
priority = ClosedPriority.META;
} else {
priority = ClosedPriority.USER;
}
}
@Override
public int getPriority() {
return priority.getValue();
}
@Override
public HRegionInfo getHRegionInfo() {
return this.regionInfo;
}
@Override
public String toString() {
String name = "UnknownServerName";
if(server != null && server.getServerName() != null) {
name = server.getServerName().toString();
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
}
@Override
public void process() {
LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName());
// Check if this table is being disabled or not
if (this.assignmentManager.getTableStateManager().isTableState(this.regionInfo.getTable(),
ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
assignmentManager.getReplicasToClose().contains(regionInfo)) {
assignmentManager.offlineDisabledRegion(regionInfo);
return;
}
// ZK Node is in CLOSED state, assign it.
assignmentManager.getRegionStates().updateRegionState(
regionInfo, RegionState.State.CLOSED);
// This below has to do w/ online enable/disable of a table
assignmentManager.removeClosedRegion(regionInfo);
assignmentManager.assign(regionInfo, true);
}
}

View File

@ -1,103 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
/**
* Handles OPENED region event on Master.
*/
@InterfaceAudience.Private
public class OpenedRegionHandler extends EventHandler implements TotesHRegionInfo {
private static final Log LOG = LogFactory.getLog(OpenedRegionHandler.class);
private final AssignmentManager assignmentManager;
private final HRegionInfo regionInfo;
private final OpenedPriority priority;
private OpenRegionCoordination coordination;
private OpenRegionCoordination.OpenRegionDetails ord;
private enum OpenedPriority {
META (1),
SYSTEM (2),
USER (3);
private final int value;
OpenedPriority(int value) {
this.value = value;
}
public int getValue() {
return value;
}
};
public OpenedRegionHandler(Server server,
AssignmentManager assignmentManager, HRegionInfo regionInfo,
OpenRegionCoordination coordination,
OpenRegionCoordination.OpenRegionDetails ord) {
super(server, EventType.RS_ZK_REGION_OPENED);
this.assignmentManager = assignmentManager;
this.regionInfo = regionInfo;
this.coordination = coordination;
this.ord = ord;
if(regionInfo.isMetaRegion()) {
priority = OpenedPriority.META;
} else if(regionInfo.getTable()
.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
priority = OpenedPriority.SYSTEM;
} else {
priority = OpenedPriority.USER;
}
}
@Override
public int getPriority() {
return priority.getValue();
}
@Override
public HRegionInfo getHRegionInfo() {
return this.regionInfo;
}
@Override
public String toString() {
String name = "UnknownServerName";
if(server != null && server.getServerName() != null) {
name = server.getServerName().toString();
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
}
@Override
public void process() {
if (!coordination.commitOpenOnMasterSide(assignmentManager,regionInfo, ord)) {
assignmentManager.unassign(regionInfo);
}
}
}

View File

@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
@ -45,9 +44,6 @@ import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.zookeeper.KeeperException;
/**
* Process server shutdown.
@ -162,24 +158,15 @@ public class ServerShutdownHandler extends EventHandler {
server.getMetaTableLocator().waitMetaRegionLocation(server.getZooKeeper());
// Skip getting user regions if the server is stopped.
if (!this.server.isStopped()) {
if (ConfigUtil.useZKForAssignment(server.getConfiguration())) {
hris = MetaTableAccessor.getServerUserRegions(this.server.getShortCircuitConnection(),
this.serverName).keySet();
} else {
// Not using ZK for assignment, regionStates has everything we want
hris = am.getRegionStates().getServerRegions(serverName);
if (hris != null) {
hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
}
hris = am.getRegionStates().getServerRegions(serverName);
if (hris != null) {
hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
}
}
break;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
} catch (IOException ioe) {
LOG.info("Received exception accessing hbase:meta during server shutdown of " +
serverName + ", retrying hbase:meta read", ioe);
}
}
if (this.server.isStopped()) {
@ -249,15 +236,8 @@ public class ServerShutdownHandler extends EventHandler {
LOG.info("Skip assigning region in transition on other server" + rit);
continue;
}
try{
//clean zk node
LOG.info("Reassigning region with rs = " + rit + " and deleting zk node if exists");
ZKAssign.deleteNodeFailSilent(services.getZooKeeper(), hri);
regionStates.updateRegionState(hri, State.OFFLINE);
} catch (KeeperException ke) {
this.server.abort("Unexpected ZK exception deleting unassigned node " + hri, ke);
return;
}
LOG.info("Reassigning region with rs = " + rit);
regionStates.updateRegionState(hri, State.OFFLINE);
} else if (regionStates.isRegionInState(
hri, State.SPLITTING_NEW, State.MERGING_NEW)) {
regionStates.updateRegionState(hri, State.OFFLINE);
@ -274,7 +254,6 @@ public class ServerShutdownHandler extends EventHandler {
// but though we did assign we will not be clearing the znode in CLOSING state.
// Doing this will have no harm. See HBASE-5927
regionStates.updateRegionState(hri, State.OFFLINE);
am.deleteClosingOrClosedNode(hri, rit.getServerName());
am.offlineDisabledRegion(hri);
} else {
LOG.warn("THIS SHOULD NOT HAPPEN: unexpected region in transition "

View File

@ -1,259 +0,0 @@
/**
* 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.migration;
import java.io.IOException;
import java.util.List;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileV1Detector;
import org.apache.hadoop.hbase.util.ZKDataMigrator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
public class UpgradeTo96 extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(UpgradeTo96.class);
private Options options = new Options();
/**
* whether to do overall upgrade (namespace and znodes)
*/
private boolean upgrade;
/**
* whether to check for HFileV1
*/
private boolean checkForHFileV1;
/**
* Path of directory to check for HFileV1
*/
private String dirToCheckForHFileV1;
UpgradeTo96() {
setOptions();
}
private void setOptions() {
options.addOption("h", "help", false, "Help");
options.addOption(new Option("check", false, "Run upgrade check; looks for HFileV1 "
+ " under ${hbase.rootdir} or provided 'dir' directory."));
options.addOption(new Option("execute", false, "Run upgrade; zk and hdfs must be up, hbase down"));
Option pathOption = new Option("dir", true,
"Relative path of dir to check for HFileV1s.");
pathOption.setRequired(false);
options.addOption(pathOption);
}
private boolean parseOption(String[] args) throws ParseException {
if (args.length == 0) return false; // no args shows help.
CommandLineParser parser = new GnuParser();
CommandLine cmd = parser.parse(options, args);
if (cmd.hasOption("h")) {
return false;
}
if (cmd.hasOption("execute")) upgrade = true;
if (cmd.hasOption("check")) checkForHFileV1 = true;
if (checkForHFileV1 && cmd.hasOption("dir")) {
this.dirToCheckForHFileV1 = cmd.getOptionValue("dir");
}
return true;
}
private void printUsage() {
HelpFormatter formatter = new HelpFormatter();
formatter.printHelp("$bin/hbase upgrade -check [-dir DIR]|-execute", options);
System.out.println("Read http://hbase.apache.org/book.html#upgrade0.96 before attempting upgrade");
System.out.println();
System.out.println("Example usage:");
System.out.println();
System.out.println("Run upgrade check; looks for HFileV1s under ${hbase.rootdir}:");
System.out.println(" $ bin/hbase upgrade -check");
System.out.println();
System.out.println("Run the upgrade: ");
System.out.println(" $ bin/hbase upgrade -execute");
}
@Override
public int run(String[] args) throws Exception {
if (!parseOption(args)) {
printUsage();
return -1;
}
if (checkForHFileV1) {
int res = doHFileV1Check();
if (res == 0) LOG.info("No HFileV1 found.");
else {
LOG.warn("There are some HFileV1, or corrupt files (files with incorrect major version).");
}
return res;
}
// if the user wants to upgrade, check for any HBase live process.
// If yes, prompt the user to stop them
else if (upgrade) {
if (isAnyHBaseProcessAlive()) {
LOG.error("Some HBase processes are still alive, or znodes not expired yet. "
+ "Please stop them before upgrade or try after some time.");
throw new IOException("Some HBase processes are still alive, or znodes not expired yet");
}
return executeUpgrade();
}
return -1;
}
private boolean isAnyHBaseProcessAlive() throws IOException {
ZooKeeperWatcher zkw = null;
try {
zkw = new ZooKeeperWatcher(getConf(), "Check Live Processes.", new Abortable() {
private boolean aborted = false;
@Override
public void abort(String why, Throwable e) {
LOG.warn("Got aborted with reason: " + why + ", and error: " + e);
this.aborted = true;
}
@Override
public boolean isAborted() {
return this.aborted;
}
});
boolean liveProcessesExists = false;
if (ZKUtil.checkExists(zkw, zkw.baseZNode) == -1) {
return false;
}
if (ZKUtil.checkExists(zkw, zkw.backupMasterAddressesZNode) != -1) {
List<String> backupMasters = ZKUtil
.listChildrenNoWatch(zkw, zkw.backupMasterAddressesZNode);
if (!backupMasters.isEmpty()) {
LOG.warn("Backup master(s) " + backupMasters
+ " are alive or backup-master znodes not expired.");
liveProcessesExists = true;
}
}
if (ZKUtil.checkExists(zkw, zkw.rsZNode) != -1) {
List<String> regionServers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode);
if (!regionServers.isEmpty()) {
LOG.warn("Region server(s) " + regionServers + " are alive or rs znodes not expired.");
liveProcessesExists = true;
}
}
if (ZKUtil.checkExists(zkw, zkw.getMasterAddressZNode()) != -1) {
byte[] data = ZKUtil.getData(zkw, zkw.getMasterAddressZNode());
if (data != null && !Bytes.equals(data, HConstants.EMPTY_BYTE_ARRAY)) {
LOG.warn("Active master at address " + Bytes.toString(data)
+ " is still alive or master znode not expired.");
liveProcessesExists = true;
}
}
return liveProcessesExists;
} catch (Exception e) {
LOG.error("Got exception while checking live hbase processes", e);
throw new IOException(e);
} finally {
if (zkw != null) {
zkw.close();
}
}
}
private int doHFileV1Check() throws Exception {
String[] args = null;
if (dirToCheckForHFileV1 != null) args = new String[] { "-p" + dirToCheckForHFileV1 };
return ToolRunner.run(getConf(), new HFileV1Detector(), args);
}
/**
* Executes the upgrade process. It involves:
* <ul>
* <li> Upgrading Namespace
* <li> Upgrading Znodes
* <li> Log splitting
* </ul>
* @throws Exception
*/
private int executeUpgrade() throws Exception {
executeTool("Namespace upgrade", new NamespaceUpgrade(),
new String[] { "--upgrade" }, 0);
executeTool("Znode upgrade", new ZKDataMigrator(), null, 0);
doOfflineLogSplitting();
return 0;
}
private void executeTool(String toolMessage, Tool tool, String[] args, int expectedResult)
throws Exception {
LOG.info("Starting " + toolMessage);
int res = ToolRunner.run(getConf(), tool, new String[] { "--upgrade" });
if (res != expectedResult) {
LOG.error(toolMessage + "returned " + res + ", expected " + expectedResult);
throw new Exception("Unexpected return code from " + toolMessage);
}
LOG.info("Successfully completed " + toolMessage);
}
/**
* Performs log splitting for all regionserver directories.
* @throws Exception
*/
private void doOfflineLogSplitting() throws Exception {
LOG.info("Starting Log splitting");
final Path rootDir = FSUtils.getRootDir(getConf());
final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
Path logDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
FileStatus[] regionServerLogDirs = FSUtils.listStatus(fs, logDir);
if (regionServerLogDirs == null || regionServerLogDirs.length == 0) {
LOG.info("No log directories to split, returning");
return;
}
try {
for (FileStatus regionServerLogDir : regionServerLogDirs) {
// split its log dir, if exists
HLogSplitter.split(rootDir, regionServerLogDir.getPath(), oldLogDir, fs, getConf());
}
LOG.info("Successfully completed Log splitting");
} catch (Exception e) {
LOG.error("Got exception while doing Log splitting ", e);
throw e;
}
}
public static void main(String[] args) throws Exception {
System.exit(ToolRunner.run(HBaseConfiguration.create(), new UpgradeTo96(), args));
}
}

View File

@ -76,7 +76,6 @@ import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
@ -123,7 +122,6 @@ import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
@ -149,6 +147,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.ServiceException;
@ -163,6 +162,12 @@ public class HRegionServer extends HasThread implements
public static final Log LOG = LogFactory.getLog(HRegionServer.class);
/**
* For testing only! Set to true to skip notifying region assignment to master .
*/
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
/*
* Strings to be used in forming the exception message for
* RegionsAlreadyInTransitionException.
@ -410,8 +415,6 @@ public class HRegionServer extends HasThread implements
protected BaseCoordinatedStateManager csm;
private final boolean useZKForAssignment;
/**
* Starts a HRegionServer at the default location.
* @param conf
@ -427,10 +430,9 @@ public class HRegionServer extends HasThread implements
* @param conf
* @param csm implementation of CoordinatedStateManager to be used
* @throws IOException
* @throws InterruptedException
*/
public HRegionServer(Configuration conf, CoordinatedStateManager csm)
throws IOException, InterruptedException {
throws IOException {
this.fsOk = true;
this.conf = conf;
checkCodecs(this.conf);
@ -479,8 +481,6 @@ public class HRegionServer extends HasThread implements
}
};
useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
// Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
// underlying hadoop hdfs accessors will be going against wrong filesystem
// (unless all is set to defaults).
@ -1719,14 +1719,12 @@ public class HRegionServer extends HasThread implements
// Update flushed sequence id of a recovering region in ZK
updateRecoveringRegionLastFlushedSequenceId(r);
// Update ZK, or META
if (r.getRegionInfo().isMetaRegion()) {
MetaTableLocator.setMetaLocation(getZooKeeper(), serverName);
} else if (useZKForAssignment) {
MetaTableAccessor.updateRegionLocation(shortCircuitConnection, r.getRegionInfo(),
this.serverName, openSeqNum);
}
if (!useZKForAssignment && !reportRegionStateTransition(
// Notify master
if (!reportRegionStateTransition(
TransitionCode.OPENED, openSeqNum, r.getRegionInfo())) {
throw new IOException("Failed to report opened region to master: "
+ r.getRegionNameAsString());
@ -1743,6 +1741,22 @@ public class HRegionServer extends HasThread implements
@Override
public boolean reportRegionStateTransition(
TransitionCode code, long openSeqNum, HRegionInfo... hris) {
if (TEST_SKIP_REPORTING_TRANSITION) {
// This is for testing only in case there is no master
// to handle the region transition report at all.
if (code == TransitionCode.OPENED) {
Preconditions.checkArgument(hris != null && hris.length == 1);
try {
MetaTableAccessor.updateRegionLocation(shortCircuitConnection,
hris[0], serverName, openSeqNum);
return true;
} catch (IOException e) {
LOG.info("Failed to update meta", e);
return false;
}
}
}
ReportRegionStateTransitionRequest.Builder builder =
ReportRegionStateTransitionRequest.newBuilder();
builder.setServer(ProtobufUtil.toServerName(serverName));
@ -2428,9 +2442,7 @@ public class HRegionServer extends HasThread implements
*/
private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
try {
CloseRegionCoordination.CloseRegionDetails details =
csm.getCloseRegionCoordination().getDetaultDetails();
if (!closeRegion(region.getEncodedName(), abort, details, null)) {
if (!closeRegion(region.getEncodedName(), abort, null)) {
LOG.warn("Failed to close " + region.getRegionNameAsString() +
" - ignoring and continuing");
}
@ -2455,13 +2467,11 @@ public class HRegionServer extends HasThread implements
*
* @param encodedName Region to close
* @param abort True if we are aborting
* @param crd details about closing region coordination-coordinated task
* @return True if closed a region.
* @throws NotServingRegionException if the region is not online
* @throws RegionAlreadyInTransitionException if the region is already closing
*/
protected boolean closeRegion(String encodedName, final boolean abort,
CloseRegionCoordination.CloseRegionDetails crd, final ServerName sn)
protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
throws NotServingRegionException, RegionAlreadyInTransitionException {
//Check for permissions to close.
HRegion actualRegion = this.getFromOnlineRegions(encodedName);
@ -2485,7 +2495,7 @@ public class HRegionServer extends HasThread implements
// We're going to try to do a standard close then.
LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
" Doing a standard close now");
return closeRegion(encodedName, abort, crd, sn);
return closeRegion(encodedName, abort, sn);
}
// Let's get the region from the online region list again
actualRegion = this.getFromOnlineRegions(encodedName);
@ -2519,11 +2529,9 @@ public class HRegionServer extends HasThread implements
CloseRegionHandler crh;
final HRegionInfo hri = actualRegion.getRegionInfo();
if (hri.isMetaRegion()) {
crh = new CloseMetaHandler(this, this, hri, abort,
csm.getCloseRegionCoordination(), crd);
crh = new CloseMetaHandler(this, this, hri, abort);
} else {
crh = new CloseRegionHandler(this, this, hri, abort,
csm.getCloseRegionCoordination(), crd, sn);
crh = new CloseRegionHandler(this, this, hri, abort, sn);
}
this.service.submit(crh);
return true;

View File

@ -50,11 +50,11 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete;
@ -66,7 +66,6 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.RowMutations;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
import org.apache.hadoop.hbase.exceptions.OperationConflictException;
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
@ -146,7 +145,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.regionserver.HRegion.Operation;
import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
@ -927,10 +925,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
requestCount.increment();
LOG.info("Close " + encodedRegionName + ", moving to " + sn);
CloseRegionCoordination.CloseRegionDetails crd = regionServer.getCoordinatedStateManager()
.getCloseRegionCoordination().parseFromProtoRequest(request);
boolean closed = regionServer.closeRegion(encodedRegionName, false, crd, sn);
boolean closed = regionServer.closeRegion(encodedRegionName, false, sn);
CloseRegionResponse.Builder builder = CloseRegionResponse.newBuilder().setClosed(closed);
return builder.build();
} catch (IOException ie) {
@ -1236,11 +1231,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
}
for (RegionOpenInfo regionOpenInfo : request.getOpenInfoList()) {
final HRegionInfo region = HRegionInfo.convert(regionOpenInfo.getRegion());
OpenRegionCoordination coordination = regionServer.getCoordinatedStateManager().
getOpenRegionCoordination();
OpenRegionCoordination.OpenRegionDetails ord =
coordination.parseFromProtoRequest(regionOpenInfo);
HTableDescriptor htd;
try {
final HRegion onlineRegion = regionServer.getFromOnlineRegions(region.getEncodedName());
@ -1284,10 +1274,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
region.getEncodedNameAsBytes(), Boolean.TRUE);
if (Boolean.FALSE.equals(previous)) {
// There is a close in progress. We need to mark this open as failed in ZK.
coordination.tryTransitionFromOfflineToFailedOpen(regionServer, region, ord);
// There is a close in progress. This should not happen any more.
throw new RegionAlreadyInTransitionException("Received OPEN for the region:"
+ region.getRegionNameAsString() + " , which we are already trying to CLOSE ");
}
@ -1324,12 +1311,12 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
// Need to pass the expected version in the constructor.
if (region.isMetaRegion()) {
regionServer.service.submit(new OpenMetaHandler(
regionServer, regionServer, region, htd, coordination, ord));
regionServer, regionServer, region, htd));
} else {
regionServer.updateRegionFavoredNodesMapping(region.getEncodedName(),
regionOpenInfo.getFavoredNodesList());
regionServer.service.submit(new OpenRegionHandler(
regionServer, regionServer, region, htd, coordination, ord));
regionServer, regionServer, region, htd));
}
}

View File

@ -31,22 +31,17 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaMutationAnnotation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.RegionMergeCoordination.RegionMergeDetails;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.SplitTransaction.LoggingProgressable;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.zookeeper.KeeperException;
/**
* Executes region merge as a "transaction". It is similar with
@ -89,7 +84,6 @@ public class RegionMergeTransaction {
private final Path mergesdir;
// We only merge adjacent regions if forcible is false
private final boolean forcible;
private boolean useCoordinationForAssignment;
/**
* Types to add to the transaction journal. Each enum is a step in the merge
@ -141,8 +135,6 @@ public class RegionMergeTransaction {
private RegionServerCoprocessorHost rsCoprocessorHost = null;
private RegionMergeDetails rmd;
/**
* Constructor
* @param a region a to merge
@ -231,14 +223,6 @@ public class RegionMergeTransaction {
*/
public HRegion execute(final Server server,
final RegionServerServices services) throws IOException {
useCoordinationForAssignment =
server == null ? true : ConfigUtil.useZKForAssignment(server.getConfiguration());
if (rmd == null) {
rmd =
server != null && server.getCoordinatedStateManager() != null ? ((BaseCoordinatedStateManager) server
.getCoordinatedStateManager()).getRegionMergeCoordination().getDefaultDetails()
: null;
}
if (rsCoprocessorHost == null) {
rsCoprocessorHost = server != null ?
((HRegionServer) server).getRegionServerCoprocessorHost() : null;
@ -253,11 +237,6 @@ public class RegionMergeTransaction {
public HRegion stepsAfterPONR(final Server server, final RegionServerServices services,
HRegion mergedRegion) throws IOException {
openMergedRegion(server, services, mergedRegion);
if (useCoordination(server)) {
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getRegionMergeCoordination().completeRegionMergeTransaction(services, mergedRegionInfo,
region_a, region_b, rmd, mergedRegion);
}
if (rsCoprocessorHost != null) {
rsCoprocessorHost.postMerge(this.region_a, this.region_b, mergedRegion);
}
@ -322,35 +301,16 @@ public class RegionMergeTransaction {
// will determine whether the region is merged or not in case of failures.
// If it is successful, master will roll-forward, if not, master will
// rollback
if (!testing && useCoordinationForAssignment) {
if (metaEntries.isEmpty()) {
MetaTableAccessor.mergeRegions(server.getShortCircuitConnection(),
mergedRegion.getRegionInfo(), region_a.getRegionInfo(), region_b.getRegionInfo(),
server.getServerName());
} else {
mergeRegionsAndPutMetaEntries(server.getShortCircuitConnection(),
mergedRegion.getRegionInfo(), region_a.getRegionInfo(), region_b.getRegionInfo(),
server.getServerName(), metaEntries);
}
} else if (services != null && !useCoordinationForAssignment) {
if (!services.reportRegionStateTransition(TransitionCode.MERGE_PONR,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
// Passed PONR, let SSH clean it up
throw new IOException("Failed to notify master that merge passed PONR: "
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString());
}
if (services != null && !services.reportRegionStateTransition(TransitionCode.MERGE_PONR,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
// Passed PONR, let SSH clean it up
throw new IOException("Failed to notify master that merge passed PONR: "
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString());
}
return mergedRegion;
}
private void mergeRegionsAndPutMetaEntries(HConnection hConnection,
HRegionInfo mergedRegion, HRegionInfo regionA, HRegionInfo regionB,
ServerName serverName, List<Mutation> metaEntries) throws IOException {
prepareMutationsForMerge(mergedRegion, regionA, regionB, serverName, metaEntries);
MetaTableAccessor.mutateMetaTable(hConnection, metaEntries);
}
public void prepareMutationsForMerge(HRegionInfo mergedRegion, HRegionInfo regionA,
HRegionInfo regionB, ServerName serverName, List<Mutation> mutations) throws IOException {
HRegionInfo copyOfMerged = new HRegionInfo(mergedRegion);
@ -380,40 +340,13 @@ public class RegionMergeTransaction {
public HRegion stepsBeforePONR(final Server server, final RegionServerServices services,
boolean testing) throws IOException {
if (rmd == null) {
rmd =
server != null && server.getCoordinatedStateManager() != null ? ((BaseCoordinatedStateManager) server
.getCoordinatedStateManager()).getRegionMergeCoordination().getDefaultDetails()
: null;
}
// If server doesn't have a coordination state manager, don't do coordination actions.
if (useCoordination(server)) {
try {
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getRegionMergeCoordination().startRegionMergeTransaction(mergedRegionInfo,
server.getServerName(), region_a.getRegionInfo(), region_b.getRegionInfo());
} catch (IOException e) {
throw new IOException("Failed to start region merge transaction for "
+ this.mergedRegionInfo.getRegionNameAsString(), e);
}
} else if (services != null && !useCoordinationForAssignment) {
if (!services.reportRegionStateTransition(TransitionCode.READY_TO_MERGE,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
throw new IOException("Failed to get ok from master to merge "
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString());
}
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_MERGE,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
throw new IOException("Failed to get ok from master to merge "
+ region_a.getRegionInfo().getRegionNameAsString() + " and "
+ region_b.getRegionInfo().getRegionNameAsString());
}
this.journal.add(JournalEntry.SET_MERGING);
if (useCoordination(server)) {
// After creating the merge node, wait for master to transition it
// from PENDING_MERGE to MERGING so that we can move on. We want master
// knows about it and won't transition any region which is merging.
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getRegionMergeCoordination().waitForRegionMergeTransaction(services, mergedRegionInfo,
region_a, region_b, rmd);
}
this.region_a.getRegionFileSystem().createMergesDir();
this.journal.add(JournalEntry.CREATED_MERGE_DIR);
@ -432,19 +365,6 @@ public class RegionMergeTransaction {
// clean this up.
mergeStoreFiles(hstoreFilesOfRegionA, hstoreFilesOfRegionB);
if (useCoordination(server)) {
try {
// Do the final check in case any merging region is moved somehow. If so, the transition
// will fail.
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getRegionMergeCoordination().confirmRegionMergeTransaction(this.mergedRegionInfo,
region_a.getRegionInfo(), region_b.getRegionInfo(), server.getServerName(), rmd);
} catch (IOException e) {
throw new IOException("Failed setting MERGING on "
+ this.mergedRegionInfo.getRegionNameAsString(), e);
}
}
// Log to the journal that we are creating merged region. We could fail
// halfway through. If we do, we could have left
// stuff in fs that needs cleanup -- a storefile or two. Thats why we
@ -578,20 +498,13 @@ public class RegionMergeTransaction {
merged.openHRegion(reporter);
if (services != null) {
try {
if (useCoordinationForAssignment) {
services.postOpenDeployTasks(merged);
} else if (!services.reportRegionStateTransition(TransitionCode.MERGED,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
throw new IOException("Failed to report merged region to master: "
+ mergedRegionInfo.getShortNameToLog());
}
services.addToOnlineRegions(merged);
} catch (KeeperException ke) {
throw new IOException(ke);
if (!services.reportRegionStateTransition(TransitionCode.MERGED,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
throw new IOException("Failed to report merged region to master: "
+ mergedRegionInfo.getShortNameToLog());
}
services.addToOnlineRegions(merged);
}
}
/**
@ -652,10 +565,7 @@ public class RegionMergeTransaction {
switch (je) {
case SET_MERGING:
if (useCoordination(server)) {
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getRegionMergeCoordination().clean(this.mergedRegionInfo);
} else if (services != null && !useCoordinationForAssignment
if (services != null
&& !services.reportRegionStateTransition(TransitionCode.MERGE_REVERTED,
mergedRegionInfo, region_a.getRegionInfo(), region_b.getRegionInfo())) {
return false;
@ -734,13 +644,6 @@ public class RegionMergeTransaction {
return this.mergesdir;
}
private boolean useCoordination(final Server server) {
return server != null && useCoordinationForAssignment
&& server.getCoordinatedStateManager() != null;
}
/**
* Checks if the given region has merge qualifier in hbase:meta
* @param services

View File

@ -39,16 +39,11 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.PairOfSameType;
@ -90,8 +85,6 @@ public class SplitTransaction {
private HRegionInfo hri_a;
private HRegionInfo hri_b;
private long fileSplitTimeout = 30000;
public SplitTransactionCoordination.SplitTransactionDetails std;
boolean useZKForAssignment;
/*
* Row to split around
@ -275,52 +268,23 @@ public class SplitTransaction {
// will determine whether the region is split or not in case of failures.
// If it is successful, master will roll-forward, if not, master will rollback
// and assign the parent region.
if (!testing && useZKForAssignment) {
if (metaEntries == null || metaEntries.isEmpty()) {
MetaTableAccessor.splitRegion(server.getShortCircuitConnection(),
parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(),
daughterRegions.getSecond().getRegionInfo(), server.getServerName());
} else {
offlineParentInMetaAndputMetaEntries(server.getShortCircuitConnection(),
parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(), daughterRegions
.getSecond().getRegionInfo(), server.getServerName(), metaEntries);
}
} else if (services != null && !useZKForAssignment) {
if (!services.reportRegionStateTransition(TransitionCode.SPLIT_PONR,
parent.getRegionInfo(), hri_a, hri_b)) {
// Passed PONR, let SSH clean it up
throw new IOException("Failed to notify master that split passed PONR: "
+ parent.getRegionInfo().getRegionNameAsString());
}
if (services != null && !services.reportRegionStateTransition(TransitionCode.SPLIT_PONR,
parent.getRegionInfo(), hri_a, hri_b)) {
// Passed PONR, let SSH clean it up
throw new IOException("Failed to notify master that split passed PONR: "
+ parent.getRegionInfo().getRegionNameAsString());
}
return daughterRegions;
}
public PairOfSameType<HRegion> stepsBeforePONR(final Server server,
final RegionServerServices services, boolean testing) throws IOException {
if (useCoordinatedStateManager(server)) {
if (std == null) {
std =
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getSplitTransactionCoordination().getDefaultDetails();
}
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getSplitTransactionCoordination().startSplitTransaction(parent, server.getServerName(),
hri_a, hri_b);
} else if (services != null && !useZKForAssignment) {
if (!services.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT,
parent.getRegionInfo(), hri_a, hri_b)) {
throw new IOException("Failed to get ok from master to split "
+ parent.getRegionNameAsString());
}
if (services != null && !services.reportRegionStateTransition(TransitionCode.READY_TO_SPLIT,
parent.getRegionInfo(), hri_a, hri_b)) {
throw new IOException("Failed to get ok from master to split "
+ parent.getRegionNameAsString());
}
this.journal.add(JournalEntry.SET_SPLITTING);
if (useCoordinatedStateManager(server)) {
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getSplitTransactionCoordination().waitForSplitTransaction(services, parent, hri_a,
hri_b, std);
}
this.parent.getRegionFileSystem().createSplitsDir();
this.journal.add(JournalEntry.CREATE_SPLIT_DIR);
@ -415,24 +379,14 @@ public class SplitTransaction {
bOpener.getName(), bOpener.getException());
}
if (services != null) {
try {
if (useZKForAssignment) {
// add 2nd daughter first (see HBASE-4335)
services.postOpenDeployTasks(b);
} else if (!services.reportRegionStateTransition(TransitionCode.SPLIT,
parent.getRegionInfo(), hri_a, hri_b)) {
throw new IOException("Failed to report split region to master: "
+ parent.getRegionInfo().getShortNameToLog());
}
// Should add it to OnlineRegions
services.addToOnlineRegions(b);
if (useZKForAssignment) {
services.postOpenDeployTasks(a);
}
services.addToOnlineRegions(a);
} catch (KeeperException ke) {
throw new IOException(ke);
if (!services.reportRegionStateTransition(TransitionCode.SPLIT,
parent.getRegionInfo(), hri_a, hri_b)) {
throw new IOException("Failed to report split region to master: "
+ parent.getRegionInfo().getShortNameToLog());
}
// Should add it to OnlineRegions
services.addToOnlineRegions(b);
services.addToOnlineRegions(a);
}
}
}
@ -450,13 +404,6 @@ public class SplitTransaction {
public PairOfSameType<HRegion> execute(final Server server,
final RegionServerServices services)
throws IOException {
useZKForAssignment = server == null ? true :
ConfigUtil.useZKForAssignment(server.getConfiguration());
if (useCoordinatedStateManager(server)) {
std =
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getSplitTransactionCoordination().getDefaultDetails();
}
PairOfSameType<HRegion> regions = createDaughters(server, services);
if (this.parent.getCoprocessorHost() != null) {
this.parent.getCoprocessorHost().preSplitAfterPONR();
@ -468,44 +415,13 @@ public class SplitTransaction {
final RegionServerServices services, PairOfSameType<HRegion> regions)
throws IOException {
openDaughters(server, services, regions.getFirst(), regions.getSecond());
if (useCoordinatedStateManager(server)) {
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getSplitTransactionCoordination().completeSplitTransaction(services, regions.getFirst(),
regions.getSecond(), std, parent);
}
// Coprocessor callback
if (parent.getCoprocessorHost() != null) {
parent.getCoprocessorHost().postSplit(regions.getFirst(), regions.getSecond());
}
return regions;
}
private void offlineParentInMetaAndputMetaEntries(HConnection hConnection,
HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
ServerName serverName, List<Mutation> metaEntries) throws IOException {
List<Mutation> mutations = metaEntries;
HRegionInfo copyOfParent = new HRegionInfo(parent);
copyOfParent.setOffline(true);
copyOfParent.setSplit(true);
//Put for parent
Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
mutations.add(putParent);
//Puts for daughters
Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA);
Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB);
addLocation(putA, serverName, 1); //these are new regions, openSeqNum = 1 is fine.
addLocation(putB, serverName, 1);
mutations.add(putA);
mutations.add(putB);
MetaTableAccessor.mutateMetaTable(hConnection, mutations);
}
public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
Bytes.toBytes(sn.getHostAndPort()));
@ -588,10 +504,6 @@ public class SplitTransaction {
}
}
private boolean useCoordinatedStateManager(final Server server) {
return server != null && useZKForAssignment && server.getCoordinatedStateManager() != null;
}
private void splitStoreFiles(final Map<byte[], List<StoreFile>> hstoreFilesToSplit)
throws IOException {
if (hstoreFilesToSplit == null) {
@ -707,10 +619,7 @@ public class SplitTransaction {
switch(je) {
case SET_SPLITTING:
if (useCoordinatedStateManager(server) && server instanceof HRegionServer) {
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getSplitTransactionCoordination().clean(this.parent.getRegionInfo());
} else if (services != null && !useZKForAssignment
if (services != null
&& !services.reportRegionStateTransition(TransitionCode.SPLIT_REVERTED,
parent.getRegionInfo(), hri_a, hri_b)) {
return false;

View File

@ -23,10 +23,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
/**
* Handles closing of the root region on a region server.
* Handles closing of the meta region on a region server.
*/
@InterfaceAudience.Private
public class CloseMetaHandler extends CloseRegionHandler {
@ -35,9 +34,7 @@ public class CloseMetaHandler extends CloseRegionHandler {
public CloseMetaHandler(final Server server,
final RegionServerServices rsServices,
final HRegionInfo regionInfo,
final boolean abort, CloseRegionCoordination closeRegionCoordination,
CloseRegionCoordination.CloseRegionDetails crd) {
super(server, rsServices, regionInfo, abort, closeRegionCoordination,
crd, EventType.M_RS_CLOSE_META);
final boolean abort) {
super(server, rsServices, regionInfo, abort, EventType.M_RS_CLOSE_META, null);
}
}

View File

@ -26,13 +26,11 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.ConfigUtil;
/**
* Handles closing of a region on a region server.
@ -41,7 +39,7 @@ import org.apache.hadoop.hbase.util.ConfigUtil;
public class CloseRegionHandler extends EventHandler {
// NOTE on priorities shutting down. There are none for close. There are some
// for open. I think that is right. On shutdown, we want the meta to close
// before root and both to close after the user regions have closed. What
// after the user regions have closed. What
// about the case where master tells us to shutdown a catalog region and we
// have a running queue of user regions to close?
private static final Log LOG = LogFactory.getLog(CloseRegionHandler.class);
@ -53,9 +51,6 @@ public class CloseRegionHandler extends EventHandler {
// when we are aborting.
private final boolean abort;
private ServerName destination;
private CloseRegionCoordination closeRegionCoordination;
private CloseRegionCoordination.CloseRegionDetails closeRegionDetails;
private final boolean useZKForAssignment;
/**
* This method used internally by the RegionServer to close out regions.
@ -63,49 +58,25 @@ public class CloseRegionHandler extends EventHandler {
* @param rsServices
* @param regionInfo
* @param abort If the regionserver is aborting.
* @param closeRegionCoordination consensus for closing regions
* @param crd object carrying details about region close task.
* @param destination
*/
public CloseRegionHandler(final Server server,
final RegionServerServices rsServices,
final HRegionInfo regionInfo, final boolean abort,
CloseRegionCoordination closeRegionCoordination,
CloseRegionCoordination.CloseRegionDetails crd) {
this(server, rsServices, regionInfo, abort, closeRegionCoordination, crd,
EventType.M_RS_CLOSE_REGION, null);
}
public CloseRegionHandler(final Server server,
final RegionServerServices rsServices,
final HRegionInfo regionInfo, final boolean abort,
CloseRegionCoordination closeRegionCoordination,
CloseRegionCoordination.CloseRegionDetails crd,
ServerName destination) {
this(server, rsServices, regionInfo, abort, closeRegionCoordination, crd,
this(server, rsServices, regionInfo, abort,
EventType.M_RS_CLOSE_REGION, destination);
}
public CloseRegionHandler(final Server server,
protected CloseRegionHandler(final Server server,
final RegionServerServices rsServices, HRegionInfo regionInfo,
boolean abort, CloseRegionCoordination closeRegionCoordination,
CloseRegionCoordination.CloseRegionDetails crd, EventType eventType) {
this(server, rsServices, regionInfo, abort, closeRegionCoordination, crd, eventType, null);
}
protected CloseRegionHandler(final Server server,
final RegionServerServices rsServices, HRegionInfo regionInfo,
boolean abort, CloseRegionCoordination closeRegionCoordination,
CloseRegionCoordination.CloseRegionDetails crd,
EventType eventType, ServerName destination) {
boolean abort, EventType eventType, ServerName destination) {
super(server, eventType);
this.server = server;
this.rsServices = rsServices;
this.regionInfo = regionInfo;
this.abort = abort;
this.destination = destination;
this.closeRegionCoordination = closeRegionCoordination;
this.closeRegionDetails = crd;
useZKForAssignment = ConfigUtil.useZKForAssignment(server.getConfiguration());
}
public HRegionInfo getRegionInfo() {
@ -128,16 +99,8 @@ public class CloseRegionHandler extends EventHandler {
// Close the region
try {
if (useZKForAssignment && closeRegionCoordination.checkClosingState(
regionInfo, closeRegionDetails)) {
return;
}
// TODO: If we need to keep updating CLOSING stamp to prevent against
// a timeout if this is long-running, need to spin up a thread?
if (region.close(abort) == null) {
// This region got closed. Most likely due to a split. So instead
// of doing the setClosedState() below, let's just ignore cont
// This region got closed. Most likely due to a split.
// The split message will clean up the master state.
LOG.warn("Can't close region: was already closed during close(): " +
regionInfo.getRegionNameAsString());
@ -153,12 +116,7 @@ public class CloseRegionHandler extends EventHandler {
}
this.rsServices.removeFromOnlineRegions(region, destination);
if (!useZKForAssignment) {
rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo);
} else {
closeRegionCoordination.setClosedState(region, this.server.getServerName(),
closeRegionDetails);
}
rsServices.reportRegionStateTransition(TransitionCode.CLOSED, regionInfo);
// Done! Region is closed on this RS
LOG.debug("Closed " + region.getRegionNameAsString());

View File

@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
/**
* Handles opening of a meta region on a region server.
@ -35,9 +34,7 @@ import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
public class OpenMetaHandler extends OpenRegionHandler {
public OpenMetaHandler(final Server server,
final RegionServerServices rsServices, HRegionInfo regionInfo,
final HTableDescriptor htd, OpenRegionCoordination coordination,
OpenRegionCoordination.OpenRegionDetails ord) {
super(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_META,
coordination, ord);
final HTableDescriptor htd) {
super(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_META);
}
}

View File

@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
@ -35,7 +34,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.ConfigUtil;
/**
* Handles opening of a region on a region server.
* <p>
@ -50,30 +48,19 @@ public class OpenRegionHandler extends EventHandler {
private final HRegionInfo regionInfo;
private final HTableDescriptor htd;
private OpenRegionCoordination coordination;
private OpenRegionCoordination.OpenRegionDetails ord;
private final boolean useZKForAssignment;
public OpenRegionHandler(final Server server,
final RegionServerServices rsServices, HRegionInfo regionInfo,
HTableDescriptor htd, OpenRegionCoordination coordination,
OpenRegionCoordination.OpenRegionDetails ord) {
this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION,
coordination, ord);
HTableDescriptor htd) {
this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION);
}
protected OpenRegionHandler(final Server server,
final RegionServerServices rsServices, final HRegionInfo regionInfo,
final HTableDescriptor htd, EventType eventType,
OpenRegionCoordination coordination, OpenRegionCoordination.OpenRegionDetails ord) {
final HTableDescriptor htd, EventType eventType) {
super(server, eventType);
this.rsServices = rsServices;
this.regionInfo = regionInfo;
this.htd = htd;
this.coordination = coordination;
this.ord = ord;
useZKForAssignment = ConfigUtil.useZKForAssignment(server.getConfiguration());
}
public HRegionInfo getRegionInfo() {
@ -83,7 +70,6 @@ public class OpenRegionHandler extends EventHandler {
@Override
public void process() throws IOException {
boolean openSuccessful = false;
boolean transitionedToOpening = false;
final String regionName = regionInfo.getRegionNameAsString();
HRegion region = null;
@ -93,10 +79,9 @@ public class OpenRegionHandler extends EventHandler {
}
final String encodedName = regionInfo.getEncodedName();
// 3 different difficult situations can occur
// 2 different difficult situations can occur
// 1) The opening was cancelled. This is an expected situation
// 2) The region was hijacked, we no longer have the znode
// 3) The region is now marked as online while we're suppose to open. This would be a bug.
// 2) The region is now marked as online while we're suppose to open. This would be a bug.
// Check that this region is not already online
if (this.rsServices.getFromOnlineRegions(encodedName) != null) {
@ -106,21 +91,13 @@ public class OpenRegionHandler extends EventHandler {
return;
}
// Check that we're still supposed to open the region and transition.
// Check that we're still supposed to open the region.
// If fails, just return. Someone stole the region from under us.
// Calling transitionFromOfflineToOpening initializes this.version.
if (!isRegionStillOpening()){
LOG.error("Region " + encodedName + " opening cancelled");
return;
}
if (useZKForAssignment
&& !coordination.transitionFromOfflineToOpening(regionInfo, ord)) {
LOG.warn("Region was hijacked? Opening cancelled for encodedName=" + encodedName);
// This is a desperate attempt: the znode is unlikely to be ours. But we can't do more.
return;
}
transitionedToOpening = true;
// Open region. After a successful open, failures in subsequent
// processing needs to do a close as part of cleanup.
region = openRegion();
@ -128,37 +105,15 @@ public class OpenRegionHandler extends EventHandler {
return;
}
boolean failed = true;
if (!useZKForAssignment ||
coordination.tickleOpening(ord, regionInfo, rsServices, "post_region_open")) {
if (updateMeta(region)) {
failed = false;
}
}
if (failed || this.server.isStopped() ||
if (!updateMeta(region) || this.server.isStopped() ||
this.rsServices.isStopping()) {
return;
}
if (!isRegionStillOpening() ||
(useZKForAssignment && !coordination.transitionToOpened(region, ord))) {
// If we fail to transition to opened, it's because of one of two cases:
// (a) we lost our ZK lease
// OR (b) someone else opened the region before us
// OR (c) someone cancelled the open
// In all cases, we try to transition to failed_open to be safe.
if (!isRegionStillOpening()) {
return;
}
// We have a znode in the opened state now. We can't really delete it as the master job.
// Transitioning to failed open would create a race condition if the master has already
// acted the transition to opened.
// Cancelling the open is dangerous, because we would have a state where the master thinks
// the region is opened while the region is actually closed. It is a dangerous state
// to be in. For this reason, from now on, we're not going back. There is a message in the
// finally close to let the admin knows where we stand.
// Successful region open, and add it to OnlineRegions
this.rsServices.addToOnlineRegions(region);
openSuccessful = true;
@ -166,12 +121,10 @@ public class OpenRegionHandler extends EventHandler {
// Done! Successful region open
LOG.debug("Opened " + regionName + " on " +
this.server.getServerName());
} finally {
// Do all clean up here
if (!openSuccessful) {
doCleanUpOnFailedOpen(region, transitionedToOpening, ord);
doCleanUpOnFailedOpen(region);
}
final Boolean current = this.rsServices.getRegionsInTransitionInRS().
remove(this.regionInfo.getEncodedNameAsBytes());
@ -180,9 +133,7 @@ public class OpenRegionHandler extends EventHandler {
// A better solution would be to not have any race condition.
// this.rsServices.getRegionsInTransitionInRS().remove(
// this.regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
// would help, but we would still have a consistency issue to manage with
// 1) this.rsServices.addToOnlineRegions(region);
// 2) the ZK state.
// would help.
if (openSuccessful) {
if (current == null) { // Should NEVER happen, but let's be paranoid.
LOG.error("Bad state: we've just opened a region that was NOT in transition. Region="
@ -198,29 +149,14 @@ public class OpenRegionHandler extends EventHandler {
}
}
private void doCleanUpOnFailedOpen(HRegion region, boolean transitionedToOpening,
OpenRegionCoordination.OpenRegionDetails ord)
private void doCleanUpOnFailedOpen(HRegion region)
throws IOException {
if (transitionedToOpening) {
try {
if (region != null) {
cleanupFailedOpen(region);
}
} finally {
if (!useZKForAssignment) {
rsServices.reportRegionStateTransition(TransitionCode.FAILED_OPEN, regionInfo);
} else {
// Even if cleanupFailed open fails we need to do this transition
// See HBASE-7698
coordination.tryTransitionFromOpeningToFailedOpen(regionInfo, ord);
}
try {
if (region != null) {
cleanupFailedOpen(region);
}
} else if (!useZKForAssignment) {
} finally {
rsServices.reportRegionStateTransition(TransitionCode.FAILED_OPEN, regionInfo);
} else {
// If still transition to OPENING is not done, we need to transition znode
// to FAILED_OPEN
coordination.tryTransitionFromOfflineToFailedOpen(this.rsServices, regionInfo, ord);
}
}
@ -244,8 +180,6 @@ public class OpenRegionHandler extends EventHandler {
// Post open deploy task:
// meta => update meta location in ZK
// other region => update meta
// It could fail if ZK/meta is not available and
// the update runs out of retries.
long now = System.currentTimeMillis();
long lastUpdate = now;
boolean tickleOpening = true;
@ -255,10 +189,6 @@ public class OpenRegionHandler extends EventHandler {
if (elapsed > 120000) { // 2 minutes, no need to tickleOpening too often
// Only tickle OPENING if postOpenDeployTasks is taking some time.
lastUpdate = now;
if (useZKForAssignment) {
tickleOpening = coordination.tickleOpening(
ord, regionInfo, rsServices, "post_open_deploy");
}
}
synchronized (signaller) {
try {
@ -356,11 +286,6 @@ public class OpenRegionHandler extends EventHandler {
this.rsServices,
new CancelableProgressable() {
public boolean progress() {
if (useZKForAssignment) {
// if tickle failed, we need to cancel opening region.
return coordination.tickleOpening(ord, regionInfo,
rsServices, "open_region_progress");
}
if (!isRegionStillOpening()) {
LOG.warn("Open region aborted since it isn't opening any more");
return false;

View File

@ -1,33 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
/**
* Some configuration related utilities
*/
@InterfaceAudience.Private
public class ConfigUtil {
public static boolean useZKForAssignment(Configuration conf) {
// To change the default, please also update ZooKeeperWatcher.java
return conf.getBoolean("hbase.assignment.usezk", false);
}
}

View File

@ -147,12 +147,13 @@ public class HBaseFsckRepair {
* Contacts a region server and waits up to hbase.hbck.close.timeout ms
* (default 120s) to close the region. This bypasses the active hmaster.
*/
@SuppressWarnings("deprecation")
public static void closeRegionSilentlyAndWait(Admin admin,
ServerName server, HRegionInfo region) throws IOException, InterruptedException {
HConnection connection = admin.getConnection();
AdminService.BlockingInterface rs = connection.getAdmin(server);
try {
ProtobufUtil.closeRegion(rs, server, region.getRegionName(), false);
ProtobufUtil.closeRegion(rs, server, region.getRegionName());
} catch (IOException e) {
LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
}

View File

@ -99,7 +99,6 @@ import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.DFSClient;
@ -109,8 +108,6 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.MiniMRCluster;
import org.apache.hadoop.mapred.TaskLog;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.ZooKeeper.States;
@ -1470,6 +1467,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Modify a table, synchronous. Waiting logic similar to that of {@code admin.rb#alter_status}.
*/
@SuppressWarnings("serial")
public static void modifyTableSync(Admin admin, HTableDescriptor desc)
throws IOException, InterruptedException {
admin.modifyTable(desc.getTableName(), desc);
@ -3009,30 +3007,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return zkw;
}
/**
* Creates a znode with OPENED state.
* @param TEST_UTIL
* @param region
* @param serverName
* @return
* @throws IOException
* @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
* @throws KeeperException
* @throws NodeExistsException
*/
public static ZooKeeperWatcher createAndForceNodeToOpenedState(
HBaseTestingUtility TEST_UTIL, HRegion region,
ServerName serverName) throws ZooKeeperConnectionException,
IOException, KeeperException, NodeExistsException {
ZooKeeperWatcher zkw = getZooKeeperWatcher(TEST_UTIL);
ZKAssign.createNodeOffline(zkw, region.getRegionInfo(), serverName);
int version = ZKAssign.transitionNodeOpening(zkw, region
.getRegionInfo(), serverName);
ZKAssign.transitionNodeOpened(zkw, region.getRegionInfo(), serverName,
version);
return zkw;
}
public static void assertKVListsEqual(String additionalMsg,
final List<? extends Cell> expected,
final List<? extends Cell> actual) {
@ -3440,6 +3414,16 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
};
}
/**
* Wait until no regions in transition.
* @param timeout How long to wait.
* @throws Exception
*/
public void waitUntilNoRegionsInTransition(
final long timeout) throws Exception {
waitFor(timeout, predicateNoRegionsInTransition());
}
/**
* Create a set of column descriptors with the combination of compression,
* encoding, bloom codecs available.

View File

@ -1,305 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.executor.ExecutorType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
/**
* Test the draining servers feature.
*/
@Category(MediumTests.class)
public class TestDrainingServer {
private static final Log LOG = LogFactory.getLog(TestDrainingServer.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private Abortable abortable = new Abortable() {
@Override
public boolean isAborted() {
return false;
}
@Override
public void abort(String why, Throwable e) {
}
};
@AfterClass
public static void afterClass() throws Exception {
TEST_UTIL.shutdownMiniZKCluster();
}
@BeforeClass
public static void beforeClass() throws Exception {
TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
TEST_UTIL.startMiniZKCluster();
}
@Test
public void testAssignmentManagerDoesntUseDrainingServer() throws Exception {
AssignmentManager am;
Configuration conf = TEST_UTIL.getConfiguration();
final HMaster master = Mockito.mock(HMaster.class);
final Server server = Mockito.mock(Server.class);
final ServerManager serverManager = Mockito.mock(ServerManager.class);
final ServerName SERVERNAME_A = ServerName.valueOf("mockserver_a.org", 1000, 8000);
final ServerName SERVERNAME_B = ServerName.valueOf("mockserver_b.org", 1001, 8000);
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf);
final HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("table_test"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
"zkWatcher-Test", abortable, true);
Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
Mockito.when(server.getConfiguration()).thenReturn(conf);
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
cp.initialize(server);
cp.start();
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
Mockito.when(serverManager.getOnlineServersList())
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList())
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList(null))
.thenReturn(new ArrayList<ServerName>(onlineServers.keySet()));
for (ServerName sn : onlineServers.keySet()) {
Mockito.when(serverManager.isServerOnline(sn)).thenReturn(true);
Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1)).thenReturn(true);
Mockito.when(serverManager.sendRegionClose(sn, REGIONINFO, -1, null, false)).thenReturn(true);
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, new ArrayList<ServerName>()))
.thenReturn(RegionOpeningState.OPENED);
Mockito.when(serverManager.sendRegionOpen(sn, REGIONINFO, -1, null))
.thenReturn(RegionOpeningState.OPENED);
Mockito.when(serverManager.addServerToDrainList(sn)).thenReturn(true);
}
Mockito.when(master.getServerManager()).thenReturn(serverManager);
am = new AssignmentManager(server, serverManager,
balancer, startupMasterExecutor("mockExecutorService"), null, null);
Mockito.when(master.getAssignmentManager()).thenReturn(am);
Mockito.when(master.getZooKeeper()).thenReturn(zkWatcher);
am.addPlan(REGIONINFO.getEncodedName(), new RegionPlan(REGIONINFO, null, SERVERNAME_A));
zkWatcher.registerListenerFirst(am);
addServerToDrainedList(SERVERNAME_A, onlineServers, serverManager);
am.assign(REGIONINFO, true);
setRegionOpenedOnZK(zkWatcher, SERVERNAME_A, REGIONINFO);
setRegionOpenedOnZK(zkWatcher, SERVERNAME_B, REGIONINFO);
am.waitForAssignment(REGIONINFO);
assertTrue(am.getRegionStates().isRegionOnline(REGIONINFO));
assertNotEquals(am.getRegionStates().getRegionServerOfRegion(REGIONINFO), SERVERNAME_A);
}
@Test
public void testAssignmentManagerDoesntUseDrainedServerWithBulkAssign() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
LoadBalancer balancer = LoadBalancerFactory.getLoadBalancer(conf);
AssignmentManager am;
final HMaster master = Mockito.mock(HMaster.class);
final Server server = Mockito.mock(Server.class);
final ServerManager serverManager = Mockito.mock(ServerManager.class);
final ServerName SERVERNAME_A = ServerName.valueOf("mockserverbulk_a.org", 1000, 8000);
final ServerName SERVERNAME_B = ServerName.valueOf("mockserverbulk_b.org", 1001, 8000);
final ServerName SERVERNAME_C = ServerName.valueOf("mockserverbulk_c.org", 1002, 8000);
final ServerName SERVERNAME_D = ServerName.valueOf("mockserverbulk_d.org", 1003, 8000);
final ServerName SERVERNAME_E = ServerName.valueOf("mockserverbulk_e.org", 1004, 8000);
final Map<HRegionInfo, ServerName> bulk = new HashMap<HRegionInfo, ServerName>();
Set<ServerName> bunchServersAssigned = new HashSet<ServerName>();
HRegionInfo REGIONINFO_A = new HRegionInfo(TableName.valueOf("table_A"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
HRegionInfo REGIONINFO_B = new HRegionInfo(TableName.valueOf("table_B"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
HRegionInfo REGIONINFO_C = new HRegionInfo(TableName.valueOf("table_C"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
HRegionInfo REGIONINFO_D = new HRegionInfo(TableName.valueOf("table_D"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
HRegionInfo REGIONINFO_E = new HRegionInfo(TableName.valueOf("table_E"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
Map<ServerName, ServerLoad> onlineServers = new HashMap<ServerName, ServerLoad>();
List<ServerName> drainedServers = new ArrayList<ServerName>();
onlineServers.put(SERVERNAME_A, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_B, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_C, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_D, ServerLoad.EMPTY_SERVERLOAD);
onlineServers.put(SERVERNAME_E, ServerLoad.EMPTY_SERVERLOAD);
bulk.put(REGIONINFO_A, SERVERNAME_A);
bulk.put(REGIONINFO_B, SERVERNAME_B);
bulk.put(REGIONINFO_C, SERVERNAME_C);
bulk.put(REGIONINFO_D, SERVERNAME_D);
bulk.put(REGIONINFO_E, SERVERNAME_E);
ZooKeeperWatcher zkWatcher = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
"zkWatcher-BulkAssignTest", abortable, true);
Mockito.when(server.getConfiguration()).thenReturn(conf);
Mockito.when(server.getServerName()).thenReturn(ServerName.valueOf("masterMock,1,1"));
Mockito.when(server.getZooKeeper()).thenReturn(zkWatcher);
CoordinatedStateManager cp = new ZkCoordinatedStateManager();
cp.initialize(server);
cp.start();
Mockito.when(server.getCoordinatedStateManager()).thenReturn(cp);
Mockito.when(serverManager.getOnlineServers()).thenReturn(onlineServers);
Mockito.when(serverManager.getOnlineServersList()).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList()).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet()));
Mockito.when(serverManager.createDestinationServersList(null)).thenReturn(
new ArrayList<ServerName>(onlineServers.keySet()));
for (Entry<HRegionInfo, ServerName> entry : bulk.entrySet()) {
Mockito.when(serverManager.isServerOnline(entry.getValue())).thenReturn(true);
Mockito.when(serverManager.sendRegionClose(entry.getValue(),
entry.getKey(), -1)).thenReturn(true);
Mockito.when(serverManager.sendRegionOpen(entry.getValue(),
entry.getKey(), -1, null)).thenReturn(RegionOpeningState.OPENED);
Mockito.when(serverManager.addServerToDrainList(entry.getValue())).thenReturn(true);
}
Mockito.when(master.getServerManager()).thenReturn(serverManager);
drainedServers.add(SERVERNAME_A);
drainedServers.add(SERVERNAME_B);
drainedServers.add(SERVERNAME_C);
drainedServers.add(SERVERNAME_D);
am = new AssignmentManager(server, serverManager,
balancer, startupMasterExecutor("mockExecutorServiceBulk"), null, null);
Mockito.when(master.getAssignmentManager()).thenReturn(am);
zkWatcher.registerListener(am);
for (ServerName drained : drainedServers) {
addServerToDrainedList(drained, onlineServers, serverManager);
}
am.assign(bulk);
Map<String, RegionState> regionsInTransition = am.getRegionStates().getRegionsInTransition();
for (Entry<String, RegionState> entry : regionsInTransition.entrySet()) {
setRegionOpenedOnZK(zkWatcher, entry.getValue().getServerName(),
entry.getValue().getRegion());
}
am.waitForAssignment(REGIONINFO_A);
am.waitForAssignment(REGIONINFO_B);
am.waitForAssignment(REGIONINFO_C);
am.waitForAssignment(REGIONINFO_D);
am.waitForAssignment(REGIONINFO_E);
Map<HRegionInfo, ServerName> regionAssignments = am.getRegionStates().getRegionAssignments();
for (Entry<HRegionInfo, ServerName> entry : regionAssignments.entrySet()) {
LOG.info("Region Assignment: "
+ entry.getKey().getRegionNameAsString() + " Server: " + entry.getValue());
bunchServersAssigned.add(entry.getValue());
}
for (ServerName sn : drainedServers) {
assertFalse(bunchServersAssigned.contains(sn));
}
}
private void addServerToDrainedList(ServerName serverName,
Map<ServerName, ServerLoad> onlineServers, ServerManager serverManager) {
onlineServers.remove(serverName);
List<ServerName> availableServers = new ArrayList<ServerName>(onlineServers.keySet());
Mockito.when(serverManager.createDestinationServersList()).thenReturn(availableServers);
Mockito.when(serverManager.createDestinationServersList(null)).thenReturn(availableServers);
}
private void setRegionOpenedOnZK(final ZooKeeperWatcher zkWatcher, final ServerName serverName,
HRegionInfo hregionInfo) throws Exception {
int version = ZKAssign.getVersion(zkWatcher, hregionInfo);
int versionTransition = ZKAssign.transitionNode(zkWatcher,
hregionInfo, serverName, EventType.M_ZK_REGION_OFFLINE,
EventType.RS_ZK_REGION_OPENING, version);
ZKAssign.transitionNodeOpened(zkWatcher, hregionInfo, serverName, versionTransition);
}
private ExecutorService startupMasterExecutor(final String name) {
ExecutorService executor = new ExecutorService(name);
executor.startExecutorService(ExecutorType.MASTER_OPEN_REGION, 3);
executor.startExecutorService(ExecutorType.MASTER_CLOSE_REGION, 3);
executor.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS, 3);
executor.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS, 3);
return executor;
}
}

View File

@ -244,7 +244,7 @@ public class TestIOFencing {
c.setClass(HConstants.REGION_IMPL, regionClass, HRegion.class);
c.setBoolean("dfs.support.append", true);
// Encourage plenty of flushes
c.setLong("hbase.hregion.memstore.flush.size", 200000);
c.setLong("hbase.hregion.memstore.flush.size", 100000);
c.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, ConstantSizeRegionSplitPolicy.class.getName());
// Only run compaction when we tell it to
c.setInt("hbase.hstore.compaction.min",1);

View File

@ -26,12 +26,12 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.client.HTable;
@ -48,6 +48,7 @@ import org.junit.experimental.categories.Category;
* Test {@link org.apache.hadoop.hbase.MetaTableAccessor}.
*/
@Category(MediumTests.class)
@SuppressWarnings("deprecation")
public class TestMetaTableAccessor {
private static final Log LOG = LogFactory.getLog(TestMetaTableAccessor.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();

View File

@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -500,8 +499,7 @@ public class TestZooKeeper {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
htd.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
admin.createTable(htd, SPLIT_KEYS);
ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
m.getZooKeeper().close();
MockLoadBalancer.retainAssignCalled = false;
m.abort("Test recovery from zk session expired",
@ -524,8 +522,7 @@ public class TestZooKeeper {
* RS goes down.
*/
@Test(timeout = 300000)
public void testLogSplittingAfterMasterRecoveryDueToZKExpiry() throws IOException,
KeeperException, InterruptedException {
public void testLogSplittingAfterMasterRecoveryDueToZKExpiry() throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
cluster.startRegionServer();
HMaster m = cluster.getMaster();
@ -541,8 +538,7 @@ public class TestZooKeeper {
HColumnDescriptor hcd = new HColumnDescriptor("col");
htd.addFamily(hcd);
admin.createTable(htd, SPLIT_KEYS);
ZooKeeperWatcher zooKeeperWatcher = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
ZKAssign.blockUntilNoRIT(zooKeeperWatcher);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
table = new HTable(TEST_UTIL.getConfiguration(), tableName);
Put p;
int numberOfPuts;

View File

@ -19,6 +19,18 @@
package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -43,7 +55,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
import org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
@ -53,23 +64,12 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
/**
* Tests for region replicas. Sad that we cannot isolate these without bringing up a whole
* cluster. See {@link org.apache.hadoop.hbase.regionserver.TestRegionServerNoMaster}.
*/
@Category(MediumTests.class)
@SuppressWarnings("deprecation")
public class TestReplicasClient {
private static final Log LOG = LogFactory.getLog(TestReplicasClient.class);
@ -187,11 +187,13 @@ public class TestReplicasClient {
} catch (MasterNotRunningException ignored) {
}
}
ha.close();
LOG.info("Master has stopped");
}
@AfterClass
public static void afterClass() throws Exception {
HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
if (table != null) table.close();
HTU.shutdownMiniCluster();
}
@ -219,8 +221,6 @@ public class TestReplicasClient {
closeRegion(hriPrimary);
} catch (Exception ignored) {
}
ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hriPrimary);
ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hriSecondary);
HTU.getHBaseAdmin().getConnection().clearRegionCache();
}
@ -233,10 +233,9 @@ public class TestReplicasClient {
try {
if (isRegionOpened(hri)) return;
} catch (Exception e){}
ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
// first version is '0'
AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
getRS().getServerName(), hri, 0, null, null);
getRS().getServerName(), hri, null, null);
AdminProtos.OpenRegionResponse responseOpen = getRS().getRSRpcServices().openRegion(null, orr);
Assert.assertEquals(responseOpen.getOpeningStateCount(), 1);
Assert.assertEquals(responseOpen.getOpeningState(0),
@ -245,27 +244,19 @@ public class TestReplicasClient {
}
private void closeRegion(HRegionInfo hri) throws Exception {
ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(
getRS().getServerName(), hri.getEncodedName(), true);
getRS().getServerName(), hri.getEncodedName());
AdminProtos.CloseRegionResponse responseClose = getRS()
.getRSRpcServices().closeRegion(null, crr);
Assert.assertTrue(responseClose.getClosed());
checkRegionIsClosed(hri.getEncodedName());
ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(), null);
}
private void checkRegionIsOpened(HRegionInfo hri) throws Exception {
while (!getRS().getRegionsInTransitionInRS().isEmpty()) {
Thread.sleep(1);
}
Assert.assertTrue(
ZKAssign.deleteOpenedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(), null));
}
private boolean isRegionOpened(HRegionInfo hri) throws Exception {

View File

@ -40,10 +40,7 @@ import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -474,7 +471,7 @@ public class TestScannersFromClientSide {
int i = cluster.getServerWith(regionName);
HRegionServer rs = cluster.getRegionServer(i);
ProtobufUtil.closeRegion(
rs.getRSRpcServices(), rs.getServerName(), regionName, false);
rs.getRSRpcServices(), rs.getServerName(), regionName);
long startTime = EnvironmentEdgeManager.currentTimeMillis();
long timeOut = 300000;
while (true) {
@ -487,27 +484,19 @@ public class TestScannersFromClientSide {
}
// Now open the region again.
ZooKeeperWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
try {
HMaster master = cluster.getMaster();
RegionStates states = master.getAssignmentManager().getRegionStates();
states.regionOffline(hri);
states.updateRegionState(hri, State.OPENING);
if (ConfigUtil.useZKForAssignment(TEST_UTIL.getConfiguration())) {
ZKAssign.createNodeOffline(zkw, hri, loc.getServerName());
HMaster master = cluster.getMaster();
RegionStates states = master.getAssignmentManager().getRegionStates();
states.regionOffline(hri);
states.updateRegionState(hri, State.OPENING);
ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
startTime = EnvironmentEdgeManager.currentTimeMillis();
while (true) {
if (rs.getOnlineRegion(regionName) != null) {
break;
}
ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
startTime = EnvironmentEdgeManager.currentTimeMillis();
while (true) {
if (rs.getOnlineRegion(regionName) != null) {
break;
}
assertTrue("Timed out in open the testing region",
EnvironmentEdgeManager.currentTimeMillis() < startTime + timeOut);
Thread.sleep(500);
}
} finally {
ZKAssign.deleteNodeFailSilent(zkw, hri);
assertTrue("Timed out in open the testing region",
EnvironmentEdgeManager.currentTimeMillis() < startTime + timeOut);
Thread.sleep(500);
}
// c0:0, c1:1

View File

@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@ -194,7 +193,7 @@ public class TestChangingEncoding {
// wait for regions out of transition. Otherwise, for online
// encoding change, verification phase may be flaky because
// regions could be still in transition.
ZKAssign.blockUntilNoRIT(TEST_UTIL.getZooKeeperWatcher());
TEST_UTIL.waitUntilNoRegionsInTransition(TIMEOUT_MS);
}
@Test(timeout=TIMEOUT_MS)

View File

@ -1,110 +0,0 @@
/**
* 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.junit.Assert.assertNotSame;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
* Package scoped mocking utility.
*/
public class Mocking {
static void waitForRegionFailedToCloseAndSetToPendingClose(
AssignmentManager am, HRegionInfo hri) throws InterruptedException {
// Since region server is fake, sendRegionClose will fail, and closing
// region will fail. For testing purpose, moving it back to pending close
boolean wait = true;
while (wait) {
RegionState state = am.getRegionStates().getRegionState(hri);
if (state != null && state.isFailedClose()){
am.getRegionStates().updateRegionState(hri, State.PENDING_CLOSE);
wait = false;
} else {
Thread.sleep(1);
}
}
}
static void waitForRegionPendingOpenInRIT(AssignmentManager am, String encodedName)
throws InterruptedException {
// We used to do a check like this:
//!Mocking.verifyRegionState(this.watcher, REGIONINFO, EventType.M_ZK_REGION_OFFLINE)) {
// There is a race condition with this: because we may do the transition to
// RS_ZK_REGION_OPENING before the RIT is internally updated. We need to wait for the
// RIT to be as we need it to be instead. This cannot happen in a real cluster as we
// update the RIT before sending the openRegion request.
boolean wait = true;
while (wait) {
RegionState state = am.getRegionStates()
.getRegionsInTransition().get(encodedName);
if (state != null && state.isPendingOpen()){
wait = false;
} else {
Thread.sleep(1);
}
}
}
/**
* Verifies that the specified region is in the specified state in ZooKeeper.
* <p>
* Returns true if region is in transition and in the specified state in
* ZooKeeper. Returns false if the region does not exist in ZK or is in
* a different state.
* <p>
* Method synchronizes() with ZK so will yield an up-to-date result but is
* a slow read.
* @param zkw
* @param region
* @param expectedState
* @return true if region exists and is in expected state
* @throws DeserializationException
*/
static boolean verifyRegionState(ZooKeeperWatcher zkw, HRegionInfo region, EventType expectedState)
throws KeeperException, DeserializationException {
String encoded = region.getEncodedName();
String node = ZKAssign.getNodeName(zkw, encoded);
zkw.sync(node);
// Read existing data of the node
byte [] existingBytes = null;
try {
existingBytes = ZKUtil.getDataAndWatch(zkw, node);
} catch (KeeperException.NoNodeException nne) {
return false;
} catch (KeeperException e) {
throw e;
}
if (existingBytes == null) return false;
RegionTransition rt = RegionTransition.parseFrom(existingBytes);
return rt.getEventType().equals(expectedState);
}
}

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.apache.hadoop.hbase.ServerLoad;
@ -47,28 +48,22 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -85,7 +80,8 @@ public class TestAssignmentManagerOnCluster {
final static Configuration conf = TEST_UTIL.getConfiguration();
private static Admin admin;
static void setupOnce() throws Exception {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Using the our load balancer to control region plans
conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
MyLoadBalancer.class, LoadBalancer.class);
@ -98,13 +94,6 @@ public class TestAssignmentManagerOnCluster {
admin = TEST_UTIL.getHBaseAdmin();
}
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Use ZK for region assignment
conf.setBoolean("hbase.assignment.usezk", true);
setupOnce();
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
@ -187,11 +176,6 @@ public class TestAssignmentManagerOnCluster {
am.addPlan(hri.getEncodedName(), plan);
master.assignRegion(hri);
int version = ZKAssign.transitionNode(master.getZooKeeper(), hri,
destServer, EventType.M_ZK_REGION_OFFLINE,
EventType.RS_ZK_REGION_OPENING, 0);
assertEquals("TansitionNode should fail", -1, version);
TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
@ -402,7 +386,7 @@ public class TestAssignmentManagerOnCluster {
// region is closing now, will be re-assigned automatically.
// now, let's forcefully assign it again. it should be
// assigned properly and no double-assignment
am.assign(hri, true, true);
am.assign(hri, true);
// let's check if it's assigned after it's out of transition
am.waitOnRegionToClearRegionsInTransition(hri);
@ -572,20 +556,7 @@ public class TestAssignmentManagerOnCluster {
}
}
am.regionOffline(hri);
ZooKeeperWatcher zkw = TEST_UTIL.getHBaseCluster().getMaster().getZooKeeper();
am.getRegionStates().updateRegionState(hri, State.PENDING_OPEN, destServerName);
if (ConfigUtil.useZKForAssignment(conf)) {
ZKAssign.createNodeOffline(zkw, hri, destServerName);
ZKAssign.transitionNodeOpening(zkw, hri, destServerName);
// Wait till the event is processed and the region is in transition
long timeoutTime = System.currentTimeMillis() + 20000;
while (!am.getRegionStates().isRegionInTransition(hri)) {
assertTrue("Failed to process ZK opening event in time",
System.currentTimeMillis() < timeoutTime);
Thread.sleep(100);
}
}
am.getTableStateManager().setTableState(table, ZooKeeperProtos.Table.State.DISABLING);
List<HRegionInfo> toAssignRegions = am.processServerShutdown(destServerName);
@ -775,7 +746,7 @@ public class TestAssignmentManagerOnCluster {
}
// You can't assign a dead region before SSH
am.assign(hri, true, true);
am.assign(hri, true);
RegionState state = regionStates.getRegionState(hri);
assertTrue(state.isFailedClose());
@ -833,7 +804,7 @@ public class TestAssignmentManagerOnCluster {
assertTrue(regionStates.isRegionOffline(hri));
// You can't assign a disabled region
am.assign(hri, true, true);
am.assign(hri, true);
assertTrue(regionStates.isRegionOffline(hri));
// You can't unassign a disabled region either
@ -991,7 +962,7 @@ public class TestAssignmentManagerOnCluster {
}
// Wait till no more RIT, the region should be offline.
am.waitUntilNoRegionsInTransition(60000);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
assertTrue(regionStates.isRegionOffline(hri));
} finally {
MyRegionServer.abortedServer = null;

View File

@ -95,13 +95,11 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@ -1505,10 +1503,8 @@ public class TestDistributedLogSplitting {
return count;
}
private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
throws KeeperException, InterruptedException {
ZKAssign.blockUntilNoRIT(zkw);
master.assignmentManager.waitUntilNoRegionsInTransition(60000);
private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master) throws Exception {
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
}
private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,

View File

@ -18,28 +18,39 @@
*/
package org.apache.hadoop.hbase.master;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import com.google.common.base.Joiner;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.*;
import com.google.common.base.Joiner;
@Category(MediumTests.class)
public class TestMaster {
@ -55,7 +66,6 @@ public class TestMaster {
// Start a cluster of two regionservers.
TEST_UTIL.startMiniCluster(2);
admin = TEST_UTIL.getHBaseAdmin();
TEST_UTIL.getHBaseCluster().getMaster().assignmentManager.initializeHandlerTrackers();
}
@AfterClass
@ -64,6 +74,7 @@ public class TestMaster {
}
@Test
@SuppressWarnings("deprecation")
public void testMasterOpsWhileSplitting() throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster m = cluster.getMaster();

View File

@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
@ -200,7 +199,7 @@ public class TestMasterNoCluster {
// Fake a successful close.
Mockito.doReturn(true).when(spy).
sendRegionClose((ServerName)Mockito.any(), (HRegionInfo)Mockito.any(),
Mockito.anyInt(), (ServerName)Mockito.any(), Mockito.anyBoolean());
(ServerName)Mockito.any());
return spy;
}
@ -235,13 +234,8 @@ public class TestMasterNoCluster {
request.setLoad(ServerLoad.EMPTY_SERVERLOAD.obtainServerLoadPB());
master.getMasterRpcServices().regionServerReport(null, request.build());
}
ZooKeeperWatcher zkw = master.getZooKeeper();
// Master should now come up.
// Master should now come up.
while (!master.isInitialized()) {
// Fake meta is closed on rs0, try several times in case the event is lost
// due to race with HMaster#assignMeta
ZKAssign.transitionNodeClosed(zkw,
HRegionInfo.FIRST_META_REGIONINFO, sn0, -1);
Threads.sleep(100);
}
assertTrue(master.isInitialized());

View File

@ -37,9 +37,6 @@ import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -63,8 +60,6 @@ public class TestMasterRestartAfterDisablingTable {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
log("Waiting for active/ready master");
cluster.waitForActiveAndReadyMaster();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testmasterRestart", null);
HMaster master = cluster.getMaster();
// Create a table with regions
TableName table = TableName.valueOf("tableRestart");
@ -75,7 +70,7 @@ public class TestMasterRestartAfterDisablingTable {
NUM_REGIONS_TO_CREATE);
numRegions += 1; // catalogs
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
log("Disabling table\n");
TEST_UTIL.getHBaseAdmin().disableTable(table);
@ -106,7 +101,7 @@ public class TestMasterRestartAfterDisablingTable {
admin.enableTable(table);
admin.close();
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
log("Verifying there are " + numRegions + " assigned on cluster\n");
regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
assertEquals("The assigned regions were not onlined after master"
@ -122,11 +117,5 @@ public class TestMasterRestartAfterDisablingTable {
private void log(String msg) {
LOG.debug("\n\nTRR: " + msg + "\n");
}
private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
throws KeeperException, InterruptedException {
ZKAssign.blockUntilNoRIT(zkw);
master.assignmentManager.waitUntilNoRegionsInTransition(60000);
}
}

View File

@ -1,227 +0,0 @@
/**
* 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.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MockServer;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKTableStateManager;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
@Category(MediumTests.class)
public class TestOpenedRegionHandler {
private static final Log LOG = LogFactory
.getLog(TestOpenedRegionHandler.class);
private HBaseTestingUtility TEST_UTIL;
private final int NUM_MASTERS = 1;
private final int NUM_RS = 1;
private Configuration conf;
private Configuration resetConf;
private ZooKeeperWatcher zkw;
@Before
public void setUp() throws Exception {
conf = HBaseConfiguration.create();
conf.setBoolean("hbase.assignment.usezk", true);
TEST_UTIL = HBaseTestingUtility.createLocalHTU(conf);
}
@After
public void tearDown() throws Exception {
// Stop the cluster
TEST_UTIL.shutdownMiniCluster();
TEST_UTIL = new HBaseTestingUtility(resetConf);
}
@Test
public void testOpenedRegionHandlerOnMasterRestart() throws Exception {
// Start the cluster
log("Starting cluster");
conf = HBaseConfiguration.create();
conf.setBoolean("hbase.assignment.usezk", true);
resetConf = conf;
TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
String tableName = "testOpenedRegionHandlerOnMasterRestart";
MiniHBaseCluster cluster = createRegions(tableName);
abortMaster(cluster);
HRegionServer regionServer = cluster.getRegionServer(0);
HRegion region = getRegionBeingServed(cluster, regionServer);
// forcefully move a region to OPENED state in zk
// Create a ZKW to use in the test
zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
region, regionServer.getServerName());
// Start up a new master
log("Starting up a new master");
cluster.startMaster().getMaster();
log("Waiting for master to be ready");
cluster.waitForActiveAndReadyMaster();
log("Master is ready");
// Failover should be completed, now wait for no RIT
log("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
}
@Test
public void testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches()
throws Exception {
HRegion region = null;
try {
int testIndex = 0;
TEST_UTIL.startMiniZKCluster();
final Server server = new MockServer(TEST_UTIL);
HTableDescriptor htd = new HTableDescriptor(
TableName.valueOf("testShouldNotCompeleteOpenedRegionSuccessfullyIfVersionMismatches"));
HRegionInfo hri = new HRegionInfo(htd.getTableName(),
Bytes.toBytes(testIndex), Bytes.toBytes(testIndex + 1));
region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
assertNotNull(region);
AssignmentManager am = Mockito.mock(AssignmentManager.class);
RegionStates rsm = Mockito.mock(RegionStates.class);
Mockito.doReturn(rsm).when(am).getRegionStates();
when(rsm.isRegionInTransition(hri)).thenReturn(false);
when(rsm.getRegionState(hri)).thenReturn(
new RegionState(region.getRegionInfo(), RegionState.State.OPEN,
System.currentTimeMillis(), server.getServerName()));
// create a node with OPENED state
zkw = HBaseTestingUtility.createAndForceNodeToOpenedState(TEST_UTIL,
region, server.getServerName());
when(am.getTableStateManager()).thenReturn(new ZKTableStateManager(zkw));
Stat stat = new Stat();
String nodeName = ZKAssign.getNodeName(zkw, region.getRegionInfo()
.getEncodedName());
ZKUtil.getDataAndWatch(zkw, nodeName, stat);
// use the version for the OpenedRegionHandler
BaseCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(server);
csm.start();
OpenRegionCoordination orc = csm.getOpenRegionCoordination();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkOrd.setServerName(server.getServerName());
zkOrd.setVersion(stat.getVersion());
OpenedRegionHandler handler = new OpenedRegionHandler(server, am, region
.getRegionInfo(), orc, zkOrd);
// Once again overwrite the same znode so that the version changes.
ZKAssign.transitionNode(zkw, region.getRegionInfo(), server
.getServerName(), EventType.RS_ZK_REGION_OPENED,
EventType.RS_ZK_REGION_OPENED, stat.getVersion());
// Should not invoke assignmentmanager.regionOnline. If it is
// invoked as per current mocking it will throw null pointer exception.
boolean expectedException = false;
try {
handler.process();
} catch (Exception e) {
expectedException = true;
}
assertFalse("The process method should not throw any exception.",
expectedException);
List<String> znodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw,
zkw.assignmentZNode);
String regionName = znodes.get(0);
assertEquals("The region should not be opened successfully.", regionName,
region.getRegionInfo().getEncodedName());
} finally {
HRegion.closeHRegion(region);
TEST_UTIL.shutdownMiniZKCluster();
}
}
private MiniHBaseCluster createRegions(String tableName)
throws InterruptedException, ZooKeeperConnectionException, IOException,
KeeperException {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
log("Waiting for active/ready master");
cluster.waitForActiveAndReadyMaster();
zkw = new ZooKeeperWatcher(conf, "testOpenedRegionHandler", null);
// Create a table with regions
byte[] table = Bytes.toBytes(tableName);
byte[] family = Bytes.toBytes("family");
TEST_UTIL.createTable(table, family);
//wait till the regions are online
log("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
return cluster;
}
private void abortMaster(MiniHBaseCluster cluster) {
// Stop the master
log("Aborting master");
cluster.abortMaster(0);
cluster.waitOnMaster(0);
log("Master has aborted");
}
private HRegion getRegionBeingServed(MiniHBaseCluster cluster,
HRegionServer regionServer) {
Collection<HRegion> onlineRegionsLocalContext = regionServer
.getOnlineRegionsLocalContext();
Iterator<HRegion> iterator = onlineRegionsLocalContext.iterator();
HRegion region = null;
while (iterator.hasNext()) {
region = iterator.next();
if (!region.getRegionInfo().isMetaTable()) {
break;
}
}
return region;
}
private void log(String msg) {
LOG.debug("\n\nTRR: " + msg + "\n");
}
}

View File

@ -28,15 +28,18 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.After;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -46,8 +49,6 @@ public class TestRestartCluster {
private static final Log LOG = LogFactory.getLog(TestRestartCluster.class);
private HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static final byte[] TABLENAME = Bytes.toBytes("master_transitions");
private static final byte [][] FAMILIES = {Bytes.toBytes("a")};
private static final byte [][] TABLES = {
Bytes.toBytes("restartTableOne"),
Bytes.toBytes("restartTableTwo"),
@ -59,35 +60,6 @@ public class TestRestartCluster {
UTIL.shutdownMiniCluster();
}
@Test (timeout=300000) public void testRestartClusterAfterKill()
throws Exception {
UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
UTIL.startMiniZKCluster();
ZooKeeperWatcher zooKeeper =
new ZooKeeperWatcher(UTIL.getConfiguration(), "cluster1", null, true);
// create the unassigned region, throw up a region opened state for META
String unassignedZNode = zooKeeper.assignmentZNode;
ZKUtil.createAndFailSilent(zooKeeper, unassignedZNode);
ServerName sn = ServerName.valueOf(HMaster.MASTER, 1, System.currentTimeMillis());
ZKAssign.createNodeOffline(zooKeeper, HRegionInfo.FIRST_META_REGIONINFO, sn);
LOG.debug("Created UNASSIGNED zNode for ROOT and hbase:meta regions in state " +
EventType.M_ZK_REGION_OFFLINE);
// start the HB cluster
LOG.info("Starting HBase cluster...");
UTIL.startMiniCluster(2);
UTIL.createTable(TABLENAME, FAMILIES);
LOG.info("Created a table, waiting for table to be available...");
UTIL.waitTableAvailable(TABLENAME, 60*1000);
LOG.info("Master deleted unassigned region and started up successfully.");
}
@Test (timeout=300000)
public void testClusterRestart() throws Exception {
UTIL.startMiniCluster(3);
@ -153,8 +125,7 @@ public class TestRestartCluster {
}
HMaster master = UTIL.getMiniHBaseCluster().getMaster();
AssignmentManager am = master.getAssignmentManager();
am.waitUntilNoRegionsInTransition(120000);
UTIL.waitUntilNoRegionsInTransition(120000);
// We don't have to use SnapshotOfRegionAssignmentFromMeta.
// We use it here because AM used to use it to load all user region placements

View File

@ -41,9 +41,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -72,9 +69,6 @@ public class TestRollingRestart {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
log("Waiting for active/ready master");
cluster.waitForActiveAndReadyMaster();
ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testRollingRestart",
null);
HMaster master = cluster.getMaster();
// Create a table with regions
TableName table = TableName.valueOf("tableRestart");
@ -85,11 +79,11 @@ public class TestRollingRestart {
NUM_REGIONS_TO_CREATE);
numRegions += 1; // catalogs
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
log("Disabling table\n");
TEST_UTIL.getHBaseAdmin().disableTable(table);
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
NavigableSet<String> regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
log("Verifying only catalog and namespace regions are assigned\n");
if (regions.size() != 2) {
@ -99,7 +93,7 @@ public class TestRollingRestart {
log("Enabling table\n");
TEST_UTIL.getHBaseAdmin().enableTable(table);
log("Waiting for no more RIT\n");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
log("Verifying there are " + numRegions + " assigned on cluster\n");
regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
assertRegionsAssigned(cluster, regions);
@ -112,7 +106,7 @@ public class TestRollingRestart {
restarted.waitForServerOnline();
log("Additional RS is online");
log("Waiting for no more RIT");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
@ -144,7 +138,6 @@ public class TestRollingRestart {
log("Restarting primary master\n\n");
activeMaster = cluster.startMaster();
cluster.waitForActiveAndReadyMaster();
master = activeMaster.getMaster();
// Start backup master
log("Restarting backup master\n\n");
@ -168,7 +161,7 @@ public class TestRollingRestart {
log("Waiting for RS shutdown to be handled by master");
waitForRSShutdownToStartAndFinish(activeMaster, serverName);
log("RS shutdown done, waiting for no more RIT");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
expectedNumRS--;
@ -179,7 +172,7 @@ public class TestRollingRestart {
expectedNumRS++;
log("Region server " + num + " is back online");
log("Waiting for no more RIT");
blockUntilNoRIT(zkw, master);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
log("Verifying there are " + numRegions + " assigned on cluster");
assertRegionsAssigned(cluster, regions);
assertEquals(expectedNumRS, cluster.getRegionServerThreads().size());
@ -195,12 +188,6 @@ public class TestRollingRestart {
TEST_UTIL.shutdownMiniCluster();
}
private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
throws KeeperException, InterruptedException {
ZKAssign.blockUntilNoRIT(zkw);
master.assignmentManager.waitUntilNoRegionsInTransition(60000);
}
private void waitForRSShutdownToStartAndFinish(MasterThread activeMaster,
ServerName serverName) throws InterruptedException {
ServerManager sm = activeMaster.getMaster().getServerManager();

View File

@ -1,302 +0,0 @@
/**
*
* 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.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Collection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
import org.mockito.internal.util.reflection.Whitebox;
/**
* Test open and close of regions using zk.
*/
@Category(MediumTests.class)
public class TestZKBasedOpenCloseRegion {
private static final Log LOG = LogFactory.getLog(TestZKBasedOpenCloseRegion.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static final TableName TABLENAME =
TableName.valueOf("TestZKBasedOpenCloseRegion");
private static final byte [][] FAMILIES = new byte [][] {Bytes.toBytes("a"),
Bytes.toBytes("b"), Bytes.toBytes("c")};
private static int countOfRegions;
@BeforeClass public static void beforeAllTests() throws Exception {
Configuration c = TEST_UTIL.getConfiguration();
c.setBoolean("hbase.assignment.usezk", true);
c.setBoolean("dfs.support.append", true);
c.setInt("hbase.regionserver.info.port", 0);
TEST_UTIL.startMiniCluster(2);
TEST_UTIL.createTable(TABLENAME, FAMILIES);
HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
countOfRegions = TEST_UTIL.createMultiRegions(t, getTestFamily());
waitUntilAllRegionsAssigned();
addToEachStartKey(countOfRegions);
t.close();
TEST_UTIL.getHBaseCluster().getMaster().assignmentManager.initializeHandlerTrackers();
}
@AfterClass public static void afterAllTests() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Before public void setup() throws IOException {
if (TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size() < 2) {
// Need at least two servers.
LOG.info("Started new server=" +
TEST_UTIL.getHBaseCluster().startRegionServer());
}
waitUntilAllRegionsAssigned();
waitOnRIT();
}
/**
* Test we reopen a region once closed.
* @throws Exception
*/
@Test (timeout=300000) public void testReOpenRegion()
throws Exception {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
LOG.info("Number of region servers = " +
cluster.getLiveRegionServerThreads().size());
int rsIdx = 0;
HRegionServer regionServer =
TEST_UTIL.getHBaseCluster().getRegionServer(rsIdx);
HRegionInfo hri = getNonMetaRegion(
ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices()));
LOG.debug("Asking RS to close region " + hri.getRegionNameAsString());
LOG.info("Unassign " + hri.getRegionNameAsString());
cluster.getMaster().assignmentManager.unassign(hri);
while (!cluster.getMaster().assignmentManager.wasClosedHandlerCalled(hri)) {
Threads.sleep(100);
}
while (!cluster.getMaster().assignmentManager.wasOpenedHandlerCalled(hri)) {
Threads.sleep(100);
}
LOG.info("Done with testReOpenRegion");
}
private HRegionInfo getNonMetaRegion(final Collection<HRegionInfo> regions) {
HRegionInfo hri = null;
for (HRegionInfo i: regions) {
LOG.info(i.getRegionNameAsString());
if (!i.isMetaRegion()) {
hri = i;
break;
}
}
return hri;
}
/**
* This test shows how a region won't be able to be assigned to a RS
* if it's already "processing" it.
* @throws Exception
*/
@Test
public void testRSAlreadyProcessingRegion() throws Exception {
LOG.info("starting testRSAlreadyProcessingRegion");
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HRegionServer hr0 =
cluster.getLiveRegionServerThreads().get(0).getRegionServer();
HRegionServer hr1 =
cluster.getLiveRegionServerThreads().get(1).getRegionServer();
HRegionInfo hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr0.getRSRpcServices()));
// fake that hr1 is processing the region
hr1.getRegionsInTransitionInRS().putIfAbsent(hri.getEncodedNameAsBytes(), true);
// now ask the master to move the region to hr1, will fail
TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
Bytes.toBytes(hr1.getServerName().toString()));
// make sure the region came back
assertEquals(hr1.getOnlineRegion(hri.getEncodedNameAsBytes()), null);
// remove the block and reset the boolean
hr1.getRegionsInTransitionInRS().remove(hri.getEncodedNameAsBytes());
// now try moving a region when there is no region in transition.
hri = getNonMetaRegion(ProtobufUtil.getOnlineRegions(hr1.getRSRpcServices()));
TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
Bytes.toBytes(hr0.getServerName().toString()));
while (!cluster.getMaster().assignmentManager.wasOpenedHandlerCalled(hri)) {
Threads.sleep(100);
}
// make sure the region has moved from the original RS
assertTrue(hr1.getOnlineRegion(hri.getEncodedNameAsBytes()) == null);
}
private void waitOnRIT() {
// Close worked but we are going to open the region elsewhere. Before going on, make sure
// this completes.
while (TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().isRegionsInTransition()) {
LOG.info("Waiting on regions in transition: " +
TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
getRegionStates().getRegionsInTransition());
Threads.sleep(10);
}
}
/**
* If region open fails with IOException in openRegion() while doing tableDescriptors.get()
* the region should not add into regionsInTransitionInRS map
* @throws Exception
*/
@Test
public void testRegionOpenFailsDueToIOException() throws Exception {
HRegionInfo REGIONINFO = new HRegionInfo(TableName.valueOf("t"),
HConstants.EMPTY_START_ROW, HConstants.EMPTY_START_ROW);
HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0);
TableDescriptors htd = Mockito.mock(TableDescriptors.class);
Object orizinalState = Whitebox.getInternalState(regionServer,"tableDescriptors");
Whitebox.setInternalState(regionServer, "tableDescriptors", htd);
Mockito.doThrow(new IOException()).when(htd).get((TableName) Mockito.any());
try {
ProtobufUtil.openRegion(regionServer.getRSRpcServices(),
regionServer.getServerName(), REGIONINFO);
fail("It should throw IOException ");
} catch (IOException e) {
}
Whitebox.setInternalState(regionServer, "tableDescriptors", orizinalState);
assertFalse("Region should not be in RIT",
regionServer.getRegionsInTransitionInRS().containsKey(REGIONINFO.getEncodedNameAsBytes()));
}
private static void waitUntilAllRegionsAssigned()
throws IOException {
HTable meta = new HTable(TEST_UTIL.getConfiguration(), TableName.META_TABLE_NAME);
while (true) {
int rows = 0;
Scan scan = new Scan();
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
ResultScanner s = meta.getScanner(scan);
for (Result r = null; (r = s.next()) != null;) {
byte [] b =
r.getValue(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
if (b == null || b.length <= 0) {
break;
}
rows++;
}
s.close();
// If I get to here and all rows have a Server, then all have been assigned.
if (rows >= countOfRegions) {
break;
}
LOG.info("Found=" + rows);
Threads.sleep(1000);
}
meta.close();
}
/*
* Add to each of the regions in hbase:meta a value. Key is the startrow of the
* region (except its 'aaa' for first region). Actual value is the row name.
* @param expected
* @return
* @throws IOException
*/
private static int addToEachStartKey(final int expected) throws IOException {
HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
HTable meta = new HTable(TEST_UTIL.getConfiguration(),
TableName.META_TABLE_NAME);
int rows = 0;
Scan scan = new Scan();
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
ResultScanner s = meta.getScanner(scan);
for (Result r = null; (r = s.next()) != null;) {
HRegionInfo hri = HRegionInfo.getHRegionInfo(r);
if (hri == null) break;
if(!hri.getTable().equals(TABLENAME)) {
continue;
}
// If start key, add 'aaa'.
byte [] row = getStartKey(hri);
Put p = new Put(row);
p.setDurability(Durability.SKIP_WAL);
p.add(getTestFamily(), getTestQualifier(), row);
t.put(p);
rows++;
}
s.close();
Assert.assertEquals(expected, rows);
t.close();
meta.close();
return rows;
}
private static byte [] getStartKey(final HRegionInfo hri) {
return Bytes.equals(HConstants.EMPTY_START_ROW, hri.getStartKey())?
Bytes.toBytes("aaa"): hri.getStartKey();
}
private static byte [] getTestFamily() {
return FAMILIES[0];
}
private static byte [] getTestQualifier() {
return getTestFamily();
}
}

View File

@ -1,42 +0,0 @@
/**
* 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 org.apache.hadoop.hbase.MediumTests;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
/**
* This tests AssignmentManager with a testing cluster.
*/
@Category(MediumTests.class)
public class TestZKLessAMOnCluster extends TestAssignmentManagerOnCluster {
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Don't use ZK for region assignment
conf.setBoolean("hbase.assignment.usezk", false);
setupOnce();
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TestAssignmentManagerOnCluster.tearDownAfterClass();
}
}

View File

@ -1,354 +0,0 @@
/**
* Copyright The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.migration;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.util.ToolRunner;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test upgrade from no namespace in 0.94 to namespace directory structure.
* Mainly tests that tables are migrated and consistent. Also verifies
* that snapshots have been migrated correctly.
*
* <p>Uses a tarball which is an image of an 0.94 hbase.rootdir.
*
* <p>Contains tables with currentKeys as the stored keys:
* foo, ns1.foo, ns2.foo
*
* <p>Contains snapshots with snapshot{num}Keys as the contents:
* snapshot1Keys, snapshot2Keys
*
* Image also contains _acl_ table with one region and two storefiles.
* This is needed to test the acl table migration.
*
*/
@Category(MediumTests.class)
public class TestNamespaceUpgrade {
static final Log LOG = LogFactory.getLog(TestNamespaceUpgrade.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final static String snapshot1Keys[] =
{"1","10","2","3","4","5","6","7","8","9"};
private final static String snapshot2Keys[] =
{"1","2","3","4","5","6","7","8","9"};
private final static String currentKeys[] =
{"1","2","3","4","5","6","7","8","9","A"};
private final static TableName tables[] =
{TableName.valueOf("foo"), TableName.valueOf("ns1.foo"), TableName.valueOf("ns.two.foo")};
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Start up our mini cluster on top of an 0.94 root.dir that has data from
// a 0.94 hbase run and see if we can migrate to 0.96
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
Path testdir = TEST_UTIL.getDataTestDir("TestNamespaceUpgrade");
// Untar our test dir.
File untar = untar(new File(testdir.toString()));
// Now copy the untar up into hdfs so when we start hbase, we'll run from it.
Configuration conf = TEST_UTIL.getConfiguration();
conf.setBoolean("hbase.assignment.usezk", true);
FsShell shell = new FsShell(conf);
FileSystem fs = FileSystem.get(conf);
// find where hbase will root itself, so we can copy filesystem there
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
if (!fs.isDirectory(hbaseRootDir.getParent())) {
// mkdir at first
fs.mkdirs(hbaseRootDir.getParent());
}
if(org.apache.hadoop.util.VersionInfo.getVersion().startsWith("2.")) {
LOG.info("Hadoop version is 2.x, pre-migrating snapshot dir");
FileSystem localFS = FileSystem.getLocal(conf);
if(!localFS.rename(new Path(untar.toString(), HConstants.OLD_SNAPSHOT_DIR_NAME),
new Path(untar.toString(), HConstants.SNAPSHOT_DIR_NAME))) {
throw new IllegalStateException("Failed to move snapshot dir to 2.x expectation");
}
}
doFsCommand(shell,
new String [] {"-put", untar.toURI().toString(), hbaseRootDir.toString()});
doFsCommand(shell, new String [] {"-lsr", "/"});
// See whats in minihdfs.
Configuration toolConf = TEST_UTIL.getConfiguration();
conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
assertTrue(FSUtils.getVersion(fs, hbaseRootDir).equals(HConstants.FILE_SYSTEM_VERSION));
doFsCommand(shell, new String [] {"-lsr", "/"});
TEST_UTIL.startMiniHBaseCluster(1, 1);
for(TableName table: tables) {
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), table).getScanner(new Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
Assert.assertEquals(currentKeys.length, count);
}
assertEquals(2, TEST_UTIL.getHBaseAdmin().listNamespaceDescriptors().length);
//verify ACL table is migrated
HTable secureTable = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
ResultScanner scanner = secureTable.getScanner(new Scan());
int count = 0;
for(Result r : scanner) {
count++;
}
assertEquals(3, count);
assertFalse(TEST_UTIL.getHBaseAdmin().tableExists(TableName.valueOf("_acl_")));
//verify ACL table was compacted
List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(secureTable.getName());
for(HRegion region : regions) {
assertEquals(1, region.getStores().size());
}
}
static File untar(final File testdir) throws IOException {
// Find the src data under src/test/data
final String datafile = "TestNamespaceUpgrade";
File srcTarFile = new File(
System.getProperty("project.build.testSourceDirectory", "src/test") +
File.separator + "data" + File.separator + datafile + ".tgz");
File homedir = new File(testdir.toString());
File tgtUntarDir = new File(homedir, "hbase");
if (tgtUntarDir.exists()) {
if (!FileUtil.fullyDelete(tgtUntarDir)) {
throw new IOException("Failed delete of " + tgtUntarDir.toString());
}
}
if (!srcTarFile.exists()) {
throw new IOException(srcTarFile+" does not exist");
}
LOG.info("Untarring " + srcTarFile + " into " + homedir.toString());
FileUtil.unTar(srcTarFile, homedir);
Assert.assertTrue(tgtUntarDir.exists());
return tgtUntarDir;
}
private static void doFsCommand(final FsShell shell, final String [] args)
throws Exception {
// Run the 'put' command.
int errcode = shell.run(args);
if (errcode != 0) throw new IOException("Failed put; errcode=" + errcode);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test (timeout=300000)
public void testSnapshots() throws IOException, InterruptedException {
String snapshots[][] = {snapshot1Keys, snapshot2Keys};
for(int i = 1; i <= snapshots.length; i++) {
for(TableName table: tables) {
TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot"+i, TableName.valueOf(table+"_clone"+i));
FSUtils.logFileSystemState(FileSystem.get(TEST_UTIL.getConfiguration()),
FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
LOG);
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), table+"_clone"+i).getScanner(new
Scan())) {
assertEquals(snapshots[i-1][count++], Bytes.toString(res.getRow()));
}
Assert.assertEquals(table+"_snapshot"+i, snapshots[i-1].length, count);
}
}
}
@Test (timeout=300000)
public void testRenameUsingSnapshots() throws Exception {
String newNS = "newNS";
TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(newNS).build());
for(TableName table: tables) {
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), table).getScanner(new
Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot3", table);
final TableName newTableName =
TableName.valueOf(newNS + TableName.NAMESPACE_DELIM + table + "_clone3");
TEST_UTIL.getHBaseAdmin().cloneSnapshot(table + "_snapshot3", newTableName);
Thread.sleep(1000);
count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath()
, LOG);
Assert.assertEquals(newTableName + "", currentKeys.length, count);
TEST_UTIL.getHBaseAdmin().flush(newTableName.toBytes());
TEST_UTIL.getHBaseAdmin().majorCompact(newTableName.toBytes());
TEST_UTIL.waitFor(30000, new Waiter.Predicate<IOException>() {
@Override
public boolean evaluate() throws IOException {
try {
return TEST_UTIL.getHBaseAdmin().getCompactionState(newTableName.toBytes()) ==
AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
} catch (InterruptedException e) {
throw new IOException(e);
}
}
});
}
String nextNS = "nextNS";
TEST_UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(nextNS).build());
for(TableName table: tables) {
TableName srcTable = TableName.valueOf(newNS + TableName.NAMESPACE_DELIM + table + "_clone3");
TEST_UTIL.getHBaseAdmin().snapshot(table + "_snapshot4", srcTable);
TableName newTableName =
TableName.valueOf(nextNS + TableName.NAMESPACE_DELIM + table + "_clone4");
TEST_UTIL.getHBaseAdmin().cloneSnapshot(table+"_snapshot4", newTableName);
FSUtils.logFileSystemState(TEST_UTIL.getTestFileSystem(), TEST_UTIL.getDefaultRootDirPath(),
LOG);
int count = 0;
for(Result res: new HTable(TEST_UTIL.getConfiguration(), newTableName).getScanner(new
Scan())) {
assertEquals(currentKeys[count++], Bytes.toString(res.getRow()));
}
Assert.assertEquals(newTableName + "", currentKeys.length, count);
}
}
@Test (timeout=300000)
public void testOldDirsAreGonePostMigration() throws IOException {
FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
List <String> dirs = new ArrayList<String>(NamespaceUpgrade.NON_USER_TABLE_DIRS);
// Remove those that are not renamed
dirs.remove(HConstants.HBCK_SIDELINEDIR_NAME);
dirs.remove(HConstants.SNAPSHOT_DIR_NAME);
dirs.remove(HConstants.HBASE_TEMP_DIRECTORY);
for (String dir: dirs) {
assertFalse(fs.exists(new Path(hbaseRootDir, dir)));
}
}
@Test (timeout=300000)
public void testNewDirsArePresentPostMigration() throws IOException {
FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
// Below list does not include 'corrupt' because there is no 'corrupt' in the tgz
String [] newdirs = new String [] {HConstants.BASE_NAMESPACE_DIR,
HConstants.HREGION_LOGDIR_NAME};
Path hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
for (String dir: newdirs) {
assertTrue(dir, fs.exists(new Path(hbaseRootDir, dir)));
}
}
@Test (timeout = 300000)
public void testACLTableMigration() throws IOException {
Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("testACLTable");
FileSystem fs = TEST_UTIL.getTestFileSystem();
Configuration conf = TEST_UTIL.getConfiguration();
byte[] FAMILY = Bytes.toBytes("l");
byte[] QUALIFIER = Bytes.toBytes("testUser");
byte[] VALUE = Bytes.toBytes("RWCA");
// Create a Region
HTableDescriptor aclTable = new HTableDescriptor(TableName.valueOf("testACLTable"));
aclTable.addFamily(new HColumnDescriptor(FAMILY));
FSTableDescriptors fstd = new FSTableDescriptors(fs, rootDir);
fstd.createTableDescriptor(aclTable);
HRegionInfo hriAcl = new HRegionInfo(aclTable.getTableName(), null, null);
HRegion region = HRegion.createHRegion(hriAcl, rootDir, conf, aclTable);
try {
// Create rows
Put p = new Put(Bytes.toBytes("-ROOT-"));
p.addImmutable(FAMILY, QUALIFIER, VALUE);
region.put(p);
p = new Put(Bytes.toBytes(".META."));
p.addImmutable(FAMILY, QUALIFIER, VALUE);
region.put(p);
p = new Put(Bytes.toBytes("_acl_"));
p.addImmutable(FAMILY, QUALIFIER, VALUE);
region.put(p);
NamespaceUpgrade upgrade = new NamespaceUpgrade();
upgrade.updateAcls(region);
// verify rows -ROOT- is removed
Get g = new Get(Bytes.toBytes("-ROOT-"));
Result r = region.get(g);
assertTrue(r == null || r.size() == 0);
// verify rows _acl_ is renamed to hbase:acl
g = new Get(AccessControlLists.ACL_TABLE_NAME.toBytes());
r = region.get(g);
assertTrue(r != null && r.size() == 1);
assertTrue(Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)) == 0);
// verify rows .META. is renamed to hbase:meta
g = new Get(TableName.META_TABLE_NAME.toBytes());
r = region.get(g);
assertTrue(r != null && r.size() == 1);
assertTrue(Bytes.compareTo(VALUE, r.getValue(FAMILY, QUALIFIER)) == 0);
} finally {
region.close();
// Delete the region
HRegionFileSystem.deleteRegionFromFileSystem(conf, fs,
FSUtils.getTableDir(rootDir, hriAcl.getTable()), hriAcl);
}
}
}

View File

@ -1,270 +0,0 @@
/**
* 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.migration;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsShell;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.io.FileLink;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.ReplicationPeer;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table.State;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileV1Detector;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.util.ToolRunner;
import org.apache.zookeeper.KeeperException;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* Upgrade to 0.96 involves detecting HFileV1 in existing cluster, updating namespace and
* updating znodes. This class tests for HFileV1 detection and upgrading znodes.
* Uprading namespace is tested in {@link TestNamespaceUpgrade}.
*/
@Category(MediumTests.class)
public class TestUpgradeTo96 {
static final Log LOG = LogFactory.getLog(TestUpgradeTo96.class);
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
/**
* underlying file system instance
*/
private static FileSystem fs;
/**
* hbase root dir
*/
private static Path hbaseRootDir;
private static ZooKeeperWatcher zkw;
/**
* replication peer znode (/hbase/replication/peers)
*/
private static String replicationPeerZnode;
/**
* znode of a table
*/
private static String tableAZnode;
private static ReplicationPeer peer1;
/**
* znode for replication peer1 (/hbase/replication/peers/1)
*/
private static String peer1Znode;
@BeforeClass
public static void setUpBeforeClass() throws Exception {
// Start up the mini cluster on top of an 0.94 root.dir that has data from
// a 0.94 hbase run and see if we can migrate to 0.96
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.startMiniDFSCluster(1);
hbaseRootDir = TEST_UTIL.getDefaultRootDirPath();
fs = FileSystem.get(TEST_UTIL.getConfiguration());
FSUtils.setRootDir(TEST_UTIL.getConfiguration(), hbaseRootDir);
zkw = TEST_UTIL.getZooKeeperWatcher();
Path testdir = TEST_UTIL.getDataTestDir("TestUpgradeTo96");
// get the untar 0.94 file structure
set94FSLayout(testdir);
setUp94Znodes();
}
/**
* Lays out 0.94 file system layout using {@link TestNamespaceUpgrade} apis.
* @param testdir
* @throws IOException
* @throws Exception
*/
private static void set94FSLayout(Path testdir) throws IOException, Exception {
File untar = TestNamespaceUpgrade.untar(new File(testdir.toString()));
if (!fs.exists(hbaseRootDir.getParent())) {
// mkdir at first
fs.mkdirs(hbaseRootDir.getParent());
}
FsShell shell = new FsShell(TEST_UTIL.getConfiguration());
shell.run(new String[] { "-put", untar.toURI().toString(), hbaseRootDir.toString() });
// See whats in minihdfs.
shell.run(new String[] { "-lsr", "/" });
}
/**
* Sets znodes used in 0.94 version. Only table and replication znodes will be upgraded to PB,
* others would be deleted.
* @throws KeeperException
*/
private static void setUp94Znodes() throws IOException, KeeperException {
// add some old znodes, which would be deleted after upgrade.
String rootRegionServerZnode = ZKUtil.joinZNode(zkw.baseZNode, "root-region-server");
ZKUtil.createWithParents(zkw, rootRegionServerZnode);
ZKUtil.createWithParents(zkw, zkw.backupMasterAddressesZNode);
// add table znode, data of its children would be protobuffized
tableAZnode = ZKUtil.joinZNode(zkw.tableZNode, "a");
ZKUtil.createWithParents(zkw, tableAZnode,
Bytes.toBytes(ZooKeeperProtos.Table.State.ENABLED.toString()));
// add replication znodes, data of its children would be protobuffized
String replicationZnode = ZKUtil.joinZNode(zkw.baseZNode, "replication");
replicationPeerZnode = ZKUtil.joinZNode(replicationZnode, "peers");
peer1Znode = ZKUtil.joinZNode(replicationPeerZnode, "1");
peer1 = ReplicationPeer.newBuilder().setClusterkey("abc:123:/hbase").build();
ZKUtil.createWithParents(zkw, peer1Znode, Bytes.toBytes(peer1.getClusterkey()));
}
/**
* Tests a 0.94 filesystem for any HFileV1.
* @throws Exception
*/
@Test
public void testHFileV1Detector() throws Exception {
assertEquals(0, ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), null));
}
/**
* Creates a corrupt file, and run HFileV1 detector tool
* @throws Exception
*/
@Test
public void testHFileV1DetectorWithCorruptFiles() throws Exception {
// add a corrupt file.
Path tablePath = new Path(hbaseRootDir, "foo");
FileStatus[] regionsDir = fs.listStatus(tablePath);
if (regionsDir == null) throw new IOException("No Regions found for table " + "foo");
Path columnFamilyDir = null;
Path targetRegion = null;
for (FileStatus s : regionsDir) {
if (fs.exists(new Path(s.getPath(), HRegionFileSystem.REGION_INFO_FILE))) {
targetRegion = s.getPath();
break;
}
}
FileStatus[] cfs = fs.listStatus(targetRegion);
for (FileStatus f : cfs) {
if (f.isDirectory()) {
columnFamilyDir = f.getPath();
break;
}
}
LOG.debug("target columnFamilyDir: " + columnFamilyDir);
// now insert a corrupt file in the columnfamily.
Path corruptFile = new Path(columnFamilyDir, "corrupt_file");
if (!fs.createNewFile(corruptFile)) throw new IOException("Couldn't create corrupt file: "
+ corruptFile);
assertEquals(1, ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), null));
// remove the corrupt file
FileSystem.get(TEST_UTIL.getConfiguration()).delete(corruptFile, false);
}
@Test
public void testHFileLink() throws Exception {
// pass a link, and verify that correct paths are returned.
Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration());
Path aFileLink = new Path(rootDir, "table/2086db948c48/cf/table=21212abcdc33-0906db948c48");
Path preNamespaceTablePath = new Path(rootDir, "table/21212abcdc33/cf/0906db948c48");
Path preNamespaceArchivePath =
new Path(rootDir, ".archive/table/21212abcdc33/cf/0906db948c48");
Path preNamespaceTempPath = new Path(rootDir, ".tmp/table/21212abcdc33/cf/0906db948c48");
boolean preNSTablePathExists = false;
boolean preNSArchivePathExists = false;
boolean preNSTempPathExists = false;
assertTrue(HFileLink.isHFileLink(aFileLink));
HFileLink hFileLink = new HFileLink(TEST_UTIL.getConfiguration(), aFileLink);
assertTrue(hFileLink.getArchivePath().toString().startsWith(rootDir.toString()));
HFileV1Detector t = new HFileV1Detector();
t.setConf(TEST_UTIL.getConfiguration());
FileLink fileLink = t.getFileLinkWithPreNSPath(aFileLink);
//assert it has 6 paths (2 NS, 2 Pre NS, and 2 .tmp) to look.
assertTrue(fileLink.getLocations().length == 6);
for (Path p : fileLink.getLocations()) {
if (p.equals(preNamespaceArchivePath)) preNSArchivePathExists = true;
if (p.equals(preNamespaceTablePath)) preNSTablePathExists = true;
if (p.equals(preNamespaceTempPath)) preNSTempPathExists = true;
}
assertTrue(preNSArchivePathExists & preNSTablePathExists & preNSTempPathExists);
}
@Test
public void testADirForHFileV1() throws Exception {
Path tablePath = new Path(hbaseRootDir, "foo");
System.out.println("testADirForHFileV1: " + tablePath.makeQualified(fs));
System.out.println("Passed: " + hbaseRootDir + "/foo");
assertEquals(0,
ToolRunner.run(TEST_UTIL.getConfiguration(), new HFileV1Detector(), new String[] { "-p"
+ "foo" }));
}
@Test
public void testZnodeMigration() throws Exception {
String rootRSZnode = ZKUtil.joinZNode(zkw.baseZNode, "root-region-server");
assertTrue(ZKUtil.checkExists(zkw, rootRSZnode) > -1);
ToolRunner.run(TEST_UTIL.getConfiguration(), new UpgradeTo96(), new String[] { "-execute" });
assertEquals(-1, ZKUtil.checkExists(zkw, rootRSZnode));
byte[] data = ZKUtil.getData(zkw, tableAZnode);
assertTrue(ProtobufUtil.isPBMagicPrefix(data));
checkTableState(data, ZooKeeperProtos.Table.State.ENABLED);
// ensure replication znodes are there, and protobuffed.
data = ZKUtil.getData(zkw, peer1Znode);
assertTrue(ProtobufUtil.isPBMagicPrefix(data));
checkReplicationPeerData(data, peer1);
}
private void checkTableState(byte[] data, State expectedState)
throws InvalidProtocolBufferException {
ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
int magicLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
assertTrue(t.getState() == expectedState);
}
private void checkReplicationPeerData(byte[] data, ReplicationPeer peer)
throws InvalidProtocolBufferException {
int magicLen = ProtobufUtil.lengthOfPBMagic();
ZooKeeperProtos.ReplicationPeer.Builder builder = ZooKeeperProtos.ReplicationPeer.newBuilder();
assertEquals(builder.mergeFrom(data, magicLen, data.length - magicLen).build().getClusterkey(),
peer.getClusterkey());
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniHBaseCluster();
TEST_UTIL.shutdownMiniDFSCluster();
TEST_UTIL.shutdownMiniZKCluster();
}
}

View File

@ -34,16 +34,15 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.HConnectionManager;
@ -52,14 +51,13 @@ import org.apache.hadoop.hbase.client.MetaScanner;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.StoppableImplementation;
@ -74,6 +72,7 @@ import com.google.common.collect.Sets;
import com.google.protobuf.ServiceException;
@Category(LargeTests.class)
@SuppressWarnings("deprecation")
public class TestEndToEndSplitTransaction {
private static final Log LOG = LogFactory.getLog(TestEndToEndSplitTransaction.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@ -109,7 +108,6 @@ public class TestEndToEndSplitTransaction {
.getRegionName();
HRegion region = server.getRegion(regionName);
SplitTransaction split = new SplitTransaction(region, splitRow);
split.useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
split.prepare();
// 1. phase I
@ -126,14 +124,10 @@ public class TestEndToEndSplitTransaction {
// 3. finish phase II
// note that this replicates some code from SplitTransaction
// 2nd daughter first
if (split.useZKForAssignment) {
server.postOpenDeployTasks(regions.getSecond());
} else {
server.reportRegionStateTransition(
RegionServerStatusProtos.RegionStateTransition.TransitionCode.SPLIT,
region.getRegionInfo(), regions.getFirst().getRegionInfo(),
regions.getSecond().getRegionInfo());
}
// Add to online regions
server.addToOnlineRegions(regions.getSecond());
@ -143,21 +137,11 @@ public class TestEndToEndSplitTransaction {
// past splitkey is ok.
assertTrue(test(con, tableName, lastRow, server));
// first daughter second
if (split.useZKForAssignment) {
server.postOpenDeployTasks(regions.getFirst());
}
// Add to online regions
server.addToOnlineRegions(regions.getFirst());
assertTrue(test(con, tableName, firstRow, server));
assertTrue(test(con, tableName, lastRow, server));
if (split.useZKForAssignment) {
// 4. phase III
((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
.getSplitTransactionCoordination().completeSplitTransaction(server, regions.getFirst(),
regions.getSecond(), split.std, region);
}
assertTrue(test(con, tableName, firstRow, server));
assertTrue(test(con, tableName, lastRow, server));
}
@ -341,7 +325,7 @@ public class TestEndToEndSplitTransaction {
verifyStartEndKeys(keys);
//HTable.getRegionsInfo()
Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
Map<HRegionInfo, ServerName> regions = table.getRegionLocations();
verifyTableRegions(regions.keySet());
} finally {
IOUtils.closeQuietly(table);

View File

@ -37,13 +37,12 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@ -94,7 +93,8 @@ public class TestRegionMergeTransactionOnCluster {
private static HMaster master;
private static Admin admin;
static void setupOnce() throws Exception {
@BeforeClass
public static void beforeAllTests() throws Exception {
// Start a cluster
TEST_UTIL.startMiniCluster(NB_SERVERS);
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
@ -103,13 +103,6 @@ public class TestRegionMergeTransactionOnCluster {
admin = TEST_UTIL.getHBaseAdmin();
}
@BeforeClass
public static void beforeAllTests() throws Exception {
// Use ZK for region assignment
TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
setupOnce();
}
@AfterClass
public static void afterAllTests() throws Exception {
TEST_UTIL.shutdownMiniCluster();
@ -148,7 +141,7 @@ public class TestRegionMergeTransactionOnCluster {
}
// We should not be able to assign it again
am.assign(hri, true, true);
am.assign(hri, true);
assertFalse("Merged region can't be assigned",
regionStates.isRegionInTransition(hri));
assertTrue(regionStates.isRegionInState(hri, State.MERGED));
@ -162,6 +155,7 @@ public class TestRegionMergeTransactionOnCluster {
table.close();
}
@SuppressWarnings("deprecation")
@Test
public void testCleanMergeReference() throws Exception {
LOG.info("Starting testCleanMergeReference");

View File

@ -45,9 +45,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.util.StringUtils;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -94,26 +92,18 @@ public class TestRegionReplicas {
@AfterClass
public static void afterClass() throws Exception {
HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
table.close();
HTU.shutdownMiniCluster();
}
@After
public void after() throws Exception {
// Clean the state if the test failed before cleaning the znode
// It does not manage all bad failures, so if there are multiple failures, only
// the first one should be looked at.
ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hriPrimary);
}
private HRegionServer getRS() {
return HTU.getMiniHBaseCluster().getRegionServer(0);
}
private void openRegion(HRegionInfo hri) throws Exception {
ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
// first version is '0'
AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, 0, null, null);
AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, null, null);
AdminProtos.OpenRegionResponse responseOpen = getRS().getRSRpcServices().openRegion(null, orr);
Assert.assertTrue(responseOpen.getOpeningStateCount() == 1);
Assert.assertTrue(responseOpen.getOpeningState(0).
@ -122,16 +112,12 @@ public class TestRegionReplicas {
}
private void closeRegion(HRegionInfo hri) throws Exception {
ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(getRS().getServerName(),
hri.getEncodedName(), true);
hri.getEncodedName());
AdminProtos.CloseRegionResponse responseClose = getRS().getRSRpcServices().closeRegion(null, crr);
Assert.assertTrue(responseClose.getClosed());
checkRegionIsClosed(hri.getEncodedName());
ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(), getRS().getServerName());
}
private void checkRegionIsOpened(String encodedRegionName) throws Exception {
@ -141,9 +127,6 @@ public class TestRegionReplicas {
}
Assert.assertTrue(getRS().getRegionByEncodedName(encodedRegionName).isAvailable());
Assert.assertTrue(
ZKAssign.deleteOpenedNode(HTU.getZooKeeperWatcher(), encodedRegionName, getRS().getServerName()));
}

View File

@ -26,29 +26,22 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@ -77,7 +70,6 @@ public class TestRegionServerNoMaster {
@BeforeClass
public static void before() throws Exception {
HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
HTU.startMiniCluster(NB_SERVERS);
final byte[] tableName = Bytes.toBytes(TestRegionServerNoMaster.class.getSimpleName());
@ -94,27 +86,37 @@ public class TestRegionServerNoMaster {
}
public static void stopMasterAndAssignMeta(HBaseTestingUtility HTU)
throws NodeExistsException, KeeperException, IOException, InterruptedException {
// No master
HTU.getHBaseCluster().getMaster().stopMaster();
throws IOException, InterruptedException {
// Stop master
HMaster master = HTU.getHBaseCluster().getMaster();
ServerName masterAddr = master.getServerName();
master.stopMaster();
Log.info("Waiting until master thread exits");
while (HTU.getHBaseCluster().getMasterThread() != null
&& HTU.getHBaseCluster().getMasterThread().isAlive()) {
Threads.sleep(100);
}
HRegionServer.TEST_SKIP_REPORTING_TRANSITION = true;
// Master is down, so is the meta. We need to assign it somewhere
// so that regions can be assigned during the mocking phase.
HRegionServer hrs = HTU.getHBaseCluster().getRegionServer(0);
HRegionServer hrs = HTU.getHBaseCluster()
.getLiveRegionServerThreads().get(0).getRegionServer();
ZooKeeperWatcher zkw = hrs.getZooKeeper();
ZKAssign.createNodeOffline(
zkw, HRegionInfo.FIRST_META_REGIONINFO, hrs.getServerName());
MetaTableLocator mtl = new MetaTableLocator();
ServerName sn = mtl.getMetaRegionLocation(zkw);
if (sn != null && !masterAddr.equals(sn)) {
return;
}
ProtobufUtil.openRegion(hrs.getRSRpcServices(),
hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
MetaTableLocator mtl = new MetaTableLocator();
while (true) {
ServerName sn = mtl.getMetaRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())) {
sn = mtl.getMetaRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())
&& hrs.onlineRegions.containsKey(
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
break;
}
Thread.sleep(100);
@ -135,19 +137,11 @@ public class TestRegionServerNoMaster {
@AfterClass
public static void afterClass() throws Exception {
HRegionServer.TEST_SKIP_REPORTING_TRANSITION = false;
table.close();
HTU.shutdownMiniCluster();
}
@After
public void after() throws Exception {
// Clean the state if the test failed before cleaning the znode
// It does not manage all bad failures, so if there are multiple failures, only
// the first one should be looked at.
ZKAssign.deleteNodeFailSilent(HTU.getZooKeeperWatcher(), hri);
}
private static HRegionServer getRS() {
return HTU.getHBaseCluster().getLiveRegionServerThreads().get(0).getRegionServer();
}
@ -157,11 +151,8 @@ public class TestRegionServerNoMaster {
* Reopen the region. Reused in multiple tests as we always leave the region open after a test.
*/
private void reopenRegion() throws Exception {
// We reopen. We need a ZK node here, as a open is always triggered by a master.
ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
// first version is '0'
AdminProtos.OpenRegionRequest orr =
RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, 0, null, null);
RequestConverter.buildOpenRegionRequest(getRS().getServerName(), hri, null, null);
AdminProtos.OpenRegionResponse responseOpen = getRS().rpcServices.openRegion(null, orr);
Assert.assertTrue(responseOpen.getOpeningStateCount() == 1);
Assert.assertTrue(responseOpen.getOpeningState(0).
@ -178,10 +169,6 @@ public class TestRegionServerNoMaster {
}
Assert.assertTrue(getRS().getRegion(regionName).isAvailable());
Assert.assertTrue(
ZKAssign.deleteOpenedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(),
getRS().getServerName()));
}
@ -196,8 +183,6 @@ public class TestRegionServerNoMaster {
} catch (NotServingRegionException expected) {
// That's how it work: if the region is closed we have an exception.
}
// We don't delete the znode here, because there is not always a znode.
}
@ -207,7 +192,7 @@ public class TestRegionServerNoMaster {
private void closeNoZK() throws Exception {
// no transition in ZK
AdminProtos.CloseRegionRequest crr =
RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, false);
RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName);
AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
Assert.assertTrue(responseClose.getClosed());
@ -222,42 +207,6 @@ public class TestRegionServerNoMaster {
reopenRegion();
}
@Test(timeout = 60000)
public void testCloseByMasterWithoutZNode() throws Exception {
// Transition in ZK on. This should fail, as there is no znode
AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(
getRS().getServerName(), regionName, true);
AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
Assert.assertTrue(responseClose.getClosed());
// now waiting. After a while, the transition should be done
while (!getRS().getRegionsInTransitionInRS().isEmpty()) {
Thread.sleep(1);
}
// the region is still available, the close got rejected at the end
Assert.assertTrue("The close should have failed", getRS().getRegion(regionName).isAvailable());
}
@Test(timeout = 60000)
public void testOpenCloseByMasterWithZNode() throws Exception {
ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
AdminProtos.CloseRegionRequest crr = RequestConverter.buildCloseRegionRequest(
getRS().getServerName(), regionName, true);
AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
Assert.assertTrue(responseClose.getClosed());
checkRegionIsClosed();
ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(),
getRS().getServerName());
reopenRegion();
}
/**
* Test that we can send multiple openRegion to the region server.
* This is used when:
@ -275,13 +224,10 @@ public class TestRegionServerNoMaster {
closeNoZK();
checkRegionIsClosed();
// We reopen. We need a ZK node here, as a open is always triggered by a master.
ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
// We're sending multiple requests in a row. The region server must handle this nicely.
for (int i = 0; i < 10; i++) {
AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
getRS().getServerName(), hri, 0, null, null);
getRS().getServerName(), hri, null, null);
AdminProtos.OpenRegionResponse responseOpen = getRS().rpcServices.openRegion(null, orr);
Assert.assertTrue(responseOpen.getOpeningStateCount() == 1);
@ -307,7 +253,7 @@ public class TestRegionServerNoMaster {
// fake region to be closing now, need to clear state afterwards
getRS().regionsInTransitionInRS.put(hri.getEncodedNameAsBytes(), Boolean.FALSE);
AdminProtos.OpenRegionRequest orr =
RequestConverter.buildOpenRegionRequest(sn, hri, 0, null, null);
RequestConverter.buildOpenRegionRequest(sn, hri, null, null);
getRS().rpcServices.openRegion(null, orr);
Assert.fail("The closing region should not be opened");
} catch (ServiceException se) {
@ -320,12 +266,9 @@ public class TestRegionServerNoMaster {
@Test(timeout = 60000)
public void testMultipleCloseFromMaster() throws Exception {
// As opening, we must support multiple requests on the same region
ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
for (int i = 0; i < 10; i++) {
AdminProtos.CloseRegionRequest crr =
RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, 0, null, true);
RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, null);
try {
AdminProtos.CloseRegionResponse responseClose = getRS().rpcServices.closeRegion(null, crr);
Assert.assertEquals("The first request should succeeds", 0, i);
@ -338,11 +281,6 @@ public class TestRegionServerNoMaster {
checkRegionIsClosed();
Assert.assertTrue(
ZKAssign.deleteClosedNode(HTU.getZooKeeperWatcher(), hri.getEncodedName(),
getRS().getServerName())
);
reopenRegion();
}
@ -356,12 +294,11 @@ public class TestRegionServerNoMaster {
checkRegionIsClosed();
// Let do the initial steps, without having a handler
ZKAssign.createNodeOffline(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
getRS().getRegionsInTransitionInRS().put(hri.getEncodedNameAsBytes(), Boolean.TRUE);
// That's a close without ZK.
AdminProtos.CloseRegionRequest crr =
RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, false);
RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName);
try {
getRS().rpcServices.closeRegion(null, crr);
Assert.assertTrue(false);
@ -375,89 +312,11 @@ public class TestRegionServerNoMaster {
// Let's start the open handler
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
BaseCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(getRS());
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(getRS().getServerName());
zkCrd.setVersionOfOfflineNode(0);
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd,
csm.getOpenRegionCoordination(), zkCrd));
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd));
// The open handler should have removed the region from RIT but kept the region closed
checkRegionIsClosed();
// The open handler should have updated the value in ZK.
Assert.assertTrue(ZKAssign.deleteNode(
getRS().getZooKeeper(), hri.getEncodedName(),
EventType.RS_ZK_REGION_FAILED_OPEN, 1)
);
reopenRegion();
}
/**
* Test an open then a close with ZK. This is going to mess-up the ZK states, so
* the opening will fail as well because it doesn't find what it expects in ZK.
*/
@Test(timeout = 60000)
public void testCancelOpeningWithZK() throws Exception {
// We close
closeNoZK();
checkRegionIsClosed();
// Let do the initial steps, without having a handler
getRS().getRegionsInTransitionInRS().put(hri.getEncodedNameAsBytes(), Boolean.TRUE);
// That's a close without ZK.
ZKAssign.createNodeClosing(HTU.getZooKeeperWatcher(), hri, getRS().getServerName());
AdminProtos.CloseRegionRequest crr =
RequestConverter.buildCloseRegionRequest(getRS().getServerName(), regionName, false);
try {
getRS().rpcServices.closeRegion(null, crr);
Assert.assertTrue(false);
} catch (ServiceException expected) {
Assert.assertTrue(expected.getCause() instanceof NotServingRegionException);
}
// The close should have left the ZK state as it is: it's the job the AM to delete it
Assert.assertTrue(ZKAssign.deleteNode(
getRS().getZooKeeper(), hri.getEncodedName(),
EventType.M_ZK_REGION_CLOSING, 0)
);
// The state in RIT should have changed to close
Assert.assertEquals(Boolean.FALSE, getRS().getRegionsInTransitionInRS().get(
hri.getEncodedNameAsBytes()));
// Let's start the open handler
// It should not succeed for two reasons:
// 1) There is no ZK node
// 2) The region in RIT was changed.
// The order is more or less implementation dependant.
HTableDescriptor htd = getRS().tableDescriptors.get(hri.getTable());
BaseCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(getRS());
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(getRS().getServerName());
zkCrd.setVersionOfOfflineNode(0);
getRS().service.submit(new OpenRegionHandler(getRS(), getRS(), hri, htd,
csm.getOpenRegionCoordination(), zkCrd));
// The open handler should have removed the region from RIT but kept the region closed
checkRegionIsClosed();
// We should not find any znode here.
Assert.assertEquals(-1, ZKAssign.getVersion(HTU.getZooKeeperWatcher(), hri));
reopenRegion();
}
@ -473,7 +332,7 @@ public class TestRegionServerNoMaster {
ServerName earlierServerName = ServerName.valueOf(sn.getHostname(), sn.getPort(), 1);
try {
CloseRegionRequest request = RequestConverter.buildCloseRegionRequest(earlierServerName, regionName, true);
CloseRegionRequest request = RequestConverter.buildCloseRegionRequest(earlierServerName, regionName);
getRS().getRSRpcServices().closeRegion(null, request);
Assert.fail("The closeRegion should have been rejected");
} catch (ServiceException se) {
@ -485,7 +344,7 @@ public class TestRegionServerNoMaster {
closeNoZK();
try {
AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
earlierServerName, hri, 0, null, null);
earlierServerName, hri, null, null);
getRS().getRSRpcServices().openRegion(null, orr);
Assert.fail("The openRegion should have been rejected");
} catch (ServiceException se) {

View File

@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -48,15 +47,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.LargeTests;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -68,14 +66,10 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.TestReplicasClient.SlowMeCopro;
import org.apache.hadoop.hbase.coordination.ZKSplitTransactionCoordination;
import org.apache.hadoop.hbase.coordination.ZkCloseRegionCoordination;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.RegionState;
@ -90,12 +84,9 @@ import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.apache.zookeeper.data.Stat;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@ -120,24 +111,13 @@ public class TestSplitTransactionOnCluster {
private static final int NB_SERVERS = 3;
private static CountDownLatch latch = new CountDownLatch(1);
private static volatile boolean secondSplit = false;
private static volatile boolean callRollBack = false;
private static volatile boolean firstSplitCompleted = false;
private static boolean useZKForAssignment = true;
static final HBaseTestingUtility TESTING_UTIL =
new HBaseTestingUtility();
static void setupOnce() throws Exception {
TESTING_UTIL.getConfiguration().setInt("hbase.balancer.period", 60000);
useZKForAssignment = TESTING_UTIL.getConfiguration().getBoolean(
"hbase.assignment.usezk", false);
TESTING_UTIL.startMiniCluster(NB_SERVERS);
}
@BeforeClass public static void before() throws Exception {
// Use ZK for region assignment
TESTING_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", true);
setupOnce();
TESTING_UTIL.getConfiguration().setInt("hbase.balancer.period", 60000);
TESTING_UTIL.startMiniCluster(NB_SERVERS);
}
@AfterClass public static void after() throws Exception {
@ -181,100 +161,6 @@ public class TestSplitTransactionOnCluster {
return hri;
}
@SuppressWarnings("deprecation")
@Test(timeout = 60000)
public void testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack() throws Exception {
final TableName tableName =
TableName.valueOf("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack");
if (!useZKForAssignment) {
// This test doesn't apply if not using ZK for assignment
return;
}
try {
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName.getName(), Bytes.toBytes("cf"));
final List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
final HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
insertData(tableName.getName(), admin, t);
t.close();
// Turn off balancer so it doesn't cut in and mess up our placements.
this.admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
// find a splittable region
final HRegion region = findSplittableRegion(regions);
assertTrue("not able to find a splittable region", region != null);
MockedCoordinatedStateManager cp = new MockedCoordinatedStateManager();
cp.initialize(regionServer, region);
cp.start();
regionServer.csm = cp;
new Thread() {
@Override
public void run() {
SplitTransaction st = null;
st = new MockedSplitTransaction(region, Bytes.toBytes("row2"));
try {
st.prepare();
st.execute(regionServer, regionServer);
} catch (IOException e) {
}
}
}.start();
for (int i = 0; !callRollBack && i < 100; i++) {
Thread.sleep(100);
}
assertTrue("Waited too long for rollback", callRollBack);
SplitTransaction st = new MockedSplitTransaction(region, Bytes.toBytes("row3"));
try {
secondSplit = true;
// make region splittable
region.initialize();
st.prepare();
st.execute(regionServer, regionServer);
} catch (IOException e) {
LOG.debug("Rollback started :"+ e.getMessage());
st.rollback(regionServer, regionServer);
}
for (int i=0; !firstSplitCompleted && i<100; i++) {
Thread.sleep(100);
}
assertTrue("fist split did not complete", firstSplitCompleted);
RegionStates regionStates = cluster.getMaster().getAssignmentManager().getRegionStates();
Map<String, RegionState> rit = regionStates.getRegionsInTransition();
for (int i=0; rit.containsKey(hri.getTable()) && i<100; i++) {
Thread.sleep(100);
}
assertFalse("region still in transition", rit.containsKey(
rit.containsKey(hri.getTable())));
List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName);
// Region server side split is successful.
assertEquals("The parent region should be splitted", 2, onlineRegions.size());
//Should be present in RIT
List<HRegionInfo> regionsOfTable = cluster.getMaster().getAssignmentManager()
.getRegionStates().getRegionsOfTable(tableName);
// Master side should also reflect the same
assertEquals("No of regions in master", 2, regionsOfTable.size());
} finally {
admin.setBalancerRunning(true, false);
secondSplit = false;
firstSplitCompleted = false;
callRollBack = false;
cluster.getMaster().setCatalogJanitorEnabled(true);
TESTING_UTIL.deleteTable(tableName);
}
}
@Test(timeout = 60000)
public void testRITStateForRollback() throws Exception {
final TableName tableName =
@ -382,20 +268,15 @@ public class TestSplitTransactionOnCluster {
/**
* A test that intentionally has master fail the processing of the split message.
* Tests that the regionserver split ephemeral node gets cleaned up if it
* crashes and that after we process server shutdown, the daughters are up on
* line.
* Tests that after we process server shutdown, the daughters are up on line.
* @throws IOException
* @throws InterruptedException
* @throws NodeExistsException
* @throws KeeperException
* @throws DeserializationException
* @throws ServiceException
*/
@Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
throws IOException, InterruptedException, NodeExistsException, KeeperException,
DeserializationException, ServiceException {
@Test (timeout = 300000) public void testRSSplitDaughtersAreOnlinedAfterShutdownHandling()
throws IOException, InterruptedException, ServiceException {
final byte [] tableName =
Bytes.toBytes("testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling");
Bytes.toBytes("testRSSplitDaughtersAreOnlinedAfterShutdownHandling");
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
@ -418,48 +299,15 @@ public class TestSplitTransactionOnCluster {
// Now, before we split, set special flag in master, a flag that has
// it FAIL the processing of split.
AssignmentManager.TEST_SKIP_SPLIT_HANDLING = true;
// Now try splitting and it should work.
split(hri, server, regionCount);
String path = ZKAssign.getNodeName(TESTING_UTIL.getZooKeeperWatcher(),
hri.getEncodedName());
RegionTransition rt = null;
Stat stats = null;
List<HRegion> daughters = null;
if (useZKForAssignment) {
daughters = checkAndGetDaughters(tableName);
// Wait till the znode moved to SPLIT
for (int i=0; i<100; i++) {
stats = TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
rt = RegionTransition.parseFrom(ZKAssign.getData(TESTING_UTIL.getZooKeeperWatcher(),
hri.getEncodedName()));
if (rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT)) break;
Thread.sleep(100);
}
LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats=" + stats);
assertTrue(rt != null && rt.getEventType().equals(EventType.RS_ZK_REGION_SPLIT));
// Now crash the server, for ZK-less assignment, the server is auto aborted
cluster.abortRegionServer(tableRegionIndex);
try {
// Now try splitting and it should work.
split(hri, server, regionCount);
} catch (RegionServerStoppedException rsse) {
// Expected. The regionserver should crash
}
waitUntilRegionServerDead();
awaitDaughters(tableName, 2);
if (useZKForAssignment) {
regions = cluster.getRegions(tableName);
for (HRegion r: regions) {
assertTrue(daughters.contains(r));
}
// Finally assert that the ephemeral SPLIT znode was cleaned up.
for (int i=0; i<100; i++) {
// wait a bit (10s max) for the node to disappear
stats = TESTING_UTIL.getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
if (stats == null) break;
Thread.sleep(100);
}
LOG.info("EPHEMERAL NODE AFTER SERVER ABORT, path=" + path + ", stats=" + stats);
assertTrue(stats == null);
}
} finally {
// Set this flag back.
AssignmentManager.TEST_SKIP_SPLIT_HANDLING = false;
@ -495,15 +343,8 @@ public class TestSplitTransactionOnCluster {
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
printOutRegions(server, "Initial regions: ");
int regionCount = ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size();
// Insert into zk a blocking znode, a znode of same name as region
// so it gets in way of our splitting.
ServerName fakedServer = ServerName.valueOf("any.old.server", 1234, -1);
if (useZKForAssignment) {
ZKAssign.createNodeClosing(TESTING_UTIL.getZooKeeperWatcher(),
hri, fakedServer);
} else {
regionStates.updateRegionState(hri, RegionState.State.CLOSING);
}
regionStates.updateRegionState(hri, RegionState.State.CLOSING);
// Now try splitting.... should fail. And each should successfully
// rollback.
this.admin.split(hri.getRegionNameAsString());
@ -515,13 +356,8 @@ public class TestSplitTransactionOnCluster {
assertEquals(regionCount, ProtobufUtil.getOnlineRegions(
server.getRSRpcServices()).size());
}
if (useZKForAssignment) {
// Now clear the zknode
ZKAssign.deleteClosingNode(TESTING_UTIL.getZooKeeperWatcher(),
hri, fakedServer);
} else {
regionStates.regionOnline(hri, server.getServerName());
}
regionStates.regionOnline(hri, server.getServerName());
// Now try splitting and it should work.
split(hri, server, regionCount);
// Get daughters
@ -541,7 +377,7 @@ public class TestSplitTransactionOnCluster {
* @throws InterruptedException
*/
@Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
throws IOException, InterruptedException, ServiceException {
throws IOException, InterruptedException {
final byte [] tableName =
Bytes.toBytes("testShutdownFixupWhenDaughterHasSplit");
@ -697,103 +533,6 @@ public class TestSplitTransactionOnCluster {
}
}
/**
* Verifies HBASE-5806. When splitting is partially done and the master goes down
* when the SPLIT node is in either SPLIT or SPLITTING state.
*
* @throws IOException
* @throws InterruptedException
* @throws NodeExistsException
* @throws KeeperException
* @throws DeserializationException
*/
@Test(timeout = 400000)
public void testMasterRestartWhenSplittingIsPartial()
throws IOException, InterruptedException, NodeExistsException,
KeeperException, DeserializationException, ServiceException {
final byte[] tableName = Bytes.toBytes("testMasterRestartWhenSplittingIsPartial");
if (!useZKForAssignment) {
// This test doesn't apply if not using ZK for assignment
return;
}
// Create table then get the single region for our new table.
HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
List<HRegion> regions = cluster.getRegions(tableName);
HRegionInfo hri = getAndCheckSingleTableRegion(regions);
int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
// Turn off balancer so it doesn't cut in and mess up our placements.
this.admin.setBalancerRunning(false, true);
// Turn off the meta scanner so it don't remove parent on us.
cluster.getMaster().setCatalogJanitorEnabled(false);
ZooKeeperWatcher zkw = new ZooKeeperWatcher(t.getConfiguration(),
"testMasterRestartWhenSplittingIsPartial", new UselessTestAbortable());
try {
// Add a bit of load up into the table so splittable.
TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY, false);
// Get region pre-split.
HRegionServer server = cluster.getRegionServer(tableRegionIndex);
printOutRegions(server, "Initial regions: ");
// Now, before we split, set special flag in master, a flag that has
// it FAIL the processing of split.
AssignmentManager.TEST_SKIP_SPLIT_HANDLING = true;
// Now try splitting and it should work.
this.admin.split(hri.getRegionNameAsString());
checkAndGetDaughters(tableName);
// Assert the ephemeral node is up in zk.
String path = ZKAssign.getNodeName(zkw, hri.getEncodedName());
Stat stats = zkw.getRecoverableZooKeeper().exists(path, false);
LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
+ stats);
byte[] bytes = ZKAssign.getData(zkw, hri.getEncodedName());
RegionTransition rtd = RegionTransition.parseFrom(bytes);
// State could be SPLIT or SPLITTING.
assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT)
|| rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
// abort and wait for new master.
MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
// Update the region to be offline and split, so that HRegionInfo#equals
// returns true in checking rebuilt region states map.
hri.setOffline(true);
hri.setSplit(true);
ServerName regionServerOfRegion = master.getAssignmentManager()
.getRegionStates().getRegionServerOfRegion(hri);
assertTrue(regionServerOfRegion != null);
// Remove the block so that split can move ahead.
AssignmentManager.TEST_SKIP_SPLIT_HANDLING = false;
String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
Stat stat = new Stat();
byte[] data = ZKUtil.getDataNoWatch(zkw, node, stat);
// ZKUtil.create
for (int i=0; data != null && i<60; i++) {
Thread.sleep(1000);
data = ZKUtil.getDataNoWatch(zkw, node, stat);
}
assertNull("Waited too long for ZK node to be removed: "+node, data);
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
assertTrue("Split parent should be in SPLIT state",
regionStates.isRegionInState(hri, State.SPLIT));
regionServerOfRegion = regionStates.getRegionServerOfRegion(hri);
assertTrue(regionServerOfRegion == null);
} finally {
// Set this flag back.
AssignmentManager.TEST_SKIP_SPLIT_HANDLING = false;
admin.setBalancerRunning(true, false);
cluster.getMaster().setCatalogJanitorEnabled(true);
t.close();
zkw.close();
}
}
/**
* Verifies HBASE-5806. Here the case is that splitting is completed but before the
* CJ could remove the parent region the master is killed and restarted.
@ -830,20 +569,6 @@ public class TestSplitTransactionOnCluster {
this.admin.split(hri.getRegionNameAsString());
checkAndGetDaughters(tableName);
// Assert the ephemeral node is up in zk.
String path = ZKAssign.getNodeName(zkw, hri.getEncodedName());
Stat stats = zkw.getRecoverableZooKeeper().exists(path, false);
LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
+ stats);
String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
Stat stat = new Stat();
byte[] data = ZKUtil.getDataNoWatch(zkw, node, stat);
// ZKUtil.create
for (int i=0; data != null && i<60; i++) {
Thread.sleep(1000);
data = ZKUtil.getDataNoWatch(zkw, node, stat);
}
assertNull("Waited too long for ZK node to be removed: "+node, data);
MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
@ -885,7 +610,6 @@ public class TestSplitTransactionOnCluster {
@Test(timeout = 60000)
public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TESTING_UTIL);
final TableName tableName =
TableName.valueOf("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
// Create table then get the single region for our new table.
@ -916,11 +640,6 @@ public class TestSplitTransactionOnCluster {
tableName);
assertEquals("The specified table should present.", true, tableExists);
} finally {
if (regions != null) {
String node = ZKAssign.getNodeName(zkw, regions.get(0).getRegionInfo()
.getEncodedName());
ZKUtil.deleteNodeFailSilent(zkw, node);
}
admin.setBalancerRunning(true, false);
cluster.getMaster().setCatalogJanitorEnabled(true);
t.close();
@ -929,7 +648,6 @@ public class TestSplitTransactionOnCluster {
@Test
public void testSplitWithRegionReplicas() throws Exception {
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TESTING_UTIL);
final TableName tableName =
TableName.valueOf("foobar");
HTableDescriptor htd = TESTING_UTIL.createTableDescriptor("foobar");
@ -938,7 +656,6 @@ public class TestSplitTransactionOnCluster {
// Create table then get the single region for our new table.
HTable t = TESTING_UTIL.createTable(htd, new byte[][]{Bytes.toBytes("cf")},
TESTING_UTIL.getConfiguration());
int count;
List<HRegion> oldRegions;
do {
oldRegions = cluster.getRegions(tableName);
@ -960,9 +677,6 @@ public class TestSplitTransactionOnCluster {
regionServerIndex = cluster.getServerWith(region.getRegionName());
regionServer = cluster.getRegionServer(regionServerIndex);
assertTrue("not able to find a splittable region", region != null);
String node = ZKAssign.getNodeName(regionServer.getZooKeeper(),
region.getRegionInfo().getEncodedName());
regionServer.getZooKeeper().sync(node);
SplitTransaction st = new SplitTransaction(region, Bytes.toBytes("row2"));
try {
st.prepare();
@ -1093,7 +807,7 @@ public class TestSplitTransactionOnCluster {
}
// We should not be able to assign it again
am.assign(hri, true, true);
am.assign(hri, true);
assertFalse("Split region can't be assigned",
regionStates.isRegionInTransition(hri));
assertTrue(regionStates.isRegionInState(hri, State.SPLIT));
@ -1176,17 +890,6 @@ public class TestSplitTransactionOnCluster {
throw new SplittingNodeCreationFailedException ();
}
};
String node = ZKAssign.getNodeName(regionServer.getZooKeeper(),
region.getRegionInfo().getEncodedName());
regionServer.getZooKeeper().sync(node);
for (int i = 0; i < 100; i++) {
// We expect the znode to be deleted by this time. Here the
// znode could be in OPENED state and the
// master has not yet deleted the znode.
if (ZKUtil.checkExists(regionServer.getZooKeeper(), node) != -1) {
Thread.sleep(100);
}
}
try {
st.prepare();
st.execute(regionServer, regionServer);
@ -1196,13 +899,7 @@ public class TestSplitTransactionOnCluster {
// This will at least make the test to fail;
assertTrue("Should be instance of CreateSplittingNodeFailedException",
e instanceof SplittingNodeCreationFailedException );
node = ZKAssign.getNodeName(regionServer.getZooKeeper(),
region.getRegionInfo().getEncodedName());
{
assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
}
assertTrue(st.rollback(regionServer, regionServer));
assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
}
} finally {
TESTING_UTIL.deleteTable(tableName);
@ -1213,8 +910,6 @@ public class TestSplitTransactionOnCluster {
public void initialize(Server server, HRegion region) {
this.server = server;
this.watcher = server.getZooKeeper();
splitTransactionCoordination = new MockedSplitTransactionCoordination(this, watcher, region);
closeRegionCoordination = new ZkCloseRegionCoordination(this, watcher);
}
}
@ -1237,42 +932,8 @@ public class TestSplitTransactionOnCluster {
}
return super.rollback(server, services);
}
}
public static class MockedSplitTransactionCoordination extends ZKSplitTransactionCoordination {
private HRegion currentRegion;
public MockedSplitTransactionCoordination(CoordinatedStateManager coordinationProvider,
ZooKeeperWatcher watcher, HRegion region) {
super(coordinationProvider, watcher);
currentRegion = region;
}
@Override
public void completeSplitTransaction(RegionServerServices services, HRegion a, HRegion b,
SplitTransactionDetails std, HRegion parent) throws IOException {
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
try {
if (!secondSplit){
callRollBack = true;
latch.await();
}
} catch (InterruptedException e) {
}
}
super.completeSplitTransaction(services, a, b, std, parent);
if (this.currentRegion.getRegionInfo().getTable().getNameAsString()
.equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
firstSplitCompleted = true;
}
}
}
private HRegion findSplittableRegion(final List<HRegion> regions) throws InterruptedException {
for (int i = 0; i < 5; ++i) {
for (HRegion r: regions) {
@ -1313,22 +974,14 @@ public class TestSplitTransactionOnCluster {
private void split(final HRegionInfo hri, final HRegionServer server, final int regionCount)
throws IOException, InterruptedException {
this.admin.split(hri.getRegionNameAsString());
try {
for (int i = 0; ProtobufUtil.getOnlineRegions(
server.getRSRpcServices()).size() <= regionCount && i < 300; i++) {
LOG.debug("Waiting on region to split");
Thread.sleep(100);
}
assertFalse("Waited too long for split",
ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size() <= regionCount);
} catch (RegionServerStoppedException e) {
if (useZKForAssignment) {
// If not using ZK for assignment, the exception may be expected.
LOG.error(e);
throw e;
}
for (int i = 0; ProtobufUtil.getOnlineRegions(
server.getRSRpcServices()).size() <= regionCount && i < 300; i++) {
LOG.debug("Waiting on region to split");
Thread.sleep(100);
}
assertFalse("Waited too long for split",
ProtobufUtil.getOnlineRegions(server.getRSRpcServices()).size() <= regionCount);
}
/**
@ -1411,13 +1064,14 @@ public class TestSplitTransactionOnCluster {
private void waitUntilRegionServerDead() throws InterruptedException, InterruptedIOException {
// Wait until the master processes the RS shutdown
for (int i=0; cluster.getMaster().getClusterStatus().
getServers().size() > NB_SERVERS && i<100; i++) {
for (int i=0; (cluster.getMaster().getClusterStatus().getServers().size() > NB_SERVERS
|| cluster.getLiveRegionServerThreads().size() > NB_SERVERS) && i<100; i++) {
LOG.info("Waiting on server to go down");
Thread.sleep(100);
}
assertFalse("Waited too long for RS to die", cluster.getMaster().getClusterStatus().
getServers().size() > NB_SERVERS);
assertFalse("Waited too long for RS to die",
cluster.getMaster().getClusterStatus(). getServers().size() > NB_SERVERS
|| cluster.getLiveRegionServerThreads().size() > NB_SERVERS);
}
private void awaitDaughters(byte[] tableName, int numDaughters) throws InterruptedException {

View File

@ -1,45 +0,0 @@
/**
* Copyright The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.LargeTests;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
/**
* Like {@link TestRegionMergeTransaction} in that we're testing
* {@link RegionMergeTransaction} only the below tests are against a running
* cluster where {@link TestRegionMergeTransaction} is tests against bare
* {@link HRegion}.
*/
@Category(LargeTests.class)
public class TestZKLessMergeOnCluster extends TestRegionMergeTransactionOnCluster {
@BeforeClass
public static void beforeAllTests() throws Exception {
// Don't use ZK for region assignment
TEST_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", false);
setupOnce();
}
@AfterClass
public static void afterAllTests() throws Exception {
TestRegionMergeTransactionOnCluster.afterAllTests();
}
}

View File

@ -1,45 +0,0 @@
/**
*
* 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.regionserver;
import org.apache.hadoop.hbase.LargeTests;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.experimental.categories.Category;
/**
* Like {@link TestSplitTransaction} in that we're testing {@link SplitTransaction}
* only the below tests are against a running cluster where {@link TestSplitTransaction}
* is tests against a bare {@link HRegion}.
*/
@Category(LargeTests.class)
public class TestZKLessSplitOnCluster extends TestSplitTransactionOnCluster {
@BeforeClass
public static void before() throws Exception {
// Don't use ZK for region assignment
TESTING_UTIL.getConfiguration().setBoolean("hbase.assignment.usezk", false);
setupOnce();
}
@AfterClass
public static void after() throws Exception {
TestSplitTransactionOnCluster.after();
}
}

View File

@ -1,255 +0,0 @@
/**
* 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.regionserver.handler;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.RegionTransition;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.coordination.ZkCloseRegionCoordination;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MockServer;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
/**
* Test of the {@link CloseRegionHandler}.
*/
@Category(MediumTests.class)
public class TestCloseRegionHandler {
static final Log LOG = LogFactory.getLog(TestCloseRegionHandler.class);
private final static HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU();
private static final HTableDescriptor TEST_HTD =
new HTableDescriptor(TableName.valueOf("TestCloseRegionHandler"));
private HRegionInfo TEST_HRI;
private int testIndex = 0;
@BeforeClass public static void before() throws Exception {
HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
HTU.startMiniZKCluster();
}
@AfterClass public static void after() throws IOException {
HTU.shutdownMiniZKCluster();
}
/**
* Before each test, use a different HRI, so the different tests
* don't interfere with each other. This allows us to use just
* a single ZK cluster for the whole suite.
*/
@Before
public void setupHRI() {
TEST_HRI = new HRegionInfo(TEST_HTD.getTableName(),
Bytes.toBytes(testIndex),
Bytes.toBytes(testIndex + 1));
testIndex++;
}
/**
* Test that if we fail a flush, abort gets set on close.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-4270">HBASE-4270</a>
* @throws IOException
* @throws NodeExistsException
* @throws KeeperException
*/
@Test public void testFailedFlushAborts()
throws IOException, NodeExistsException, KeeperException {
final Server server = new MockServer(HTU, false);
final RegionServerServices rss = HTU.createMockRegionServerService();
HTableDescriptor htd = TEST_HTD;
final HRegionInfo hri =
new HRegionInfo(htd.getTableName(), HConstants.EMPTY_END_ROW,
HConstants.EMPTY_END_ROW);
HRegion region = HTU.createLocalHRegion(hri, htd);
try {
assertNotNull(region);
// Spy on the region so can throw exception when close is called.
HRegion spy = Mockito.spy(region);
final boolean abort = false;
Mockito.when(spy.close(abort)).
thenThrow(new IOException("Mocked failed close!"));
// The CloseRegionHandler will try to get an HRegion that corresponds
// to the passed hri -- so insert the region into the online region Set.
rss.addToOnlineRegions(spy);
// Assert the Server is NOT stopped before we call close region.
assertFalse(server.isStopped());
ZkCoordinatedStateManager consensusProvider = new ZkCoordinatedStateManager();
consensusProvider.initialize(server);
consensusProvider.start();
ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
new ZkCloseRegionCoordination.ZkCloseRegionDetails();
zkCrd.setPublishStatusInZk(false);
zkCrd.setExpectedVersion(-1);
CloseRegionHandler handler = new CloseRegionHandler(server, rss, hri, false,
consensusProvider.getCloseRegionCoordination(), zkCrd);
boolean throwable = false;
try {
handler.process();
} catch (Throwable t) {
throwable = true;
} finally {
assertTrue(throwable);
// Abort calls stop so stopped flag should be set.
assertTrue(server.isStopped());
}
} finally {
HRegion.closeHRegion(region);
}
}
/**
* Test if close region can handle ZK closing node version mismatch
* @throws IOException
* @throws NodeExistsException
* @throws KeeperException
* @throws DeserializationException
*/
@Test public void testZKClosingNodeVersionMismatch()
throws IOException, NodeExistsException, KeeperException, DeserializationException {
final Server server = new MockServer(HTU);
final RegionServerServices rss = HTU.createMockRegionServerService();
HTableDescriptor htd = TEST_HTD;
final HRegionInfo hri = TEST_HRI;
ZkCoordinatedStateManager coordinationProvider = new ZkCoordinatedStateManager();
coordinationProvider.initialize(server);
coordinationProvider.start();
// open a region first so that it can be closed later
OpenRegion(server, rss, htd, hri, coordinationProvider.getOpenRegionCoordination());
// close the region
// Create it CLOSING, which is what Master set before sending CLOSE RPC
int versionOfClosingNode = ZKAssign.createNodeClosing(server.getZooKeeper(),
hri, server.getServerName());
// The CloseRegionHandler will validate the expected version
// Given it is set to invalid versionOfClosingNode+1,
// CloseRegionHandler should be M_ZK_REGION_CLOSING
ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
new ZkCloseRegionCoordination.ZkCloseRegionDetails();
zkCrd.setPublishStatusInZk(true);
zkCrd.setExpectedVersion(versionOfClosingNode+1);
CloseRegionHandler handler = new CloseRegionHandler(server, rss, hri, false,
coordinationProvider.getCloseRegionCoordination(), zkCrd);
handler.process();
// Handler should remain in M_ZK_REGION_CLOSING
RegionTransition rt =
RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
assertTrue(rt.getEventType().equals(EventType.M_ZK_REGION_CLOSING ));
}
/**
* Test if the region can be closed properly
* @throws IOException
* @throws NodeExistsException
* @throws KeeperException
* @throws org.apache.hadoop.hbase.exceptions.DeserializationException
*/
@Test public void testCloseRegion()
throws IOException, NodeExistsException, KeeperException, DeserializationException {
final Server server = new MockServer(HTU);
final RegionServerServices rss = HTU.createMockRegionServerService();
HTableDescriptor htd = TEST_HTD;
HRegionInfo hri = TEST_HRI;
ZkCoordinatedStateManager coordinationProvider = new ZkCoordinatedStateManager();
coordinationProvider.initialize(server);
coordinationProvider.start();
// open a region first so that it can be closed later
OpenRegion(server, rss, htd, hri, coordinationProvider.getOpenRegionCoordination());
// close the region
// Create it CLOSING, which is what Master set before sending CLOSE RPC
int versionOfClosingNode = ZKAssign.createNodeClosing(server.getZooKeeper(),
hri, server.getServerName());
// The CloseRegionHandler will validate the expected version
// Given it is set to correct versionOfClosingNode,
// CloseRegionHandlerit should be RS_ZK_REGION_CLOSED
ZkCloseRegionCoordination.ZkCloseRegionDetails zkCrd =
new ZkCloseRegionCoordination.ZkCloseRegionDetails();
zkCrd.setPublishStatusInZk(true);
zkCrd.setExpectedVersion(versionOfClosingNode);
CloseRegionHandler handler = new CloseRegionHandler(server, rss, hri, false,
coordinationProvider.getCloseRegionCoordination(), zkCrd);
handler.process();
// Handler should have transitioned it to RS_ZK_REGION_CLOSED
RegionTransition rt = RegionTransition.parseFrom(
ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
assertTrue(rt.getEventType().equals(EventType.RS_ZK_REGION_CLOSED));
}
private void OpenRegion(Server server, RegionServerServices rss,
HTableDescriptor htd, HRegionInfo hri, OpenRegionCoordination coordination)
throws IOException, NodeExistsException, KeeperException, DeserializationException {
// Create it OFFLINE node, which is what Master set before sending OPEN RPC
ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
OpenRegionCoordination.OpenRegionDetails ord =
coordination.getDetailsForNonCoordinatedOpening();
OpenRegionHandler openHandler =
new OpenRegionHandler(server, rss, hri, htd, coordination, ord);
rss.getRegionsInTransitionInRS().put(hri.getEncodedNameAsBytes(), Boolean.TRUE);
openHandler.process();
// This parse is not used?
RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(), hri.getEncodedName()));
// delete the node, which is what Master do after the region is opened
ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(),
EventType.RS_ZK_REGION_OPENED, server.getServerName());
}
}

View File

@ -1,360 +0,0 @@
/**
*
* 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.regionserver.handler;
import static org.junit.Assert.*;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.MockServer;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NodeExistsException;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test of the {@link OpenRegionHandler}.
*/
@Category(MediumTests.class)
public class TestOpenRegionHandler {
static final Log LOG = LogFactory.getLog(TestOpenRegionHandler.class);
private final static HBaseTestingUtility HTU = HBaseTestingUtility.createLocalHTU();
private static HTableDescriptor TEST_HTD;
private HRegionInfo TEST_HRI;
private int testIndex = 0;
@BeforeClass public static void before() throws Exception {
HTU.getConfiguration().setBoolean("hbase.assignment.usezk", true);
HTU.startMiniZKCluster();
TEST_HTD = new HTableDescriptor(TableName.valueOf("TestOpenRegionHandler.java"));
}
@AfterClass public static void after() throws IOException {
TEST_HTD = null;
HTU.shutdownMiniZKCluster();
}
/**
* Before each test, use a different HRI, so the different tests
* don't interfere with each other. This allows us to use just
* a single ZK cluster for the whole suite.
*/
@Before
public void setupHRI() {
TEST_HRI = new HRegionInfo(TEST_HTD.getTableName(),
Bytes.toBytes(testIndex),
Bytes.toBytes(testIndex + 1));
testIndex++;
}
/**
* Test the openregionhandler can deal with its znode being yanked out from
* under it.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-3627">HBASE-3627</a>
* @throws IOException
* @throws NodeExistsException
* @throws KeeperException
*/
@Test public void testYankingRegionFromUnderIt()
throws IOException, NodeExistsException, KeeperException {
final Server server = new MockServer(HTU);
final RegionServerServices rss = HTU.createMockRegionServerService();
HTableDescriptor htd = TEST_HTD;
final HRegionInfo hri = TEST_HRI;
HRegion region =
HRegion.createHRegion(hri, HTU.getDataTestDir(), HTU
.getConfiguration(), htd);
assertNotNull(region);
try {
ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(server);
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(server.getServerName());
OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri,
htd, csm.getOpenRegionCoordination(), zkCrd) {
HRegion openRegion() {
// Open region first, then remove znode as though it'd been hijacked.
HRegion region = super.openRegion();
// Don't actually open region BUT remove the znode as though it'd
// been hijacked on us.
ZooKeeperWatcher zkw = this.server.getZooKeeper();
String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
try {
ZKUtil.deleteNodeFailSilent(zkw, node);
} catch (KeeperException e) {
throw new RuntimeException("Ugh failed delete of " + node, e);
}
return region;
}
};
rss.getRegionsInTransitionInRS().put(
hri.getEncodedNameAsBytes(), Boolean.TRUE);
// Call process without first creating OFFLINE region in zk, see if
// exception or just quiet return (expected).
handler.process();
rss.getRegionsInTransitionInRS().put(
hri.getEncodedNameAsBytes(), Boolean.TRUE);
ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
// Call process again but this time yank the zk znode out from under it
// post OPENING; again will expect it to come back w/o NPE or exception.
handler.process();
} finally {
HRegion.closeHRegion(region);
}
}
/**
* Test the openregionhandler can deal with perceived failure of transitioning to OPENED state
* due to intermittent zookeeper malfunctioning.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-9387">HBASE-9387</a>
* @throws IOException
* @throws NodeExistsException
* @throws KeeperException
*/
@Test
public void testRegionServerAbortionDueToFailureTransitioningToOpened()
throws IOException, NodeExistsException, KeeperException {
final Server server = new MockServer(HTU);
final RegionServerServices rss = HTU.createMockRegionServerService();
HTableDescriptor htd = TEST_HTD;
final HRegionInfo hri = TEST_HRI;
HRegion region =
HRegion.createHRegion(hri, HTU.getDataTestDir(), HTU
.getConfiguration(), htd);
assertNotNull(region);
try {
ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(server);
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(server.getServerName());
ZkOpenRegionCoordination openRegionCoordination =
new ZkOpenRegionCoordination(csm, server.getZooKeeper()) {
@Override
public boolean transitionToOpened(final HRegion r, OpenRegionDetails ord)
throws IOException {
// remove znode simulating intermittent zookeeper connection issue
ZooKeeperWatcher zkw = server.getZooKeeper();
String node = ZKAssign.getNodeName(zkw, hri.getEncodedName());
try {
ZKUtil.deleteNodeFailSilent(zkw, node);
} catch (KeeperException e) {
throw new RuntimeException("Ugh failed delete of " + node, e);
}
// then try to transition to OPENED
return super.transitionToOpened(r, ord);
}
};
OpenRegionHandler handler = new OpenRegionHandler(server, rss, hri, htd,
openRegionCoordination, zkCrd);
rss.getRegionsInTransitionInRS().put(
hri.getEncodedNameAsBytes(), Boolean.TRUE);
// Call process without first creating OFFLINE region in zk, see if
// exception or just quiet return (expected).
handler.process();
rss.getRegionsInTransitionInRS().put(
hri.getEncodedNameAsBytes(), Boolean.TRUE);
ZKAssign.createNodeOffline(server.getZooKeeper(), hri, server.getServerName());
// Call process again but this time yank the zk znode out from under it
// post OPENING; again will expect it to come back w/o NPE or exception.
handler.process();
} catch (IOException ioe) {
} finally {
HRegion.closeHRegion(region);
}
// Region server is expected to abort due to OpenRegionHandler perceiving transitioning
// to OPENED as failed
// This was corresponding to the second handler.process() call above.
assertTrue("region server should have aborted", server.isAborted());
}
@Test
public void testFailedOpenRegion() throws Exception {
Server server = new MockServer(HTU);
RegionServerServices rsServices = HTU.createMockRegionServerService();
// Create it OFFLINE, which is what it expects
ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(server);
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(server.getServerName());
// Create the handler
OpenRegionHandler handler =
new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
csm.getOpenRegionCoordination(), zkCrd) {
@Override
HRegion openRegion() {
// Fake failure of opening a region due to an IOE, which is caught
return null;
}
};
rsServices.getRegionsInTransitionInRS().put(
TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
handler.process();
// Handler should have transitioned it to FAILED_OPEN
RegionTransition rt = RegionTransition.parseFrom(
ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
}
@Test
public void testFailedUpdateMeta() throws Exception {
Server server = new MockServer(HTU);
RegionServerServices rsServices = HTU.createMockRegionServerService();
// Create it OFFLINE, which is what it expects
ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
// Create the handler
ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(server);
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(server.getServerName());
OpenRegionHandler handler = new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
csm.getOpenRegionCoordination(), zkCrd) {
@Override
boolean updateMeta(final HRegion r) {
// Fake failure of updating META
return false;
}
};
rsServices.getRegionsInTransitionInRS().put(
TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
handler.process();
// Handler should have transitioned it to FAILED_OPEN
RegionTransition rt = RegionTransition.parseFrom(
ZKAssign.getData(server.getZooKeeper(), TEST_HRI.getEncodedName()));
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
}
@Test
public void testTransitionToFailedOpenEvenIfCleanupFails() throws Exception {
Server server = new MockServer(HTU);
RegionServerServices rsServices = HTU.createMockRegionServerService();
// Create it OFFLINE, which is what it expects
ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
// Create the handler
ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(server);
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(server.getServerName());
OpenRegionHandler handler = new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
csm.getOpenRegionCoordination(), zkCrd) {
@Override
boolean updateMeta(HRegion r) {
return false;
};
@Override
void cleanupFailedOpen(HRegion region) throws IOException {
throw new IOException("FileSystem got closed.");
}
};
rsServices.getRegionsInTransitionInRS().put(TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
try {
handler.process();
} catch (Exception e) {
// Ignore the IOException that we have thrown from cleanupFailedOpen
}
RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(),
TEST_HRI.getEncodedName()));
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
}
@Test
public void testTransitionToFailedOpenFromOffline() throws Exception {
Server server = new MockServer(HTU);
RegionServerServices rsServices = HTU.createMockRegionServerService(server.getServerName());
// Create it OFFLINE, which is what it expects
ZKAssign.createNodeOffline(server.getZooKeeper(), TEST_HRI, server.getServerName());
// Create the handler
ZkCoordinatedStateManager csm = new ZkCoordinatedStateManager();
csm.initialize(server);
csm.start();
ZkOpenRegionCoordination.ZkOpenRegionDetails zkCrd =
new ZkOpenRegionCoordination.ZkOpenRegionDetails();
zkCrd.setServerName(server.getServerName());
ZkOpenRegionCoordination openRegionCoordination =
new ZkOpenRegionCoordination(csm, server.getZooKeeper()) {
@Override
public boolean transitionFromOfflineToOpening(HRegionInfo regionInfo,
OpenRegionDetails ord) {
return false;
}
};
OpenRegionHandler handler = new OpenRegionHandler(server, rsServices, TEST_HRI, TEST_HTD,
openRegionCoordination, zkCrd);
rsServices.getRegionsInTransitionInRS().put(TEST_HRI.getEncodedNameAsBytes(), Boolean.TRUE);
handler.process();
RegionTransition rt = RegionTransition.parseFrom(ZKAssign.getData(server.getZooKeeper(),
TEST_HRI.getEncodedName()));
assertEquals(EventType.RS_ZK_REGION_FAILED_OPEN, rt.getEventType());
}
}

View File

@ -29,17 +29,15 @@ import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.HConnectionManager;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.HBaseTestingUtility;
/**
* This builds a table, removes info from meta, and then rebuilds meta.
*/
@Category(MediumTests.class)
public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
@SuppressWarnings("deprecation")
@Test(timeout = 120000)
public void testMetaRebuild() throws Exception {
wipeOutMeta();
@ -68,10 +66,9 @@ public class TestOfflineMetaRebuildBase extends OfflineMetaRebuildTestCore {
TEST_UTIL.startMiniZKCluster();
TEST_UTIL.restartHBaseCluster(3);
TEST_UTIL.getHBaseAdmin().enableTable(table);
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
LOG.info("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
LOG.info("No more RIT in ZK, now doing final test verification");
// everything is good again.

View File

@ -24,13 +24,10 @@ import static org.junit.Assert.assertFalse;
import java.util.Arrays;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -71,10 +68,8 @@ public class TestOfflineMetaRebuildHole extends OfflineMetaRebuildTestCore {
TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default
TEST_UTIL.restartHBaseCluster(3);
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
LOG.info("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
LOG.info("No more RIT in ZK, now doing final test verification");
int tries = 60;
while(TEST_UTIL.getHBaseCluster()

View File

@ -24,14 +24,11 @@ import static org.junit.Assert.assertFalse;
import java.util.Arrays;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.util.HBaseFsck;
import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
import org.apache.hadoop.hbase.util.HBaseFsck.HbckInfo;
import org.apache.hadoop.hbase.zookeeper.ZKAssign;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -80,10 +77,8 @@ public class TestOfflineMetaRebuildOverlap extends OfflineMetaRebuildTestCore {
TEST_UTIL.startMiniZKCluster(); // tables seem enabled by default
TEST_UTIL.restartHBaseCluster(3);
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL);
LOG.info("Waiting for no more RIT");
ZKAssign.blockUntilNoRIT(zkw);
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
LOG.info("No more RIT in ZK, now doing final test verification");
int tries = 60;
while(TEST_UTIL.getHBaseCluster()