HBASE-11611 Clean up ZK-based region assignment
This commit is contained in:
parent
e17a3ca091
commit
17dff6818e
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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));
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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)?
|
||||
|
|
|
@ -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,
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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.
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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 {
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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);
|
||||
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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,17 +672,12 @@ 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
|
||||
|
@ -701,7 +694,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
|||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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,17 +128,12 @@ 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());
|
||||
|
@ -150,7 +142,6 @@ public class RegionStateStore {
|
|||
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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
@ -731,7 +729,7 @@ public class ServerManager {
|
|||
return RegionOpeningState.FAILED_OPENING;
|
||||
}
|
||||
OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server,
|
||||
region, versionOfOfflineNode, favoredNodes,
|
||||
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) {
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
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);
|
||||
LOG.info("Reassigning region with rs = " + rit);
|
||||
regionStates.updateRegionState(hri, State.OFFLINE);
|
||||
} catch (KeeperException ke) {
|
||||
this.server.abort("Unexpected ZK exception deleting unassigned node " + hri, ke);
|
||||
return;
|
||||
}
|
||||
} 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 "
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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,
|
||||
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,
|
||||
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,22 +498,15 @@ public class RegionMergeTransaction {
|
|||
merged.openHRegion(reporter);
|
||||
|
||||
if (services != null) {
|
||||
try {
|
||||
if (useCoordinationForAssignment) {
|
||||
services.postOpenDeployTasks(merged);
|
||||
} else if (!services.reportRegionStateTransition(TransitionCode.MERGED,
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Create reference file(s) of merging regions under the region_a merges dir
|
||||
* @param hstoreFilesOfRegionA
|
||||
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
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,
|
||||
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,
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
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);
|
||||
}
|
||||
|
||||
// Done! Region is closed on this RS
|
||||
LOG.debug("Closed " + region.getRegionNameAsString());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
} else if (!useZKForAssignment) {
|
||||
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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
|
Binary file not shown.
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,15 +484,10 @@ 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());
|
||||
}
|
||||
ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
|
||||
startTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
while (true) {
|
||||
|
@ -506,9 +498,6 @@ public class TestScannersFromClientSide {
|
|||
EnvironmentEdgeManager.currentTimeMillis() < startTime + timeOut);
|
||||
Thread.sleep(500);
|
||||
}
|
||||
} finally {
|
||||
ZKAssign.deleteNodeFailSilent(zkw, hri);
|
||||
}
|
||||
|
||||
// c0:0, c1:1
|
||||
kvListExp = new ArrayList<Cell>();
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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.
|
||||
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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
try {
|
||||
// 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);
|
||||
} 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);
|
||||
}
|
||||
|
||||
// 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());
|
||||
}
|
||||
|
||||
// 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,40 +932,6 @@ 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 {
|
||||
|
@ -1313,7 +974,6 @@ 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");
|
||||
|
@ -1322,13 +982,6 @@ public class TestSplitTransactionOnCluster {
|
|||
|
||||
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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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.
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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()
|
||||
|
|
Loading…
Reference in New Issue