HBASE-19128 Purge Distributed Log Replay from codebase, configurations, text; mark the feature as unsupported, broken.
This commit is contained in:
parent
8095f96289
commit
a6d8023ff5
|
@ -1,44 +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.exceptions;
|
||||
|
||||
import org.apache.hadoop.hbase.NotServingRegionException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Thrown when a read request issued against a region which is in recovering state.
|
||||
*/
|
||||
@InterfaceAudience.Public
|
||||
public class RegionInRecoveryException extends NotServingRegionException {
|
||||
private static final long serialVersionUID = 327302071153799L;
|
||||
|
||||
/** default constructor */
|
||||
public RegionInRecoveryException() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* @param s message
|
||||
*/
|
||||
public RegionInRecoveryException(String s) {
|
||||
super(s);
|
||||
}
|
||||
|
||||
}
|
|
@ -259,8 +259,7 @@ public enum EventType {
|
|||
RS_PARALLEL_SEEK (80, ExecutorType.RS_PARALLEL_SEEK),
|
||||
|
||||
/**
|
||||
* RS wal recovery work items(either creating recover.edits or directly replay wals)
|
||||
* to be executed on the RS.<br>
|
||||
* RS wal recovery work items (splitting wals) to be executed on the RS.<br>
|
||||
*
|
||||
* RS_LOG_REPLAY
|
||||
*/
|
||||
|
|
|
@ -1806,7 +1806,7 @@ public final class ProtobufUtil {
|
|||
final AdminService.BlockingInterface admin, ServerName server, final org.apache.hadoop.hbase.client.RegionInfo region)
|
||||
throws IOException {
|
||||
OpenRegionRequest request =
|
||||
RequestConverter.buildOpenRegionRequest(server, region, null, null);
|
||||
RequestConverter.buildOpenRegionRequest(server, region, null);
|
||||
try {
|
||||
admin.openRegion(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
|
|
|
@ -866,15 +866,14 @@ public final class RequestConverter {
|
|||
* Create a protocol buffer OpenRegionRequest to open a list of regions
|
||||
* @param server the serverName for the RPC
|
||||
* @param regionOpenInfos info of a list of regions to open
|
||||
* @param openForReplay whether open for replay
|
||||
* @return a protocol buffer OpenRegionRequest
|
||||
*/
|
||||
public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
|
||||
final List<Pair<RegionInfo, List<ServerName>>> regionOpenInfos, Boolean openForReplay) {
|
||||
final List<Pair<RegionInfo, List<ServerName>>> regionOpenInfos) {
|
||||
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
|
||||
for (Pair<RegionInfo, List<ServerName>> regionOpenInfo : regionOpenInfos) {
|
||||
builder.addOpenInfo(buildRegionOpenInfo(regionOpenInfo.getFirst(),
|
||||
regionOpenInfo.getSecond(), openForReplay));
|
||||
regionOpenInfo.getSecond()));
|
||||
}
|
||||
if (server != null) {
|
||||
builder.setServerStartCode(server.getStartcode());
|
||||
|
@ -889,13 +888,12 @@ public final class RequestConverter {
|
|||
* @param server the serverName for the RPC
|
||||
* @param region the region to open
|
||||
* @param favoredNodes a list of favored nodes
|
||||
* @param openForReplay whether open for replay
|
||||
* @return a protocol buffer OpenRegionRequest
|
||||
*/
|
||||
public static OpenRegionRequest buildOpenRegionRequest(ServerName server,
|
||||
final RegionInfo region, List<ServerName> favoredNodes, Boolean openForReplay) {
|
||||
final RegionInfo region, List<ServerName> favoredNodes) {
|
||||
OpenRegionRequest.Builder builder = OpenRegionRequest.newBuilder();
|
||||
builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes, openForReplay));
|
||||
builder.addOpenInfo(buildRegionOpenInfo(region, favoredNodes));
|
||||
if (server != null) {
|
||||
builder.setServerStartCode(server.getStartcode());
|
||||
}
|
||||
|
@ -1522,8 +1520,7 @@ public final class RequestConverter {
|
|||
* Create a RegionOpenInfo based on given region info and version of offline node
|
||||
*/
|
||||
public static RegionOpenInfo buildRegionOpenInfo(
|
||||
final RegionInfo region,
|
||||
final List<ServerName> favoredNodes, Boolean openForReplay) {
|
||||
final RegionInfo region, final List<ServerName> favoredNodes) {
|
||||
RegionOpenInfo.Builder builder = RegionOpenInfo.newBuilder();
|
||||
builder.setRegion(ProtobufUtil.toRegionInfo(region));
|
||||
if (favoredNodes != null) {
|
||||
|
@ -1531,9 +1528,6 @@ public final class RequestConverter {
|
|||
builder.addFavoredNodes(ProtobufUtil.toServerName(server));
|
||||
}
|
||||
}
|
||||
if(openForReplay != null) {
|
||||
builder.setOpenForDistributedLogReplay(openForReplay);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -46,10 +46,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.security.Superusers;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -2089,55 +2086,4 @@ public class ZKUtil {
|
|||
return 0;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param regionLastFlushedSequenceId the flushed sequence id of a region which is the min of its
|
||||
* store max seq ids
|
||||
* @param storeSequenceIds column family to sequence Id map
|
||||
* @return Serialized protobuf of <code>RegionSequenceIds</code> with pb magic prefix prepended
|
||||
* suitable for use to filter wal edits in distributedLogReplay mode
|
||||
*/
|
||||
public static byte[] regionSequenceIdsToByteArray(final Long regionLastFlushedSequenceId,
|
||||
final Map<byte[], Long> storeSequenceIds) {
|
||||
ClusterStatusProtos.RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
|
||||
ClusterStatusProtos.RegionStoreSequenceIds.newBuilder();
|
||||
ClusterStatusProtos.StoreSequenceId.Builder storeSequenceIdBuilder =
|
||||
ClusterStatusProtos.StoreSequenceId.newBuilder();
|
||||
if (storeSequenceIds != null) {
|
||||
for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){
|
||||
byte[] columnFamilyName = e.getKey();
|
||||
Long curSeqId = e.getValue();
|
||||
storeSequenceIdBuilder.setFamilyName(UnsafeByteOperations.unsafeWrap(columnFamilyName));
|
||||
storeSequenceIdBuilder.setSequenceId(curSeqId);
|
||||
regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
|
||||
storeSequenceIdBuilder.clear();
|
||||
}
|
||||
}
|
||||
regionSequenceIdsBuilder.setLastFlushedSequenceId(regionLastFlushedSequenceId);
|
||||
byte[] result = regionSequenceIdsBuilder.build().toByteArray();
|
||||
return ProtobufUtil.prependPBMagic(result);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes Content of serialized data of RegionStoreSequenceIds
|
||||
* @return a RegionStoreSequenceIds object
|
||||
* @throws DeserializationException
|
||||
*/
|
||||
public static RegionStoreSequenceIds parseRegionStoreSequenceIds(final byte[] bytes)
|
||||
throws DeserializationException {
|
||||
if (bytes == null || !ProtobufUtil.isPBMagicPrefix(bytes)) {
|
||||
throw new DeserializationException("Unable to parse RegionStoreSequenceIds.");
|
||||
}
|
||||
RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
|
||||
ClusterStatusProtos.RegionStoreSequenceIds.newBuilder();
|
||||
int pblen = ProtobufUtil.lengthOfPBMagic();
|
||||
RegionStoreSequenceIds storeIds = null;
|
||||
try {
|
||||
ProtobufUtil.mergeFrom(regionSequenceIdsBuilder, bytes, pblen, bytes.length - pblen);
|
||||
storeIds = regionSequenceIdsBuilder.build();
|
||||
} catch (IOException e) {
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
return storeIds;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -72,8 +72,6 @@ public class ZNodePaths {
|
|||
public final String switchZNode;
|
||||
// znode containing the lock for the tables
|
||||
public final String tableLockZNode;
|
||||
// znode containing the state of recovering regions
|
||||
public final String recoveringRegionsZNode;
|
||||
// znode containing namespace descriptors
|
||||
public final String namespaceZNode;
|
||||
// znode of indicating master maintenance mode
|
||||
|
@ -116,8 +114,6 @@ public class ZNodePaths {
|
|||
switchZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.switch", "switch"));
|
||||
tableLockZNode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.tableLock", "table-lock"));
|
||||
recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.recovering.regions", "recovering-regions"));
|
||||
namespaceZNode = ZKUtil.joinZNode(baseZNode,
|
||||
conf.get("zookeeper.znode.namespace", "namespace"));
|
||||
masterMaintZNode = ZKUtil.joinZNode(baseZNode,
|
||||
|
@ -142,10 +138,9 @@ public class ZNodePaths {
|
|||
+ ", clusterIdZNode=" + clusterIdZNode + ", splitLogZNode=" + splitLogZNode
|
||||
+ ", balancerZNode=" + balancerZNode + ", regionNormalizerZNode=" + regionNormalizerZNode
|
||||
+ ", switchZNode=" + switchZNode + ", tableLockZNode=" + tableLockZNode
|
||||
+ ", recoveringRegionsZNode=" + recoveringRegionsZNode + ", namespaceZNode="
|
||||
+ namespaceZNode + ", masterMaintZNode=" + masterMaintZNode + ", replicationZNode="
|
||||
+ replicationZNode + ", peersZNode=" + peersZNode + ", queuesZNode=" + queuesZNode
|
||||
+ ", hfileRefsZNode=" + hfileRefsZNode + "]";
|
||||
+ ", namespaceZNode=" + namespaceZNode + ", masterMaintZNode=" + masterMaintZNode
|
||||
+ ", replicationZNode=" + replicationZNode + ", peersZNode=" + peersZNode
|
||||
+ ", queuesZNode=" + queuesZNode + ", hfileRefsZNode=" + hfileRefsZNode + "]";
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -158,7 +158,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
ZKUtil.createAndFailSilent(this, znodePaths.splitLogZNode);
|
||||
ZKUtil.createAndFailSilent(this, znodePaths.backupMasterAddressesZNode);
|
||||
ZKUtil.createAndFailSilent(this, znodePaths.tableLockZNode);
|
||||
ZKUtil.createAndFailSilent(this, znodePaths.recoveringRegionsZNode);
|
||||
ZKUtil.createAndFailSilent(this, znodePaths.masterMaintZNode);
|
||||
} catch (KeeperException e) {
|
||||
throw new ZooKeeperConnectionException(
|
||||
|
|
|
@ -24,7 +24,6 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -33,7 +32,7 @@ import org.junit.experimental.categories.Category;
|
|||
public class TestZooKeeperWatcher {
|
||||
|
||||
@Test
|
||||
public void testIsClientReadable() throws ZooKeeperConnectionException, IOException {
|
||||
public void testIsClientReadable() throws IOException {
|
||||
ZooKeeperWatcher watcher = new ZooKeeperWatcher(HBaseConfiguration.create(),
|
||||
"testIsClientReadable", null, false);
|
||||
|
||||
|
@ -50,7 +49,6 @@ public class TestZooKeeperWatcher {
|
|||
assertFalse(watcher.isClientReadable(watcher.znodePaths.regionNormalizerZNode));
|
||||
assertFalse(watcher.isClientReadable(watcher.znodePaths.clusterStateZNode));
|
||||
assertFalse(watcher.isClientReadable(watcher.znodePaths.drainingZNode));
|
||||
assertFalse(watcher.isClientReadable(watcher.znodePaths.recoveringRegionsZNode));
|
||||
assertFalse(watcher.isClientReadable(watcher.znodePaths.splitLogZNode));
|
||||
assertFalse(watcher.isClientReadable(watcher.znodePaths.backupMasterAddressesZNode));
|
||||
|
||||
|
|
|
@ -1013,13 +1013,6 @@ public final class HConstants {
|
|||
|
||||
public static final String LOCALHOST_IP = "127.0.0.1";
|
||||
|
||||
/** Conf key that enables unflushed WAL edits directly being replayed to region servers */
|
||||
public static final String DISTRIBUTED_LOG_REPLAY_KEY = "hbase.master.distributed.log.replay";
|
||||
public static final boolean DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG = false;
|
||||
public static final String DISALLOW_WRITES_IN_RECOVERING =
|
||||
"hbase.regionserver.disallow.writes.when.recovering";
|
||||
public static final boolean DEFAULT_DISALLOW_WRITES_IN_RECOVERING_CONFIG = false;
|
||||
|
||||
public static final String REGION_SERVER_HANDLER_COUNT = "hbase.regionserver.handler.count";
|
||||
public static final int DEFAULT_REGION_SERVER_HANDLER_COUNT = 30;
|
||||
|
||||
|
|
|
@ -930,8 +930,7 @@ possible configurations would overwhelm and obscure the important.
|
|||
<value>3</value>
|
||||
<description>The HFile format version to use for new files.
|
||||
Version 3 adds support for tags in hfiles (See http://hbase.apache.org/book.html#hbase.tags).
|
||||
Distributed Log Replay requires that tags are enabled. Also see the configuration
|
||||
'hbase.replication.rpc.codec'.
|
||||
Also see the configuration 'hbase.replication.rpc.codec'.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
|
@ -1653,9 +1652,8 @@ possible configurations would overwhelm and obscure the important.
|
|||
of written. This configure is to set how long (in ms) we will wait before next checking if a
|
||||
log can not push right now because there are some logs written before it have not been pushed.
|
||||
A larger waiting will decrease the number of queries on hbase:meta but will enlarge the delay
|
||||
of replication. This feature relies on zk-less assignment, and conflicts with distributed log
|
||||
replay. So users must set hbase.assignment.usezk and hbase.master.distributed.log.replay to
|
||||
false to support it.
|
||||
of replication. This feature relies on zk-less assignment, so users must set
|
||||
hbase.assignment.usezk to false to support it.
|
||||
</description>
|
||||
</property>
|
||||
<!-- Static Web User Filter properties. -->
|
||||
|
|
|
@ -1,72 +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.wal;
|
||||
|
||||
import org.apache.hadoop.hbase.metrics.BaseSource;
|
||||
|
||||
/**
|
||||
* Interface of the source that will export metrics about log replay statistics when recovering a
|
||||
* region server in distributedLogReplay mode
|
||||
*/
|
||||
public interface MetricsEditsReplaySource extends BaseSource {
|
||||
|
||||
/**
|
||||
* The name of the metrics
|
||||
*/
|
||||
String METRICS_NAME = "replay";
|
||||
|
||||
/**
|
||||
* The name of the metrics context that metrics will be under.
|
||||
*/
|
||||
String METRICS_CONTEXT = "regionserver";
|
||||
|
||||
/**
|
||||
* Description
|
||||
*/
|
||||
String METRICS_DESCRIPTION = "Metrics about HBase RegionServer WAL Edits Replay";
|
||||
|
||||
/**
|
||||
* The name of the metrics context that metrics will be under in jmx
|
||||
*/
|
||||
String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;
|
||||
|
||||
|
||||
String REPLAY_TIME_NAME = "replayTime";
|
||||
String REPLAY_TIME_DESC = "Time an replay operation took.";
|
||||
String REPLAY_BATCH_SIZE_NAME = "replayBatchSize";
|
||||
String REPLAY_BATCH_SIZE_DESC = "Number of changes in each replay batch.";
|
||||
String REPLAY_DATA_SIZE_NAME = "replayDataSize";
|
||||
String REPLAY_DATA_SIZE_DESC = "Size (in bytes) of the data of each replay.";
|
||||
|
||||
/**
|
||||
* Add the time a replay command took
|
||||
*/
|
||||
void updateReplayTime(long time);
|
||||
|
||||
/**
|
||||
* Add the batch size of each replay
|
||||
*/
|
||||
void updateReplayBatchSize(long size);
|
||||
|
||||
/**
|
||||
* Add the payload data size of each replay
|
||||
*/
|
||||
void updateReplayDataSize(long size);
|
||||
|
||||
}
|
|
@ -1,76 +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.wal;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
|
||||
import org.apache.hadoop.metrics2.MetricHistogram;
|
||||
|
||||
/**
|
||||
* Hadoop1 implementation of MetricsMasterSource. Implements BaseSource through BaseSourceImpl,
|
||||
* following the pattern
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsEditsReplaySourceImpl extends BaseSourceImpl implements
|
||||
MetricsEditsReplaySource {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(MetricsEditsReplaySourceImpl.class.getName());
|
||||
|
||||
private MetricHistogram replayTimeHisto;
|
||||
private MetricHistogram replayBatchSizeHisto;
|
||||
private MetricHistogram replayDataSizeHisto;
|
||||
|
||||
public MetricsEditsReplaySourceImpl() {
|
||||
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
|
||||
}
|
||||
|
||||
public MetricsEditsReplaySourceImpl(String metricsName,
|
||||
String metricsDescription,
|
||||
String metricsContext,
|
||||
String metricsJmxContext) {
|
||||
super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init() {
|
||||
super.init();
|
||||
replayTimeHisto = metricsRegistry.newTimeHistogram(REPLAY_TIME_NAME, REPLAY_TIME_DESC);
|
||||
replayBatchSizeHisto = metricsRegistry.newSizeHistogram(REPLAY_BATCH_SIZE_NAME,
|
||||
REPLAY_BATCH_SIZE_DESC);
|
||||
replayDataSizeHisto = metricsRegistry
|
||||
.newSizeHistogram(REPLAY_DATA_SIZE_NAME, REPLAY_DATA_SIZE_DESC);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateReplayTime(long time) {
|
||||
replayTimeHisto.add(time);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateReplayBatchSize(long size) {
|
||||
replayBatchSizeHisto.add(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateReplayDataSize(long size) {
|
||||
replayDataSizeHisto.add(size);
|
||||
}
|
||||
}
|
|
@ -1,18 +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.
|
||||
#
|
||||
org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySourceImpl
|
|
@ -39,7 +39,7 @@ message GetRegionInfoRequest {
|
|||
message GetRegionInfoResponse {
|
||||
required RegionInfo region_info = 1;
|
||||
optional CompactionState compaction_state = 2;
|
||||
optional bool isRecovering = 3;
|
||||
// optional bool DEPRECATED_isRecovering = 3;
|
||||
// True if region is splittable, false otherwise.
|
||||
optional bool splittable = 4;
|
||||
// True if region is mergeable, false otherwise.
|
||||
|
@ -87,7 +87,7 @@ message OpenRegionRequest {
|
|||
optional uint32 version_of_offline_node = 2;
|
||||
repeated ServerName favored_nodes = 3;
|
||||
// open region for distributedLogReplay
|
||||
optional bool openForDistributedLogReplay = 4;
|
||||
// optional bool DEPRECATED_openForDistributedLogReplay = 4;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -278,7 +278,7 @@ message MergeTableRegionsStateData {
|
|||
|
||||
message ServerCrashStateData {
|
||||
required ServerName server_name = 1;
|
||||
optional bool distributed_log_replay = 2;
|
||||
// optional bool DEPRECATED_distributed_log_replay = 2;
|
||||
repeated RegionInfo regions_on_crashed_server = 3;
|
||||
repeated RegionInfo regions_assigned = 4;
|
||||
optional bool carrying_meta = 5;
|
||||
|
@ -297,7 +297,7 @@ enum ServerCrashState {
|
|||
SERVER_CRASH_GET_REGIONS = 3;
|
||||
SERVER_CRASH_NO_SPLIT_LOGS = 4;
|
||||
SERVER_CRASH_SPLIT_LOGS = 5;
|
||||
SERVER_CRASH_PREPARE_LOG_REPLAY = 6;
|
||||
// Removed SERVER_CRASH_PREPARE_LOG_REPLAY = 6;
|
||||
// Removed SERVER_CRASH_CALC_REGIONS_TO_ASSIGN = 7;
|
||||
SERVER_CRASH_ASSIGN = 8;
|
||||
SERVER_CRASH_WAIT_ON_ASSIGN = 9;
|
||||
|
|
|
@ -78,14 +78,9 @@ message SplitLogTask {
|
|||
DONE = 3;
|
||||
ERR = 4;
|
||||
}
|
||||
enum RecoveryMode {
|
||||
UNKNOWN = 0;
|
||||
LOG_SPLITTING = 1;
|
||||
LOG_REPLAY = 2;
|
||||
}
|
||||
required State state = 1;
|
||||
required ServerName server_name = 2;
|
||||
optional RecoveryMode mode = 3 [default = UNKNOWN];
|
||||
// optional RecoveryMode DEPRECATED_mode = 3 [default = UNKNOWN];
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -36,7 +36,7 @@ message GetRegionInfoRequest {
|
|||
message GetRegionInfoResponse {
|
||||
required RegionInfo region_info = 1;
|
||||
optional CompactionState compaction_state = 2;
|
||||
optional bool isRecovering = 3;
|
||||
// optional bool DEPRECATED_isRecovering = 3;
|
||||
|
||||
enum CompactionState {
|
||||
NONE = 0;
|
||||
|
@ -78,7 +78,7 @@ message OpenRegionRequest {
|
|||
optional uint32 version_of_offline_node = 2;
|
||||
repeated ServerName favored_nodes = 3;
|
||||
// open region for distributedLogReplay
|
||||
optional bool openForDistributedLogReplay = 4;
|
||||
// optional bool DEPRECATED_openForDistributedLogReplay = 4;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -78,14 +78,9 @@ message SplitLogTask {
|
|||
DONE = 3;
|
||||
ERR = 4;
|
||||
}
|
||||
enum RecoveryMode {
|
||||
UNKNOWN = 0;
|
||||
LOG_SPLITTING = 1;
|
||||
LOG_REPLAY = 2;
|
||||
}
|
||||
required State state = 1;
|
||||
required ServerName server_name = 2;
|
||||
optional RecoveryMode mode = 3 [default = UNKNOWN];
|
||||
// optional RecoveryMode DEPRECATED_mode = 3 [default = UNKNOWN];
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
/**
|
||||
|
@ -37,64 +36,51 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
public class SplitLogTask {
|
||||
private final ServerName originServer;
|
||||
private final ZooKeeperProtos.SplitLogTask.State state;
|
||||
private final ZooKeeperProtos.SplitLogTask.RecoveryMode mode;
|
||||
|
||||
public static class Unassigned extends SplitLogTask {
|
||||
public Unassigned(final ServerName originServer, final RecoveryMode mode) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.UNASSIGNED, mode);
|
||||
public Unassigned(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.UNASSIGNED);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Owned extends SplitLogTask {
|
||||
public Owned(final ServerName originServer) {
|
||||
this(originServer, ZooKeeperProtos.SplitLogTask.RecoveryMode.LOG_SPLITTING);
|
||||
}
|
||||
|
||||
public Owned(final ServerName originServer, final RecoveryMode mode) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.OWNED, mode);
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.OWNED);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Resigned extends SplitLogTask {
|
||||
public Resigned(final ServerName originServer, final RecoveryMode mode) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.RESIGNED, mode);
|
||||
public Resigned(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.RESIGNED);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Done extends SplitLogTask {
|
||||
public Done(final ServerName originServer, final RecoveryMode mode) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.DONE, mode);
|
||||
public Done(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.DONE);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Err extends SplitLogTask {
|
||||
public Err(final ServerName originServer, final RecoveryMode mode) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.ERR, mode);
|
||||
public Err(final ServerName originServer) {
|
||||
super(originServer, ZooKeeperProtos.SplitLogTask.State.ERR);
|
||||
}
|
||||
}
|
||||
|
||||
SplitLogTask(final ZooKeeperProtos.SplitLogTask slt) {
|
||||
this.originServer = ProtobufUtil.toServerName(slt.getServerName());
|
||||
this.state = slt.getState();
|
||||
this.mode = (slt.hasMode()) ? slt.getMode() :
|
||||
ZooKeeperProtos.SplitLogTask.RecoveryMode.UNKNOWN;
|
||||
}
|
||||
|
||||
SplitLogTask(final ServerName originServer, final ZooKeeperProtos.SplitLogTask.State state,
|
||||
final ZooKeeperProtos.SplitLogTask.RecoveryMode mode) {
|
||||
SplitLogTask(final ServerName originServer, final ZooKeeperProtos.SplitLogTask.State state) {
|
||||
this.originServer = originServer;
|
||||
this.state = state;
|
||||
this.mode = mode;
|
||||
}
|
||||
|
||||
public ServerName getServerName() {
|
||||
return this.originServer;
|
||||
}
|
||||
|
||||
public ZooKeeperProtos.SplitLogTask.RecoveryMode getMode() {
|
||||
return this.mode;
|
||||
}
|
||||
|
||||
public boolean isUnassigned(final ServerName sn) {
|
||||
return this.originServer.equals(sn) && isUnassigned();
|
||||
}
|
||||
|
@ -182,8 +168,7 @@ public class SplitLogTask {
|
|||
// pbs just created.
|
||||
HBaseProtos.ServerName snpb = ProtobufUtil.toServerName(this.originServer);
|
||||
ZooKeeperProtos.SplitLogTask slts =
|
||||
ZooKeeperProtos.SplitLogTask.newBuilder().setServerName(snpb).setState(this.state).
|
||||
setMode(this.mode).build();
|
||||
ZooKeeperProtos.SplitLogTask.newBuilder().setServerName(snpb).setState(this.state).build();
|
||||
return ProtobufUtil.prependPBMagic(slts.toByteArray());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,19 +20,16 @@
|
|||
package org.apache.hadoop.hbase.coordination;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.master.MasterServices;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
|
||||
/**
|
||||
* Coordination for SplitLogManager. It creates and works with tasks for split log operations<BR>
|
||||
|
@ -41,20 +38,11 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLo
|
|||
* {@link #remainingTasksInCoordination()} and waits until it become zero.
|
||||
* <P>
|
||||
* Methods required for task life circle: <BR>
|
||||
* {@link #markRegionsRecovering(ServerName, Set)} mark regions for log replaying. Used by
|
||||
* {@link org.apache.hadoop.hbase.master.MasterFileSystem} <BR>
|
||||
* {@link #removeRecoveringRegions(Set, Boolean)} make regions cleanup that previous were marked as
|
||||
* recovering. Called after all tasks processed <BR>
|
||||
* {@link #removeStaleRecoveringRegions(Set)} remove stale recovering. called by
|
||||
* {@link org.apache.hadoop.hbase.master.MasterFileSystem} after Active Master is initialized <BR>
|
||||
* {@link #getLastRecoveryTime()} required for garbage collector and should indicate when the last
|
||||
* recovery has been made<BR>
|
||||
* {@link #checkTaskStillAvailable(String)} Check that task is still there <BR>
|
||||
* {@link #checkTasks()} check for unassigned tasks and resubmit them
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface SplitLogManagerCoordination {
|
||||
|
||||
/**
|
||||
* Detail class that shares data between coordination and split log manager
|
||||
*/
|
||||
|
@ -116,29 +104,11 @@ public interface SplitLogManagerCoordination {
|
|||
*/
|
||||
String prepareTask(String taskName);
|
||||
|
||||
/**
|
||||
* Mark regions in recovering state for distributed log replay
|
||||
* @param serverName server name
|
||||
* @param userRegions set of regions to be marked
|
||||
* @throws IOException in case of failure
|
||||
* @throws InterruptedIOException
|
||||
*/
|
||||
void markRegionsRecovering(final ServerName serverName, Set<RegionInfo> userRegions)
|
||||
throws IOException, InterruptedIOException;
|
||||
|
||||
/**
|
||||
* tells Coordination that it should check for new tasks
|
||||
*/
|
||||
void checkTasks();
|
||||
|
||||
/**
|
||||
* It removes recovering regions from Coordination
|
||||
* @param serverNames servers which are just recovered
|
||||
* @param isMetaRecovery whether current recovery is for the meta region on
|
||||
* <code>serverNames</code>
|
||||
*/
|
||||
void removeRecoveringRegions(Set<String> serverNames, Boolean isMetaRecovery) throws IOException;
|
||||
|
||||
/**
|
||||
* Return the number of remaining tasks
|
||||
*/
|
||||
|
@ -150,23 +120,6 @@ public interface SplitLogManagerCoordination {
|
|||
*/
|
||||
void checkTaskStillAvailable(String task);
|
||||
|
||||
/**
|
||||
* Change the recovery mode.
|
||||
* @param b the recovery mode state
|
||||
* @throws InterruptedIOException
|
||||
* @throws IOException in case of failure
|
||||
*/
|
||||
void setRecoveryMode(boolean b) throws IOException;
|
||||
|
||||
/**
|
||||
* Removes known stale servers
|
||||
* @param knownServers set of previously failed servers
|
||||
* @throws IOException in case of failure
|
||||
* @throws InterruptedIOException
|
||||
*/
|
||||
void removeStaleRecoveringRegions(Set<String> knownServers) throws IOException,
|
||||
InterruptedIOException;
|
||||
|
||||
/**
|
||||
* Resubmit the task in case if found unassigned or failed
|
||||
* @param taskName path related to task
|
||||
|
@ -187,28 +140,6 @@ public interface SplitLogManagerCoordination {
|
|||
*/
|
||||
void deleteTask(String taskName);
|
||||
|
||||
/**
|
||||
* @return shows whether the log recovery mode is in replaying state
|
||||
*/
|
||||
boolean isReplaying();
|
||||
|
||||
/**
|
||||
* @return shows whether the log recovery mode is in splitting state
|
||||
*/
|
||||
boolean isSplitting();
|
||||
|
||||
/**
|
||||
* @return the time of last attempt to recover
|
||||
*/
|
||||
long getLastRecoveryTime();
|
||||
|
||||
/**
|
||||
* Temporary function, mostly for UTs. In the regular code isReplaying or isSplitting should be
|
||||
* used.
|
||||
* @return the current log recovery mode.
|
||||
*/
|
||||
RecoveryMode getRecoveryMode();
|
||||
|
||||
/**
|
||||
* Support method to init constants such as timeout. Mostly required for UTs.
|
||||
* @throws IOException
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.coordination;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -25,7 +24,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
|
@ -33,7 +31,7 @@ import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
|||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Coordinated operations for {@link SplitLogWorker} and
|
||||
* Coordinated operations for {@link SplitLogWorker} and
|
||||
* {@link org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler} Important
|
||||
* methods for SplitLogWorker: <BR>
|
||||
* {@link #isReady()} called from {@link SplitLogWorker#run()} to check whether the coordination is
|
||||
|
@ -51,7 +49,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
|
|||
public interface SplitLogWorkerCoordination {
|
||||
|
||||
/* SplitLogWorker part */
|
||||
public static final int DEFAULT_MAX_SPLITTERS = 2;
|
||||
int DEFAULT_MAX_SPLITTERS = 2;
|
||||
|
||||
/**
|
||||
* Initialize internal values. This method should be used when corresponding SplitLogWorker
|
||||
|
@ -127,15 +125,11 @@ public interface SplitLogWorkerCoordination {
|
|||
* Interface for log-split tasks Used to carry implementation details in encapsulated way through
|
||||
* Handlers to the coordination API.
|
||||
*/
|
||||
static interface SplitTaskDetails {
|
||||
interface SplitTaskDetails {
|
||||
|
||||
/**
|
||||
* @return full file path in HDFS for the WAL file to be split.
|
||||
*/
|
||||
String getWALFile();
|
||||
}
|
||||
|
||||
RegionStoreSequenceIds getRegionFlushedSequenceId(String failedServerName, String key)
|
||||
throws IOException;
|
||||
|
||||
}
|
||||
|
|
|
@ -27,23 +27,15 @@ import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.S
|
|||
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination.TaskFinisher.Status;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective;
|
||||
|
@ -51,7 +43,6 @@ import org.apache.hadoop.hbase.master.SplitLogManager.Task;
|
|||
import org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
|
@ -67,8 +58,6 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
|
|||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
|
||||
/**
|
||||
* ZooKeeper based implementation of
|
||||
* {@link SplitLogManagerCoordination}
|
||||
|
@ -92,16 +81,8 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
|
||||
SplitLogManagerDetails details;
|
||||
|
||||
// When lastRecoveringNodeCreationTime is older than the following threshold, we'll check
|
||||
// whether to GC stale recovering znodes
|
||||
private volatile long lastRecoveringNodeCreationTime = 0;
|
||||
|
||||
public boolean ignoreZKDeleteForTesting = false;
|
||||
|
||||
private RecoveryMode recoveryMode;
|
||||
|
||||
private boolean isDrainingDone = false;
|
||||
|
||||
public ZKSplitLogManagerCoordination(Configuration conf, ZooKeeperWatcher watcher) {
|
||||
super(watcher);
|
||||
this.conf = conf;
|
||||
|
@ -124,7 +105,6 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
this.zkretries = conf.getLong("hbase.splitlog.zk.retries", DEFAULT_ZK_RETRIES);
|
||||
this.resubmitThreshold = conf.getLong("hbase.splitlog.max.resubmit", DEFAULT_MAX_RESUBMIT);
|
||||
this.timeout = conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT, DEFAULT_TIMEOUT);
|
||||
setRecoveryMode(true);
|
||||
if (this.watcher != null) {
|
||||
this.watcher.registerListener(this);
|
||||
lookForOrphans();
|
||||
|
@ -222,7 +202,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
}
|
||||
LOG.info("resubmitting task " + path);
|
||||
task.incarnation.incrementAndGet();
|
||||
boolean result = resubmit(this.details.getServerName(), path, version);
|
||||
boolean result = resubmit(path, version);
|
||||
if (!result) {
|
||||
task.heartbeatNoDetails(EnvironmentEdgeManager.currentTime());
|
||||
return false;
|
||||
|
@ -254,7 +234,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
// might miss the watch-trigger that creation of RESCAN node provides.
|
||||
// Since the TimeoutMonitor will keep resubmitting UNASSIGNED tasks
|
||||
// therefore this behavior is safe.
|
||||
SplitLogTask slt = new SplitLogTask.Done(this.details.getServerName(), getRecoveryMode());
|
||||
SplitLogTask slt = new SplitLogTask.Done(this.details.getServerName());
|
||||
this.watcher
|
||||
.getRecoverableZooKeeper()
|
||||
.getZooKeeper()
|
||||
|
@ -278,83 +258,6 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
SplitLogCounters.tot_mgr_get_data_queued.increment();
|
||||
}
|
||||
|
||||
/**
|
||||
* It removes recovering regions under /hbase/recovering-regions/[encoded region name] so that the
|
||||
* region server hosting the region can allow reads to the recovered region
|
||||
* @param recoveredServerNameSet servers which are just recovered
|
||||
* @param isMetaRecovery whether current recovery is for the meta region on
|
||||
* <code>serverNames</code>
|
||||
*/
|
||||
@Override
|
||||
public void removeRecoveringRegions(final Set<String> recoveredServerNameSet,
|
||||
Boolean isMetaRecovery)
|
||||
throws IOException {
|
||||
final String metaEncodeRegionName = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName();
|
||||
int count = 0;
|
||||
try {
|
||||
List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode);
|
||||
if (tasks != null) {
|
||||
int listSize = tasks.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
if (!ZKSplitLog.isRescanNode(tasks.get(i))) {
|
||||
count++;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (count == 0 && this.details.getMaster().isInitialized()
|
||||
&& !this.details.getMaster().getServerManager().areDeadServersInProgress()) {
|
||||
// No splitting work items left
|
||||
ZKSplitLog.deleteRecoveringRegionZNodes(watcher, null);
|
||||
// reset lastRecoveringNodeCreationTime because we cleared all recovering znodes at
|
||||
// this point.
|
||||
lastRecoveringNodeCreationTime = Long.MAX_VALUE;
|
||||
} else if (!recoveredServerNameSet.isEmpty()) {
|
||||
// Remove recovering regions which don't have any RS associated with it
|
||||
List<String> regions = ZKUtil.listChildrenNoWatch(watcher,
|
||||
watcher.znodePaths.recoveringRegionsZNode);
|
||||
if (regions != null) {
|
||||
int listSize = regions.size();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Processing recovering " + regions + " and servers " +
|
||||
recoveredServerNameSet + ", isMetaRecovery=" + isMetaRecovery);
|
||||
}
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
String region = regions.get(i);
|
||||
if (isMetaRecovery != null) {
|
||||
if ((isMetaRecovery && !region.equalsIgnoreCase(metaEncodeRegionName))
|
||||
|| (!isMetaRecovery && region.equalsIgnoreCase(metaEncodeRegionName))) {
|
||||
// skip non-meta regions when recovering the meta region or
|
||||
// skip the meta region when recovering user regions
|
||||
continue;
|
||||
}
|
||||
}
|
||||
String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, region);
|
||||
List<String> failedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
|
||||
if (failedServers == null || failedServers.isEmpty()) {
|
||||
ZKUtil.deleteNode(watcher, nodePath);
|
||||
continue;
|
||||
}
|
||||
if (recoveredServerNameSet.containsAll(failedServers)) {
|
||||
ZKUtil.deleteNodeRecursively(watcher, nodePath);
|
||||
} else {
|
||||
int tmpFailedServerSize = failedServers.size();
|
||||
for (int j = 0; j < tmpFailedServerSize; j++) {
|
||||
String failedServer = failedServers.get(j);
|
||||
if (recoveredServerNameSet.contains(failedServer)) {
|
||||
String tmpPath = ZKUtil.joinZNode(nodePath, failedServer);
|
||||
ZKUtil.deleteNode(watcher, tmpPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (KeeperException ke) {
|
||||
LOG.warn("removeRecoveringRegionsFromZK got zookeeper exception. Will retry", ke);
|
||||
throw new IOException(ke);
|
||||
}
|
||||
}
|
||||
|
||||
private void deleteNode(String path, Long retries) {
|
||||
SplitLogCounters.tot_mgr_node_delete_queued.increment();
|
||||
// Once a task znode is ready for delete, that is it is in the TASK_DONE
|
||||
|
@ -415,7 +318,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
}
|
||||
|
||||
private void createNode(String path, Long retry_count) {
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(details.getServerName(), getRecoveryMode());
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(details.getServerName());
|
||||
ZKUtil.asyncCreate(this.watcher, path, slt.toByteArray(), new CreateAsyncCallback(),
|
||||
retry_count);
|
||||
SplitLogCounters.tot_mgr_node_create_queued.increment();
|
||||
|
@ -439,7 +342,6 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
SplitLogCounters.tot_mgr_get_data_queued.increment();
|
||||
}
|
||||
|
||||
|
||||
private void getDataSetWatchSuccess(String path, byte[] data, int version)
|
||||
throws DeserializationException {
|
||||
if (data == null) {
|
||||
|
@ -589,77 +491,6 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
+ " rescan nodes");
|
||||
}
|
||||
|
||||
/**
|
||||
* Create znodes /hbase/recovering-regions/[region_ids...]/[failed region server names ...] for
|
||||
* all regions of the passed in region servers
|
||||
* @param serverName the name of a region server
|
||||
* @param userRegions user regiones assigned on the region server
|
||||
*/
|
||||
@Override
|
||||
public void markRegionsRecovering(final ServerName serverName, Set<RegionInfo> userRegions)
|
||||
throws IOException, InterruptedIOException {
|
||||
this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTime();
|
||||
for (RegionInfo region : userRegions) {
|
||||
String regionEncodeName = region.getEncodedName();
|
||||
long retries = this.zkretries;
|
||||
|
||||
do {
|
||||
String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode,
|
||||
regionEncodeName);
|
||||
long lastRecordedFlushedSequenceId = -1;
|
||||
try {
|
||||
long lastSequenceId =
|
||||
this.details.getMaster().getServerManager()
|
||||
.getLastFlushedSequenceId(regionEncodeName.getBytes()).getLastFlushedSequenceId();
|
||||
|
||||
/*
|
||||
* znode layout: .../region_id[last known flushed sequence id]/failed server[last known
|
||||
* flushed sequence id for the server]
|
||||
*/
|
||||
byte[] data = ZKUtil.getData(this.watcher, nodePath);
|
||||
if (data == null) {
|
||||
ZKUtil
|
||||
.createSetData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId));
|
||||
} else {
|
||||
lastRecordedFlushedSequenceId =
|
||||
ZKSplitLog.parseLastFlushedSequenceIdFrom(data);
|
||||
if (lastRecordedFlushedSequenceId < lastSequenceId) {
|
||||
// update last flushed sequence id in the region level
|
||||
ZKUtil.setData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId));
|
||||
}
|
||||
}
|
||||
// go one level deeper with server name
|
||||
nodePath = ZKUtil.joinZNode(nodePath, serverName.getServerName());
|
||||
if (lastSequenceId <= lastRecordedFlushedSequenceId) {
|
||||
// the newly assigned RS failed even before any flush to the region
|
||||
lastSequenceId = lastRecordedFlushedSequenceId;
|
||||
}
|
||||
ZKUtil.createSetData(this.watcher, nodePath,
|
||||
ZKUtil.regionSequenceIdsToByteArray(lastSequenceId, null));
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Marked " + regionEncodeName + " recovering from " + serverName +
|
||||
": " + nodePath);
|
||||
}
|
||||
// break retry loop
|
||||
break;
|
||||
} catch (KeeperException e) {
|
||||
// ignore ZooKeeper exceptions inside retry loop
|
||||
if (retries <= 1) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
// wait a little bit for retry
|
||||
try {
|
||||
Thread.sleep(20);
|
||||
} catch (InterruptedException e1) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
} while ((--retries) > 0);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void nodeDataChanged(String path) {
|
||||
Task task;
|
||||
|
@ -672,216 +503,11 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* ZooKeeper implementation of
|
||||
* {@link SplitLogManagerCoordination#removeStaleRecoveringRegions(Set)}
|
||||
*/
|
||||
@Override
|
||||
public void removeStaleRecoveringRegions(final Set<String> knownFailedServers)
|
||||
throws IOException, InterruptedIOException {
|
||||
|
||||
try {
|
||||
List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.znodePaths.splitLogZNode);
|
||||
if (tasks != null) {
|
||||
int listSize = tasks.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
String t = tasks.get(i);
|
||||
byte[] data;
|
||||
try {
|
||||
data = ZKUtil.getData(this.watcher,
|
||||
ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, t));
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
if (data != null) {
|
||||
SplitLogTask slt = null;
|
||||
try {
|
||||
slt = SplitLogTask.parseFrom(data);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse data for znode " + t, e);
|
||||
}
|
||||
if (slt != null && slt.isDone()) {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
// decode the file name
|
||||
t = ZKSplitLog.getFileName(t);
|
||||
ServerName serverName = AbstractFSWALProvider
|
||||
.getServerNameFromWALDirectoryName(new Path(t));
|
||||
if (serverName != null) {
|
||||
knownFailedServers.add(serverName.getServerName());
|
||||
} else {
|
||||
LOG.warn("Found invalid WAL log file name:" + t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// remove recovering regions which doesn't have any RS associated with it
|
||||
List<String> regions = ZKUtil.listChildrenNoWatch(watcher,
|
||||
watcher.znodePaths.recoveringRegionsZNode);
|
||||
if (regions != null) {
|
||||
int listSize = regions.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode,
|
||||
regions.get(i));
|
||||
List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
|
||||
if (regionFailedServers == null || regionFailedServers.isEmpty()) {
|
||||
ZKUtil.deleteNode(watcher, nodePath);
|
||||
continue;
|
||||
}
|
||||
boolean needMoreRecovery = false;
|
||||
int tmpFailedServerSize = regionFailedServers.size();
|
||||
for (int j = 0; j < tmpFailedServerSize; j++) {
|
||||
if (knownFailedServers.contains(regionFailedServers.get(j))) {
|
||||
needMoreRecovery = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!needMoreRecovery) {
|
||||
ZKUtil.deleteNodeRecursively(watcher, nodePath);
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean isReplaying() {
|
||||
return this.recoveryMode == RecoveryMode.LOG_REPLAY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized boolean isSplitting() {
|
||||
return this.recoveryMode == RecoveryMode.LOG_SPLITTING;
|
||||
}
|
||||
|
||||
private List<String> listSplitLogTasks() throws KeeperException {
|
||||
List<String> taskOrRescanList = ZKUtil.listChildrenNoWatch(watcher,
|
||||
watcher.znodePaths.splitLogZNode);
|
||||
if (taskOrRescanList == null || taskOrRescanList.isEmpty()) {
|
||||
return Collections.<String> emptyList();
|
||||
}
|
||||
List<String> taskList = new ArrayList<>();
|
||||
for (String taskOrRescan : taskOrRescanList) {
|
||||
// Remove rescan nodes
|
||||
if (!ZKSplitLog.isRescanNode(taskOrRescan)) {
|
||||
taskList.add(taskOrRescan);
|
||||
}
|
||||
}
|
||||
return taskList;
|
||||
}
|
||||
|
||||
/**
|
||||
* This function is to set recovery mode from outstanding split log tasks from before or current
|
||||
* configuration setting
|
||||
* @param isForInitialization
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public void setRecoveryMode(boolean isForInitialization) throws IOException {
|
||||
synchronized(this) {
|
||||
if (this.isDrainingDone) {
|
||||
// when there is no outstanding splitlogtask after master start up, we already have up to
|
||||
// date recovery mode
|
||||
return;
|
||||
}
|
||||
}
|
||||
if (this.watcher == null) {
|
||||
// when watcher is null(testing code) and recovery mode can only be LOG_SPLITTING
|
||||
synchronized(this) {
|
||||
this.isDrainingDone = true;
|
||||
this.recoveryMode = RecoveryMode.LOG_SPLITTING;
|
||||
}
|
||||
return;
|
||||
}
|
||||
boolean hasSplitLogTask = false;
|
||||
boolean hasRecoveringRegions = false;
|
||||
RecoveryMode previousRecoveryMode = RecoveryMode.UNKNOWN;
|
||||
RecoveryMode recoveryModeInConfig =
|
||||
(isDistributedLogReplay(conf)) ? RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING;
|
||||
|
||||
// Firstly check if there are outstanding recovering regions
|
||||
try {
|
||||
List<String> regions = ZKUtil.listChildrenNoWatch(watcher,
|
||||
watcher.znodePaths.recoveringRegionsZNode);
|
||||
if (regions != null && !regions.isEmpty()) {
|
||||
hasRecoveringRegions = true;
|
||||
previousRecoveryMode = RecoveryMode.LOG_REPLAY;
|
||||
}
|
||||
if (previousRecoveryMode == RecoveryMode.UNKNOWN) {
|
||||
// Secondly check if there are outstanding split log task
|
||||
List<String> tasks = listSplitLogTasks();
|
||||
if (!tasks.isEmpty()) {
|
||||
hasSplitLogTask = true;
|
||||
if (isForInitialization) {
|
||||
// during initialization, try to get recovery mode from splitlogtask
|
||||
int listSize = tasks.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
String task = tasks.get(i);
|
||||
try {
|
||||
byte[] data = ZKUtil.getData(this.watcher,
|
||||
ZKUtil.joinZNode(watcher.znodePaths.splitLogZNode, task));
|
||||
if (data == null) continue;
|
||||
SplitLogTask slt = SplitLogTask.parseFrom(data);
|
||||
previousRecoveryMode = slt.getMode();
|
||||
if (previousRecoveryMode == RecoveryMode.UNKNOWN) {
|
||||
// created by old code base where we don't set recovery mode in splitlogtask
|
||||
// we can safely set to LOG_SPLITTING because we're in master initialization code
|
||||
// before SSH is enabled & there is no outstanding recovering regions
|
||||
previousRecoveryMode = RecoveryMode.LOG_SPLITTING;
|
||||
}
|
||||
break;
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Failed parse data for znode " + task, e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
synchronized (this) {
|
||||
if (this.isDrainingDone) {
|
||||
return;
|
||||
}
|
||||
if (!hasSplitLogTask && !hasRecoveringRegions) {
|
||||
this.isDrainingDone = true;
|
||||
this.recoveryMode = recoveryModeInConfig;
|
||||
return;
|
||||
} else if (!isForInitialization) {
|
||||
// splitlogtask hasn't drained yet, keep existing recovery mode
|
||||
return;
|
||||
}
|
||||
|
||||
if (previousRecoveryMode != RecoveryMode.UNKNOWN) {
|
||||
this.isDrainingDone = (previousRecoveryMode == recoveryModeInConfig);
|
||||
this.recoveryMode = previousRecoveryMode;
|
||||
} else {
|
||||
this.recoveryMode = recoveryModeInConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns if distributed log replay is turned on or not
|
||||
* @param conf
|
||||
* @return true when distributed log replay is turned on
|
||||
*/
|
||||
private boolean isDistributedLogReplay(Configuration conf) {
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean resubmit(ServerName serverName, String path, int version) {
|
||||
private boolean resubmit(String path, int version) {
|
||||
try {
|
||||
// blocking zk call but this is done from the timeout thread
|
||||
SplitLogTask slt =
|
||||
new SplitLogTask.Unassigned(this.details.getServerName(), getRecoveryMode());
|
||||
new SplitLogTask.Unassigned(this.details.getServerName());
|
||||
if (ZKUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) {
|
||||
LOG.debug("failed to resubmit task " + path + " version changed");
|
||||
return false;
|
||||
|
@ -1109,16 +735,6 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
|
|||
return details;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized RecoveryMode getRecoveryMode() {
|
||||
return recoveryMode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLastRecoveryTime() {
|
||||
return lastRecoveringNodeCreationTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Temporary function that is used by unit tests only
|
||||
*/
|
||||
|
|
|
@ -19,10 +19,7 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coordination;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
|
@ -39,13 +36,9 @@ import org.apache.hadoop.hbase.SplitLogCounters;
|
|||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -78,7 +71,6 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
|
||||
private TaskExecutor splitTaskExecutor;
|
||||
|
||||
private final Object taskReadyLock = new Object();
|
||||
private AtomicInteger taskReadySeq = new AtomicInteger(0);
|
||||
private volatile String currentTask = null;
|
||||
private int currentVersion;
|
||||
|
@ -103,10 +95,12 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
@Override
|
||||
public void nodeChildrenChanged(String path) {
|
||||
if (path.equals(watcher.znodePaths.splitLogZNode)) {
|
||||
if (LOG.isTraceEnabled()) LOG.trace("tasks arrived or departed on " + path);
|
||||
synchronized (taskReadyLock) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("tasks arrived or departed on " + path);
|
||||
}
|
||||
synchronized (taskReadySeq) {
|
||||
this.taskReadySeq.incrementAndGet();
|
||||
taskReadyLock.notify();
|
||||
taskReadySeq.notify();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -240,8 +234,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
}
|
||||
|
||||
currentVersion =
|
||||
attemptToOwnTask(true, watcher, server.getServerName(), path,
|
||||
slt.getMode(), stat.getVersion());
|
||||
attemptToOwnTask(true, watcher, server.getServerName(), path, stat.getVersion());
|
||||
if (currentVersion < 0) {
|
||||
SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.increment();
|
||||
return;
|
||||
|
@ -253,7 +246,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
splitTaskDetails.setTaskNode(currentTask);
|
||||
splitTaskDetails.setCurTaskZKVersion(new MutableInt(currentVersion));
|
||||
|
||||
endTask(new SplitLogTask.Done(server.getServerName(), slt.getMode()),
|
||||
endTask(new SplitLogTask.Done(server.getServerName()),
|
||||
SplitLogCounters.tot_wkr_task_acquired_rescan, splitTaskDetails);
|
||||
return;
|
||||
}
|
||||
|
@ -262,7 +255,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
SplitLogCounters.tot_wkr_task_acquired.increment();
|
||||
getDataSetWatchAsync();
|
||||
|
||||
submitTask(path, slt.getMode(), currentVersion, reportPeriod);
|
||||
submitTask(path, currentVersion, reportPeriod);
|
||||
|
||||
// after a successful submit, sleep a little bit to allow other RSs to grab the rest tasks
|
||||
try {
|
||||
|
@ -287,8 +280,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
* @param curTask task to submit
|
||||
* @param curTaskZKVersion current version of task
|
||||
*/
|
||||
void submitTask(final String curTask, final RecoveryMode mode, final int curTaskZKVersion,
|
||||
final int reportPeriod) {
|
||||
void submitTask(final String curTask, final int curTaskZKVersion, final int reportPeriod) {
|
||||
final MutableInt zkVersion = new MutableInt(curTaskZKVersion);
|
||||
|
||||
CancelableProgressable reporter = new CancelableProgressable() {
|
||||
|
@ -301,7 +293,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
last_report_at = t;
|
||||
int latestZKVersion =
|
||||
attemptToOwnTask(false, watcher, server.getServerName(), curTask,
|
||||
mode, zkVersion.intValue());
|
||||
zkVersion.intValue());
|
||||
if (latestZKVersion < 0) {
|
||||
LOG.warn("Failed to heartbeat the task" + curTask);
|
||||
return false;
|
||||
|
@ -318,7 +310,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
|
||||
WALSplitterHandler hsh =
|
||||
new WALSplitterHandler(server, this, splitTaskDetails, reporter,
|
||||
this.tasksInProgress, splitTaskExecutor, mode);
|
||||
this.tasksInProgress, splitTaskExecutor);
|
||||
server.getExecutorService().submit(hsh);
|
||||
}
|
||||
|
||||
|
@ -361,10 +353,10 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
* @return non-negative integer value when task can be owned by current region server otherwise -1
|
||||
*/
|
||||
protected static int attemptToOwnTask(boolean isFirstTime, ZooKeeperWatcher zkw,
|
||||
ServerName server, String task, RecoveryMode mode, int taskZKVersion) {
|
||||
ServerName server, String task, int taskZKVersion) {
|
||||
int latestZKVersion = FAILED_TO_OWN_TASK;
|
||||
try {
|
||||
SplitLogTask slt = new SplitLogTask.Owned(server, mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(server);
|
||||
Stat stat = zkw.getRecoverableZooKeeper().setData(task, slt.toByteArray(), taskZKVersion);
|
||||
if (stat == null) {
|
||||
LOG.warn("zk.setData() returned null for path " + task);
|
||||
|
@ -398,7 +390,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
* policy puts an upper-limit on the number of simultaneous log splitting that could be happening
|
||||
* in a cluster.
|
||||
* <p>
|
||||
* Synchronization using <code>taskReadyLock</code> ensures that it will try to grab every task
|
||||
* Synchronization using <code>taskReadySeq</code> ensures that it will try to grab every task
|
||||
* that has been put up
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
|
@ -406,7 +398,7 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
public void taskLoop() throws InterruptedException {
|
||||
while (!shouldStop) {
|
||||
int seq_start = taskReadySeq.get();
|
||||
List<String> paths = null;
|
||||
List<String> paths;
|
||||
paths = getTaskList();
|
||||
if (paths == null) {
|
||||
LOG.warn("Could not get tasks, did someone remove " + watcher.znodePaths.splitLogZNode
|
||||
|
@ -438,41 +430,9 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
}
|
||||
}
|
||||
SplitLogCounters.tot_wkr_task_grabing.increment();
|
||||
synchronized (taskReadyLock) {
|
||||
synchronized (taskReadySeq) {
|
||||
while (seq_start == taskReadySeq.get()) {
|
||||
taskReadyLock.wait(checkInterval);
|
||||
if (server != null) {
|
||||
// check to see if we have stale recovering regions in our internal memory state
|
||||
Map<String, HRegion> recoveringRegions = server.getRecoveringRegions();
|
||||
if (!recoveringRegions.isEmpty()) {
|
||||
// Make a local copy to prevent ConcurrentModificationException when other threads
|
||||
// modify recoveringRegions
|
||||
List<String> tmpCopy = new ArrayList<>(recoveringRegions.keySet());
|
||||
int listSize = tmpCopy.size();
|
||||
for (int i = 0; i < listSize; i++) {
|
||||
String region = tmpCopy.get(i);
|
||||
String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode,
|
||||
region);
|
||||
try {
|
||||
if (ZKUtil.checkExists(watcher, nodePath) == -1) {
|
||||
server.getExecutorService().submit(
|
||||
new FinishRegionRecoveringHandler(server, region, nodePath));
|
||||
} else {
|
||||
// current check is a defensive(or redundant) mechanism to prevent us from
|
||||
// having stale recovering regions in our internal RS memory state while
|
||||
// zookeeper(source of truth) says differently. We stop at the first good one
|
||||
// because we should not have a single instance such as this in normal case so
|
||||
// check the first one is good enough.
|
||||
break;
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
// ignore zookeeper error
|
||||
LOG.debug("Got a zookeeper when trying to open a recovering region", e);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
taskReadySeq.wait(checkInterval);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -550,12 +510,6 @@ public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
|
|||
return shouldStop;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RegionStoreSequenceIds getRegionFlushedSequenceId(String failedServerName, String key)
|
||||
throws IOException {
|
||||
return ZKSplitLog.getRegionFlushedSequenceId(watcher, failedServerName, key);
|
||||
}
|
||||
|
||||
/**
|
||||
* Asynchronous handler for zk get-data-set-watch on node results.
|
||||
*/
|
||||
|
|
|
@ -118,12 +118,6 @@ public interface RegionObserver {
|
|||
*/
|
||||
default void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {}
|
||||
|
||||
/**
|
||||
* Called after the log replay on the region is over.
|
||||
* @param c the environment provided by the region server
|
||||
*/
|
||||
default void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> c) {}
|
||||
|
||||
/**
|
||||
* Called before the memstore is flushed to disk.
|
||||
* @param c the environment provided by the region server
|
||||
|
@ -965,6 +959,7 @@ public interface RegionObserver {
|
|||
* @param info the RegionInfo for this region
|
||||
* @param edits the file of recovered edits
|
||||
*/
|
||||
// todo: what about these?
|
||||
default void preReplayWALs(ObserverContext<? extends RegionCoprocessorEnvironment> ctx,
|
||||
RegionInfo info, Path edits) throws IOException {}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
|
@ -37,7 +36,6 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
|
@ -84,7 +82,6 @@ public class MasterWalManager {
|
|||
// create the split log lock
|
||||
private final Lock splitLogLock = new ReentrantLock();
|
||||
private final SplitLogManager splitLogManager;
|
||||
private final boolean distributedLogReplay;
|
||||
|
||||
// Is the fileystem ok?
|
||||
private volatile boolean fsOk = true;
|
||||
|
@ -101,7 +98,6 @@ public class MasterWalManager {
|
|||
this.rootDir = rootDir;
|
||||
this.services = services;
|
||||
this.splitLogManager = new SplitLogManager(services, conf);
|
||||
this.distributedLogReplay = this.splitLogManager.isLogReplaying();
|
||||
|
||||
this.oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
|
||||
}
|
||||
|
@ -279,35 +275,10 @@ public class MasterWalManager {
|
|||
return logDirs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark regions in recovering state when distributedLogReplay are set true
|
||||
* @param serverName Failed region server whose wals to be replayed
|
||||
* @param regions Set of regions to be recovered
|
||||
*/
|
||||
public void prepareLogReplay(ServerName serverName, Set<RegionInfo> regions) throws IOException {
|
||||
if (!this.distributedLogReplay) {
|
||||
return;
|
||||
}
|
||||
// mark regions in recovering state
|
||||
if (regions == null || regions.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
this.splitLogManager.markRegionsRecovering(serverName, regions);
|
||||
}
|
||||
|
||||
public void splitLog(final Set<ServerName> serverNames) throws IOException {
|
||||
splitLog(serverNames, NON_META_FILTER);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wrapper function on {@link SplitLogManager#removeStaleRecoveringRegions(Set)}
|
||||
* @param failedServers A set of known failed servers
|
||||
*/
|
||||
void removeStaleRecoveringRegionsFromZK(final Set<ServerName> failedServers)
|
||||
throws IOException, InterruptedIOException {
|
||||
this.splitLogManager.removeStaleRecoveringRegions(failedServers);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is the base split method that splits WAL files matching a filter. Callers should
|
||||
* pass the appropriate filter for meta and non-meta WALs.
|
||||
|
|
|
@ -25,7 +25,6 @@ import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.I
|
|||
import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
@ -36,7 +35,6 @@ import java.util.Set;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -50,19 +48,16 @@ import org.apache.hadoop.hbase.ScheduledChore;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
|
||||
/**
|
||||
* Distributes the task of log splitting to the available region servers.
|
||||
|
@ -106,15 +101,6 @@ public class SplitLogManager {
|
|||
|
||||
private long unassignedTimeout;
|
||||
private long lastTaskCreateTime = Long.MAX_VALUE;
|
||||
private long checkRecoveringTimeThreshold = 15000; // 15 seconds
|
||||
private final List<Pair<Set<ServerName>, Boolean>> failedRecoveringRegionDeletions = Collections
|
||||
.synchronizedList(new ArrayList<Pair<Set<ServerName>, Boolean>>());
|
||||
|
||||
/**
|
||||
* In distributedLogReplay mode, we need touch both splitlog and recovering-regions znodes in one
|
||||
* operation. So the lock is used to guard such cases.
|
||||
*/
|
||||
protected final ReentrantLock recoveringRegionLock = new ReentrantLock();
|
||||
|
||||
@VisibleForTesting
|
||||
final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<>();
|
||||
|
@ -141,7 +127,6 @@ public class SplitLogManager {
|
|||
SplitLogManagerDetails details = new SplitLogManagerDetails(tasks, master, failedDeletions);
|
||||
coordination.setDetails(details);
|
||||
coordination.init();
|
||||
// Determine recovery mode
|
||||
}
|
||||
this.unassignedTimeout =
|
||||
conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);
|
||||
|
@ -252,7 +237,6 @@ public class SplitLogManager {
|
|||
long t = EnvironmentEdgeManager.currentTime();
|
||||
long totalSize = 0;
|
||||
TaskBatch batch = new TaskBatch();
|
||||
Boolean isMetaRecovery = (filter == null) ? null : false;
|
||||
for (FileStatus lf : logfiles) {
|
||||
// TODO If the log file is still being written to - which is most likely
|
||||
// the case for the last log file - then its length will show up here
|
||||
|
@ -266,13 +250,6 @@ public class SplitLogManager {
|
|||
}
|
||||
}
|
||||
waitForSplittingCompletion(batch, status);
|
||||
// remove recovering regions
|
||||
if (filter == MasterWalManager.META_FILTER /* reference comparison */) {
|
||||
// we split meta regions and user regions separately therefore logfiles are either all for
|
||||
// meta or user regions but won't for both( we could have mixed situations in tests)
|
||||
isMetaRecovery = true;
|
||||
}
|
||||
removeRecoveringRegions(serverNames, isMetaRecovery);
|
||||
|
||||
if (batch.done != batch.installed) {
|
||||
batch.isDead = true;
|
||||
|
@ -383,61 +360,6 @@ public class SplitLogManager {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* It removes recovering regions under /hbase/recovering-regions/[encoded region name] so that the
|
||||
* region server hosting the region can allow reads to the recovered region
|
||||
* @param serverNames servers which are just recovered
|
||||
* @param isMetaRecovery whether current recovery is for the meta region on {@code serverNames}
|
||||
*/
|
||||
private void removeRecoveringRegions(final Set<ServerName> serverNames, Boolean isMetaRecovery) {
|
||||
if (!isLogReplaying()) {
|
||||
// the function is only used in WALEdit direct replay mode
|
||||
return;
|
||||
}
|
||||
if (serverNames == null || serverNames.isEmpty()) return;
|
||||
|
||||
Set<String> recoveredServerNameSet = new HashSet<>();
|
||||
for (ServerName tmpServerName : serverNames) {
|
||||
recoveredServerNameSet.add(tmpServerName.getServerName());
|
||||
}
|
||||
|
||||
this.recoveringRegionLock.lock();
|
||||
try {
|
||||
getSplitLogManagerCoordination().removeRecoveringRegions(
|
||||
recoveredServerNameSet, isMetaRecovery);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("removeRecoveringRegions got exception. Will retry", e);
|
||||
if (serverNames != null && !serverNames.isEmpty()) {
|
||||
this.failedRecoveringRegionDeletions.add(new Pair<>(serverNames, isMetaRecovery));
|
||||
}
|
||||
} finally {
|
||||
this.recoveringRegionLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* It removes stale recovering regions under /hbase/recovering-regions/[encoded region name]
|
||||
* during master initialization phase.
|
||||
* @param failedServers A set of known failed servers
|
||||
* @throws IOException
|
||||
*/
|
||||
void removeStaleRecoveringRegions(final Set<ServerName> failedServers) throws IOException,
|
||||
InterruptedIOException {
|
||||
Set<String> knownFailedServers = new HashSet<>();
|
||||
if (failedServers != null) {
|
||||
for (ServerName tmpServerName : failedServers) {
|
||||
knownFailedServers.add(tmpServerName.getServerName());
|
||||
}
|
||||
}
|
||||
|
||||
this.recoveringRegionLock.lock();
|
||||
try {
|
||||
getSplitLogManagerCoordination().removeStaleRecoveringRegions(knownFailedServers);
|
||||
} finally {
|
||||
this.recoveringRegionLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @param path
|
||||
* @param batch
|
||||
|
@ -533,54 +455,6 @@ public class SplitLogManager {
|
|||
LOG.info("dead splitlog workers " + serverNames);
|
||||
}
|
||||
|
||||
/**
|
||||
* This function is to set recovery mode from outstanding split log tasks from before or current
|
||||
* configuration setting
|
||||
* @param isForInitialization
|
||||
* @throws IOException throws if it's impossible to set recovery mode
|
||||
*/
|
||||
public void setRecoveryMode(boolean isForInitialization) throws IOException {
|
||||
getSplitLogManagerCoordination().setRecoveryMode(isForInitialization);
|
||||
}
|
||||
|
||||
public void markRegionsRecovering(ServerName server, Set<RegionInfo> userRegions)
|
||||
throws InterruptedIOException, IOException {
|
||||
if (userRegions == null || (!isLogReplaying())) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
this.recoveringRegionLock.lock();
|
||||
// mark that we're creating recovering regions
|
||||
getSplitLogManagerCoordination().markRegionsRecovering(server, userRegions);
|
||||
} finally {
|
||||
this.recoveringRegionLock.unlock();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether log is replaying
|
||||
*/
|
||||
public boolean isLogReplaying() {
|
||||
if (server.getCoordinatedStateManager() == null) return false;
|
||||
return getSplitLogManagerCoordination().isReplaying();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether log is splitting
|
||||
*/
|
||||
public boolean isLogSplitting() {
|
||||
if (server.getCoordinatedStateManager() == null) return false;
|
||||
return getSplitLogManagerCoordination().isSplitting();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the current log recovery mode
|
||||
*/
|
||||
public RecoveryMode getRecoveryMode() {
|
||||
return getSplitLogManagerCoordination().getRecoveryMode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Keeps track of the batch of tasks submitted together by a caller in splitLogDistributed().
|
||||
* Clients threads use this object to wait for all their tasks to be done.
|
||||
|
@ -753,30 +627,11 @@ public class SplitLogManager {
|
|||
}
|
||||
failedDeletions.removeAll(tmpPaths);
|
||||
}
|
||||
|
||||
// Garbage collect left-over
|
||||
long timeInterval =
|
||||
EnvironmentEdgeManager.currentTime()
|
||||
- getSplitLogManagerCoordination().getLastRecoveryTime();
|
||||
if (!failedRecoveringRegionDeletions.isEmpty()
|
||||
|| (tot == 0 && tasks.isEmpty() && (timeInterval > checkRecoveringTimeThreshold))) {
|
||||
// inside the function there have more checks before GC anything
|
||||
if (!failedRecoveringRegionDeletions.isEmpty()) {
|
||||
List<Pair<Set<ServerName>, Boolean>> previouslyFailedDeletions =
|
||||
new ArrayList<>(failedRecoveringRegionDeletions);
|
||||
failedRecoveringRegionDeletions.removeAll(previouslyFailedDeletions);
|
||||
for (Pair<Set<ServerName>, Boolean> failedDeletion : previouslyFailedDeletions) {
|
||||
removeRecoveringRegions(failedDeletion.getFirst(), failedDeletion.getSecond());
|
||||
}
|
||||
} else {
|
||||
removeRecoveringRegions(null, null);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public enum ResubmitDirective {
|
||||
CHECK(), FORCE();
|
||||
CHECK(), FORCE()
|
||||
}
|
||||
|
||||
public enum TerminationStatus {
|
||||
|
|
|
@ -510,7 +510,7 @@ public class RSProcedureDispatcher
|
|||
public OpenRegionRequest.RegionOpenInfo buildRegionOpenInfoRequest(
|
||||
final MasterProcedureEnv env) {
|
||||
return RequestConverter.buildRegionOpenInfo(getRegionInfo(),
|
||||
env.getAssignmentManager().getFavoredNodes(getRegionInfo()), false);
|
||||
env.getAssignmentManager().getFavoredNodes(getRegionInfo()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -124,7 +124,6 @@ import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
|
|||
import org.apache.hadoop.hbase.coprocessor.RegionObserver.MutationType;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
|
||||
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
|
||||
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
|
||||
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
|
||||
import org.apache.hadoop.hbase.filter.ByteArrayComparable;
|
||||
|
@ -369,14 +368,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
/** Saved state from replaying prepare flush cache */
|
||||
private PrepareFlushResult prepareFlushResult = null;
|
||||
|
||||
/**
|
||||
* Config setting for whether to allow writes when a region is in recovering or not.
|
||||
*/
|
||||
private boolean disallowWritesInRecovering = false;
|
||||
|
||||
// When a region is in recovering state, it can only accept writes not reads
|
||||
private volatile boolean recovering = false;
|
||||
|
||||
private volatile Optional<ConfigurationManager> configurationManager;
|
||||
|
||||
// Used for testing.
|
||||
|
@ -798,13 +789,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
|
||||
this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
|
||||
this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper);
|
||||
|
||||
Map<String, HRegion> recoveringRegions = rsServices.getRecoveringRegions();
|
||||
String encodedName = getRegionInfo().getEncodedName();
|
||||
if (recoveringRegions != null && recoveringRegions.containsKey(encodedName)) {
|
||||
this.recovering = true;
|
||||
recoveringRegions.put(encodedName, this);
|
||||
}
|
||||
} else {
|
||||
this.metricsRegionWrapper = null;
|
||||
this.metricsRegion = null;
|
||||
|
@ -814,10 +798,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
LOG.debug("Instantiated " + this);
|
||||
}
|
||||
|
||||
// by default, we allow writes against a region when it's in recovering
|
||||
this.disallowWritesInRecovering =
|
||||
conf.getBoolean(HConstants.DISALLOW_WRITES_IN_RECOVERING,
|
||||
HConstants.DEFAULT_DISALLOW_WRITES_IN_RECOVERING_CONFIG);
|
||||
configurationManager = Optional.empty();
|
||||
|
||||
// disable stats tracking system tables, but check the config for everything else
|
||||
|
@ -959,13 +939,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
// Use maximum of log sequenceid or that which was found in stores
|
||||
// (particularly if no recovered edits, seqid will be -1).
|
||||
long nextSeqid = maxSeqId;
|
||||
|
||||
// In distributedLogReplay mode, we don't know the last change sequence number because region
|
||||
// is opened before recovery completes. So we add a safety bumper to avoid new sequence number
|
||||
// overlaps used sequence numbers
|
||||
if (this.writestate.writesEnabled) {
|
||||
nextSeqid = WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(), this.fs
|
||||
.getRegionDir(), nextSeqid, (this.recovering ? (this.flushPerChanges + 10000000) : 1));
|
||||
nextSeqid = WALSplitter.writeRegionSequenceIdFile(this.fs.getFileSystem(),
|
||||
this.fs.getRegionDir(), nextSeqid, 1);
|
||||
} else {
|
||||
nextSeqid++;
|
||||
}
|
||||
|
@ -1329,64 +1305,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return this.writestate.isReadOnly();
|
||||
}
|
||||
|
||||
/**
|
||||
* Reset recovering state of current region
|
||||
*/
|
||||
public void setRecovering(boolean newState) {
|
||||
boolean wasRecovering = this.recovering;
|
||||
// Before we flip the recovering switch (enabling reads) we should write the region open
|
||||
// event to WAL if needed
|
||||
if (wal != null && getRegionServerServices() != null && !writestate.readOnly
|
||||
&& wasRecovering && !newState) {
|
||||
|
||||
// force a flush only if region replication is set up for this region. Otherwise no need.
|
||||
boolean forceFlush = getTableDescriptor().getRegionReplication() > 1;
|
||||
|
||||
MonitoredTask status = TaskMonitor.get().createStatus("Recovering region " + this);
|
||||
|
||||
try {
|
||||
// force a flush first
|
||||
if (forceFlush) {
|
||||
status.setStatus("Flushing region " + this + " because recovery is finished");
|
||||
internalFlushcache(status);
|
||||
}
|
||||
|
||||
status.setStatus("Writing region open event marker to WAL because recovery is finished");
|
||||
try {
|
||||
long seqId = openSeqNum;
|
||||
// obtain a new seqId because we possibly have writes and flushes on top of openSeqNum
|
||||
if (wal != null) {
|
||||
seqId = getNextSequenceId(wal);
|
||||
}
|
||||
writeRegionOpenMarker(wal, seqId);
|
||||
} catch (IOException e) {
|
||||
// We cannot rethrow this exception since we are being called from the zk thread. The
|
||||
// region has already opened. In this case we log the error, but continue
|
||||
LOG.warn(getRegionInfo().getEncodedName() + " : was not able to write region opening "
|
||||
+ "event to WAL, continuing", e);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
// Distributed log replay semantics does not necessarily require a flush, since the replayed
|
||||
// data is already written again in the WAL. So failed flush should be fine.
|
||||
LOG.warn(getRegionInfo().getEncodedName() + " : was not able to flush "
|
||||
+ "event to WAL, continuing", ioe);
|
||||
} finally {
|
||||
status.cleanup();
|
||||
}
|
||||
}
|
||||
|
||||
this.recovering = newState;
|
||||
if (wasRecovering && !recovering) {
|
||||
// Call only when wal replay is over.
|
||||
coprocessorHost.postLogReplay();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isRecovering() {
|
||||
return this.recovering;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAvailable() {
|
||||
return !isClosed() && !isClosing();
|
||||
|
@ -7026,11 +6944,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
checkClassLoading();
|
||||
this.openSeqNum = initialize(reporter);
|
||||
this.mvcc.advanceTo(openSeqNum);
|
||||
if (wal != null && getRegionServerServices() != null && !writestate.readOnly
|
||||
&& !recovering) {
|
||||
// Only write the region open event marker to WAL if (1) we are not read-only
|
||||
// (2) dist log replay is off or we are not recovering. In case region is
|
||||
// recovering, the open event will be written at setRecovering(false)
|
||||
if (wal != null && getRegionServerServices() != null && !writestate.readOnly) {
|
||||
// Only write the region open event marker to WAL if we are not read-only.
|
||||
writeRegionOpenMarker(wal, openSeqNum);
|
||||
}
|
||||
return this;
|
||||
|
@ -7843,9 +7758,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
public static final long FIXED_OVERHEAD = ClassSize.align(
|
||||
ClassSize.OBJECT +
|
||||
ClassSize.ARRAY +
|
||||
51 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
|
||||
50 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
|
||||
(14 * Bytes.SIZEOF_LONG) +
|
||||
6 * Bytes.SIZEOF_BOOLEAN);
|
||||
3 * Bytes.SIZEOF_BOOLEAN);
|
||||
|
||||
// woefully out of date - currently missing:
|
||||
// 1 x HashMap - coprocessorServiceHandlers
|
||||
|
@ -8017,12 +7932,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
return null;
|
||||
}
|
||||
|
||||
// Can't split region which is in recovering state
|
||||
if (this.isRecovering()) {
|
||||
LOG.info("Cannot split region " + this.getRegionInfo().getEncodedName() + " in recovery.");
|
||||
return null;
|
||||
}
|
||||
|
||||
// Can't split a region that is closing.
|
||||
if (this.isClosing()) {
|
||||
return null;
|
||||
|
@ -8077,22 +7986,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
case GET: // read operations
|
||||
case SCAN:
|
||||
checkReadsEnabled();
|
||||
case INCREMENT: // write operations
|
||||
case APPEND:
|
||||
case SPLIT_REGION:
|
||||
case MERGE_REGION:
|
||||
case PUT:
|
||||
case DELETE:
|
||||
case BATCH_MUTATE:
|
||||
case COMPACT_REGION:
|
||||
case SNAPSHOT:
|
||||
// when a region is in recovering state, no read, split, merge or snapshot is allowed
|
||||
if (isRecovering() && (this.disallowWritesInRecovering ||
|
||||
(op != Operation.PUT && op != Operation.DELETE && op != Operation.BATCH_MUTATE))) {
|
||||
throw new RegionInRecoveryException(getRegionInfo().getRegionNameAsString() +
|
||||
" is recovering; cannot take reads");
|
||||
}
|
||||
break;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import javax.management.MalformedObjectNameException;
|
|||
import javax.management.ObjectName;
|
||||
import javax.servlet.http.HttpServlet;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.lang.Thread.UncaughtExceptionHandler;
|
||||
import java.lang.management.MemoryType;
|
||||
import java.lang.management.MemoryUsage;
|
||||
|
@ -33,7 +32,6 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -160,9 +158,7 @@ import org.apache.hadoop.hbase.wal.WALFactory;
|
|||
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -172,8 +168,6 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.KeeperException.NoNodeException;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
|
||||
|
@ -262,10 +256,6 @@ public class HRegionServer extends HasThread implements
|
|||
*/
|
||||
protected MetaTableLocator metaTableLocator;
|
||||
|
||||
// Watch if a region is out of recovering state from ZooKeeper
|
||||
@SuppressWarnings("unused")
|
||||
private RecoveringRegionWatcher recoveringRegionWatcher;
|
||||
|
||||
/**
|
||||
* Go here to get table descriptors.
|
||||
*/
|
||||
|
@ -296,13 +286,6 @@ public class HRegionServer extends HasThread implements
|
|||
protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap =
|
||||
new ConcurrentHashMap<>();
|
||||
|
||||
/**
|
||||
* Set of regions currently being in recovering state which means it can accept writes(edits from
|
||||
* previous failed region server) but not reads. A recovering region is also an online region.
|
||||
*/
|
||||
protected final Map<String, HRegion> recoveringRegions = Collections
|
||||
.synchronizedMap(new HashMap<String, HRegion>());
|
||||
|
||||
// Leases
|
||||
protected Leases leases;
|
||||
|
||||
|
@ -898,8 +881,6 @@ public class HRegionServer extends HasThread implements
|
|||
} catch (KeeperException e) {
|
||||
this.abort("Failed to reach coordination cluster when creating procedure handler.", e);
|
||||
}
|
||||
// register watcher for recovering regions
|
||||
this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED",
|
||||
|
@ -2229,9 +2210,6 @@ public class HRegionServer extends HasThread implements
|
|||
openSeqNum = 0;
|
||||
}
|
||||
|
||||
// Update flushed sequence id of a recovering region in ZK
|
||||
updateRecoveringRegionLastFlushedSequenceId(r);
|
||||
|
||||
// Notify master
|
||||
if (!reportRegionStateTransition(new RegionStateTransitionContext(
|
||||
TransitionCode.OPENED, openSeqNum, masterSystemTime, r.getRegionInfo()))) {
|
||||
|
@ -2756,11 +2734,6 @@ public class HRegionServer extends HasThread implements
|
|||
return this.stopping;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HRegion> getRecoveringRegions() {
|
||||
return this.recoveringRegions;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @return the configuration
|
||||
|
@ -3563,91 +3536,6 @@ public class HRegionServer extends HasThread implements
|
|||
return this.compactSplitThread;
|
||||
}
|
||||
|
||||
/**
|
||||
* A helper function to store the last flushed sequence Id with the previous failed RS for a
|
||||
* recovering region. The Id is used to skip wal edits which are flushed. Since the flushed
|
||||
* sequence id is only valid for each RS, we associate the Id with corresponding failed RS.
|
||||
* @throws KeeperException
|
||||
* @throws IOException
|
||||
*/
|
||||
private void updateRecoveringRegionLastFlushedSequenceId(Region r) throws KeeperException,
|
||||
IOException {
|
||||
if (!r.isRecovering()) {
|
||||
// return immdiately for non-recovering regions
|
||||
return;
|
||||
}
|
||||
|
||||
RegionInfo regionInfo = r.getRegionInfo();
|
||||
ZooKeeperWatcher zkw = getZooKeeper();
|
||||
String previousRSName = this.getLastFailedRSFromZK(regionInfo.getEncodedName());
|
||||
Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqId();
|
||||
long minSeqIdForLogReplay = -1;
|
||||
for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) {
|
||||
if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) {
|
||||
minSeqIdForLogReplay = storeSeqIdForReplay;
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
long lastRecordedFlushedSequenceId = -1;
|
||||
String nodePath = ZKUtil.joinZNode(this.zooKeeper.znodePaths.recoveringRegionsZNode,
|
||||
regionInfo.getEncodedName());
|
||||
// recovering-region level
|
||||
byte[] data;
|
||||
try {
|
||||
data = ZKUtil.getData(zkw, nodePath);
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
if (data != null) {
|
||||
lastRecordedFlushedSequenceId = ZKSplitLog.parseLastFlushedSequenceIdFrom(data);
|
||||
}
|
||||
if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
|
||||
ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
|
||||
}
|
||||
if (previousRSName != null) {
|
||||
// one level deeper for the failed RS
|
||||
nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
|
||||
ZKUtil.setData(zkw, nodePath,
|
||||
ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
|
||||
LOG.debug("Update last flushed sequence id of region " + regionInfo.getEncodedName() +
|
||||
" for " + previousRSName);
|
||||
} else {
|
||||
LOG.warn("Can't find failed region server for recovering region " +
|
||||
regionInfo.getEncodedName());
|
||||
}
|
||||
} catch (NoNodeException ignore) {
|
||||
LOG.debug("Region " + regionInfo.getEncodedName() +
|
||||
" must have completed recovery because its recovery znode has been removed", ignore);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the last failed RS name under /hbase/recovering-regions/encodedRegionName
|
||||
* @param encodedRegionName
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
|
||||
String result = null;
|
||||
long maxZxid = 0;
|
||||
ZooKeeperWatcher zkw = this.getZooKeeper();
|
||||
String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, encodedRegionName);
|
||||
List<String> failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath);
|
||||
if (failedServers == null || failedServers.isEmpty()) {
|
||||
return result;
|
||||
}
|
||||
for (String failedServer : failedServers) {
|
||||
String rsPath = ZKUtil.joinZNode(nodePath, failedServer);
|
||||
Stat stat = new Stat();
|
||||
ZKUtil.getDataNoWatch(zkw, rsPath, stat);
|
||||
if (maxZxid < stat.getCzxid()) {
|
||||
maxZxid = stat.getCzxid();
|
||||
result = failedServer;
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public CoprocessorServiceResponse execRegionServerService(
|
||||
@SuppressWarnings("UnusedParameters") final RpcController controller,
|
||||
final CoprocessorServiceRequest serviceRequest) throws ServiceException {
|
||||
|
|
|
@ -125,9 +125,7 @@ import org.apache.hadoop.hbase.wal.WAL;
|
|||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.wal.WALKey;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.cache.Cache;
|
||||
|
@ -1686,7 +1684,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
}
|
||||
builder.setSplittable(region.isSplittable());
|
||||
builder.setMergeable(region.isMergeable());
|
||||
builder.setIsRecovering(region.isRecovering());
|
||||
if (request.hasBestSplitRow() && request.getBestSplitRow() && bestSplitRow != null) {
|
||||
builder.setBestSplitRow(UnsafeByteOperations.unsafeWrap(bestSplitRow));
|
||||
}
|
||||
|
@ -1933,23 +1930,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
regionServer.removeFromMovedRegions(region.getEncodedName());
|
||||
|
||||
if (previous == null || !previous.booleanValue()) {
|
||||
// check if the region to be opened is marked in recovering state in ZK
|
||||
if (ZKSplitLog.isRegionMarkedRecoveringInZK(regionServer.getZooKeeper(),
|
||||
region.getEncodedName())) {
|
||||
// Check if current region open is for distributedLogReplay. This check is to support
|
||||
// rolling restart/upgrade where we want to Master/RS see same configuration
|
||||
if (!regionOpenInfo.hasOpenForDistributedLogReplay()
|
||||
|| regionOpenInfo.getOpenForDistributedLogReplay()) {
|
||||
regionServer.recoveringRegions.put(region.getEncodedName(), null);
|
||||
} else {
|
||||
// Remove stale recovery region from ZK when we open region not for recovering which
|
||||
// could happen when turn distributedLogReplay off from on.
|
||||
List<String> tmpRegions = new ArrayList<>();
|
||||
tmpRegions.add(region.getEncodedName());
|
||||
ZKSplitLog.deleteRecoveringRegionZNodes(regionServer.getZooKeeper(),
|
||||
tmpRegions);
|
||||
}
|
||||
}
|
||||
htd = htds.get(region.getTable());
|
||||
if (htd == null) {
|
||||
htd = regionServer.tableDescriptors.get(region.getTable());
|
||||
|
@ -1983,10 +1963,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
}
|
||||
|
||||
builder.addOpeningState(RegionOpeningState.OPENED);
|
||||
|
||||
} catch (KeeperException zooKeeperEx) {
|
||||
LOG.error("Can't retrieve recovering state from zookeeper", zooKeeperEx);
|
||||
throw new ServiceException(zooKeeperEx);
|
||||
} catch (IOException ie) {
|
||||
LOG.warn("Failed opening region " + region.getRegionNameAsString(), ie);
|
||||
if (isBulkAssign) {
|
||||
|
|
|
@ -72,9 +72,6 @@ public interface Region extends ConfigurationObserver {
|
|||
/** @return True if closing process has started */
|
||||
boolean isClosing();
|
||||
|
||||
/** @return True if region is in recovering state */
|
||||
boolean isRecovering();
|
||||
|
||||
/** @return True if region is read only */
|
||||
boolean isReadOnly();
|
||||
|
||||
|
|
|
@ -545,25 +545,6 @@ public class RegionCoprocessorHost
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Invoked after log replay on region
|
||||
*/
|
||||
public void postLogReplay() {
|
||||
if (coprocEnvironments.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
execOperation(new RegionObserverOperationWithoutResult() {
|
||||
@Override
|
||||
public void call(RegionObserver observer) throws IOException {
|
||||
observer.postLogReplay(this);
|
||||
}
|
||||
});
|
||||
} catch (IOException e) {
|
||||
LOG.warn(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Invoked before a region is closed
|
||||
* @param abortRequested true if the server is aborting
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
|
@ -175,16 +174,11 @@ public interface RegionServerServices extends Server, MutableOnlineRegions, Favo
|
|||
*/
|
||||
ExecutorService getExecutorService();
|
||||
|
||||
/**
|
||||
* @return set of recovering regions on the hosting region server
|
||||
*/
|
||||
Map<String, HRegion> getRecoveringRegions();
|
||||
|
||||
/**
|
||||
* Only required for "old" log replay; if it's removed, remove this.
|
||||
* @return The RegionServer's NonceManager
|
||||
*/
|
||||
public ServerNonceManager getNonceManager();
|
||||
ServerNonceManager getNonceManager();
|
||||
|
||||
/**
|
||||
* Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to be
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.NotServingRegionException;
|
|||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.wal.WALFactory;
|
||||
import org.apache.hadoop.hbase.wal.WALSplitter;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
|
@ -83,7 +82,7 @@ public class SplitLogWorker implements Runnable {
|
|||
final WALFactory factory) {
|
||||
this(hserver, conf, server, new TaskExecutor() {
|
||||
@Override
|
||||
public Status exec(String filename, RecoveryMode mode, CancelableProgressable p) {
|
||||
public Status exec(String filename, CancelableProgressable p) {
|
||||
Path walDir;
|
||||
FileSystem fs;
|
||||
try {
|
||||
|
@ -99,8 +98,7 @@ public class SplitLogWorker implements Runnable {
|
|||
try {
|
||||
if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)),
|
||||
fs, conf, p, sequenceIdChecker,
|
||||
server.getCoordinatedStateManager().getSplitLogWorkerCoordination(),
|
||||
server.getConnection(), mode, factory)) {
|
||||
server.getCoordinatedStateManager().getSplitLogWorkerCoordination(), factory)) {
|
||||
return Status.PREEMPTED;
|
||||
}
|
||||
} catch (InterruptedIOException iioe) {
|
||||
|
@ -194,7 +192,7 @@ public class SplitLogWorker implements Runnable {
|
|||
RESIGNED(),
|
||||
PREEMPTED()
|
||||
}
|
||||
Status exec(String name, RecoveryMode mode, CancelableProgressable p);
|
||||
Status exec(String name, CancelableProgressable p);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,56 +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 java.io.IOException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
|
||||
public class FinishRegionRecoveringHandler extends EventHandler {
|
||||
private static final Log LOG = LogFactory.getLog(FinishRegionRecoveringHandler.class);
|
||||
|
||||
protected final RegionServerServices rss;
|
||||
protected final String regionName;
|
||||
protected final String path;
|
||||
|
||||
public FinishRegionRecoveringHandler(RegionServerServices rss,
|
||||
String regionName, String path) {
|
||||
// we are using the open region handlers, since this operation is in the region open lifecycle
|
||||
super(rss, EventType.M_RS_OPEN_REGION);
|
||||
this.rss = rss;
|
||||
this.regionName = regionName;
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
Region region = this.rss.getRecoveringRegions().remove(regionName);
|
||||
if (region != null) {
|
||||
((HRegion)region).setRecovering(false);
|
||||
LOG.info(path + " deleted; " + regionName + " recovered.");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.SplitLogTask;
|
|||
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
|
@ -46,14 +45,13 @@ public class WALSplitterHandler extends EventHandler {
|
|||
private final CancelableProgressable reporter;
|
||||
private final AtomicInteger inProgressTasks;
|
||||
private final TaskExecutor splitTaskExecutor;
|
||||
private final RecoveryMode mode;
|
||||
private final SplitLogWorkerCoordination.SplitTaskDetails splitTaskDetails;
|
||||
private final SplitLogWorkerCoordination coordination;
|
||||
|
||||
|
||||
public WALSplitterHandler(final Server server, SplitLogWorkerCoordination coordination,
|
||||
SplitLogWorkerCoordination.SplitTaskDetails splitDetails, CancelableProgressable reporter,
|
||||
AtomicInteger inProgressTasks, TaskExecutor splitTaskExecutor, RecoveryMode mode) {
|
||||
AtomicInteger inProgressTasks, TaskExecutor splitTaskExecutor) {
|
||||
super(server, EventType.RS_LOG_REPLAY);
|
||||
this.splitTaskDetails = splitDetails;
|
||||
this.coordination = coordination;
|
||||
|
@ -62,17 +60,17 @@ public class WALSplitterHandler extends EventHandler {
|
|||
this.inProgressTasks.incrementAndGet();
|
||||
this.serverName = server.getServerName();
|
||||
this.splitTaskExecutor = splitTaskExecutor;
|
||||
this.mode = mode;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process() throws IOException {
|
||||
long startTime = System.currentTimeMillis();
|
||||
Status status = null;
|
||||
try {
|
||||
Status status = this.splitTaskExecutor.exec(splitTaskDetails.getWALFile(), mode, reporter);
|
||||
status = this.splitTaskExecutor.exec(splitTaskDetails.getWALFile(), reporter);
|
||||
switch (status) {
|
||||
case DONE:
|
||||
coordination.endTask(new SplitLogTask.Done(this.serverName,this.mode),
|
||||
coordination.endTask(new SplitLogTask.Done(this.serverName),
|
||||
SplitLogCounters.tot_wkr_task_done, splitTaskDetails);
|
||||
break;
|
||||
case PREEMPTED:
|
||||
|
@ -81,7 +79,7 @@ public class WALSplitterHandler extends EventHandler {
|
|||
break;
|
||||
case ERR:
|
||||
if (server != null && !server.isStopped()) {
|
||||
coordination.endTask(new SplitLogTask.Err(this.serverName, this.mode),
|
||||
coordination.endTask(new SplitLogTask.Err(this.serverName),
|
||||
SplitLogCounters.tot_wkr_task_err, splitTaskDetails);
|
||||
break;
|
||||
}
|
||||
|
@ -93,13 +91,13 @@ public class WALSplitterHandler extends EventHandler {
|
|||
LOG.info("task execution interrupted because worker is exiting "
|
||||
+ splitTaskDetails.toString());
|
||||
}
|
||||
coordination.endTask(new SplitLogTask.Resigned(this.serverName, this.mode),
|
||||
coordination.endTask(new SplitLogTask.Resigned(this.serverName),
|
||||
SplitLogCounters.tot_wkr_task_resigned, splitTaskDetails);
|
||||
break;
|
||||
}
|
||||
} finally {
|
||||
LOG.info("worker " + serverName + " done with task " + splitTaskDetails.toString() + " in "
|
||||
+ (System.currentTimeMillis() - startTime) + "ms");
|
||||
LOG.info("Worker " + serverName + " done with task " + splitTaskDetails.toString() + " in "
|
||||
+ (System.currentTimeMillis() - startTime) + "ms. Status = " + status);
|
||||
this.inProgressTasks.decrementAndGet();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,59 +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.wal;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
|
||||
/**
|
||||
* Class used to push numbers about WAL edits replay into the metrics subsystem. This will take a
|
||||
* single function call and turn it into multiple manipulations of the hadoop metrics system.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsWALEditsReplay {
|
||||
private static final Log LOG = LogFactory.getLog(MetricsWALEditsReplay.class);
|
||||
|
||||
private final MetricsEditsReplaySource source;
|
||||
|
||||
public MetricsWALEditsReplay() {
|
||||
source = CompatibilitySingletonFactory.getInstance(MetricsEditsReplaySource.class);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the time a replay command took
|
||||
*/
|
||||
void updateReplayTime(long time) {
|
||||
source.updateReplayTime(time);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the batch size of each replay
|
||||
*/
|
||||
void updateReplayBatchSize(long size) {
|
||||
source.updateReplayDataSize(size);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the payload data size of each replay
|
||||
*/
|
||||
void updateReplayDataSize(long size) {
|
||||
}
|
||||
}
|
|
@ -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.regionserver.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ClientServiceCallable;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.WAL.Entry;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
|
||||
/**
|
||||
* This class is responsible for replaying the edits coming from a failed region server.
|
||||
* <p>
|
||||
* This class uses the native HBase client in order to replay WAL entries.
|
||||
* </p>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALEditsReplaySink {
|
||||
private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class);
|
||||
private static final int MAX_BATCH_SIZE = 1024;
|
||||
private final Configuration conf;
|
||||
private final ClusterConnection conn;
|
||||
private final TableName tableName;
|
||||
private final MetricsWALEditsReplay metrics;
|
||||
private final AtomicLong totalReplayedEdits = new AtomicLong();
|
||||
private final boolean skipErrors;
|
||||
private final int replayTimeout;
|
||||
private final RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
/**
|
||||
* Create a sink for WAL log entries replay
|
||||
* @param conf configuration
|
||||
* @param tableName of table to replay edits of
|
||||
* @param conn connection to use
|
||||
* @throws IOException on IO failure
|
||||
*/
|
||||
public WALEditsReplaySink(Configuration conf, TableName tableName, ClusterConnection conn)
|
||||
throws IOException {
|
||||
this.conf = conf;
|
||||
this.metrics = new MetricsWALEditsReplay();
|
||||
this.conn = conn;
|
||||
this.tableName = tableName;
|
||||
this.skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
|
||||
HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS);
|
||||
// a single replay operation time out and default is 60 seconds
|
||||
this.replayTimeout = conf.getInt("hbase.regionserver.logreplay.timeout", 60000);
|
||||
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Replay an array of actions of the same region directly into the newly assigned Region Server
|
||||
* @param entries to replay
|
||||
* @throws IOException on IO failure
|
||||
*/
|
||||
public void replayEntries(List<Pair<HRegionLocation, Entry>> entries) throws IOException {
|
||||
if (entries.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
int batchSize = entries.size();
|
||||
Map<HRegionInfo, List<Entry>> entriesByRegion =
|
||||
new HashMap<>();
|
||||
HRegionLocation loc = null;
|
||||
Entry entry = null;
|
||||
List<Entry> regionEntries = null;
|
||||
// Build the action list.
|
||||
for (int i = 0; i < batchSize; i++) {
|
||||
loc = entries.get(i).getFirst();
|
||||
entry = entries.get(i).getSecond();
|
||||
if (entriesByRegion.containsKey(loc.getRegionInfo())) {
|
||||
regionEntries = entriesByRegion.get(loc.getRegionInfo());
|
||||
} else {
|
||||
regionEntries = new ArrayList<>();
|
||||
entriesByRegion.put(loc.getRegionInfo(), regionEntries);
|
||||
}
|
||||
regionEntries.add(entry);
|
||||
}
|
||||
|
||||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
|
||||
// replaying edits by region
|
||||
for (Map.Entry<HRegionInfo, List<Entry>> _entry : entriesByRegion.entrySet()) {
|
||||
HRegionInfo curRegion = _entry.getKey();
|
||||
List<Entry> allActions = _entry.getValue();
|
||||
// send edits in chunks
|
||||
int totalActions = allActions.size();
|
||||
int replayedActions = 0;
|
||||
int curBatchSize = 0;
|
||||
for (; replayedActions < totalActions;) {
|
||||
curBatchSize = (totalActions > (MAX_BATCH_SIZE + replayedActions)) ? MAX_BATCH_SIZE
|
||||
: (totalActions - replayedActions);
|
||||
replayEdits(loc, curRegion, allActions.subList(replayedActions,
|
||||
replayedActions + curBatchSize));
|
||||
replayedActions += curBatchSize;
|
||||
}
|
||||
}
|
||||
|
||||
long endTime = EnvironmentEdgeManager.currentTime() - startTime;
|
||||
LOG.debug("number of rows:" + entries.size() + " are sent by batch! spent " + endTime
|
||||
+ "(ms)!");
|
||||
|
||||
metrics.updateReplayTime(endTime);
|
||||
metrics.updateReplayBatchSize(batchSize);
|
||||
|
||||
this.totalReplayedEdits.addAndGet(batchSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a string representation of this sink's metrics
|
||||
* @return string with the total replayed edits count
|
||||
*/
|
||||
public String getStats() {
|
||||
return this.totalReplayedEdits.get() == 0 ? "" : "Sink: total replayed edits: "
|
||||
+ this.totalReplayedEdits;
|
||||
}
|
||||
|
||||
private void replayEdits(final HRegionLocation regionLoc, final HRegionInfo regionInfo,
|
||||
final List<Entry> entries) throws IOException {
|
||||
try {
|
||||
RpcRetryingCallerFactory factory = RpcRetryingCallerFactory.instantiate(conf, null);
|
||||
ReplayServerCallable<ReplicateWALEntryResponse> callable =
|
||||
new ReplayServerCallable<>(this.conn, this.rpcControllerFactory,
|
||||
this.tableName, regionLoc, entries);
|
||||
factory.<ReplicateWALEntryResponse> newCaller().callWithRetries(callable, this.replayTimeout);
|
||||
} catch (IOException ie) {
|
||||
if (skipErrors) {
|
||||
LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS
|
||||
+ "=true so continuing replayEdits with error:" + ie.getMessage());
|
||||
} else {
|
||||
throw ie;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Callable that handles the <code>replay</code> method call going against a single regionserver
|
||||
* @param <R>
|
||||
*/
|
||||
class ReplayServerCallable<R> extends ClientServiceCallable<ReplicateWALEntryResponse> {
|
||||
private List<Entry> entries;
|
||||
|
||||
ReplayServerCallable(final Connection connection, RpcControllerFactory rpcControllerFactory,
|
||||
final TableName tableName, final HRegionLocation regionLoc, final List<Entry> entries) {
|
||||
super(connection, tableName, HConstants.EMPTY_BYTE_ARRAY,
|
||||
rpcControllerFactory.newController(), HConstants.PRIORITY_UNSET);
|
||||
this.entries = entries;
|
||||
setLocation(regionLoc);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ReplicateWALEntryResponse rpcCall() throws Exception {
|
||||
if (entries.isEmpty()) return null;
|
||||
|
||||
Entry[] entriesArray = new Entry[entries.size()];
|
||||
entriesArray = entries.toArray(entriesArray);
|
||||
AdminService.BlockingInterface remoteSvr = conn.getAdmin(getLocation().getServerName());
|
||||
|
||||
Pair<AdminProtos.ReplicateWALEntryRequest, CellScanner> p =
|
||||
ReplicationProtbufUtil.buildReplicateWALEntryRequest(entriesArray);
|
||||
setRpcControllerCellScanner(p.getSecond());
|
||||
return remoteSvr.replay(getRpcController(), p.getFirst());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void prepare(boolean reload) throws IOException {
|
||||
if (!reload) return;
|
||||
// relocate regions in case we have a new dead server or network hiccup
|
||||
// if not due to connection issue, the following code should run fast because it uses
|
||||
// cached location
|
||||
boolean skip = false;
|
||||
for (Entry entry : this.entries) {
|
||||
WALEdit edit = entry.getEdit();
|
||||
List<Cell> cells = edit.getCells();
|
||||
for (Cell cell : cells) {
|
||||
// filtering WAL meta entries
|
||||
setLocation(conn.locateRegion(tableName, CellUtil.cloneRow(cell)));
|
||||
skip = true;
|
||||
break;
|
||||
}
|
||||
// use first log entry to relocate region because all entries are for one region
|
||||
if (skip) break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -57,8 +57,6 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
|
|||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil;
|
||||
import org.apache.hadoop.hbase.replication.BaseWALEntryFilter;
|
||||
import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
|
||||
import org.apache.hadoop.hbase.replication.HBaseReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.replication.WALEntryFilter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -75,7 +73,6 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.cache.Cache;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.cache.CacheBuilder;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
|
||||
|
||||
|
@ -109,44 +106,6 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint {
|
|||
|
||||
private ExecutorService pool;
|
||||
|
||||
/**
|
||||
* Skips the entries which has original seqId. Only entries persisted via distributed log replay
|
||||
* have their original seq Id fields set.
|
||||
*/
|
||||
private static class SkipReplayedEditsFilter extends BaseWALEntryFilter {
|
||||
@Override
|
||||
public Entry filter(Entry entry) {
|
||||
// if orig seq id is set, skip replaying the entry
|
||||
if (entry.getKey().getOrigLogSeqNum() > 0) {
|
||||
return null;
|
||||
}
|
||||
return entry;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WALEntryFilter getWALEntryfilter() {
|
||||
WALEntryFilter superFilter = super.getWALEntryfilter();
|
||||
WALEntryFilter skipReplayedEditsFilter = getSkipReplayedEditsFilter();
|
||||
|
||||
if (superFilter == null) {
|
||||
return skipReplayedEditsFilter;
|
||||
}
|
||||
|
||||
if (skipReplayedEditsFilter == null) {
|
||||
return superFilter;
|
||||
}
|
||||
|
||||
ArrayList<WALEntryFilter> filters = Lists.newArrayList();
|
||||
filters.add(superFilter);
|
||||
filters.add(skipReplayedEditsFilter);
|
||||
return new ChainWALEntryFilter(filters);
|
||||
}
|
||||
|
||||
protected WALEntryFilter getSkipReplayedEditsFilter() {
|
||||
return new SkipReplayedEditsFilter();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(Context context) throws IOException {
|
||||
super.init(context);
|
||||
|
|
|
@ -1524,31 +1524,15 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
|
|||
}
|
||||
if (AccessControlLists.isAclRegion(region)) {
|
||||
aclRegion = true;
|
||||
// When this region is under recovering state, initialize will be handled by postLogReplay
|
||||
if (!region.isRecovering()) {
|
||||
try {
|
||||
initialize(env);
|
||||
} catch (IOException ex) {
|
||||
// if we can't obtain permissions, it's better to fail
|
||||
// than perform checks incorrectly
|
||||
throw new RuntimeException("Failed to initialize permissions cache", ex);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
initialized = true;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> c) {
|
||||
if (aclRegion) {
|
||||
try {
|
||||
initialize(c.getEnvironment());
|
||||
initialize(env);
|
||||
} catch (IOException ex) {
|
||||
// if we can't obtain permissions, it's better to fail
|
||||
// than perform checks incorrectly
|
||||
throw new RuntimeException("Failed to initialize permissions cache", ex);
|
||||
}
|
||||
} else {
|
||||
initialized = true;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
|||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
|
@ -224,8 +223,6 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
|
|||
// the system.
|
||||
labelsTable.setValue(HTableDescriptor.SPLIT_POLICY,
|
||||
DisabledRegionSplitPolicy.class.getName());
|
||||
labelsTable.setValue(Bytes.toBytes(HConstants.DISALLOW_WRITES_IN_RECOVERING),
|
||||
Bytes.toBytes(true));
|
||||
try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
|
||||
admin.createTable(labelsTable);
|
||||
}
|
||||
|
@ -265,10 +262,7 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
|
|||
this.accessControllerAvailable = CoprocessorHost.getLoadedCoprocessors()
|
||||
.contains(AccessController.class.getName());
|
||||
}
|
||||
// Defer the init of VisibilityLabelService on labels region until it is in recovering state.
|
||||
if (!e.getEnvironment().getRegion().isRecovering()) {
|
||||
initVisibilityLabelService(e.getEnvironment());
|
||||
}
|
||||
initVisibilityLabelService(e.getEnvironment());
|
||||
} else {
|
||||
checkAuths = e.getEnvironment().getConfiguration()
|
||||
.getBoolean(VisibilityConstants.CHECK_AUTHS_FOR_MUTATION, false);
|
||||
|
@ -276,14 +270,6 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postLogReplay(ObserverContext<RegionCoprocessorEnvironment> e) {
|
||||
if (this.labelsRegion) {
|
||||
initVisibilityLabelService(e.getEnvironment());
|
||||
LOG.debug("post labels region log replay");
|
||||
}
|
||||
}
|
||||
|
||||
private void initVisibilityLabelService(RegionCoprocessorEnvironment env) {
|
||||
try {
|
||||
this.visibilityLabelService.init(env);
|
||||
|
|
|
@ -18,9 +18,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.wal;
|
||||
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
|
||||
import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
|
@ -32,7 +30,6 @@ import java.io.InterruptedIOException;
|
|||
import java.text.ParseException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -50,7 +47,6 @@ import java.util.concurrent.Future;
|
|||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Matcher;
|
||||
|
@ -67,26 +63,13 @@ import org.apache.hadoop.fs.PathFilter;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
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.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
|
@ -95,24 +78,15 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.LastSequenceId;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEditsReplaySink;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -122,11 +96,12 @@ import org.apache.hadoop.hbase.wal.WALProvider.Writer;
|
|||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.io.MultipleIOException;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* This class is responsible for splitting up a bunch of regionserver commit log
|
||||
* files that are no longer being written to, into new files, one per region for
|
||||
* region to replay on startup. Delete the old log files when finished.
|
||||
* files that are no longer being written to, into new files, one per region, for
|
||||
* recovering data on startup. Delete the old log files when finished.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class WALSplitter {
|
||||
|
@ -142,14 +117,10 @@ public class WALSplitter {
|
|||
|
||||
// Major subcomponents of the split process.
|
||||
// These are separated into inner classes to make testing easier.
|
||||
PipelineController controller;
|
||||
OutputSink outputSink;
|
||||
EntryBuffers entryBuffers;
|
||||
private EntryBuffers entryBuffers;
|
||||
|
||||
private Map<TableName, TableState> tableStatesCache =
|
||||
new ConcurrentHashMap<>();
|
||||
private SplitLogWorkerCoordination splitLogWorkerCoordination;
|
||||
private Connection connection;
|
||||
private final WALFactory walFactory;
|
||||
|
||||
private MonitoredTask status;
|
||||
|
@ -157,31 +128,19 @@ public class WALSplitter {
|
|||
// For checking the latest flushed sequence id
|
||||
protected final LastSequenceId sequenceIdChecker;
|
||||
|
||||
protected boolean distributedLogReplay;
|
||||
|
||||
// Map encodedRegionName -> lastFlushedSequenceId
|
||||
protected Map<String, Long> lastFlushedSequenceIds = new ConcurrentHashMap<>();
|
||||
|
||||
// Map encodedRegionName -> maxSeqIdInStores
|
||||
protected Map<String, Map<byte[], Long>> regionMaxSeqIdInStores = new ConcurrentHashMap<>();
|
||||
|
||||
// Failed region server that the wal file being split belongs to
|
||||
protected String failedServerName = "";
|
||||
|
||||
// Number of writer threads
|
||||
private final int numWriterThreads;
|
||||
|
||||
// Min batch size when replay WAL edits
|
||||
private final int minBatchSize;
|
||||
|
||||
// the file being split currently
|
||||
private FileStatus fileBeingSplit;
|
||||
|
||||
@VisibleForTesting
|
||||
WALSplitter(final WALFactory factory, Configuration conf, Path rootDir,
|
||||
FileSystem fs, LastSequenceId idChecker,
|
||||
SplitLogWorkerCoordination splitLogWorkerCoordination, Connection connection,
|
||||
RecoveryMode mode) {
|
||||
SplitLogWorkerCoordination splitLogWorkerCoordination) {
|
||||
this.conf = HBaseConfiguration.create(conf);
|
||||
String codecClassName = conf
|
||||
.get(WALCellCodec.WAL_CELL_CODEC_CLASS_KEY, WALCellCodec.class.getName());
|
||||
|
@ -190,31 +149,15 @@ public class WALSplitter {
|
|||
this.fs = fs;
|
||||
this.sequenceIdChecker = idChecker;
|
||||
this.splitLogWorkerCoordination = splitLogWorkerCoordination;
|
||||
this.connection = connection;
|
||||
|
||||
this.walFactory = factory;
|
||||
this.controller = new PipelineController();
|
||||
PipelineController controller = new PipelineController();
|
||||
|
||||
entryBuffers = new EntryBuffers(controller,
|
||||
this.conf.getInt("hbase.regionserver.hlog.splitlog.buffersize",
|
||||
128*1024*1024));
|
||||
|
||||
// a larger minBatchSize may slow down recovery because replay writer has to wait for
|
||||
// enough edits before replaying them
|
||||
this.minBatchSize = this.conf.getInt("hbase.regionserver.wal.logreplay.batch.size", 64);
|
||||
this.distributedLogReplay = (RecoveryMode.LOG_REPLAY == mode);
|
||||
|
||||
this.numWriterThreads = this.conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
|
||||
if (this.splitLogWorkerCoordination != null && this.distributedLogReplay) {
|
||||
outputSink = new LogReplayOutputSink(controller, entryBuffers, numWriterThreads);
|
||||
} else {
|
||||
if (this.distributedLogReplay) {
|
||||
LOG.info("ZooKeeperWatcher is passed in as NULL so disable distrubitedLogRepaly.");
|
||||
}
|
||||
this.distributedLogReplay = false;
|
||||
outputSink = new LogRecoveredEditsOutputSink(controller, entryBuffers, numWriterThreads);
|
||||
}
|
||||
this.conf.getInt("hbase.regionserver.hlog.splitlog.buffersize", 128 * 1024 * 1024));
|
||||
|
||||
int numWriterThreads = this.conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
|
||||
outputSink = new LogRecoveredEditsOutputSink(controller, entryBuffers, numWriterThreads);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -227,10 +170,10 @@ public class WALSplitter {
|
|||
*/
|
||||
public static boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs,
|
||||
Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
|
||||
SplitLogWorkerCoordination splitLogWorkerCoordination, Connection connection,
|
||||
RecoveryMode mode, final WALFactory factory) throws IOException {
|
||||
SplitLogWorkerCoordination splitLogWorkerCoordination, final WALFactory factory)
|
||||
throws IOException {
|
||||
WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, idChecker,
|
||||
splitLogWorkerCoordination, connection, mode);
|
||||
splitLogWorkerCoordination);
|
||||
return s.splitLogFile(logfile, reporter);
|
||||
}
|
||||
|
||||
|
@ -246,8 +189,7 @@ public class WALSplitter {
|
|||
List<Path> splits = new ArrayList<>();
|
||||
if (logfiles != null && logfiles.length > 0) {
|
||||
for (FileStatus logfile: logfiles) {
|
||||
WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, null, null, null,
|
||||
RecoveryMode.LOG_SPLITTING);
|
||||
WALSplitter s = new WALSplitter(factory, conf, rootDir, fs, null, null);
|
||||
if (s.splitLogFile(logfile, null)) {
|
||||
finishSplitLogFile(rootDir, oldLogDir, logfile.getPath(), conf);
|
||||
if (s.outputSink.splits != null) {
|
||||
|
@ -281,22 +223,20 @@ public class WALSplitter {
|
|||
int editsCount = 0;
|
||||
int editsSkipped = 0;
|
||||
|
||||
status =
|
||||
TaskMonitor.get().createStatus(
|
||||
status = TaskMonitor.get().createStatus(
|
||||
"Splitting log file " + logfile.getPath() + "into a temporary staging area.");
|
||||
Reader in = null;
|
||||
Reader logFileReader = null;
|
||||
this.fileBeingSplit = logfile;
|
||||
try {
|
||||
long logLength = logfile.getLen();
|
||||
LOG.info("Splitting WAL=" + logPath + ", length=" + logLength +
|
||||
", distributedLogReplay=" + this.distributedLogReplay);
|
||||
LOG.info("Splitting WAL=" + logPath + ", length=" + logLength);
|
||||
status.setStatus("Opening log file");
|
||||
if (reporter != null && !reporter.progress()) {
|
||||
progress_failed = true;
|
||||
return false;
|
||||
}
|
||||
in = getReader(logfile, skipErrors, reporter);
|
||||
if (in == null) {
|
||||
logFileReader = getReader(logfile, skipErrors, reporter);
|
||||
if (logFileReader == null) {
|
||||
LOG.warn("Nothing to split in WAL=" + logPath);
|
||||
return true;
|
||||
}
|
||||
|
@ -307,26 +247,12 @@ public class WALSplitter {
|
|||
outputSinkStarted = true;
|
||||
Entry entry;
|
||||
Long lastFlushedSequenceId = -1L;
|
||||
// THIS IS BROKEN!!!! GETTING SERVERNAME FROM PATH IS NOT GOING TO WORK IF LAYOUT CHANGES!!!
|
||||
// TODO: Fix.
|
||||
ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(logPath);
|
||||
failedServerName = (serverName == null) ? "" : serverName.getServerName();
|
||||
while ((entry = getNextLogLine(in, logPath, skipErrors)) != null) {
|
||||
while ((entry = getNextLogLine(logFileReader, logPath, skipErrors)) != null) {
|
||||
byte[] region = entry.getKey().getEncodedRegionName();
|
||||
String encodedRegionNameAsStr = Bytes.toString(region);
|
||||
lastFlushedSequenceId = lastFlushedSequenceIds.get(encodedRegionNameAsStr);
|
||||
if (lastFlushedSequenceId == null) {
|
||||
if (this.distributedLogReplay) {
|
||||
RegionStoreSequenceIds ids = splitLogWorkerCoordination.getRegionFlushedSequenceId(
|
||||
failedServerName, encodedRegionNameAsStr);
|
||||
if (ids != null) {
|
||||
lastFlushedSequenceId = ids.getLastFlushedSequenceId();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("DLR Last flushed sequenceid for " + encodedRegionNameAsStr + ": " +
|
||||
TextFormat.shortDebugString(ids));
|
||||
}
|
||||
}
|
||||
} else if (sequenceIdChecker != null) {
|
||||
if (sequenceIdChecker != null) {
|
||||
RegionStoreSequenceIds ids = sequenceIdChecker.getLastSequenceId(region);
|
||||
Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
for (StoreSequenceId storeSeqId : ids.getStoreSequenceIdList()) {
|
||||
|
@ -390,8 +316,8 @@ public class WALSplitter {
|
|||
} finally {
|
||||
LOG.debug("Finishing writing output logs and closing down.");
|
||||
try {
|
||||
if (null != in) {
|
||||
in.close();
|
||||
if (null != logFileReader) {
|
||||
logFileReader.close();
|
||||
}
|
||||
} catch (IOException exception) {
|
||||
LOG.warn("Could not close WAL reader: " + exception.getMessage());
|
||||
|
@ -954,7 +880,7 @@ public class WALSplitter {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return RegionEntryBuffer a buffer of edits to be written or replayed.
|
||||
* @return RegionEntryBuffer a buffer of edits to be written.
|
||||
*/
|
||||
synchronized RegionEntryBuffer getChunkToWrite() {
|
||||
long biggestSize = 0;
|
||||
|
@ -1127,8 +1053,8 @@ public class WALSplitter {
|
|||
}
|
||||
|
||||
/**
|
||||
* The following class is an abstraction class to provide a common interface to support both
|
||||
* existing recovered edits file sink and region server WAL edits replay sink
|
||||
* The following class is an abstraction class to provide a common interface to support
|
||||
* different ways of consuming recovered edits.
|
||||
*/
|
||||
public static abstract class OutputSink {
|
||||
|
||||
|
@ -1195,10 +1121,6 @@ public class WALSplitter {
|
|||
}
|
||||
}
|
||||
|
||||
Long getRegionMaximumEditLogSeqNum(byte[] region) {
|
||||
return regionMaximumEditLogSeqNum.get(region);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the number of currently opened writers
|
||||
*/
|
||||
|
@ -1692,583 +1614,6 @@ public class WALSplitter {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Class that manages to replay edits from WAL files directly to assigned fail over region servers
|
||||
*/
|
||||
class LogReplayOutputSink extends OutputSink {
|
||||
private static final double BUFFER_THRESHOLD = 0.35;
|
||||
private static final String KEY_DELIMITER = "#";
|
||||
|
||||
private final long waitRegionOnlineTimeOut;
|
||||
private final Set<String> recoveredRegions = Collections.synchronizedSet(new HashSet<String>());
|
||||
private final Map<String, RegionServerWriter> writers = new ConcurrentHashMap<>();
|
||||
// online encoded region name -> region location map
|
||||
private final Map<String, HRegionLocation> onlineRegions = new ConcurrentHashMap<>();
|
||||
|
||||
private final Map<TableName, ClusterConnection> tableNameToHConnectionMap = Collections
|
||||
.synchronizedMap(new TreeMap<TableName, ClusterConnection>());
|
||||
/**
|
||||
* Map key -> value layout
|
||||
* {@literal <servername>:<table name> -> Queue<Row>}
|
||||
*/
|
||||
private final Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
|
||||
new ConcurrentHashMap<>();
|
||||
private final List<Throwable> thrown = new ArrayList<>();
|
||||
|
||||
// The following sink is used in distrubitedLogReplay mode for entries of regions in a disabling
|
||||
// table. It's a limitation of distributedLogReplay. Because log replay needs a region is
|
||||
// assigned and online before it can replay wal edits while regions of disabling/disabled table
|
||||
// won't be assigned by AM. We can retire this code after HBASE-8234.
|
||||
private final LogRecoveredEditsOutputSink logRecoveredEditsOutputSink;
|
||||
private boolean hasEditsInDisablingOrDisabledTables = false;
|
||||
|
||||
public LogReplayOutputSink(PipelineController controller, EntryBuffers entryBuffers,
|
||||
int numWriters) {
|
||||
super(controller, entryBuffers, numWriters);
|
||||
this.waitRegionOnlineTimeOut =
|
||||
conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT,
|
||||
ZKSplitLogManagerCoordination.DEFAULT_TIMEOUT);
|
||||
this.logRecoveredEditsOutputSink = new LogRecoveredEditsOutputSink(controller,
|
||||
entryBuffers, numWriters);
|
||||
this.logRecoveredEditsOutputSink.setReporter(reporter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void append(RegionEntryBuffer buffer) throws IOException {
|
||||
List<Entry> entries = buffer.entryBuffer;
|
||||
if (entries.isEmpty()) {
|
||||
LOG.warn("got an empty buffer, skipping");
|
||||
return;
|
||||
}
|
||||
|
||||
// check if current region in a disabling or disabled table
|
||||
if (isTableDisabledOrDisabling(buffer.tableName)) {
|
||||
// need fall back to old way
|
||||
logRecoveredEditsOutputSink.append(buffer);
|
||||
hasEditsInDisablingOrDisabledTables = true;
|
||||
// store regions we have recovered so far
|
||||
addToRecoveredRegions(Bytes.toString(buffer.encodedRegionName));
|
||||
return;
|
||||
}
|
||||
|
||||
// group entries by region servers
|
||||
groupEditsByServer(entries);
|
||||
|
||||
// process workitems
|
||||
String maxLocKey = null;
|
||||
int maxSize = 0;
|
||||
List<Pair<HRegionLocation, Entry>> maxQueue = null;
|
||||
synchronized (this.serverToBufferQueueMap) {
|
||||
for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry :
|
||||
this.serverToBufferQueueMap.entrySet()) {
|
||||
List<Pair<HRegionLocation, Entry>> curQueue = entry.getValue();
|
||||
if (curQueue.size() > maxSize) {
|
||||
maxSize = curQueue.size();
|
||||
maxQueue = curQueue;
|
||||
maxLocKey = entry.getKey();
|
||||
}
|
||||
}
|
||||
if (maxSize < minBatchSize
|
||||
&& entryBuffers.totalBuffered < BUFFER_THRESHOLD * entryBuffers.maxHeapUsage) {
|
||||
// buffer more to process
|
||||
return;
|
||||
} else if (maxSize > 0) {
|
||||
this.serverToBufferQueueMap.remove(maxLocKey);
|
||||
}
|
||||
}
|
||||
|
||||
if (maxSize > 0) {
|
||||
processWorkItems(maxLocKey, maxQueue);
|
||||
}
|
||||
}
|
||||
|
||||
private void addToRecoveredRegions(String encodedRegionName) {
|
||||
if (!recoveredRegions.contains(encodedRegionName)) {
|
||||
recoveredRegions.add(encodedRegionName);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper function to group WALEntries to individual region servers
|
||||
* @throws IOException
|
||||
*/
|
||||
private void groupEditsByServer(List<Entry> entries) throws IOException {
|
||||
Set<TableName> nonExistentTables = null;
|
||||
Long cachedLastFlushedSequenceId = -1l;
|
||||
for (Entry entry : entries) {
|
||||
WALEdit edit = entry.getEdit();
|
||||
TableName table = entry.getKey().getTablename();
|
||||
// clear scopes which isn't needed for recovery
|
||||
entry.getKey().serializeReplicationScope(false);
|
||||
String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName());
|
||||
// skip edits of non-existent tables
|
||||
if (nonExistentTables != null && nonExistentTables.contains(table)) {
|
||||
this.skippedEdits.incrementAndGet();
|
||||
continue;
|
||||
}
|
||||
|
||||
Map<byte[], Long> maxStoreSequenceIds = null;
|
||||
boolean needSkip = false;
|
||||
HRegionLocation loc = null;
|
||||
String locKey = null;
|
||||
List<Cell> cells = edit.getCells();
|
||||
List<Cell> skippedCells = new ArrayList<>();
|
||||
ClusterConnection cconn = this.getConnectionByTableName(table);
|
||||
|
||||
for (Cell cell : cells) {
|
||||
byte[] row = CellUtil.cloneRow(cell);
|
||||
byte[] family = CellUtil.cloneFamily(cell);
|
||||
boolean isCompactionEntry = false;
|
||||
if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
|
||||
CompactionDescriptor compaction = WALEdit.getCompaction(cell);
|
||||
if (compaction != null && compaction.hasRegionName()) {
|
||||
try {
|
||||
byte[][] regionName = HRegionInfo.parseRegionName(compaction.getRegionName()
|
||||
.toByteArray());
|
||||
row = regionName[1]; // startKey of the region
|
||||
family = compaction.getFamilyName().toByteArray();
|
||||
isCompactionEntry = true;
|
||||
} catch (Exception ex) {
|
||||
LOG.warn("Unexpected exception received, ignoring " + ex);
|
||||
skippedCells.add(cell);
|
||||
continue;
|
||||
}
|
||||
} else {
|
||||
skippedCells.add(cell);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
loc =
|
||||
locateRegionAndRefreshLastFlushedSequenceId(cconn, table, row,
|
||||
encodeRegionNameStr);
|
||||
// skip replaying the compaction if the region is gone
|
||||
if (isCompactionEntry && !encodeRegionNameStr.equalsIgnoreCase(
|
||||
loc.getRegionInfo().getEncodedName())) {
|
||||
LOG.info("Not replaying a compaction marker for an older region: "
|
||||
+ encodeRegionNameStr);
|
||||
needSkip = true;
|
||||
}
|
||||
} catch (TableNotFoundException ex) {
|
||||
// table has been deleted so skip edits of the table
|
||||
LOG.info("Table " + table + " doesn't exist. Skip log replay for region "
|
||||
+ encodeRegionNameStr);
|
||||
lastFlushedSequenceIds.put(encodeRegionNameStr, Long.MAX_VALUE);
|
||||
if (nonExistentTables == null) {
|
||||
nonExistentTables = new TreeSet<>();
|
||||
}
|
||||
nonExistentTables.add(table);
|
||||
this.skippedEdits.incrementAndGet();
|
||||
needSkip = true;
|
||||
break;
|
||||
}
|
||||
|
||||
cachedLastFlushedSequenceId =
|
||||
lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
|
||||
if (cachedLastFlushedSequenceId != null
|
||||
&& cachedLastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
|
||||
// skip the whole WAL entry
|
||||
this.skippedEdits.incrementAndGet();
|
||||
needSkip = true;
|
||||
break;
|
||||
} else {
|
||||
if (maxStoreSequenceIds == null) {
|
||||
maxStoreSequenceIds =
|
||||
regionMaxSeqIdInStores.get(loc.getRegionInfo().getEncodedName());
|
||||
}
|
||||
if (maxStoreSequenceIds != null) {
|
||||
Long maxStoreSeqId = maxStoreSequenceIds.get(family);
|
||||
if (maxStoreSeqId == null || maxStoreSeqId >= entry.getKey().getSequenceId()) {
|
||||
// skip current kv if column family doesn't exist anymore or already flushed
|
||||
skippedCells.add(cell);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// skip the edit
|
||||
if (loc == null || needSkip) continue;
|
||||
|
||||
if (!skippedCells.isEmpty()) {
|
||||
cells.removeAll(skippedCells);
|
||||
}
|
||||
|
||||
synchronized (serverToBufferQueueMap) {
|
||||
locKey = loc.getHostnamePort() + KEY_DELIMITER + table;
|
||||
List<Pair<HRegionLocation, Entry>> queue = serverToBufferQueueMap.get(locKey);
|
||||
if (queue == null) {
|
||||
queue =
|
||||
Collections.synchronizedList(new ArrayList<Pair<HRegionLocation, Entry>>());
|
||||
serverToBufferQueueMap.put(locKey, queue);
|
||||
}
|
||||
queue.add(new Pair<>(loc, entry));
|
||||
}
|
||||
// store regions we have recovered so far
|
||||
addToRecoveredRegions(loc.getRegionInfo().getEncodedName());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Locate destination region based on table name & row. This function also makes sure the
|
||||
* destination region is online for replay.
|
||||
* @throws IOException
|
||||
*/
|
||||
private HRegionLocation locateRegionAndRefreshLastFlushedSequenceId(ClusterConnection cconn,
|
||||
TableName table, byte[] row, String originalEncodedRegionName) throws IOException {
|
||||
// fetch location from cache
|
||||
HRegionLocation loc = onlineRegions.get(originalEncodedRegionName);
|
||||
if(loc != null) return loc;
|
||||
// fetch location from hbase:meta directly without using cache to avoid hit old dead server
|
||||
loc = cconn.getRegionLocation(table, row, true);
|
||||
if (loc == null) {
|
||||
throw new IOException("Can't locate location for row:" + Bytes.toString(row)
|
||||
+ " of table:" + table);
|
||||
}
|
||||
// check if current row moves to a different region due to region merge/split
|
||||
if (!originalEncodedRegionName.equalsIgnoreCase(loc.getRegionInfo().getEncodedName())) {
|
||||
// originalEncodedRegionName should have already flushed
|
||||
lastFlushedSequenceIds.put(originalEncodedRegionName, Long.MAX_VALUE);
|
||||
HRegionLocation tmpLoc = onlineRegions.get(loc.getRegionInfo().getEncodedName());
|
||||
if (tmpLoc != null) return tmpLoc;
|
||||
}
|
||||
|
||||
Long lastFlushedSequenceId = -1L;
|
||||
AtomicBoolean isRecovering = new AtomicBoolean(true);
|
||||
loc = waitUntilRegionOnline(loc, row, this.waitRegionOnlineTimeOut, isRecovering);
|
||||
if (!isRecovering.get()) {
|
||||
// region isn't in recovering at all because WAL file may contain a region that has
|
||||
// been moved to somewhere before hosting RS fails
|
||||
lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), Long.MAX_VALUE);
|
||||
LOG.info("logReplay skip region: " + loc.getRegionInfo().getEncodedName()
|
||||
+ " because it's not in recovering.");
|
||||
} else {
|
||||
Long cachedLastFlushedSequenceId =
|
||||
lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
|
||||
|
||||
// retrieve last flushed sequence Id from ZK. Because region postOpenDeployTasks will
|
||||
// update the value for the region
|
||||
RegionStoreSequenceIds ids =
|
||||
splitLogWorkerCoordination.getRegionFlushedSequenceId(failedServerName,
|
||||
loc.getRegionInfo().getEncodedName());
|
||||
if (ids != null) {
|
||||
lastFlushedSequenceId = ids.getLastFlushedSequenceId();
|
||||
Map<byte[], Long> storeIds = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
List<StoreSequenceId> maxSeqIdInStores = ids.getStoreSequenceIdList();
|
||||
for (StoreSequenceId id : maxSeqIdInStores) {
|
||||
storeIds.put(id.getFamilyName().toByteArray(), id.getSequenceId());
|
||||
}
|
||||
regionMaxSeqIdInStores.put(loc.getRegionInfo().getEncodedName(), storeIds);
|
||||
}
|
||||
|
||||
if (cachedLastFlushedSequenceId == null
|
||||
|| lastFlushedSequenceId > cachedLastFlushedSequenceId) {
|
||||
lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), lastFlushedSequenceId);
|
||||
}
|
||||
}
|
||||
|
||||
onlineRegions.put(loc.getRegionInfo().getEncodedName(), loc);
|
||||
return loc;
|
||||
}
|
||||
|
||||
private void processWorkItems(String key, List<Pair<HRegionLocation, Entry>> actions)
|
||||
throws IOException {
|
||||
RegionServerWriter rsw = null;
|
||||
|
||||
long startTime = System.nanoTime();
|
||||
try {
|
||||
rsw = getRegionServerWriter(key);
|
||||
rsw.sink.replayEntries(actions);
|
||||
|
||||
// Pass along summary statistics
|
||||
rsw.incrementEdits(actions.size());
|
||||
rsw.incrementNanoTime(System.nanoTime() - startTime);
|
||||
} catch (IOException e) {
|
||||
e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
|
||||
LOG.fatal(" Got while writing log entry to log", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait until region is online on the destination region server
|
||||
* @param loc
|
||||
* @param row
|
||||
* @param timeout How long to wait
|
||||
* @param isRecovering Recovering state of the region interested on destination region server.
|
||||
* @return True when region is online on the destination region server
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private HRegionLocation waitUntilRegionOnline(HRegionLocation loc, byte[] row,
|
||||
final long timeout, AtomicBoolean isRecovering)
|
||||
throws IOException {
|
||||
final long endTime = EnvironmentEdgeManager.currentTime() + timeout;
|
||||
final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
|
||||
boolean reloadLocation = false;
|
||||
TableName tableName = loc.getRegionInfo().getTable();
|
||||
int tries = 0;
|
||||
Throwable cause = null;
|
||||
while (endTime > EnvironmentEdgeManager.currentTime()) {
|
||||
try {
|
||||
// Try and get regioninfo from the hosting server.
|
||||
ClusterConnection cconn = getConnectionByTableName(tableName);
|
||||
if(reloadLocation) {
|
||||
loc = cconn.getRegionLocation(tableName, row, true);
|
||||
}
|
||||
BlockingInterface remoteSvr = cconn.getAdmin(loc.getServerName());
|
||||
HRegionInfo region = loc.getRegionInfo();
|
||||
try {
|
||||
GetRegionInfoRequest request =
|
||||
RequestConverter.buildGetRegionInfoRequest(region.getRegionName());
|
||||
GetRegionInfoResponse response = remoteSvr.getRegionInfo(null, request);
|
||||
if (HRegionInfo.convert(response.getRegionInfo()) != null) {
|
||||
isRecovering.set((response.hasIsRecovering()) ? response.getIsRecovering() : true);
|
||||
return loc;
|
||||
}
|
||||
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException e) {
|
||||
throw ProtobufUtil.handleRemoteException(e);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
cause = e.getCause();
|
||||
if(!(cause instanceof RegionOpeningException)) {
|
||||
reloadLocation = true;
|
||||
}
|
||||
}
|
||||
long expectedSleep = ConnectionUtils.getPauseTime(pause, tries);
|
||||
try {
|
||||
Thread.sleep(expectedSleep);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted when waiting region " +
|
||||
loc.getRegionInfo().getEncodedName() + " online.", e);
|
||||
}
|
||||
tries++;
|
||||
}
|
||||
|
||||
throw new IOException("Timeout when waiting region " + loc.getRegionInfo().getEncodedName() +
|
||||
" online for " + timeout + " milliseconds.", cause);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean flush() throws IOException {
|
||||
String curLoc = null;
|
||||
int curSize = 0;
|
||||
List<Pair<HRegionLocation, Entry>> curQueue = null;
|
||||
synchronized (this.serverToBufferQueueMap) {
|
||||
for (Map.Entry<String, List<Pair<HRegionLocation, Entry>>> entry :
|
||||
this.serverToBufferQueueMap.entrySet()) {
|
||||
curQueue = entry.getValue();
|
||||
if (!curQueue.isEmpty()) {
|
||||
curSize = curQueue.size();
|
||||
curLoc = entry.getKey();
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (curSize > 0) {
|
||||
this.serverToBufferQueueMap.remove(curLoc);
|
||||
}
|
||||
}
|
||||
|
||||
if (curSize > 0) {
|
||||
this.processWorkItems(curLoc, curQueue);
|
||||
// We should already have control of the monitor; ensure this is the case.
|
||||
synchronized(controller.dataAvailable) {
|
||||
controller.dataAvailable.notifyAll();
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean keepRegionEvent(Entry entry) {
|
||||
return true;
|
||||
}
|
||||
|
||||
void addWriterError(Throwable t) {
|
||||
thrown.add(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Path> finishWritingAndClose() throws IOException {
|
||||
try {
|
||||
if (!finishWriting(false)) {
|
||||
return null;
|
||||
}
|
||||
if (hasEditsInDisablingOrDisabledTables) {
|
||||
splits = logRecoveredEditsOutputSink.finishWritingAndClose();
|
||||
} else {
|
||||
splits = new ArrayList<>();
|
||||
}
|
||||
// returns an empty array in order to keep interface same as old way
|
||||
return splits;
|
||||
} finally {
|
||||
List<IOException> thrown = closeRegionServerWriters();
|
||||
if (thrown != null && !thrown.isEmpty()) {
|
||||
throw MultipleIOException.createIOException(thrown);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
int getNumOpenWriters() {
|
||||
return this.writers.size() + this.logRecoveredEditsOutputSink.getNumOpenWriters();
|
||||
}
|
||||
|
||||
private List<IOException> closeRegionServerWriters() throws IOException {
|
||||
List<IOException> result = null;
|
||||
if (!writersClosed) {
|
||||
result = Lists.newArrayList();
|
||||
try {
|
||||
for (WriterThread t : writerThreads) {
|
||||
while (t.isAlive()) {
|
||||
t.shouldStop = true;
|
||||
t.interrupt();
|
||||
try {
|
||||
t.join(10);
|
||||
} catch (InterruptedException e) {
|
||||
IOException iie = new InterruptedIOException();
|
||||
iie.initCause(e);
|
||||
throw iie;
|
||||
}
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
synchronized (writers) {
|
||||
for (Map.Entry<String, RegionServerWriter> entry : writers.entrySet()) {
|
||||
RegionServerWriter tmpW = entry.getValue();
|
||||
try {
|
||||
tmpW.close();
|
||||
} catch (IOException ioe) {
|
||||
LOG.error("Couldn't close writer for region server:" + entry.getKey(), ioe);
|
||||
result.add(ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// close connections
|
||||
synchronized (this.tableNameToHConnectionMap) {
|
||||
for (Map.Entry<TableName, ClusterConnection> entry :
|
||||
this.tableNameToHConnectionMap.entrySet()) {
|
||||
ClusterConnection cconn = entry.getValue();
|
||||
try {
|
||||
cconn.clearRegionCache();
|
||||
cconn.close();
|
||||
} catch (IOException ioe) {
|
||||
result.add(ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
writersClosed = true;
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<byte[], Long> getOutputCounts() {
|
||||
TreeMap<byte[], Long> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
synchronized (writers) {
|
||||
for (Map.Entry<String, RegionServerWriter> entry : writers.entrySet()) {
|
||||
ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumberOfRecoveredRegions() {
|
||||
return this.recoveredRegions.size();
|
||||
}
|
||||
|
||||
private boolean isTableDisabledOrDisabling(TableName tableName) {
|
||||
if (connection == null)
|
||||
return false; // we can't get state without CoordinatedStateManager
|
||||
if (tableName.isSystemTable())
|
||||
return false; // assume that system tables never can be disabled
|
||||
TableState tableState = tableStatesCache.get(tableName);
|
||||
if (tableState == null) {
|
||||
try {
|
||||
tableState = MetaTableAccessor.getTableState(connection, tableName);
|
||||
if (tableState != null)
|
||||
tableStatesCache.put(tableName, tableState);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("State is not accessible for table " + tableName, e);
|
||||
}
|
||||
}
|
||||
return tableState != null && tableState
|
||||
.inStates(TableState.State.DISABLED, TableState.State.DISABLING);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a writer and path for a log starting at the given entry. This function is threadsafe so
|
||||
* long as multiple threads are always acting on different regions.
|
||||
* @return null if this region shouldn't output any logs
|
||||
*/
|
||||
private RegionServerWriter getRegionServerWriter(String loc) throws IOException {
|
||||
RegionServerWriter ret = writers.get(loc);
|
||||
if (ret != null) {
|
||||
return ret;
|
||||
}
|
||||
|
||||
TableName tableName = getTableFromLocationStr(loc);
|
||||
if(tableName == null){
|
||||
throw new IOException("Invalid location string:" + loc + " found. Replay aborted.");
|
||||
}
|
||||
|
||||
ClusterConnection hconn = getConnectionByTableName(tableName);
|
||||
synchronized (writers) {
|
||||
ret = writers.get(loc);
|
||||
if (ret == null) {
|
||||
ret = new RegionServerWriter(conf, tableName, hconn);
|
||||
writers.put(loc, ret);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
private ClusterConnection getConnectionByTableName(final TableName tableName) throws IOException {
|
||||
ClusterConnection cconn = this.tableNameToHConnectionMap.get(tableName);
|
||||
if (cconn == null) {
|
||||
synchronized (this.tableNameToHConnectionMap) {
|
||||
cconn = this.tableNameToHConnectionMap.get(tableName);
|
||||
if (cconn == null) {
|
||||
cconn = (ClusterConnection) ConnectionFactory.createConnection(conf);
|
||||
this.tableNameToHConnectionMap.put(tableName, cconn);
|
||||
}
|
||||
}
|
||||
}
|
||||
return cconn;
|
||||
}
|
||||
private TableName getTableFromLocationStr(String loc) {
|
||||
/**
|
||||
* location key is in format {@literal <server name:port>#<table name>}
|
||||
*/
|
||||
String[] splits = loc.split(KEY_DELIMITER);
|
||||
if (splits.length != 2) {
|
||||
return null;
|
||||
}
|
||||
return TableName.valueOf(splits[1]);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Private data structure that wraps a receiving RS and collecting statistics about the data
|
||||
* written to this newly assigned RS.
|
||||
*/
|
||||
private final static class RegionServerWriter extends SinkWriter {
|
||||
final WALEditsReplaySink sink;
|
||||
|
||||
RegionServerWriter(final Configuration conf, final TableName tableName, final ClusterConnection conn)
|
||||
throws IOException {
|
||||
this.sink = new WALEditsReplaySink(conf, tableName, conn);
|
||||
}
|
||||
|
||||
void close() throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
static class CorruptedLogFileException extends Exception {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
|
|
|
@ -1,92 +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.zookeeper;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* Watcher used to be notified of the recovering region coming out of recovering state
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class RecoveringRegionWatcher extends ZooKeeperListener {
|
||||
private static final Log LOG = LogFactory.getLog(RecoveringRegionWatcher.class);
|
||||
|
||||
private HRegionServer server;
|
||||
|
||||
/**
|
||||
* Construct a ZooKeeper event listener.
|
||||
*/
|
||||
public RecoveringRegionWatcher(ZooKeeperWatcher watcher, HRegionServer server) {
|
||||
super(watcher);
|
||||
watcher.registerListener(this);
|
||||
this.server = server;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when a node has been deleted
|
||||
* @param path full path of the deleted node
|
||||
*/
|
||||
@Override
|
||||
public void nodeDeleted(String path) {
|
||||
if (this.server.isStopped() || this.server.isStopping()) {
|
||||
return;
|
||||
}
|
||||
|
||||
String parentPath = path.substring(0, path.lastIndexOf('/'));
|
||||
if (!this.watcher.znodePaths.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) {
|
||||
return;
|
||||
}
|
||||
|
||||
String regionName = path.substring(parentPath.length() + 1);
|
||||
|
||||
server.getExecutorService().submit(new FinishRegionRecoveringHandler(server, regionName, path));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void nodeDataChanged(String path) {
|
||||
registerWatcher(path);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void nodeChildrenChanged(String path) {
|
||||
registerWatcher(path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Reinstall watcher because watcher only fire once though we're only interested in nodeDeleted
|
||||
* event we need to register the watcher in case other event happens
|
||||
*/
|
||||
private void registerWatcher(String path) {
|
||||
String parentPath = path.substring(0, path.lastIndexOf('/'));
|
||||
if (!this.watcher.znodePaths.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) {
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
ZKUtil.getDataAndWatch(watcher, path);
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Can't register watcher on znode " + path, e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -18,11 +18,9 @@
|
|||
package org.apache.hadoop.hbase.zookeeper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.net.URLDecoder;
|
||||
import java.net.URLEncoder;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -30,12 +28,9 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
|
||||
/**
|
||||
* Common methods and attributes used by {@link org.apache.hadoop.hbase.master.SplitLogManager}
|
||||
* Common methods and attributes used by {@link org.apache.hadoop.hbase.master.SplitLogManager}
|
||||
* and {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker}
|
||||
* running distributed splitting of WAL logs.
|
||||
*/
|
||||
|
@ -104,20 +99,10 @@ public class ZKSplitLog {
|
|||
return true;
|
||||
}
|
||||
|
||||
public static boolean isTaskPath(ZooKeeperWatcher zkw, String path) {
|
||||
String dirname = path.substring(0, path.lastIndexOf('/'));
|
||||
return dirname.equals(zkw.znodePaths.splitLogZNode);
|
||||
}
|
||||
|
||||
public static Path getSplitLogDir(Path rootdir, String tmpname) {
|
||||
return new Path(new Path(rootdir, HConstants.SPLIT_LOGDIR_NAME), tmpname);
|
||||
}
|
||||
|
||||
|
||||
public static String getSplitLogDirTmpComponent(final String worker, String file) {
|
||||
return worker + "_" + ZKSplitLog.encode(file);
|
||||
}
|
||||
|
||||
public static void markCorrupted(Path rootdir, String logFileName,
|
||||
FileSystem fs) {
|
||||
Path file = new Path(getSplitLogDir(rootdir, logFileName), "corrupt");
|
||||
|
@ -136,101 +121,4 @@ public class ZKSplitLog {
|
|||
isCorrupt = fs.exists(file);
|
||||
return isCorrupt;
|
||||
}
|
||||
|
||||
/*
|
||||
* Following methods come from SplitLogManager
|
||||
*/
|
||||
|
||||
/**
|
||||
* check if /hbase/recovering-regions/<current region encoded name>
|
||||
* exists. Returns true if exists and set watcher as well.
|
||||
* @param zkw
|
||||
* @param regionEncodedName region encode name
|
||||
* @return true when /hbase/recovering-regions/<current region encoded name> exists
|
||||
* @throws KeeperException
|
||||
*/
|
||||
public static boolean
|
||||
isRegionMarkedRecoveringInZK(ZooKeeperWatcher zkw, String regionEncodedName)
|
||||
throws KeeperException {
|
||||
boolean result = false;
|
||||
String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, regionEncodedName);
|
||||
|
||||
byte[] node = ZKUtil.getDataAndWatch(zkw, nodePath);
|
||||
if (node != null) {
|
||||
result = true;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param bytes - Content of a failed region server or recovering region znode.
|
||||
* @return long - The last flushed sequence Id for the region server
|
||||
*/
|
||||
public static long parseLastFlushedSequenceIdFrom(final byte[] bytes) {
|
||||
long lastRecordedFlushedSequenceId = -1l;
|
||||
try {
|
||||
lastRecordedFlushedSequenceId = ZKUtil.parseWALPositionFrom(bytes);
|
||||
} catch (DeserializationException e) {
|
||||
lastRecordedFlushedSequenceId = -1l;
|
||||
LOG.warn("Can't parse last flushed sequence Id", e);
|
||||
}
|
||||
return lastRecordedFlushedSequenceId;
|
||||
}
|
||||
|
||||
public static void deleteRecoveringRegionZNodes(ZooKeeperWatcher watcher, List<String> regions) {
|
||||
try {
|
||||
if (regions == null) {
|
||||
// remove all children under /home/recovering-regions
|
||||
LOG.debug("Garbage collecting all recovering region znodes");
|
||||
ZKUtil.deleteChildrenRecursively(watcher, watcher.znodePaths.recoveringRegionsZNode);
|
||||
} else {
|
||||
for (String curRegion : regions) {
|
||||
String nodePath = ZKUtil.joinZNode(watcher.znodePaths.recoveringRegionsZNode, curRegion);
|
||||
ZKUtil.deleteNodeRecursively(watcher, nodePath);
|
||||
}
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
LOG.warn("Cannot remove recovering regions from ZooKeeper", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This function is used in distributedLogReplay to fetch last flushed sequence id from ZK
|
||||
* @param zkw
|
||||
* @param serverName
|
||||
* @param encodedRegionName
|
||||
* @return the last flushed sequence ids recorded in ZK of the region for <code>serverName</code>
|
||||
* @throws IOException
|
||||
*/
|
||||
|
||||
public static RegionStoreSequenceIds getRegionFlushedSequenceId(ZooKeeperWatcher zkw,
|
||||
String serverName, String encodedRegionName) throws IOException {
|
||||
// when SplitLogWorker recovers a region by directly replaying unflushed WAL edits,
|
||||
// last flushed sequence Id changes when newly assigned RS flushes writes to the region.
|
||||
// If the newly assigned RS fails again(a chained RS failures scenario), the last flushed
|
||||
// sequence Id name space (sequence Id only valid for a particular RS instance), changes
|
||||
// when different newly assigned RS flushes the region.
|
||||
// Therefore, in this mode we need to fetch last sequence Ids from ZK where we keep history of
|
||||
// last flushed sequence Id for each failed RS instance.
|
||||
RegionStoreSequenceIds result = null;
|
||||
String nodePath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, encodedRegionName);
|
||||
nodePath = ZKUtil.joinZNode(nodePath, serverName);
|
||||
try {
|
||||
byte[] data;
|
||||
try {
|
||||
data = ZKUtil.getData(zkw, nodePath);
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
if (data != null) {
|
||||
result = ZKUtil.parseRegionStoreSequenceIds(data);
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Cannot get lastFlushedSequenceId from ZooKeeper for server="
|
||||
+ serverName + "; region=" + encodedRegionName, e);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Can't parse last flushed sequence Id from znode:" + nodePath, e);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -264,12 +264,6 @@ public class MockRegionServerServices implements RegionServerServices {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HRegion> getRecoveringRegions() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerNonceManager getNonceManager() {
|
||||
// TODO Auto-generated method stub
|
||||
|
|
|
@ -323,7 +323,7 @@ public class TestIOFencing {
|
|||
@Override
|
||||
public boolean evaluate() throws Exception {
|
||||
Region newRegion = newServer.getOnlineRegion(REGION_NAME);
|
||||
return newRegion != null && !newRegion.isRecovering();
|
||||
return newRegion != null;
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -36,20 +36,16 @@ import java.util.Set;
|
|||
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.filter.BinaryComparator;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||
import org.apache.hadoop.hbase.filter.RowFilter;
|
||||
import org.apache.hadoop.hbase.io.TimeRange;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -121,15 +117,6 @@ public class TestSerialization {
|
|||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSplitLogTask() throws DeserializationException {
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"),
|
||||
RecoveryMode.LOG_REPLAY);
|
||||
byte [] bytes = slt.toByteArray();
|
||||
SplitLogTask sltDeserialized = SplitLogTask.parseFrom(bytes);
|
||||
assertTrue(slt.equals(sltDeserialized));
|
||||
}
|
||||
|
||||
@Test public void testCompareFilter() throws Exception {
|
||||
Filter f = new RowFilter(CompareOperator.EQUAL,
|
||||
new BinaryComparator(Bytes.toBytes("testRowOne-2")));
|
||||
|
|
|
@ -264,7 +264,7 @@ public class TestReplicasClient {
|
|||
} catch (Exception e){}
|
||||
// first version is '0'
|
||||
AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
|
||||
getRS().getServerName(), hri, null, null);
|
||||
getRS().getServerName(), hri, null);
|
||||
AdminProtos.OpenRegionResponse responseOpen = getRS().getRSRpcServices().openRegion(null, orr);
|
||||
Assert.assertEquals(responseOpen.getOpeningStateCount(), 1);
|
||||
Assert.assertEquals(responseOpen.getOpeningState(0),
|
||||
|
|
|
@ -556,11 +556,6 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, HRegion> getRecoveringRegions() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public UpdateFavoredNodesResponse updateFavoredNodes(RpcController controller,
|
||||
UpdateFavoredNodesRequest request) throws ServiceException {
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1,105 +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 java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Test the master wal manager in a local cluster
|
||||
*/
|
||||
@Category({MasterTests.class, MediumTests.class})
|
||||
public class TestMasterWalManager {
|
||||
private static final Log LOG = LogFactory.getLog(TestMasterWalManager.class);
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
@BeforeClass
|
||||
public static void setupTest() throws Exception {
|
||||
UTIL.startMiniCluster();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void teardownTest() throws Exception {
|
||||
UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveStaleRecoveringRegionsDuringMasterInitialization() throws Exception {
|
||||
// this test is for when distributed log replay is enabled
|
||||
if (!UTIL.getConfiguration().getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false)) return;
|
||||
|
||||
LOG.info("Starting testRemoveStaleRecoveringRegionsDuringMasterInitialization");
|
||||
HMaster master = UTIL.getMiniHBaseCluster().getMaster();
|
||||
MasterWalManager mwm = master.getMasterWalManager();
|
||||
|
||||
String failedRegion = "failedRegoin1";
|
||||
String staleRegion = "staleRegion";
|
||||
ServerName inRecoveryServerName = ServerName.valueOf("mgr,1,1");
|
||||
ServerName previouselyFaildServerName = ServerName.valueOf("previous,1,1");
|
||||
String walPath = "/hbase/data/.logs/" + inRecoveryServerName.getServerName()
|
||||
+ "-splitting/test";
|
||||
// Create a ZKW to use in the test
|
||||
ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(UTIL);
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, walPath),
|
||||
new SplitLogTask.Owned(inRecoveryServerName).toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
String staleRegionPath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode, staleRegion);
|
||||
ZKUtil.createWithParents(zkw, staleRegionPath);
|
||||
String inRecoveringRegionPath = ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode,
|
||||
failedRegion);
|
||||
inRecoveringRegionPath = ZKUtil.joinZNode(inRecoveringRegionPath,
|
||||
inRecoveryServerName.getServerName());
|
||||
ZKUtil.createWithParents(zkw, inRecoveringRegionPath);
|
||||
Set<ServerName> servers = new HashSet<>();
|
||||
servers.add(previouselyFaildServerName);
|
||||
mwm.removeStaleRecoveringRegionsFromZK(servers);
|
||||
|
||||
// verification
|
||||
assertFalse(ZKUtil.checkExists(zkw, staleRegionPath) != -1);
|
||||
assertTrue(ZKUtil.checkExists(zkw, inRecoveringRegionPath) != -1);
|
||||
|
||||
ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.recoveringRegionsZNode);
|
||||
ZKUtil.deleteChildrenRecursively(zkw, zkw.znodePaths.splitLogZNode);
|
||||
zkw.close();
|
||||
}
|
||||
}
|
|
@ -36,7 +36,6 @@ import static org.junit.Assert.assertFalse;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
@ -49,7 +48,6 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.SplitLogTask;
|
||||
|
@ -59,7 +57,6 @@ import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
|
|||
import org.apache.hadoop.hbase.master.SplitLogManager.Task;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
|
||||
import org.apache.hadoop.hbase.regionserver.TestMasterAddressTracker.NodeCreationListener;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
|
||||
|
@ -73,7 +70,6 @@ import org.apache.zookeeper.ZooDefs.Ids;
|
|||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -93,7 +89,6 @@ public class TestSplitLogManager {
|
|||
private SplitLogManager slm;
|
||||
private Configuration conf;
|
||||
private int to;
|
||||
private RecoveryMode mode;
|
||||
|
||||
private static HBaseTestingUtility TEST_UTIL;
|
||||
|
||||
|
@ -153,10 +148,6 @@ public class TestSplitLogManager {
|
|||
|
||||
conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
|
||||
to = to + 16 * 100;
|
||||
|
||||
this.mode =
|
||||
(conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ? RecoveryMode.LOG_REPLAY
|
||||
: RecoveryMode.LOG_SPLITTING);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -245,7 +236,7 @@ public class TestSplitLogManager {
|
|||
LOG.info("TestOrphanTaskAcquisition");
|
||||
|
||||
String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(master.getServerName(), this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(master.getServerName());
|
||||
zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
|
@ -270,7 +261,7 @@ public class TestSplitLogManager {
|
|||
" startup");
|
||||
String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
|
||||
//create an unassigned orphan task
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(master.getServerName(), this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(master.getServerName());
|
||||
zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
|
@ -305,19 +296,19 @@ public class TestSplitLogManager {
|
|||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
final ServerName worker2 = ServerName.valueOf("worker2,1,1");
|
||||
final ServerName worker3 = ServerName.valueOf("worker3,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
|
||||
waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
|
||||
int version1 = ZKUtil.checkExists(zkw, tasknode);
|
||||
assertTrue(version1 > version);
|
||||
slt = new SplitLogTask.Owned(worker2, this.mode);
|
||||
slt = new SplitLogTask.Owned(worker2);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 1, 2, to/2);
|
||||
waitForCounter(tot_mgr_resubmit, 1, 2, to + to/2);
|
||||
int version2 = ZKUtil.checkExists(zkw, tasknode);
|
||||
assertTrue(version2 > version1);
|
||||
slt = new SplitLogTask.Owned(worker3, this.mode);
|
||||
slt = new SplitLogTask.Owned(worker3);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 2, 3, to/2);
|
||||
waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + to/2);
|
||||
|
@ -335,7 +326,7 @@ public class TestSplitLogManager {
|
|||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
|
||||
waitForCounter(new Expr() {
|
||||
|
@ -362,7 +353,7 @@ public class TestSplitLogManager {
|
|||
TaskBatch batch = new TaskBatch();
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Done(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Done(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
synchronized (batch) {
|
||||
while (batch.installed != batch.done) {
|
||||
|
@ -383,7 +374,7 @@ public class TestSplitLogManager {
|
|||
|
||||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Err(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Err(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
|
||||
synchronized (batch) {
|
||||
|
@ -407,7 +398,7 @@ public class TestSplitLogManager {
|
|||
assertEquals(tot_mgr_resubmit.sum(), 0);
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
assertEquals(tot_mgr_resubmit.sum(), 0);
|
||||
SplitLogTask slt = new SplitLogTask.Resigned(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Resigned(worker1);
|
||||
assertEquals(tot_mgr_resubmit.sum(), 0);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
ZKUtil.checkExists(zkw, tasknode);
|
||||
|
@ -430,7 +421,7 @@ public class TestSplitLogManager {
|
|||
// create an orphan task in OWNED state
|
||||
String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
zkw.getRecoverableZooKeeper().create(tasknode1, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
|
@ -445,7 +436,7 @@ public class TestSplitLogManager {
|
|||
for (int i = 0; i < (3 * to)/100; i++) {
|
||||
Thread.sleep(100);
|
||||
final ServerName worker2 = ServerName.valueOf("worker1,1,1");
|
||||
slt = new SplitLogTask.Owned(worker2, this.mode);
|
||||
slt = new SplitLogTask.Owned(worker2);
|
||||
ZKUtil.setData(zkw, tasknode1, slt.toByteArray());
|
||||
}
|
||||
|
||||
|
@ -469,7 +460,7 @@ public class TestSplitLogManager {
|
|||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
int version = ZKUtil.checkExists(zkw, tasknode);
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
if (tot_mgr_heartbeat.sum() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
|
||||
slm.handleDeadWorker(worker1);
|
||||
|
@ -494,7 +485,7 @@ public class TestSplitLogManager {
|
|||
String tasknode = submitTaskAndWait(batch, "foo/1");
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(worker1);
|
||||
ZKUtil.setData(zkw, tasknode, slt.toByteArray());
|
||||
if (tot_mgr_heartbeat.sum() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
|
||||
|
||||
|
@ -544,7 +535,7 @@ public class TestSplitLogManager {
|
|||
while (!done) {
|
||||
for (Map.Entry<String, Task> entry : slm.getTasks().entrySet()) {
|
||||
final ServerName worker1 = ServerName.valueOf("worker1,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Done(worker1, RecoveryMode.LOG_SPLITTING);
|
||||
SplitLogTask slt = new SplitLogTask.Done(worker1);
|
||||
boolean encounteredZKException = false;
|
||||
try {
|
||||
ZKUtil.setData(zkw, entry.getKey(), slt.toByteArray());
|
||||
|
@ -564,50 +555,4 @@ public class TestSplitLogManager {
|
|||
|
||||
assertFalse(fs.exists(logDirPath));
|
||||
}
|
||||
|
||||
/**
|
||||
* The following test case is aiming to test the situation when distributedLogReplay is turned off
|
||||
* and restart a cluster there should no recovery regions in ZK left.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test(timeout = 300000)
|
||||
public void testRecoveryRegionRemovedFromZK() throws Exception {
|
||||
LOG.info("testRecoveryRegionRemovedFromZK");
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
String nodePath =
|
||||
ZKUtil.joinZNode(zkw.znodePaths.recoveringRegionsZNode,
|
||||
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
|
||||
ZKUtil.createSetData(zkw, nodePath, ZKUtil.positionToByteArray(0L));
|
||||
|
||||
slm = new SplitLogManager(master, conf);
|
||||
slm.removeStaleRecoveringRegions(null);
|
||||
|
||||
List<String> recoveringRegions =
|
||||
zkw.getRecoverableZooKeeper().getChildren(zkw.znodePaths.recoveringRegionsZNode, false);
|
||||
|
||||
assertTrue("Recovery regions isn't cleaned", recoveringRegions.isEmpty());
|
||||
}
|
||||
|
||||
@Ignore("DLR is broken by HBASE-12751") @Test(timeout=60000)
|
||||
public void testGetPreviousRecoveryMode() throws Exception {
|
||||
LOG.info("testGetPreviousRecoveryMode");
|
||||
SplitLogCounters.resetCounters();
|
||||
// Not actually enabling DLR for the cluster, just for the ZkCoordinatedStateManager to use.
|
||||
// The test is just manipulating ZK manually anyways.
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
|
||||
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, "testRecovery"),
|
||||
new SplitLogTask.Unassigned(
|
||||
ServerName.valueOf("mgr,1,1"), RecoveryMode.LOG_SPLITTING).toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
|
||||
slm = new SplitLogManager(master, conf);
|
||||
LOG.info("Mode1=" + slm.getRecoveryMode());
|
||||
assertTrue(slm.isLogSplitting());
|
||||
zkw.getRecoverableZooKeeper().delete(ZKSplitLog.getEncodedNodeName(zkw, "testRecovery"), -1);
|
||||
LOG.info("Mode2=" + slm.getRecoveryMode());
|
||||
slm.setRecoveryMode(false);
|
||||
LOG.info("Mode3=" + slm.getRecoveryMode());
|
||||
assertTrue("Mode4=" + slm.getRecoveryMode(), slm.isLogReplaying());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -48,7 +48,6 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
|
@ -134,7 +133,6 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
|||
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.Region.RowLock;
|
||||
import org.apache.hadoop.hbase.regionserver.TestHStore.FaultyFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
|
||||
|
@ -176,7 +174,6 @@ import org.mockito.invocation.InvocationOnMock;
|
|||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
|
||||
|
@ -5902,93 +5899,6 @@ public class TestHRegion {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOpenRegionWrittenToWALForLogReplay() throws Exception {
|
||||
// similar to the above test but with distributed log replay
|
||||
final ServerName serverName = ServerName.valueOf(name.getMethodName(), 100, 42);
|
||||
final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
|
||||
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
|
||||
htd.addFamily(new HColumnDescriptor(fam1));
|
||||
htd.addFamily(new HColumnDescriptor(fam2));
|
||||
|
||||
HRegionInfo hri = new HRegionInfo(htd.getTableName(),
|
||||
HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
|
||||
|
||||
// open the region w/o rss and wal and flush some files
|
||||
HRegion region =
|
||||
HBaseTestingUtility.createRegionAndWAL(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL
|
||||
.getConfiguration(), htd);
|
||||
assertNotNull(region);
|
||||
|
||||
// create a file in fam1 for the region before opening in OpenRegionHandler
|
||||
region.put(new Put(Bytes.toBytes("a")).addColumn(fam1, fam1, fam1));
|
||||
region.flush(true);
|
||||
HBaseTestingUtility.closeRegionAndWAL(region);
|
||||
|
||||
ArgumentCaptor<WALEdit> editCaptor = ArgumentCaptor.forClass(WALEdit.class);
|
||||
|
||||
// capture append() calls
|
||||
WAL wal = mockWAL();
|
||||
when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
|
||||
|
||||
// add the region to recovering regions
|
||||
HashMap<String, HRegion> recoveringRegions = Maps.newHashMap();
|
||||
recoveringRegions.put(region.getRegionInfo().getEncodedName(), null);
|
||||
when(rss.getRecoveringRegions()).thenReturn(recoveringRegions);
|
||||
|
||||
try {
|
||||
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
|
||||
conf.set(HConstants.REGION_IMPL, HRegionWithSeqId.class.getName());
|
||||
region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
|
||||
conf, rss, null);
|
||||
|
||||
// verify that we have not appended region open event to WAL because this region is still
|
||||
// recovering
|
||||
verify(wal, times(0)).append((HRegionInfo)any(), (WALKey)any()
|
||||
, editCaptor.capture(), anyBoolean());
|
||||
|
||||
// not put the region out of recovering state
|
||||
new FinishRegionRecoveringHandler(rss, region.getRegionInfo().getEncodedName(), "/foo")
|
||||
.prepare().process();
|
||||
|
||||
// now we should have put the entry
|
||||
verify(wal, times(1)).append((HRegionInfo)any(), (WALKey)any()
|
||||
, editCaptor.capture(), anyBoolean());
|
||||
|
||||
WALEdit edit = editCaptor.getValue();
|
||||
assertNotNull(edit);
|
||||
assertNotNull(edit.getCells());
|
||||
assertEquals(1, edit.getCells().size());
|
||||
RegionEventDescriptor desc = WALEdit.getRegionEventDescriptor(edit.getCells().get(0));
|
||||
assertNotNull(desc);
|
||||
|
||||
LOG.info("RegionEventDescriptor from WAL: " + desc);
|
||||
|
||||
assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
|
||||
assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getTableName().toBytes()));
|
||||
assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
|
||||
hri.getEncodedNameAsBytes()));
|
||||
assertTrue(desc.getLogSequenceNumber() > 0);
|
||||
assertEquals(serverName, ProtobufUtil.toServerName(desc.getServer()));
|
||||
assertEquals(2, desc.getStoresCount());
|
||||
|
||||
StoreDescriptor store = desc.getStores(0);
|
||||
assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam1));
|
||||
assertEquals(store.getStoreHomeDir(), Bytes.toString(fam1));
|
||||
assertEquals(1, store.getStoreFileCount()); // 1store file
|
||||
assertFalse(store.getStoreFile(0).contains("/")); // ensure path is relative
|
||||
|
||||
store = desc.getStores(1);
|
||||
assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam2));
|
||||
assertEquals(store.getStoreHomeDir(), Bytes.toString(fam2));
|
||||
assertEquals(0, store.getStoreFileCount()); // no store files
|
||||
|
||||
} finally {
|
||||
HBaseTestingUtility.closeRegionAndWAL(region);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to setup a WAL mock.
|
||||
* Needs to do the bit where we close latch on the WALKey on append else test hangs.
|
||||
|
|
|
@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
@ -417,20 +416,9 @@ public class TestPerColumnFamilyFlush {
|
|||
}
|
||||
}
|
||||
|
||||
// Test Log Replay with Distributed Replay on.
|
||||
// In distributed log replay, the log splitters ask the master for the
|
||||
// last flushed sequence id for a region. This test would ensure that we
|
||||
// are doing the book-keeping correctly.
|
||||
@Ignore("DLR is broken by HBASE-12751") @Test(timeout = 180000)
|
||||
public void testLogReplayWithDistributedReplay() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
|
||||
doTestLogReplay();
|
||||
}
|
||||
|
||||
// Test Log Replay with Distributed log split on.
|
||||
@Test(timeout = 180000)
|
||||
public void testLogReplayWithDistributedLogSplit() throws Exception {
|
||||
TEST_UTIL.getConfiguration().setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
doTestLogReplay();
|
||||
}
|
||||
|
||||
|
|
|
@ -22,8 +22,6 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
@ -57,14 +55,10 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
||||
/**
|
||||
* Tests failover of secondary region replicas.
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
@Category(LargeTests.class)
|
||||
public class TestRegionReplicaFailover {
|
||||
|
||||
|
@ -90,19 +84,6 @@ public class TestRegionReplicaFailover {
|
|||
|
||||
private HTableDescriptor htd;
|
||||
|
||||
/*
|
||||
* We are testing with dist log split and dist log replay separately
|
||||
*/
|
||||
@Parameters
|
||||
public static Collection<Object[]> getParameters() {
|
||||
Object[][] params =
|
||||
new Boolean[][] { /*{true}, Disable DLR!!! It is going to be removed*/ {false} };
|
||||
return Arrays.asList(params);
|
||||
}
|
||||
|
||||
@Parameterized.Parameter(0)
|
||||
public boolean distributedLogReplay;
|
||||
|
||||
@Before
|
||||
public void before() throws Exception {
|
||||
Configuration conf = HTU.getConfiguration();
|
||||
|
@ -112,7 +93,6 @@ public class TestRegionReplicaFailover {
|
|||
conf.setBoolean(ServerRegionReplicaUtil.REGION_REPLICA_WAIT_FOR_PRIMARY_FLUSH_CONF_KEY, true);
|
||||
conf.setInt("replication.stats.thread.period.seconds", 5);
|
||||
conf.setBoolean("hbase.tests.use.shortcircuit.reads", false);
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, distributedLogReplay);
|
||||
|
||||
HTU.startMiniCluster(NB_SERVERS);
|
||||
htd = HTU.createTableDescriptor(
|
||||
|
|
|
@ -153,7 +153,7 @@ public class TestRegionServerNoMaster {
|
|||
public static void openRegion(HBaseTestingUtility HTU, HRegionServer rs, HRegionInfo hri)
|
||||
throws Exception {
|
||||
AdminProtos.OpenRegionRequest orr =
|
||||
RequestConverter.buildOpenRegionRequest(rs.getServerName(), hri, null, null);
|
||||
RequestConverter.buildOpenRegionRequest(rs.getServerName(), hri, null);
|
||||
AdminProtos.OpenRegionResponse responseOpen = rs.rpcServices.openRegion(null, orr);
|
||||
|
||||
Assert.assertTrue(responseOpen.getOpeningStateCount() == 1);
|
||||
|
@ -295,7 +295,7 @@ public class TestRegionServerNoMaster {
|
|||
closeRegionNoZK();
|
||||
try {
|
||||
AdminProtos.OpenRegionRequest orr = RequestConverter.buildOpenRegionRequest(
|
||||
earlierServerName, hri, null, null);
|
||||
earlierServerName, hri, null);
|
||||
getRS().getRSRpcServices().openRegion(null, orr);
|
||||
Assert.fail("The openRegion should have been rejected");
|
||||
} catch (org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException se) {
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.ChoreService;
|
|||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
|
@ -47,7 +46,6 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
|
|||
import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.ExecutorType;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.CancelableProgressable;
|
||||
|
@ -78,7 +76,6 @@ public class TestSplitLogWorker {
|
|||
private ZooKeeperWatcher zkw;
|
||||
private SplitLogWorker slw;
|
||||
private ExecutorService executorService;
|
||||
private RecoveryMode mode;
|
||||
|
||||
class DummyServer implements Server {
|
||||
private ZooKeeperWatcher zkw;
|
||||
|
@ -212,8 +209,6 @@ public class TestSplitLogWorker {
|
|||
SplitLogCounters.resetCounters();
|
||||
executorService = new ExecutorService("TestSplitLogWorker");
|
||||
executorService.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, 10);
|
||||
this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -228,7 +223,7 @@ public class TestSplitLogWorker {
|
|||
new SplitLogWorker.TaskExecutor() {
|
||||
|
||||
@Override
|
||||
public Status exec(String name, RecoveryMode mode, CancelableProgressable p) {
|
||||
public Status exec(String name, CancelableProgressable p) {
|
||||
while (true) {
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
|
@ -251,7 +246,7 @@ public class TestSplitLogWorker {
|
|||
final ServerName RS = ServerName.valueOf("rs,1,1");
|
||||
RegionServerServices mockedRS = getRegionServer(RS);
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS),
|
||||
new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"), this.mode).toByteArray(),
|
||||
new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1")).toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
|
@ -287,7 +282,7 @@ public class TestSplitLogWorker {
|
|||
final ServerName SVR1 = ServerName.valueOf("svr1,1,1");
|
||||
final ServerName SVR2 = ServerName.valueOf("svr2,1,1");
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TRFT),
|
||||
new SplitLogTask.Unassigned(MANAGER, this.mode).toByteArray(),
|
||||
new SplitLogTask.Unassigned(MANAGER).toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
RegionServerServices mockedRS1 = getRegionServer(SVR1);
|
||||
RegionServerServices mockedRS2 = getRegionServer(SVR2);
|
||||
|
@ -330,7 +325,7 @@ public class TestSplitLogWorker {
|
|||
|
||||
// this time create a task node after starting the splitLogWorker
|
||||
zkw.getRecoverableZooKeeper().create(PATH,
|
||||
new SplitLogTask.Unassigned(MANAGER, this.mode).toByteArray(),
|
||||
new SplitLogTask.Unassigned(MANAGER).toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
|
||||
waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME);
|
||||
|
@ -338,7 +333,7 @@ public class TestSplitLogWorker {
|
|||
byte [] bytes = ZKUtil.getData(zkw, PATH);
|
||||
SplitLogTask slt = SplitLogTask.parseFrom(bytes);
|
||||
assertTrue(slt.isOwned(SRV));
|
||||
slt = new SplitLogTask.Owned(MANAGER, this.mode);
|
||||
slt = new SplitLogTask.Owned(MANAGER);
|
||||
ZKUtil.setData(zkw, PATH, slt.toByteArray());
|
||||
waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME);
|
||||
} finally {
|
||||
|
@ -362,7 +357,7 @@ public class TestSplitLogWorker {
|
|||
waitForCounter(SplitLogCounters.tot_wkr_task_grabing, 0, 1, WAIT_TIME);
|
||||
|
||||
SplitLogTask unassignedManager =
|
||||
new SplitLogTask.Unassigned(MANAGER, this.mode);
|
||||
new SplitLogTask.Unassigned(MANAGER);
|
||||
zkw.getRecoverableZooKeeper().create(PATH1, unassignedManager.toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
|
||||
|
@ -376,7 +371,7 @@ public class TestSplitLogWorker {
|
|||
|
||||
// preempt the first task, have it owned by another worker
|
||||
final ServerName anotherWorker = ServerName.valueOf("another-worker,1,1");
|
||||
SplitLogTask slt = new SplitLogTask.Owned(anotherWorker, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Owned(anotherWorker);
|
||||
ZKUtil.setData(zkw, PATH1, slt.toByteArray());
|
||||
waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME);
|
||||
|
||||
|
@ -402,7 +397,7 @@ public class TestSplitLogWorker {
|
|||
Thread.sleep(100);
|
||||
|
||||
String task = ZKSplitLog.getEncodedNodeName(zkw, "task");
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(MANAGER, this.mode);
|
||||
SplitLogTask slt = new SplitLogTask.Unassigned(MANAGER);
|
||||
zkw.getRecoverableZooKeeper().create(task,slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
|
||||
CreateMode.PERSISTENT);
|
||||
|
||||
|
@ -454,7 +449,7 @@ public class TestSplitLogWorker {
|
|||
RegionServerServices mockedRS = getRegionServer(RS);
|
||||
for (int i = 0; i < maxTasks; i++) {
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS + i),
|
||||
new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"), this.mode).toByteArray(),
|
||||
new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1")).toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
}
|
||||
|
||||
|
@ -497,7 +492,7 @@ public class TestSplitLogWorker {
|
|||
|
||||
for (int i = 0; i < maxTasks; i++) {
|
||||
zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS + i),
|
||||
new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"), this.mode).toByteArray(),
|
||||
new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1")).toByteArray(),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
}
|
||||
|
||||
|
@ -521,8 +516,6 @@ public class TestSplitLogWorker {
|
|||
|
||||
/**
|
||||
* Create a mocked region server service instance
|
||||
* @param server
|
||||
* @return
|
||||
*/
|
||||
private RegionServerServices getRegionServer(ServerName name) {
|
||||
|
||||
|
|
|
@ -79,7 +79,6 @@ public class TestSplitWalDataLoss {
|
|||
@Before
|
||||
public void setUp() throws Exception {
|
||||
testUtil.getConfiguration().setInt("hbase.regionserver.msginterval", 30000);
|
||||
testUtil.getConfiguration().setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
testUtil.startMiniCluster(2);
|
||||
Admin admin = testUtil.getAdmin();
|
||||
admin.createNamespace(namespace);
|
||||
|
|
|
@ -112,8 +112,6 @@ import org.mockito.Mockito;
|
|||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
|
||||
/**
|
||||
* Test replay of edits out of a WAL split.
|
||||
*/
|
||||
|
@ -127,7 +125,6 @@ public abstract class AbstractTestWALReplay {
|
|||
private Path logDir;
|
||||
private FileSystem fs;
|
||||
private Configuration conf;
|
||||
private RecoveryMode mode;
|
||||
private WALFactory wals;
|
||||
|
||||
@Rule
|
||||
|
@ -165,9 +162,6 @@ public abstract class AbstractTestWALReplay {
|
|||
if (TEST_UTIL.getDFSCluster().getFileSystem().exists(this.hbaseRootDir)) {
|
||||
TEST_UTIL.getDFSCluster().getFileSystem().delete(this.hbaseRootDir, true);
|
||||
}
|
||||
this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
|
||||
HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
this.wals = new WALFactory(conf, null, currentTest.getMethodName());
|
||||
}
|
||||
|
||||
|
@ -908,7 +902,7 @@ public abstract class AbstractTestWALReplay {
|
|||
assertNotNull(listStatus);
|
||||
assertTrue(listStatus.length > 0);
|
||||
WALSplitter.splitLogFile(hbaseRootDir, listStatus[0],
|
||||
this.fs, this.conf, null, null, null, null, mode, wals);
|
||||
this.fs, this.conf, null, null, null, wals);
|
||||
FileStatus[] listStatus1 = this.fs.listStatus(
|
||||
new Path(FSUtils.getTableDir(hbaseRootDir, tableName), new Path(hri.getEncodedName(),
|
||||
"recovered.edits")), new PathFilter() {
|
||||
|
@ -1062,10 +1056,8 @@ public abstract class AbstractTestWALReplay {
|
|||
first = fs.getFileStatus(smallFile);
|
||||
second = fs.getFileStatus(largeFile);
|
||||
}
|
||||
WALSplitter.splitLogFile(hbaseRootDir, first, fs, conf, null, null, null, null,
|
||||
RecoveryMode.LOG_SPLITTING, wals);
|
||||
WALSplitter.splitLogFile(hbaseRootDir, second, fs, conf, null, null, null, null,
|
||||
RecoveryMode.LOG_SPLITTING, wals);
|
||||
WALSplitter.splitLogFile(hbaseRootDir, first, fs, conf, null, null, null, wals);
|
||||
WALSplitter.splitLogFile(hbaseRootDir, second, fs, conf, null, null, null, wals);
|
||||
WAL wal = createWAL(this.conf, hbaseRootDir, logName);
|
||||
region = HRegion.openHRegion(conf, this.fs, hbaseRootDir, hri, htd, wal);
|
||||
assertTrue(region.getOpenSeqNum() > mvcc.getWritePoint());
|
||||
|
|
|
@ -276,56 +276,4 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
|
|||
closeRegion(HTU, rs0, hriSecondary);
|
||||
connection.close();
|
||||
}
|
||||
|
||||
@Test (timeout = 240000)
|
||||
public void testReplayedEditsAreSkipped() throws Exception {
|
||||
openRegion(HTU, rs0, hriSecondary);
|
||||
ClusterConnection connection =
|
||||
(ClusterConnection) ConnectionFactory.createConnection(HTU.getConfiguration());
|
||||
RegionReplicaReplicationEndpoint replicator = new RegionReplicaReplicationEndpoint();
|
||||
|
||||
ReplicationEndpoint.Context context = mock(ReplicationEndpoint.Context.class);
|
||||
when(context.getConfiguration()).thenReturn(HTU.getConfiguration());
|
||||
when(context.getMetrics()).thenReturn(mock(MetricsSource.class));
|
||||
|
||||
ReplicationPeer mockPeer = mock(ReplicationPeer.class);
|
||||
when(mockPeer.getNamespaces()).thenReturn(null);
|
||||
when(mockPeer.getTableCFs()).thenReturn(null);
|
||||
when(mockPeer.getPeerConfig()).thenReturn(new ReplicationPeerConfig());
|
||||
when(context.getReplicationPeer()).thenReturn(mockPeer);
|
||||
|
||||
replicator.init(context);
|
||||
replicator.startAsync();
|
||||
|
||||
// test the filter for the RE, not actual replication
|
||||
WALEntryFilter filter = replicator.getWALEntryfilter();
|
||||
|
||||
//load some data to primary
|
||||
HTU.loadNumericRows(table, f, 0, 1000);
|
||||
|
||||
Assert.assertEquals(1000, entries.size());
|
||||
for (Entry e: entries) {
|
||||
Cell _c = e.getEdit().getCells().get(0);
|
||||
if (Integer.parseInt(
|
||||
Bytes.toString(_c.getValueArray(), _c.getValueOffset(), _c.getValueLength())) % 2 == 0) {
|
||||
e.getKey().setOrigLogSeqNum(1); // simulate dist log replay by setting orig seq id
|
||||
}
|
||||
}
|
||||
|
||||
long skipped = 0, replayed = 0;
|
||||
for (Entry e : entries) {
|
||||
if (filter.filter(e) == null) {
|
||||
skipped++;
|
||||
} else {
|
||||
replayed++;
|
||||
}
|
||||
}
|
||||
|
||||
assertEquals(500, skipped);
|
||||
assertEquals(500, replayed);
|
||||
|
||||
HTU.deleteNumericRows(table, f, 0, 1000);
|
||||
closeRegion(HTU, rs0, hriSecondary);
|
||||
connection.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -77,7 +77,6 @@ public class TestVisibilityLabelReplicationWithExpAsString extends TestVisibilit
|
|||
+ "\\\"" + "\u0027&\\\\" + "\")";
|
||||
// setup configuration
|
||||
conf = HBaseConfiguration.create();
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
conf.setInt("hfile.format.version", 3);
|
||||
conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
|
||||
conf.setInt("replication.source.size.capacity", 10240);
|
||||
|
|
|
@ -64,7 +64,6 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
|
||||
|
@ -101,7 +100,7 @@ public abstract class TestVisibilityLabels {
|
|||
public static Configuration conf;
|
||||
|
||||
private volatile boolean killedRS = false;
|
||||
@Rule
|
||||
@Rule
|
||||
public final TestName TEST_NAME = new TestName();
|
||||
public static User SUPERUSER, USER1;
|
||||
|
||||
|
@ -146,7 +145,7 @@ public abstract class TestVisibilityLabels {
|
|||
current.getRowLength(), row2, 0, row2.length));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testSimpleVisibilityLabelsWithUniCodeCharacters() throws Exception {
|
||||
TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
|
||||
|
@ -398,13 +397,6 @@ public abstract class TestVisibilityLabels {
|
|||
} catch (InterruptedException e) {
|
||||
}
|
||||
}
|
||||
Region labelsTableRegion = regionServer.getRegions(LABELS_TABLE_NAME).get(0);
|
||||
while (labelsTableRegion.isRecovering()) {
|
||||
try {
|
||||
Thread.sleep(10);
|
||||
} catch (InterruptedException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -583,7 +575,7 @@ public abstract class TestVisibilityLabels {
|
|||
put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, value);
|
||||
put.setCellVisibility(new CellVisibility(SECRET));
|
||||
table.checkAndPut(row2, fam, qual, null, put);
|
||||
|
||||
|
||||
Scan scan = new Scan();
|
||||
scan.setAuthorizations(new Authorizations(SECRET));
|
||||
ResultScanner scanner = table.getScanner(scan);
|
||||
|
|
|
@ -129,7 +129,6 @@ public class TestVisibilityLabelsReplication {
|
|||
public void setup() throws Exception {
|
||||
// setup configuration
|
||||
conf = HBaseConfiguration.create();
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
conf.setInt("hfile.format.version", 3);
|
||||
conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/1");
|
||||
conf.setInt("replication.source.size.capacity", 10240);
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.NavigableMap;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
@ -42,7 +41,6 @@ public class TestVisibilityLabelsWithCustomVisLabService extends TestVisibilityL
|
|||
public static void setupBeforeClass() throws Exception {
|
||||
// setup configuration
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
VisibilityTestUtil.enableVisiblityLabels(conf);
|
||||
conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
|
||||
ScanLabelGenerator.class);
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
|
@ -66,7 +65,6 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili
|
|||
public static void setupBeforeClass() throws Exception {
|
||||
// setup configuration
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
VisibilityTestUtil.enableVisiblityLabels(conf);
|
||||
conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
|
||||
ScanLabelGenerator.class);
|
||||
|
|
|
@ -98,7 +98,6 @@ public class TestVisibilityLabelsWithDeletes {
|
|||
public static void setupBeforeClass() throws Exception {
|
||||
// setup configuration
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
VisibilityTestUtil.enableVisiblityLabels(conf);
|
||||
conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
|
||||
ScanLabelGenerator.class);
|
||||
|
@ -3287,7 +3286,7 @@ public class TestVisibilityLabelsWithDeletes {
|
|||
};
|
||||
SUPERUSER.runAs(action);
|
||||
}
|
||||
|
||||
|
||||
@SafeVarargs
|
||||
public static <T> List<T> createList(T... ts) {
|
||||
return new ArrayList<>(Arrays.asList(ts));
|
||||
|
|
|
@ -70,7 +70,6 @@ public class TestVisibilityWithCheckAuths {
|
|||
public static void setupBeforeClass() throws Exception {
|
||||
// setup configuration
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
|
||||
VisibilityTestUtil.enableVisiblityLabels(conf);
|
||||
conf.setBoolean(VisibilityConstants.CHECK_AUTHS_FOR_MUTATION, true);
|
||||
conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
|||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
|
||||
|
@ -182,11 +181,9 @@ public class TestWALReaderOnSecureWAL {
|
|||
}
|
||||
|
||||
FileStatus[] listStatus = fs.listStatus(walPath.getParent());
|
||||
RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
Path rootdir = FSUtils.getRootDir(conf);
|
||||
try {
|
||||
WALSplitter s = new WALSplitter(wals, conf, rootdir, fs, null, null, null, mode);
|
||||
WALSplitter s = new WALSplitter(wals, conf, rootdir, fs, null, null);
|
||||
s.splitLogFile(listStatus[0], null);
|
||||
Path file = new Path(ZKSplitLog.getSplitLogDir(rootdir, listStatus[0].getPath().getName()),
|
||||
"corrupt");
|
||||
|
@ -229,11 +226,9 @@ public class TestWALReaderOnSecureWAL {
|
|||
}
|
||||
|
||||
FileStatus[] listStatus = fs.listStatus(walPath.getParent());
|
||||
RecoveryMode mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
Path rootdir = FSUtils.getRootDir(conf);
|
||||
try {
|
||||
WALSplitter s = new WALSplitter(wals, conf, rootdir, fs, null, null, null, mode);
|
||||
WALSplitter s = new WALSplitter(wals, conf, rootdir, fs, null, null);
|
||||
s.splitLogFile(listStatus[0], null);
|
||||
Path file = new Path(ZKSplitLog.getSplitLogDir(rootdir, listStatus[0].getPath().getName()),
|
||||
"corrupt");
|
||||
|
|
|
@ -71,7 +71,6 @@ import org.apache.hadoop.hbase.security.User;
|
|||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -139,7 +138,6 @@ public class TestWALSplit {
|
|||
private static String ROBBER;
|
||||
private static String ZOMBIE;
|
||||
private static String [] GROUP = new String [] {"supergroup"};
|
||||
private RecoveryMode mode;
|
||||
|
||||
static enum Corruptions {
|
||||
INSERT_GARBAGE_ON_FIRST_LINE,
|
||||
|
@ -188,8 +186,6 @@ public class TestWALSplit {
|
|||
REGIONS.clear();
|
||||
Collections.addAll(REGIONS, "bbb", "ccc");
|
||||
InstrumentedLogWriter.activateFailure = false;
|
||||
this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
|
||||
RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
|
||||
wals = new WALFactory(conf, null, name.getMethodName());
|
||||
WALDIR = new Path(HBASELOGDIR,
|
||||
AbstractFSWALProvider.getWALDirectoryName(ServerName.valueOf(name.getMethodName(),
|
||||
|
@ -803,8 +799,7 @@ public class TestWALSplit {
|
|||
}
|
||||
assertTrue("There should be some log greater than size 0.", 0 < largestSize);
|
||||
// Set up a splitter that will throw an IOE on the output side
|
||||
WALSplitter logSplitter = new WALSplitter(wals,
|
||||
conf, HBASEDIR, fs, null, null, null, this.mode) {
|
||||
WALSplitter logSplitter = new WALSplitter(wals, conf, HBASEDIR, fs, null, null) {
|
||||
@Override
|
||||
protected Writer createWriter(Path logfile) throws IOException {
|
||||
Writer mockWriter = Mockito.mock(Writer.class);
|
||||
|
@ -931,7 +926,7 @@ public class TestWALSplit {
|
|||
try {
|
||||
conf.setInt("hbase.splitlog.report.period", 1000);
|
||||
boolean ret = WALSplitter.splitLogFile(
|
||||
HBASEDIR, logfile, spiedFs, conf, localReporter, null, null, null, this.mode, wals);
|
||||
HBASEDIR, logfile, spiedFs, conf, localReporter, null, null, wals);
|
||||
assertFalse("Log splitting should failed", ret);
|
||||
assertTrue(count.get() > 0);
|
||||
} catch (IOException e) {
|
||||
|
@ -989,8 +984,7 @@ public class TestWALSplit {
|
|||
makeRegionDirs(regions);
|
||||
|
||||
// Create a splitter that reads and writes the data without touching disk
|
||||
WALSplitter logSplitter = new WALSplitter(wals,
|
||||
localConf, HBASEDIR, fs, null, null, null, this.mode) {
|
||||
WALSplitter logSplitter = new WALSplitter(wals, localConf, HBASEDIR, fs, null, null) {
|
||||
|
||||
/* Produce a mock writer that doesn't write anywhere */
|
||||
@Override
|
||||
|
@ -1139,8 +1133,7 @@ public class TestWALSplit {
|
|||
assertTrue("There should be some log file",
|
||||
logfiles != null && logfiles.length > 0);
|
||||
|
||||
WALSplitter logSplitter = new WALSplitter(wals,
|
||||
conf, HBASEDIR, fs, null, null, null, this.mode) {
|
||||
WALSplitter logSplitter = new WALSplitter(wals, conf, HBASEDIR, fs, null, null) {
|
||||
@Override
|
||||
protected Writer createWriter(Path logfile)
|
||||
throws IOException {
|
||||
|
|
|
@ -1157,30 +1157,21 @@ Due to an asynchronous implementation, in very rare cases, the split log manager
|
|||
For that reason, it periodically checks for remaining uncompleted task in its task map or ZooKeeper.
|
||||
If none are found, it throws an exception so that the log splitting can be retried right away instead of hanging there waiting for something that won't happen.
|
||||
|
||||
[[wal.compression]]
|
||||
==== WAL Compression ====
|
||||
|
||||
[[distributed.log.replay]]
|
||||
====== Distributed Log Replay
|
||||
The content of the WAL can be compressed using LRU Dictionary compression.
|
||||
This can be used to speed up WAL replication to different datanodes.
|
||||
The dictionary can store up to 2^15^ elements; eviction starts after this number is exceeded.
|
||||
|
||||
After a RegionServer fails, its failed regions are assigned to another RegionServer, which are marked as "recovering" in ZooKeeper.
|
||||
A split log worker directly replays edits from the WAL of the failed RegionServer to the regions at its new location.
|
||||
When a region is in "recovering" state, it can accept writes but no reads (including Append and Increment), region splits or merges.
|
||||
To enable WAL compression, set the `hbase.regionserver.wal.enablecompression` property to `true`.
|
||||
The default value for this property is `false`.
|
||||
By default, WAL tag compression is turned on when WAL compression is enabled.
|
||||
You can turn off WAL tag compression by setting the `hbase.regionserver.wal.tags.enablecompression` property to 'false'.
|
||||
|
||||
Distributed Log Replay extends the <<distributed.log.splitting>> framework.
|
||||
It works by directly replaying WAL edits to another RegionServer instead of creating _recovered.edits_ files.
|
||||
It provides the following advantages over distributed log splitting alone:
|
||||
|
||||
* It eliminates the overhead of writing and reading a large number of _recovered.edits_ files.
|
||||
It is not unusual for thousands of _recovered.edits_ files to be created and written concurrently during a RegionServer recovery.
|
||||
Many small random writes can degrade overall system performance.
|
||||
* It allows writes even when a region is in recovering state.
|
||||
It only takes seconds for a recovering region to accept writes again.
|
||||
|
||||
.Enabling Distributed Log Replay
|
||||
To enable distributed log replay, set `hbase.master.distributed.log.replay` to `true`.
|
||||
This will be the default for HBase 0.99 (link:https://issues.apache.org/jira/browse/HBASE-10888[HBASE-10888]).
|
||||
|
||||
You must also enable HFile version 3 (which is the default HFile format starting in HBase 0.99.
|
||||
See link:https://issues.apache.org/jira/browse/HBASE-10855[HBASE-10855]). Distributed log replay is unsafe for rolling upgrades.
|
||||
A possible downside to WAL compression is that we lose more data from the last block in the WAL if it ill-terminated
|
||||
mid-write. If entries in this last block were added with new dictionary entries but we failed persist the amended
|
||||
dictionary because of an abrupt termination, a read of this last block may not be able to resolve last-written entries.
|
||||
|
||||
[[wal.disable]]
|
||||
==== Disabling the WAL
|
||||
|
|
|
@ -376,23 +376,6 @@ The WAL file writer implementation.
|
|||
.Default
|
||||
`org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter`
|
||||
|
||||
|
||||
[[hbase.master.distributed.log.replay]]
|
||||
*`hbase.master.distributed.log.replay`*::
|
||||
+
|
||||
.Description
|
||||
Enable 'distributed log replay' as default engine splitting
|
||||
WAL files on server crash. This default is new in hbase 1.0. To fall
|
||||
back to the old mode 'distributed log splitter', set the value to
|
||||
'false'. 'Disributed log replay' improves MTTR because it does not
|
||||
write intermediate files. 'DLR' required that 'hfile.format.version'
|
||||
be set to version 3 or higher.
|
||||
|
||||
+
|
||||
.Default
|
||||
`true`
|
||||
|
||||
|
||||
[[hbase.regionserver.global.memstore.size]]
|
||||
*`hbase.regionserver.global.memstore.size`*::
|
||||
+
|
||||
|
@ -1264,9 +1247,8 @@ A comma-separated list of sizes for buckets for the bucketcache
|
|||
+
|
||||
.Description
|
||||
The HFile format version to use for new files.
|
||||
Version 3 adds support for tags in hfiles (See http://hbase.apache.org/book.html#hbase.tags).
|
||||
Distributed Log Replay requires that tags are enabled. Also see the configuration
|
||||
'hbase.replication.rpc.codec'.
|
||||
Version 3 adds support for tags in hfiles (See https://hbase.apache.org/book.html#hbase.tags).
|
||||
Also see the configuration 'hbase.replication.rpc.codec'.
|
||||
|
||||
+
|
||||
.Default
|
||||
|
|
|
@ -75,7 +75,7 @@ In addition to the usual API versioning considerations HBase has other compatibi
|
|||
* APIs available in a patch version will be available in all later patch versions. However, new APIs may be added which will not be available in earlier patch versions.
|
||||
* New APIs introduced in a patch version will only be added in a source compatible way footnote:[See 'Source Compatibility' https://blogs.oracle.com/darcy/entry/kinds_of_compatibility]: i.e. code that implements public APIs will continue to compile.
|
||||
** Example: A user using a newly deprecated API does not need to modify application code with HBase API calls until the next major version.
|
||||
*
|
||||
*
|
||||
|
||||
.Client Binary compatibility
|
||||
* Client code written to APIs available in a given patch release can run unchanged (no recompilation needed) against the new jars of later patch versions.
|
||||
|
@ -355,10 +355,6 @@ You may have made use of this configuration if you are using BucketCache. If NOT
|
|||
.If you have your own customer filters.
|
||||
See the release notes on the issue link:https://issues.apache.org/jira/browse/HBASE-12068[HBASE-12068 [Branch-1\] Avoid need to always do KeyValueUtil#ensureKeyValue for Filter transformCell]; be sure to follow the recommendations therein.
|
||||
|
||||
[[dlr]]
|
||||
.Distributed Log Replay
|
||||
<<distributed.log.replay>> is off by default in HBase 1.0.0. Enabling it can make a big difference improving HBase MTTR. Enable this feature if you are doing a clean stop/start when you are upgrading. You cannot rolling upgrade to this feature (caveat if you are running on a version of HBase in excess of HBase 0.98.4 -- see link:https://issues.apache.org/jira/browse/HBASE-12577[HBASE-12577 Disable distributed log replay by default] for more).
|
||||
|
||||
.Mismatch Of `hbase.client.scanner.max.result.size` Between Client and Server
|
||||
If either the client or server version is lower than 0.98.11/1.0.0 and the server
|
||||
has a smaller value for `hbase.client.scanner.max.result.size` than the client, scan
|
||||
|
@ -380,9 +376,9 @@ There are no known issues running a <<hbase.rolling.upgrade,rolling upgrade>> fr
|
|||
In hbase-1.x, the default Scan caching 'number of rows' changed.
|
||||
Where in 0.98.x, it defaulted to 100, in later HBase versions, the
|
||||
default became Integer.MAX_VALUE. Not setting a cache size can make
|
||||
for Scans that run for a long time server-side, especially if
|
||||
for Scans that run for a long time server-side, especially if
|
||||
they are running with stringent filtering. See
|
||||
link:https://issues.apache.org/jira/browse/HBASE-16973[Revisiting default value for hbase.client.scanner.caching];
|
||||
link:https://issues.apache.org/jira/browse/HBASE-16973[Revisiting default value for hbase.client.scanner.caching];
|
||||
for further discussion.
|
||||
|
||||
[[upgrade1.0.from.0.94]]
|
||||
|
|
Loading…
Reference in New Issue