diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java index 71b3448240d..d057ce380fa 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java @@ -185,6 +185,9 @@ public class HRegionInfo implements Comparable { private byte[] tableName = null; private String tableNameAsString = null; + // when a region is in recovering state, it can only accept writes not reads + private volatile boolean recovering = false; + /** HRegionInfo for root region */ public static final HRegionInfo ROOT_REGIONINFO = new HRegionInfo(0L, Bytes.toBytes("-ROOT-")); @@ -293,6 +296,7 @@ public class HRegionInfo implements Comparable { this.startKey = startKey == null? HConstants.EMPTY_START_ROW: startKey.clone(); this.tableName = tableName.clone(); + this.recovering = false; setHashCode(); } @@ -313,6 +317,7 @@ public class HRegionInfo implements Comparable { this.hashCode = other.hashCode(); this.encodedName = other.getEncodedName(); this.tableName = other.tableName; + this.recovering = other.isRecovering(); } @@ -596,6 +601,20 @@ public class HRegionInfo implements Comparable { this.split = split; } + /** + * @return True if current region is in recovering + */ + public boolean isRecovering() { + return this.recovering; + } + + /** + * @param newState set recovering state + */ + public void setRecovering(boolean newState) { + this.recovering = newState; + } + /** * @return True if this region is offline. */ @@ -833,6 +852,7 @@ public class HRegionInfo implements Comparable { } builder.setOffline(info.isOffline()); builder.setSplit(info.isSplit()); + builder.setRecovering(info.isRecovering()); return builder.build(); } @@ -865,6 +885,9 @@ public class HRegionInfo implements Comparable { if (proto.hasOffline()) { hri.setOffline(proto.getOffline()); } + if (proto.hasRecovering()) { + hri.setRecovering(proto.getRecovering()); + } return hri; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java index d1822c8a3c2..fde258f0245 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java @@ -18,6 +18,14 @@ package org.apache.hadoop.hbase.client; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.UUID; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; @@ -31,14 +39,6 @@ import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; -import java.util.UUID; - @InterfaceAudience.Public @InterfaceStability.Evolving public abstract class Mutation extends OperationWithAttributes implements Row, CellScannable, @@ -63,6 +63,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C protected byte [] row = null; protected long ts = HConstants.LATEST_TIMESTAMP; protected Durability durability = Durability.USE_DEFAULT; + // A Map sorted by column family. protected NavigableMap> familyMap = new TreeMap>(Bytes.BYTES_COMPARATOR); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java index a04b1c21e27..fe2527219c6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerCallable.java @@ -178,7 +178,7 @@ public abstract class ServerCallable implements Callable { if (t instanceof SocketTimeoutException || t instanceof ConnectException || t instanceof RetriesExhaustedException || - getConnection().isDeadServer(location.getServerName())) { + (location != null && getConnection().isDeadServer(location.getServerName()))) { // if thrown these exceptions, we clear all the cache entries that // map to that slow/dead server; otherwise, let cache miss and ask // .META. again to find the new location @@ -261,7 +261,9 @@ public abstract class ServerCallable implements Callable { */ protected static Throwable translateException(Throwable t) throws DoNotRetryIOException { if (t instanceof UndeclaredThrowableException) { - t = t.getCause(); + if(t.getCause() != null) { + t = t.getCause(); + } } if (t instanceof RemoteException) { t = ((RemoteException)t).unwrapRemoteException(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java new file mode 100644 index 00000000000..66e0fa03d4d --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RegionInRecoveryException.java @@ -0,0 +1,45 @@ +/** + * + * 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.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Thrown when a read request issued against a region which is in recovering state. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +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); + } + +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java index ba74a6d360a..47faf9cee6b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java @@ -431,6 +431,30 @@ public class ReplicationZookeeper implements Closeable { return ProtobufUtil.prependPBMagic(bytes); } + /** + * @param position + * @return Serialized protobuf of position with pb magic prefix + * prepended suitable for use as content of an hlog position in a + * replication queue. + */ + public static byte[] positionToByteArray( + final long position) { + return ZKUtil.positionToByteArray(position); + } + + /** + * @param lockOwner + * @return Serialized protobuf of lockOwner with pb magic prefix + * prepended suitable for use as content of an replication lock during + * region server fail over. + */ + static byte[] lockToByteArray( + final String lockOwner) { + byte[] bytes = ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build() + .toByteArray(); + return ProtobufUtil.prependPBMagic(bytes); + } + /** * @param bytes Content of a peer znode. * @return ClusterKey parsed from the passed bytes. @@ -476,6 +500,42 @@ public class ReplicationZookeeper implements Closeable { } } + /** + * @param bytes - Content of a HLog position znode. + * @return long - The current HLog position. + * @throws DeserializationException + */ + public static long parseHLogPositionFrom( + final byte[] bytes) throws DeserializationException { + return ZKUtil.parseHLogPositionFrom(bytes); + } + + /** + * @param bytes - Content of a lock znode. + * @return String - The owner of the lock. + * @throws DeserializationException + */ + static String parseLockOwnerFrom( + final byte[] bytes) throws DeserializationException { + if (ProtobufUtil.isPBMagicPrefix(bytes)) { + int pblen = ProtobufUtil.lengthOfPBMagic(); + ZooKeeperProtos.ReplicationLock.Builder builder = ZooKeeperProtos.ReplicationLock + .newBuilder(); + ZooKeeperProtos.ReplicationLock lock; + try { + lock = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build(); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return lock.getLockOwner(); + } else { + if (bytes.length > 0) { + return Bytes.toString(bytes); + } + return ""; + } + } + private boolean peerExists(String id) throws KeeperException { return ZKUtil.checkExists(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)) >= 0; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java index 207cf18b39f..1fb8eba7f46 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKTable.java @@ -376,6 +376,7 @@ public class ZKTable { Set allTables = new HashSet(); List children = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode); + if(children == null) return allTables; for (String child: children) { ZooKeeperProtos.Table.State state = ZKTableReadOnly.getTableState(zkw, child); for (ZooKeeperProtos.Table.State expectedState: states) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 94ed2f56467..717ee4156ef 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -27,6 +27,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent; @@ -50,6 +52,8 @@ import org.apache.zookeeper.proto.DeleteRequest; import org.apache.zookeeper.proto.SetDataRequest; import org.apache.zookeeper.server.ZooKeeperSaslServer; +import com.google.protobuf.InvalidProtocolBufferException; + import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; import java.io.BufferedReader; @@ -1815,4 +1819,39 @@ public class ZKUtil { } } + /** + * @param position + * @return Serialized protobuf of position with pb magic prefix prepended suitable + * for use as content of an hlog position in a replication queue. + */ + public static byte[] positionToByteArray(final long position) { + byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position) + .build().toByteArray(); + return ProtobufUtil.prependPBMagic(bytes); + } + + /** + * @param bytes - Content of a HLog position znode. + * @return long - The current HLog position. + * @throws DeserializationException + */ + public static long parseHLogPositionFrom(final byte[] bytes) throws DeserializationException { + if (ProtobufUtil.isPBMagicPrefix(bytes)) { + int pblen = ProtobufUtil.lengthOfPBMagic(); + ZooKeeperProtos.ReplicationHLogPosition.Builder builder = + ZooKeeperProtos.ReplicationHLogPosition.newBuilder(); + ZooKeeperProtos.ReplicationHLogPosition position; + try { + position = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build(); + } catch (InvalidProtocolBufferException e) { + throw new DeserializationException(e); + } + return position.getPosition(); + } else { + if (bytes.length > 0) { + return Bytes.toLong(bytes); + } + return 0; + } + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index dd6bd4e5a98..ab387a6bddf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -107,6 +107,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { public String balancerZNode; // znode containing the lock for the tables public String tableLockZNode; + // znode containing the state of recovering regions + public String recoveringRegionsZNode; // Certain ZooKeeper nodes need to be world-readable public static final ArrayList CREATOR_ALL_AND_WORLD_READABLE = @@ -133,11 +135,11 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { /** * Instantiate a ZooKeeper connection and watcher. - * @param identifier string that is passed to RecoverableZookeeper to be used as - * identifier for this instance. Use null for default. * @param conf + * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for + * this instance. Use null for default. * @param abortable Can be null if there is on error there is no host to abort: e.g. client - * context. + * context. * @param canCreateBaseZNode * @throws IOException * @throws ZooKeeperConnectionException @@ -176,6 +178,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { ZKUtil.createAndFailSilent(this, splitLogZNode); ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode); ZKUtil.createAndFailSilent(this, tableLockZNode); + ZKUtil.createAndFailSilent(this, recoveringRegionsZNode); } catch (KeeperException e) { throw new ZooKeeperConnectionException( prefix("Unexpected KeeperException creating base node"), e); @@ -227,6 +230,8 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { conf.get("zookeeper.znode.balancer", "balancer")); tableLockZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.tableLock", "table-lock")); + recoveringRegionsZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.recovering.regions", "recovering-regions")); } /** diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index f2762d0995c..1c3214dc3bd 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -717,6 +717,14 @@ public final class HConstants { public static final String DISTRIBUTED_LOG_SPLITTING_KEY = "hbase.master.distributed.log.splitting"; + /** 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 = true; + + /** Conf key that specifies timeout value to wait for a region ready */ + public static final String LOG_REPLAY_WAIT_REGION_TIMEOUT = + "hbase.master.log.replay.wait.region.timeout"; + /** * The name of the configuration parameter that specifies * the number of bytes in a newly created checksum chunk. @@ -767,6 +775,7 @@ public final class HConstants { public static final int QOS_THRESHOLD = 10; public static final int HIGH_QOS = 100; public static final int REPLICATION_QOS = 5; // normal_QOS < replication_QOS < high_QOS + public static final int REPLAY_QOS = 6; // REPLICATION_QOS < REPLAY_QOS < high_QOS /** Directory under /hbase where archived hfiles are stored */ public static final String HFILE_ARCHIVE_DIRECTORY = ".archive"; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java index 7ad6e4f2de6..477ae425763 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSource.java @@ -60,6 +60,8 @@ public interface MetricsMasterSource extends BaseSource { static final String SNAPSHOT_TIME_NAME = "snapshotTime"; static final String SNAPSHOT_RESTORE_TIME_NAME = "snapshotRestoreTime"; static final String SNAPSHOT_CLONE_TIME_NAME = "snapshotCloneTime"; + static final String META_SPLIT_TIME_NAME = "metaHlogSplitTime"; + static final String META_SPLIT_SIZE_NAME = "metaHlogSplitSize"; static final String CLUSTER_REQUESTS_NAME = "clusterRequests"; static final String RIT_COUNT_NAME = "ritCount"; static final String RIT_COUNT_OVER_THRESHOLD_NAME = "ritCountOverThreshold"; @@ -78,7 +80,8 @@ public interface MetricsMasterSource extends BaseSource { static final String SNAPSHOT_TIME_DESC = "Time it takes to finish snapshot()"; static final String SNAPSHOT_RESTORE_TIME_DESC = "Time it takes to finish restoreSnapshot()"; static final String SNAPSHOT_CLONE_TIME_DESC = "Time it takes to finish cloneSnapshot()"; - + static final String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()"; + static final String META_SPLIT_SIZE_DESC = "Size of META HLog files being split"; /** * Increment the number of requests the cluster has seen. @@ -117,4 +120,9 @@ public interface MetricsMasterSource extends BaseSource { void updateSnapshotCloneTime(long time); void updateSnapshotRestoreTime(long time); + + void updateMetaWALSplitTime(long time); + + void updateMetaWALSplitSize(long size); + } diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java index 4a148491ddf..865a860333a 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java @@ -80,6 +80,13 @@ public interface MetricsRegionServerSource extends BaseSource { */ void updateAppend(long t); + /** + * Update the Replay time histogram. + * + * @param t time it took + */ + void updateReplay(long t); + /** * Increment the number of slow Puts that have happened. */ @@ -188,6 +195,7 @@ public interface MetricsRegionServerSource extends BaseSource { static final String INCREMENT_KEY = "increment"; static final String MUTATE_KEY = "mutate"; static final String APPEND_KEY = "append"; + static final String REPLAY_KEY = "replay"; static final String SCAN_NEXT_KEY = "scanNext"; static final String SLOW_MUTATE_KEY = "slowPutCount"; static final String SLOW_GET_KEY = "slowGetCount"; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java new file mode 100644 index 00000000000..e4236e07018 --- /dev/null +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySource.java @@ -0,0 +1,72 @@ +/** + * 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 + */ + static final String METRICS_NAME = "replay"; + + /** + * The name of the metrics context that metrics will be under. + */ + static final String METRICS_CONTEXT = "regionserver"; + + /** + * Description + */ + static final String METRICS_DESCRIPTION = "Metrics about HBase RegionServer HLog Edits Replay"; + + /** + * The name of the metrics context that metrics will be under in jmx + */ + static final String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + + static final String REPLAY_TIME_NAME = "replayTime"; + static final String REPLAY_TIME_DESC = "Time an replay operation took."; + static final String REPLAY_BATCH_SIZE_NAME = "replayBatchSize"; + static final String REPLAY_BATCH_SIZE_DESC = "Number of changes in each replay batch."; + static final String REPLAY_DATA_SIZE_NAME = "replayDataSize"; + static final 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); + +} diff --git a/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java index 2d7c9fb6fd3..46503077149 100644 --- a/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java +++ b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java @@ -48,6 +48,8 @@ public class MetricsMasterSourceImpl private MetricMutableStat snapshotTimeHisto; private MetricMutableStat snapshotCloneTimeHisto; private MetricMutableStat snapshotRestoreTimeHisto; + private MetricMutableHistogram metaSplitTimeHisto; + private MetricMutableHistogram metaSplitSizeHisto; public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) { this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT, masterWrapper); @@ -77,6 +79,8 @@ public class MetricsMasterSourceImpl SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true); snapshotRestoreTimeHisto = metricsRegistry.newStat( SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true); + metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC); + metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC); } public void incRequests(final int inc) { @@ -120,6 +124,16 @@ public class MetricsMasterSourceImpl snapshotRestoreTimeHisto.add(time); } + @Override + public void updateMetaWALSplitTime(long time) { + metaSplitTimeHisto.add(time); + } + + @Override + public void updateMetaWALSplitSize(long size) { + metaSplitSizeHisto.add(size); + } + /** * Method to export all the metrics. * diff --git a/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java index 8b726906db5..220a1421e2b 100644 --- a/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java +++ b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java @@ -38,6 +38,7 @@ public class MetricsRegionServerSourceImpl private final MetricHistogram getHisto; private final MetricHistogram incrementHisto; private final MetricHistogram appendHisto; + private final MetricHistogram replayHisto; private final MetricMutableCounterLong slowPut; private final MetricMutableCounterLong slowDelete; private final MetricMutableCounterLong slowGet; @@ -70,6 +71,8 @@ public class MetricsRegionServerSourceImpl appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY); slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l); + + replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY); } @Override @@ -97,6 +100,11 @@ public class MetricsRegionServerSourceImpl appendHisto.add(t); } + @Override + public void updateReplay(long t) { + replayHisto.add(t); + } + @Override public void incrSlowPut() { slowPut.incr(); diff --git a/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java new file mode 100644 index 00000000000..dfb94f2766e --- /dev/null +++ b/hbase-hadoop1-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java @@ -0,0 +1,75 @@ +/** + * 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.hadoop.hbase.metrics.BaseSourceImpl; +import org.apache.hadoop.metrics2.lib.MetricMutableHistogram; + +/** + * Hadoop1 implementation of MetricsMasterSource. + * + * Implements BaseSource through BaseSourceImpl, following the pattern + */ +public class MetricsEditsReplaySourceImpl + extends BaseSourceImpl implements MetricsEditsReplaySource { + + private static final Log LOG = LogFactory.getLog(MetricsEditsReplaySourceImpl.class.getName()); + + private MetricMutableHistogram replayTimeHisto; + private MetricMutableHistogram replayBatchSizeHisto; + private MetricMutableHistogram 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.newHistogram(REPLAY_TIME_NAME, REPLAY_TIME_DESC); + replayBatchSizeHisto = + metricsRegistry.newHistogram(REPLAY_BATCH_SIZE_NAME, REPLAY_BATCH_SIZE_DESC); + replayDataSizeHisto = + metricsRegistry.newHistogram(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); + } +} diff --git a/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySource b/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySource new file mode 100644 index 00000000000..ed95795f499 --- /dev/null +++ b/hbase-hadoop1-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySource @@ -0,0 +1 @@ +org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySourceImpl diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java index 6f9f1430361..7fced1660e0 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterSourceImpl.java @@ -45,6 +45,8 @@ public class MetricsMasterSourceImpl private MutableStat snapshotTimeHisto; private MutableStat snapshotCloneTimeHisto; private MutableStat snapshotRestoreTimeHisto; + private MutableHistogram metaSplitTimeHisto; + private MutableHistogram metaSplitSizeHisto; public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) { this(METRICS_NAME, @@ -79,6 +81,8 @@ public class MetricsMasterSourceImpl SNAPSHOT_CLONE_TIME_NAME, SNAPSHOT_CLONE_TIME_DESC, "Ops", "Time", true); snapshotRestoreTimeHisto = metricsRegistry.newStat( SNAPSHOT_RESTORE_TIME_NAME, SNAPSHOT_RESTORE_TIME_DESC, "Ops", "Time", true); + metaSplitTimeHisto = metricsRegistry.newHistogram(META_SPLIT_TIME_NAME, META_SPLIT_TIME_DESC); + metaSplitSizeHisto = metricsRegistry.newHistogram(META_SPLIT_SIZE_NAME, META_SPLIT_SIZE_DESC); } public void incRequests(final int inc) { @@ -122,6 +126,16 @@ public class MetricsMasterSourceImpl snapshotRestoreTimeHisto.add(time); } + @Override + public void updateMetaWALSplitTime(long time) { + metaSplitTimeHisto.add(time); + } + + @Override + public void updateMetaWALSplitSize(long size) { + metaSplitSizeHisto.add(size); + } + @Override public void getMetrics(MetricsCollector metricsCollector, boolean all) { diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java index 418e4a370b3..6729fa1bcce 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java @@ -41,6 +41,7 @@ public class MetricsRegionServerSourceImpl private final MetricHistogram getHisto; private final MetricHistogram incrementHisto; private final MetricHistogram appendHisto; + private final MetricHistogram replayHisto; private final MutableCounterLong slowPut; private final MutableCounterLong slowDelete; @@ -75,6 +76,8 @@ public class MetricsRegionServerSourceImpl appendHisto = getMetricsRegistry().newHistogram(APPEND_KEY); slowAppend = getMetricsRegistry().newCounter(SLOW_APPEND_KEY, SLOW_APPEND_DESC, 0l); + + replayHisto = getMetricsRegistry().newHistogram(REPLAY_KEY); } @Override @@ -102,6 +105,11 @@ public class MetricsRegionServerSourceImpl appendHisto.add(t); } + @Override + public void updateReplay(long t) { + replayHisto.add(t); + } + @Override public void incrSlowPut() { slowPut.incr(); diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java new file mode 100644 index 00000000000..6420f40f6c0 --- /dev/null +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsEditsReplaySourceImpl.java @@ -0,0 +1,74 @@ +/** + * 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.hadoop.hbase.metrics.BaseSourceImpl; +import org.apache.hadoop.metrics2.MetricHistogram; + +/** + * Hadoop1 implementation of MetricsMasterSource. Implements BaseSource through BaseSourceImpl, + * following the pattern + */ +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.newHistogram(REPLAY_TIME_NAME, REPLAY_TIME_DESC); + replayBatchSizeHisto = metricsRegistry.newHistogram(REPLAY_BATCH_SIZE_NAME, + REPLAY_BATCH_SIZE_DESC); + replayDataSizeHisto = metricsRegistry + .newHistogram(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); + } +} diff --git a/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySource b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySource new file mode 100644 index 00000000000..ed95795f499 --- /dev/null +++ b/hbase-hadoop2-compat/src/main/resources/META-INF/services/org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySource @@ -0,0 +1 @@ +org.apache.hadoop.hbase.regionserver.wal.MetricsEditsReplaySourceImpl diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java index cf054b06d05..da5d121b3fe 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AdminProtos.java @@ -14155,6 +14155,11 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request, com.google.protobuf.RpcCallback done); + public abstract void replay( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, + com.google.protobuf.RpcCallback done); + public abstract void rollWALWriter( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request, @@ -14255,6 +14260,14 @@ public final class AdminProtos { impl.replicateWALEntry(controller, request, done); } + @java.lang.Override + public void replay( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, + com.google.protobuf.RpcCallback done) { + impl.replay(controller, request, done); + } + @java.lang.Override public void rollWALWriter( com.google.protobuf.RpcController controller, @@ -14322,10 +14335,12 @@ public final class AdminProtos { case 9: return impl.replicateWALEntry(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest)request); case 10: - return impl.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request); + return impl.replay(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request); case 11: - return impl.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request); + return impl.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request); case 12: + return impl.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request); + case 13: return impl.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request); default: throw new java.lang.AssertionError("Can't get here."); @@ -14362,10 +14377,12 @@ public final class AdminProtos { case 9: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); case 10: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); case 12: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); + case 13: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -14402,10 +14419,12 @@ public final class AdminProtos { case 9: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); case 10: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); case 12: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); + case 13: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -14465,6 +14484,11 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request, com.google.protobuf.RpcCallback done); + public abstract void replay( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, + com.google.protobuf.RpcCallback done); + public abstract void rollWALWriter( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request, @@ -14553,16 +14577,21 @@ public final class AdminProtos { done)); return; case 10: + this.replay(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 11: this.rollWALWriter(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 11: + case 12: this.getServerInfo(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 12: + case 13: this.stopServer(controller, (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); @@ -14602,10 +14631,12 @@ public final class AdminProtos { case 9: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest.getDefaultInstance(); case 10: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest.getDefaultInstance(); case 12: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest.getDefaultInstance(); + case 13: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -14642,10 +14673,12 @@ public final class AdminProtos { case 9: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance(); case 10: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(); case 11: - return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(); case 12: + return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(); + case 13: return org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -14818,12 +14851,27 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryResponse.getDefaultInstance())); } + public void replay( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(10), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance())); + } + public void rollWALWriter( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(10), + getDescriptor().getMethods().get(11), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance(), @@ -14838,7 +14886,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(11), + getDescriptor().getMethods().get(12), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance(), @@ -14853,7 +14901,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(12), + getDescriptor().getMethods().get(13), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance(), @@ -14920,6 +14968,11 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ReplicateWALEntryRequest request) throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse replay( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request) + throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request) @@ -15063,12 +15116,24 @@ public final class AdminProtos { } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse replay( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(10), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance()); + } + + public org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse rollWALWriter( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(10), + getDescriptor().getMethods().get(11), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse.getDefaultInstance()); @@ -15080,7 +15145,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(11), + getDescriptor().getMethods().get(12), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse.getDefaultInstance()); @@ -15092,7 +15157,7 @@ public final class AdminProtos { org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(12), + getDescriptor().getMethods().get(13), controller, request, org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse.getDefaultInstance()); @@ -15255,78 +15320,80 @@ public final class AdminProtos { descriptor; static { java.lang.String[] descriptorData = { - "\n\013Admin.proto\032\013hbase.proto\032\tWAL.proto\"Q\n" + - "\024GetRegionInfoRequest\022 \n\006region\030\001 \002(\0132\020." + - "RegionSpecifier\022\027\n\017compactionState\030\002 \001(\010" + - "\"\301\001\n\025GetRegionInfoResponse\022\037\n\nregionInfo" + - "\030\001 \002(\0132\013.RegionInfo\022?\n\017compactionState\030\002" + - " \001(\0162&.GetRegionInfoResponse.CompactionS" + - "tate\"F\n\017CompactionState\022\010\n\004NONE\020\000\022\t\n\005MIN" + - "OR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_AND_MINOR\020\003\"G\n\023" + - "GetStoreFileRequest\022 \n\006region\030\001 \002(\0132\020.Re" + - "gionSpecifier\022\016\n\006family\030\002 \003(\014\")\n\024GetStor", - "eFileResponse\022\021\n\tstoreFile\030\001 \003(\t\"\030\n\026GetO" + - "nlineRegionRequest\":\n\027GetOnlineRegionRes" + - "ponse\022\037\n\nregionInfo\030\001 \003(\0132\013.RegionInfo\"\270" + - "\001\n\021OpenRegionRequest\0223\n\010openInfo\030\001 \003(\0132!" + - ".OpenRegionRequest.RegionOpenInfo\032n\n\016Reg" + - "ionOpenInfo\022\033\n\006region\030\001 \002(\0132\013.RegionInfo" + - "\022\034\n\024versionOfOfflineNode\030\002 \001(\r\022!\n\014favore" + - "dNodes\030\003 \003(\0132\013.ServerName\"\234\001\n\022OpenRegion" + - "Response\022<\n\014openingState\030\001 \003(\0162&.OpenReg" + - "ionResponse.RegionOpeningState\"H\n\022Region", - "OpeningState\022\n\n\006OPENED\020\000\022\022\n\016ALREADY_OPEN" + - "ED\020\001\022\022\n\016FAILED_OPENING\020\002\"\232\001\n\022CloseRegion" + - "Request\022 \n\006region\030\001 \002(\0132\020.RegionSpecifie" + - "r\022\034\n\024versionOfClosingNode\030\002 \001(\r\022\034\n\016trans" + - "itionInZK\030\003 \001(\010:\004true\022&\n\021destinationServ" + - "er\030\004 \001(\0132\013.ServerName\"%\n\023CloseRegionResp" + - "onse\022\016\n\006closed\030\001 \002(\010\"M\n\022FlushRegionReque" + - "st\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022\025\n\r" + - "ifOlderThanTs\030\002 \001(\004\"=\n\023FlushRegionRespon" + - "se\022\025\n\rlastFlushTime\030\001 \002(\004\022\017\n\007flushed\030\002 \001", - "(\010\"J\n\022SplitRegionRequest\022 \n\006region\030\001 \002(\013" + - "2\020.RegionSpecifier\022\022\n\nsplitPoint\030\002 \001(\014\"\025" + - "\n\023SplitRegionResponse\"W\n\024CompactRegionRe" + - "quest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\022" + - "\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003 \001(\014\"\027\n\025Compac" + - "tRegionResponse\"t\n\023MergeRegionsRequest\022!" + - "\n\007regionA\030\001 \002(\0132\020.RegionSpecifier\022!\n\007reg" + - "ionB\030\002 \002(\0132\020.RegionSpecifier\022\027\n\010forcible" + - "\030\003 \001(\010:\005false\"\026\n\024MergeRegionsResponse\"7\n" + - "\010WALEntry\022\024\n\003key\030\001 \002(\0132\007.WALKey\022\025\n\rkeyVa", - "lueBytes\030\002 \003(\014\"4\n\030ReplicateWALEntryReque" + - "st\022\030\n\005entry\030\001 \003(\0132\t.WALEntry\"\033\n\031Replicat" + - "eWALEntryResponse\"\026\n\024RollWALWriterReques" + - "t\".\n\025RollWALWriterResponse\022\025\n\rregionToFl" + - "ush\030\001 \003(\014\"#\n\021StopServerRequest\022\016\n\006reason" + - "\030\001 \002(\t\"\024\n\022StopServerResponse\"\026\n\024GetServe" + - "rInfoRequest\"@\n\nServerInfo\022\037\n\nserverName" + - "\030\001 \002(\0132\013.ServerName\022\021\n\twebuiPort\030\002 \001(\r\"8" + - "\n\025GetServerInfoResponse\022\037\n\nserverInfo\030\001 " + - "\002(\0132\013.ServerInfo2\266\006\n\014AdminService\022>\n\rget", - "RegionInfo\022\025.GetRegionInfoRequest\032\026.GetR" + - "egionInfoResponse\022;\n\014getStoreFile\022\024.GetS" + - "toreFileRequest\032\025.GetStoreFileResponse\022D" + - "\n\017getOnlineRegion\022\027.GetOnlineRegionReque" + - "st\032\030.GetOnlineRegionResponse\0225\n\nopenRegi" + - "on\022\022.OpenRegionRequest\032\023.OpenRegionRespo" + - "nse\0228\n\013closeRegion\022\023.CloseRegionRequest\032" + - "\024.CloseRegionResponse\0228\n\013flushRegion\022\023.F" + - "lushRegionRequest\032\024.FlushRegionResponse\022" + - "8\n\013splitRegion\022\023.SplitRegionRequest\032\024.Sp", - "litRegionResponse\022>\n\rcompactRegion\022\025.Com" + - "pactRegionRequest\032\026.CompactRegionRespons" + - "e\022;\n\014mergeRegions\022\024.MergeRegionsRequest\032" + - "\025.MergeRegionsResponse\022J\n\021replicateWALEn" + - "try\022\031.ReplicateWALEntryRequest\032\032.Replica" + - "teWALEntryResponse\022>\n\rrollWALWriter\022\025.Ro" + - "llWALWriterRequest\032\026.RollWALWriterRespon" + - "se\022>\n\rgetServerInfo\022\025.GetServerInfoReque" + - "st\032\026.GetServerInfoResponse\0225\n\nstopServer" + - "\022\022.StopServerRequest\032\023.StopServerRespons", - "eBA\n*org.apache.hadoop.hbase.protobuf.ge" + - "neratedB\013AdminProtosH\001\210\001\001\240\001\001" + "\n\013Admin.proto\032\014Client.proto\032\013hbase.proto" + + "\032\tWAL.proto\"Q\n\024GetRegionInfoRequest\022 \n\006r" + + "egion\030\001 \002(\0132\020.RegionSpecifier\022\027\n\017compact" + + "ionState\030\002 \001(\010\"\301\001\n\025GetRegionInfoResponse" + + "\022\037\n\nregionInfo\030\001 \002(\0132\013.RegionInfo\022?\n\017com" + + "pactionState\030\002 \001(\0162&.GetRegionInfoRespon" + + "se.CompactionState\"F\n\017CompactionState\022\010\n" + + "\004NONE\020\000\022\t\n\005MINOR\020\001\022\t\n\005MAJOR\020\002\022\023\n\017MAJOR_A" + + "ND_MINOR\020\003\"G\n\023GetStoreFileRequest\022 \n\006reg" + + "ion\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006family\030\002 ", + "\003(\014\")\n\024GetStoreFileResponse\022\021\n\tstoreFile" + + "\030\001 \003(\t\"\030\n\026GetOnlineRegionRequest\":\n\027GetO" + + "nlineRegionResponse\022\037\n\nregionInfo\030\001 \003(\0132" + + "\013.RegionInfo\"\270\001\n\021OpenRegionRequest\0223\n\010op" + + "enInfo\030\001 \003(\0132!.OpenRegionRequest.RegionO" + + "penInfo\032n\n\016RegionOpenInfo\022\033\n\006region\030\001 \002(" + + "\0132\013.RegionInfo\022\034\n\024versionOfOfflineNode\030\002" + + " \001(\r\022!\n\014favoredNodes\030\003 \003(\0132\013.ServerName\"" + + "\234\001\n\022OpenRegionResponse\022<\n\014openingState\030\001" + + " \003(\0162&.OpenRegionResponse.RegionOpeningS", + "tate\"H\n\022RegionOpeningState\022\n\n\006OPENED\020\000\022\022" + + "\n\016ALREADY_OPENED\020\001\022\022\n\016FAILED_OPENING\020\002\"\232" + + "\001\n\022CloseRegionRequest\022 \n\006region\030\001 \002(\0132\020." + + "RegionSpecifier\022\034\n\024versionOfClosingNode\030" + + "\002 \001(\r\022\034\n\016transitionInZK\030\003 \001(\010:\004true\022&\n\021d" + + "estinationServer\030\004 \001(\0132\013.ServerName\"%\n\023C" + + "loseRegionResponse\022\016\n\006closed\030\001 \002(\010\"M\n\022Fl" + + "ushRegionRequest\022 \n\006region\030\001 \002(\0132\020.Regio" + + "nSpecifier\022\025\n\rifOlderThanTs\030\002 \001(\004\"=\n\023Flu" + + "shRegionResponse\022\025\n\rlastFlushTime\030\001 \002(\004\022", + "\017\n\007flushed\030\002 \001(\010\"J\n\022SplitRegionRequest\022 " + + "\n\006region\030\001 \002(\0132\020.RegionSpecifier\022\022\n\nspli" + + "tPoint\030\002 \001(\014\"\025\n\023SplitRegionResponse\"W\n\024C" + + "ompactRegionRequest\022 \n\006region\030\001 \002(\0132\020.Re" + + "gionSpecifier\022\r\n\005major\030\002 \001(\010\022\016\n\006family\030\003" + + " \001(\014\"\027\n\025CompactRegionResponse\"t\n\023MergeRe" + + "gionsRequest\022!\n\007regionA\030\001 \002(\0132\020.RegionSp" + + "ecifier\022!\n\007regionB\030\002 \002(\0132\020.RegionSpecifi" + + "er\022\027\n\010forcible\030\003 \001(\010:\005false\"\026\n\024MergeRegi" + + "onsResponse\"7\n\010WALEntry\022\024\n\003key\030\001 \002(\0132\007.W", + "ALKey\022\025\n\rkeyValueBytes\030\002 \003(\014\"4\n\030Replicat" + + "eWALEntryRequest\022\030\n\005entry\030\001 \003(\0132\t.WALEnt" + + "ry\"\033\n\031ReplicateWALEntryResponse\"\026\n\024RollW" + + "ALWriterRequest\".\n\025RollWALWriterResponse" + + "\022\025\n\rregionToFlush\030\001 \003(\014\"#\n\021StopServerReq" + + "uest\022\016\n\006reason\030\001 \002(\t\"\024\n\022StopServerRespon" + + "se\"\026\n\024GetServerInfoRequest\"@\n\nServerInfo" + + "\022\037\n\nserverName\030\001 \002(\0132\013.ServerName\022\021\n\tweb" + + "uiPort\030\002 \001(\r\"8\n\025GetServerInfoResponse\022\037\n" + + "\nserverInfo\030\001 \002(\0132\013.ServerInfo2\337\006\n\014Admin", + "Service\022>\n\rgetRegionInfo\022\025.GetRegionInfo" + + "Request\032\026.GetRegionInfoResponse\022;\n\014getSt" + + "oreFile\022\024.GetStoreFileRequest\032\025.GetStore" + + "FileResponse\022D\n\017getOnlineRegion\022\027.GetOnl" + + "ineRegionRequest\032\030.GetOnlineRegionRespon" + + "se\0225\n\nopenRegion\022\022.OpenRegionRequest\032\023.O" + + "penRegionResponse\0228\n\013closeRegion\022\023.Close" + + "RegionRequest\032\024.CloseRegionResponse\0228\n\013f" + + "lushRegion\022\023.FlushRegionRequest\032\024.FlushR" + + "egionResponse\0228\n\013splitRegion\022\023.SplitRegi", + "onRequest\032\024.SplitRegionResponse\022>\n\rcompa" + + "ctRegion\022\025.CompactRegionRequest\032\026.Compac" + + "tRegionResponse\022;\n\014mergeRegions\022\024.MergeR" + + "egionsRequest\032\025.MergeRegionsResponse\022J\n\021" + + "replicateWALEntry\022\031.ReplicateWALEntryReq" + + "uest\032\032.ReplicateWALEntryResponse\022\'\n\006repl" + + "ay\022\r.MultiRequest\032\016.MultiResponse\022>\n\rrol" + + "lWALWriter\022\025.RollWALWriterRequest\032\026.Roll" + + "WALWriterResponse\022>\n\rgetServerInfo\022\025.Get" + + "ServerInfoRequest\032\026.GetServerInfoRespons", + "e\0225\n\nstopServer\022\022.StopServerRequest\032\023.St" + + "opServerResponseBA\n*org.apache.hadoop.hb" + + "ase.protobuf.generatedB\013AdminProtosH\001\210\001\001" + + "\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -15571,6 +15638,7 @@ public final class AdminProtos { com.google.protobuf.Descriptors.FileDescriptor .internalBuildGeneratedFileFrom(descriptorData, new com.google.protobuf.Descriptors.FileDescriptor[] { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.getDescriptor(), org.apache.hadoop.hbase.protobuf.generated.WALProtos.getDescriptor(), }, assigner); diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java index ff24574b742..a566592f7dc 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/HBaseProtos.java @@ -3132,6 +3132,10 @@ public final class HBaseProtos { // optional bool split = 6; boolean hasSplit(); boolean getSplit(); + + // optional bool recovering = 7; + boolean hasRecovering(); + boolean getRecovering(); } public static final class RegionInfo extends com.google.protobuf.GeneratedMessage @@ -3222,6 +3226,16 @@ public final class HBaseProtos { return split_; } + // optional bool recovering = 7; + public static final int RECOVERING_FIELD_NUMBER = 7; + private boolean recovering_; + public boolean hasRecovering() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public boolean getRecovering() { + return recovering_; + } + private void initFields() { regionId_ = 0L; tableName_ = com.google.protobuf.ByteString.EMPTY; @@ -3229,6 +3243,7 @@ public final class HBaseProtos { endKey_ = com.google.protobuf.ByteString.EMPTY; offline_ = false; split_ = false; + recovering_ = false; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -3268,6 +3283,9 @@ public final class HBaseProtos { if (((bitField0_ & 0x00000020) == 0x00000020)) { output.writeBool(6, split_); } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeBool(7, recovering_); + } getUnknownFields().writeTo(output); } @@ -3301,6 +3319,10 @@ public final class HBaseProtos { size += com.google.protobuf.CodedOutputStream .computeBoolSize(6, split_); } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(7, recovering_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -3354,6 +3376,11 @@ public final class HBaseProtos { result = result && (getSplit() == other.getSplit()); } + result = result && (hasRecovering() == other.hasRecovering()); + if (hasRecovering()) { + result = result && (getRecovering() + == other.getRecovering()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -3387,6 +3414,10 @@ public final class HBaseProtos { hash = (37 * hash) + SPLIT_FIELD_NUMBER; hash = (53 * hash) + hashBoolean(getSplit()); } + if (hasRecovering()) { + hash = (37 * hash) + RECOVERING_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getRecovering()); + } hash = (29 * hash) + getUnknownFields().hashCode(); return hash; } @@ -3515,6 +3546,8 @@ public final class HBaseProtos { bitField0_ = (bitField0_ & ~0x00000010); split_ = false; bitField0_ = (bitField0_ & ~0x00000020); + recovering_ = false; + bitField0_ = (bitField0_ & ~0x00000040); return this; } @@ -3577,6 +3610,10 @@ public final class HBaseProtos { to_bitField0_ |= 0x00000020; } result.split_ = split_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.recovering_ = recovering_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -3611,6 +3648,9 @@ public final class HBaseProtos { if (other.hasSplit()) { setSplit(other.getSplit()); } + if (other.hasRecovering()) { + setRecovering(other.getRecovering()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -3680,6 +3720,11 @@ public final class HBaseProtos { split_ = input.readBool(); break; } + case 56: { + bitField0_ |= 0x00000040; + recovering_ = input.readBool(); + break; + } } } } @@ -3821,6 +3866,27 @@ public final class HBaseProtos { return this; } + // optional bool recovering = 7; + private boolean recovering_ ; + public boolean hasRecovering() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public boolean getRecovering() { + return recovering_; + } + public Builder setRecovering(boolean value) { + bitField0_ |= 0x00000040; + recovering_ = value; + onChanged(); + return this; + } + public Builder clearRecovering() { + bitField0_ = (bitField0_ & ~0x00000040); + recovering_ = false; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:RegionInfo) } @@ -14719,60 +14785,60 @@ public final class HBaseProtos { "hema\022&\n\rconfiguration\030\004 \003(\0132\017.NameString" + "Pair\"o\n\022ColumnFamilySchema\022\014\n\004name\030\001 \002(\014" + "\022#\n\nattributes\030\002 \003(\0132\017.BytesBytesPair\022&\n" + - "\rconfiguration\030\003 \003(\0132\017.NameStringPair\"s\n", - "\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableNa" + - "me\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 \001" + - "(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\"0\n\014Fa" + - "voredNodes\022 \n\013favoredNode\030\001 \003(\0132\013.Server" + - "Name\"\225\001\n\017RegionSpecifier\0222\n\004type\030\001 \002(\0162$" + - ".RegionSpecifier.RegionSpecifierType\022\r\n\005" + - "value\030\002 \002(\014\"?\n\023RegionSpecifierType\022\017\n\013RE" + - "GION_NAME\020\001\022\027\n\023ENCODED_REGION_NAME\020\002\"\260\003\n" + - "\nRegionLoad\022)\n\017regionSpecifier\030\001 \002(\0132\020.R" + - "egionSpecifier\022\016\n\006stores\030\002 \001(\r\022\022\n\nstoref", - "iles\030\003 \001(\r\022\037\n\027storeUncompressedSizeMB\030\004 " + - "\001(\r\022\027\n\017storefileSizeMB\030\005 \001(\r\022\026\n\016memstore" + - "SizeMB\030\006 \001(\r\022\034\n\024storefileIndexSizeMB\030\007 \001" + - "(\r\022\031\n\021readRequestsCount\030\010 \001(\004\022\032\n\022writeRe" + - "questsCount\030\t \001(\004\022\032\n\022totalCompactingKVs\030" + - "\n \001(\004\022\033\n\023currentCompactedKVs\030\013 \001(\004\022\027\n\017ro" + - "otIndexSizeKB\030\014 \001(\r\022\036\n\026totalStaticIndexS" + - "izeKB\030\r \001(\r\022\036\n\026totalStaticBloomSizeKB\030\016 " + - "\001(\r\022\032\n\022completeSequenceId\030\017 \001(\004\"\372\001\n\nServ" + - "erLoad\022\030\n\020numberOfRequests\030\001 \001(\r\022\035\n\025tota", - "lNumberOfRequests\030\002 \001(\r\022\022\n\nusedHeapMB\030\003 " + - "\001(\r\022\021\n\tmaxHeapMB\030\004 \001(\r\022 \n\013regionLoads\030\005 " + - "\003(\0132\013.RegionLoad\022\"\n\014coprocessors\030\006 \003(\0132\014" + - ".Coprocessor\022\027\n\017reportStartTime\030\007 \001(\004\022\025\n" + - "\rreportEndTime\030\010 \001(\004\022\026\n\016infoServerPort\030\t" + - " \001(\r\"%\n\tTimeRange\022\014\n\004from\030\001 \001(\004\022\n\n\002to\030\002 " + - "\001(\004\"0\n\006Filter\022\014\n\004name\030\001 \002(\t\022\030\n\020serialize" + - "dFilter\030\002 \001(\014\"x\n\010KeyValue\022\013\n\003row\030\001 \002(\014\022\016" + - "\n\006family\030\002 \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\021\n\ttim" + - "estamp\030\004 \001(\004\022\032\n\007keyType\030\005 \001(\0162\t.CellType", - "\022\r\n\005value\030\006 \001(\014\"?\n\nServerName\022\020\n\010hostNam" + - "e\030\001 \002(\t\022\014\n\004port\030\002 \001(\r\022\021\n\tstartCode\030\003 \001(\004" + - "\"\033\n\013Coprocessor\022\014\n\004name\030\001 \002(\t\"-\n\016NameStr" + - "ingPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\",\n\r" + - "NameBytesPair\022\014\n\004name\030\001 \002(\t\022\r\n\005value\030\002 \001" + - "(\014\"/\n\016BytesBytesPair\022\r\n\005first\030\001 \002(\014\022\016\n\006s" + - "econd\030\002 \002(\014\",\n\rNameInt64Pair\022\014\n\004name\030\001 \001" + - "(\t\022\r\n\005value\030\002 \001(\003\"\255\001\n\023SnapshotDescriptio" + - "n\022\014\n\004name\030\001 \002(\t\022\r\n\005table\030\002 \001(\t\022\027\n\014creati" + - "onTime\030\003 \001(\003:\0010\022.\n\004type\030\004 \001(\0162\031.Snapshot", - "Description.Type:\005FLUSH\022\017\n\007version\030\005 \001(\005" + - "\"\037\n\004Type\022\014\n\010DISABLED\020\000\022\t\n\005FLUSH\020\001\"\n\n\010Emp" + - "tyMsg\"\032\n\007LongMsg\022\017\n\007longMsg\030\001 \002(\003\"&\n\rBig" + - "DecimalMsg\022\025\n\rbigdecimalMsg\030\001 \002(\014\"1\n\004UUI" + - "D\022\024\n\014leastSigBits\030\001 \002(\004\022\023\n\013mostSigBits\030\002" + - " \002(\004*`\n\010CellType\022\013\n\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n" + - "\n\006DELETE\020\010\022\021\n\rDELETE_COLUMN\020\014\022\021\n\rDELETE_" + - "FAMILY\020\016\022\014\n\007MAXIMUM\020\377\001*r\n\013CompareType\022\010\n" + - "\004LESS\020\000\022\021\n\rLESS_OR_EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n" + - "\tNOT_EQUAL\020\003\022\024\n\020GREATER_OR_EQUAL\020\004\022\013\n\007GR", - "EATER\020\005\022\t\n\005NO_OP\020\006B>\n*org.apache.hadoop." + - "hbase.protobuf.generatedB\013HBaseProtosH\001\240" + - "\001\001" + "\rconfiguration\030\003 \003(\0132\017.NameStringPair\"\207\001", + "\n\nRegionInfo\022\020\n\010regionId\030\001 \002(\004\022\021\n\ttableN" + + "ame\030\002 \002(\014\022\020\n\010startKey\030\003 \001(\014\022\016\n\006endKey\030\004 " + + "\001(\014\022\017\n\007offline\030\005 \001(\010\022\r\n\005split\030\006 \001(\010\022\022\n\nr" + + "ecovering\030\007 \001(\010\"0\n\014FavoredNodes\022 \n\013favor" + + "edNode\030\001 \003(\0132\013.ServerName\"\225\001\n\017RegionSpec" + + "ifier\0222\n\004type\030\001 \002(\0162$.RegionSpecifier.Re" + + "gionSpecifierType\022\r\n\005value\030\002 \002(\014\"?\n\023Regi" + + "onSpecifierType\022\017\n\013REGION_NAME\020\001\022\027\n\023ENCO" + + "DED_REGION_NAME\020\002\"\260\003\n\nRegionLoad\022)\n\017regi" + + "onSpecifier\030\001 \002(\0132\020.RegionSpecifier\022\016\n\006s", + "tores\030\002 \001(\r\022\022\n\nstorefiles\030\003 \001(\r\022\037\n\027store" + + "UncompressedSizeMB\030\004 \001(\r\022\027\n\017storefileSiz" + + "eMB\030\005 \001(\r\022\026\n\016memstoreSizeMB\030\006 \001(\r\022\034\n\024sto" + + "refileIndexSizeMB\030\007 \001(\r\022\031\n\021readRequestsC" + + "ount\030\010 \001(\004\022\032\n\022writeRequestsCount\030\t \001(\004\022\032" + + "\n\022totalCompactingKVs\030\n \001(\004\022\033\n\023currentCom" + + "pactedKVs\030\013 \001(\004\022\027\n\017rootIndexSizeKB\030\014 \001(\r" + + "\022\036\n\026totalStaticIndexSizeKB\030\r \001(\r\022\036\n\026tota" + + "lStaticBloomSizeKB\030\016 \001(\r\022\032\n\022completeSequ" + + "enceId\030\017 \001(\004\"\372\001\n\nServerLoad\022\030\n\020numberOfR", + "equests\030\001 \001(\r\022\035\n\025totalNumberOfRequests\030\002" + + " \001(\r\022\022\n\nusedHeapMB\030\003 \001(\r\022\021\n\tmaxHeapMB\030\004 " + + "\001(\r\022 \n\013regionLoads\030\005 \003(\0132\013.RegionLoad\022\"\n" + + "\014coprocessors\030\006 \003(\0132\014.Coprocessor\022\027\n\017rep" + + "ortStartTime\030\007 \001(\004\022\025\n\rreportEndTime\030\010 \001(" + + "\004\022\026\n\016infoServerPort\030\t \001(\r\"%\n\tTimeRange\022\014" + + "\n\004from\030\001 \001(\004\022\n\n\002to\030\002 \001(\004\"0\n\006Filter\022\014\n\004na" + + "me\030\001 \002(\t\022\030\n\020serializedFilter\030\002 \001(\014\"x\n\010Ke" + + "yValue\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002 \002(\014\022\021\n\tq" + + "ualifier\030\003 \002(\014\022\021\n\ttimestamp\030\004 \001(\004\022\032\n\007key", + "Type\030\005 \001(\0162\t.CellType\022\r\n\005value\030\006 \001(\014\"?\n\n" + + "ServerName\022\020\n\010hostName\030\001 \002(\t\022\014\n\004port\030\002 \001" + + "(\r\022\021\n\tstartCode\030\003 \001(\004\"\033\n\013Coprocessor\022\014\n\004" + + "name\030\001 \002(\t\"-\n\016NameStringPair\022\014\n\004name\030\001 \002" + + "(\t\022\r\n\005value\030\002 \002(\t\",\n\rNameBytesPair\022\014\n\004na" + + "me\030\001 \002(\t\022\r\n\005value\030\002 \001(\014\"/\n\016BytesBytesPai" + + "r\022\r\n\005first\030\001 \002(\014\022\016\n\006second\030\002 \002(\014\",\n\rName" + + "Int64Pair\022\014\n\004name\030\001 \001(\t\022\r\n\005value\030\002 \001(\003\"\255" + + "\001\n\023SnapshotDescription\022\014\n\004name\030\001 \002(\t\022\r\n\005" + + "table\030\002 \001(\t\022\027\n\014creationTime\030\003 \001(\003:\0010\022.\n\004", + "type\030\004 \001(\0162\031.SnapshotDescription.Type:\005F" + + "LUSH\022\017\n\007version\030\005 \001(\005\"\037\n\004Type\022\014\n\010DISABLE" + + "D\020\000\022\t\n\005FLUSH\020\001\"\n\n\010EmptyMsg\"\032\n\007LongMsg\022\017\n" + + "\007longMsg\030\001 \002(\003\"&\n\rBigDecimalMsg\022\025\n\rbigde" + + "cimalMsg\030\001 \002(\014\"1\n\004UUID\022\024\n\014leastSigBits\030\001" + + " \002(\004\022\023\n\013mostSigBits\030\002 \002(\004*`\n\010CellType\022\013\n" + + "\007MINIMUM\020\000\022\007\n\003PUT\020\004\022\n\n\006DELETE\020\010\022\021\n\rDELET" + + "E_COLUMN\020\014\022\021\n\rDELETE_FAMILY\020\016\022\014\n\007MAXIMUM" + + "\020\377\001*r\n\013CompareType\022\010\n\004LESS\020\000\022\021\n\rLESS_OR_" + + "EQUAL\020\001\022\t\n\005EQUAL\020\002\022\r\n\tNOT_EQUAL\020\003\022\024\n\020GRE", + "ATER_OR_EQUAL\020\004\022\013\n\007GREATER\020\005\022\t\n\005NO_OP\020\006B" + + ">\n*org.apache.hadoop.hbase.protobuf.gene" + + "ratedB\013HBaseProtosH\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -14808,7 +14874,7 @@ public final class HBaseProtos { internal_static_RegionInfo_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionInfo_descriptor, - new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", }, + new java.lang.String[] { "RegionId", "TableName", "StartKey", "EndKey", "Offline", "Split", "Recovering", }, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.class, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionInfo.Builder.class); internal_static_FavoredNodes_descriptor = diff --git a/hbase-protocol/src/main/protobuf/Admin.proto b/hbase-protocol/src/main/protobuf/Admin.proto index 0d55edd08fb..733eadff571 100644 --- a/hbase-protocol/src/main/protobuf/Admin.proto +++ b/hbase-protocol/src/main/protobuf/Admin.proto @@ -24,6 +24,7 @@ option java_generic_services = true; option java_generate_equals_and_hash = true; option optimize_for = SPEED; +import "Client.proto"; import "hbase.proto"; import "WAL.proto"; @@ -233,6 +234,9 @@ service AdminService { rpc replicateWALEntry(ReplicateWALEntryRequest) returns(ReplicateWALEntryResponse); + + rpc replay(MultiRequest) + returns(MultiResponse); rpc rollWALWriter(RollWALWriterRequest) returns(RollWALWriterResponse); diff --git a/hbase-protocol/src/main/protobuf/hbase.proto b/hbase-protocol/src/main/protobuf/hbase.proto index 9ae3ad2f1e6..f91035db10c 100644 --- a/hbase-protocol/src/main/protobuf/hbase.proto +++ b/hbase-protocol/src/main/protobuf/hbase.proto @@ -81,6 +81,7 @@ message RegionInfo { optional bytes endKey = 4; optional bool offline = 5; optional bool split = 6; + optional bool recovering = 7; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java index 897d97056f1..b4f2f576da2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java @@ -422,7 +422,8 @@ public class AssignmentManager extends ZooKeeperListener { return; } - boolean failover = !serverManager.getDeadServers().isEmpty(); + boolean failover = (!serverManager.getDeadServers().isEmpty() || !serverManager + .getRequeuedDeadServers().isEmpty()); if (!failover) { // Run through all regions. If they are not assigned and not in RIT, then @@ -2728,18 +2729,38 @@ public class AssignmentManager extends ZooKeeperListener { */ public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri) throws IOException, InterruptedException { - if (!regionStates.isRegionInTransition(hri)) return; + waitOnRegionToClearRegionsInTransition(hri, -1L); + } + + /** + * Wait on region to clear regions-in-transition or time out + * @param hri + * @param timeOut Milliseconds to wait for current region to be out of transition state. + * @return True when a region clears regions-in-transition before timeout otherwise false + * @throws IOException + * @throws InterruptedException + */ + public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut) + throws IOException, InterruptedException { + if (!regionStates.isRegionInTransition(hri)) return true; RegionState rs = null; + long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTimeMillis() + + timeOut; // There is already a timeout monitor on regions in transition so I // should not have to have one here too? - while(!this.server.isStopped() && regionStates.isRegionInTransition(hri)) { - LOG.info("Waiting on " + rs + " to clear regions-in-transition"); + LOG.info("Waiting on " + rs + " to clear regions-in-transition"); + while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) { regionStates.waitForUpdate(100); + if (EnvironmentEdgeManager.currentTimeMillis() > end) { + LOG.info("Timed out on waiting for region:" + hri.getEncodedName() + " to be assigned."); + return false; + } } if (this.server.isStopped()) { - LOG.info("Giving up wait on regions in " + - "transition because stoppable.isStopped is set"); + LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set"); + return false; } + return true; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 565d9fb9000..c8c0b51593b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -27,6 +27,7 @@ import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -347,7 +348,15 @@ MasterServices, Server { /** The health check chore. */ private HealthCheckChore healthCheckChore; + + /** + * is in distributedLogReplay mode. When true, SplitLogWorker directly replays WAL edits to newly + * assigned region servers instead of creating recovered.edits files. + */ + private final boolean distributedLogReplay; + /** flag used in test cases in order to simulate RS failures during master initialization */ + private volatile boolean initializationBeforeMetaAssignment = false; /** * Initializes the HMaster. The steps are as follows: @@ -451,6 +460,9 @@ MasterServices, Server { clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass); Threads.setDaemonThreadRunning(clusterStatusPublisherChore.getThread()); } + + distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, + HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG); } /** @@ -769,18 +781,47 @@ MasterServices, Server { this.assignmentManager.startTimeOutMonitor(); } - // TODO: Should do this in background rather than block master startup - status.setStatus("Splitting logs after master startup"); - splitLogAfterStartup(this.fileSystemManager); + // get a list for previously failed RS which need log splitting work + // we recover .META. region servers inside master initialization and + // handle other failed servers in SSH in order to start up master node ASAP + Set previouslyFailedServers = this.fileSystemManager + .getFailedServersFromLogFolders(); + // remove stale recovering regions from previous run + this.fileSystemManager.removeStaleRecoveringRegionsFromZK(previouslyFailedServers); + + // log splitting for .META. server + ServerName oldMetaServerLocation = this.catalogTracker.getMetaLocation(); + if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) { + splitMetaLogBeforeAssignment(oldMetaServerLocation); + // Note: we can't remove oldMetaServerLocation from previousFailedServers list because it + // may also host user regions + } + + this.initializationBeforeMetaAssignment = true; // Make sure meta assigned before proceeding. - if (!assignMeta(status)) return; + status.setStatus("Assigning Meta Region"); + assignMeta(status); + + if (this.distributedLogReplay && oldMetaServerLocation != null + && previouslyFailedServers.contains(oldMetaServerLocation)) { + // replay WAL edits mode need new .META. RS is assigned firstly + status.setStatus("replaying log for Meta Region"); + this.fileSystemManager.splitMetaLog(oldMetaServerLocation); + } + enableServerShutdownHandler(); + status.setStatus("Submitting log splitting work for previously failed region servers"); + // Master has recovered META region server and we put + // other failed region servers in a queue to be handled later by SSH + for (ServerName tmpServer : previouslyFailedServers) { + this.serverManager.processDeadServer(tmpServer, true); + } + // Update meta with new PB serialization if required. i.e migrate all HRI to PB serialization // in meta. This must happen before we assign all user regions or else the assignment will // fail. - // TODO: Remove this after 0.96, when we do 0.98. org.apache.hadoop.hbase.catalog.MetaMigrationConvertingToPB .updateMetaIfNecessary(this); @@ -829,14 +870,6 @@ MasterServices, Server { Threads.setDaemonThreadRunning(catalogJanitorChore.getThread()); } - /** - * Override to change master's splitLogAfterStartup. Used testing - * @param mfs - */ - protected void splitLogAfterStartup(final MasterFileSystem mfs) { - mfs.splitLogAfterStartup(); - } - /** * Create a {@link ServerManager} instance. * @param master @@ -865,52 +898,66 @@ MasterServices, Server { } /** - * Check .META. are assigned. If not, - * assign them. + * Check .META. is assigned. If not, assign it. + * @param status MonitoredTask * @throws InterruptedException * @throws IOException * @throws KeeperException - * @return True if meta is healthy, assigned */ - boolean assignMeta(MonitoredTask status) - throws InterruptedException, IOException, KeeperException { + void assignMeta(MonitoredTask status) + throws InterruptedException, IOException, KeeperException { + // Work on meta region int assigned = 0; long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000); + boolean beingExpired = false; - // Work on .META. region. Is it in zk in transition? status.setStatus("Assigning META region"); - assignmentManager.getRegionStates().createRegionState( - HRegionInfo.FIRST_META_REGIONINFO); - boolean rit = this.assignmentManager. - processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO); - ServerName currentMetaServer = null; - boolean metaRegionLocation = catalogTracker.verifyMetaRegionLocation(timeout); + + assignmentManager.getRegionStates().createRegionState(HRegionInfo.FIRST_META_REGIONINFO); + boolean rit = this.assignmentManager + .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO); + boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout); if (!rit && !metaRegionLocation) { - currentMetaServer = this.catalogTracker.getMetaLocation(); - splitLogAndExpireIfOnline(currentMetaServer); - this.assignmentManager.assignMeta(); - enableSSHandWaitForMeta(); + ServerName currentMetaServer = this.catalogTracker.getMetaLocation(); + if (currentMetaServer != null) { + beingExpired = expireIfOnline(currentMetaServer); + } + if (beingExpired) { + splitMetaLogBeforeAssignment(currentMetaServer); + } + assignmentManager.assignMeta(); // Make sure a .META. location is set. - if (!isMetaLocation()) return false; - // This guarantees that the transition assigning .META. has completed - this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO); + enableSSHandWaitForMeta(); assigned++; + if (beingExpired && this.distributedLogReplay) { + // In Replay WAL Mode, we need the new .META. server online + this.fileSystemManager.splitMetaLog(currentMetaServer); + } } else if (rit && !metaRegionLocation) { // Make sure a .META. location is set. - if (!isMetaLocation()) return false; - // This guarantees that the transition assigning .META. has completed - this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO); + enableSSHandWaitForMeta(); assigned++; - } else if (metaRegionLocation) { - // Region already assigned. We didn't assign it. Add to in-memory state. + } else { + // Region already assigned. We didn't assign it. Add to in-memory state. this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO, this.catalogTracker.getMetaLocation()); } enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME)); - LOG.info(".META. assigned=" + assigned + ", rit=" + rit + - ", location=" + catalogTracker.getMetaLocation()); + LOG.info(".META. assigned=" + assigned + ", rit=" + rit + ", location=" + + catalogTracker.getMetaLocation()); status.setStatus("META assigned."); - return true; + } + + private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException { + if (this.distributedLogReplay) { + // In log replay mode, we mark META region as recovering in ZK + Set regions = new HashSet(); + regions.add(HRegionInfo.FIRST_META_REGIONINFO); + this.fileSystemManager.prepareMetaLogReplay(currentMetaServer, regions); + } else { + // In recovered.edits mode: create recovered edits file for .META. server + this.fileSystemManager.splitMetaLog(currentMetaServer); + } } private void enableSSHandWaitForMeta() throws IOException, InterruptedException { @@ -921,24 +968,6 @@ MasterServices, Server { this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO); } - /** - * @return True if there a meta available - * @throws InterruptedException - */ - private boolean isMetaLocation() throws InterruptedException { - // Cycle up here in master rather than down in catalogtracker so we can - // check the master stopped flag every so often. - while (!this.stopped) { - try { - if (this.catalogTracker.waitForMeta(100) != null) break; - } catch (NotAllMetaRegionsOnlineException e) { - // Ignore. I know .META. is not online yet. - } - } - // We got here because we came of above loop. - return !this.stopped; - } - private void enableCatalogTables(String catalogTableName) { if (!this.assignmentManager.getZKTable().isEnabledTable(catalogTableName)) { this.assignmentManager.setEnabledTable(catalogTableName); @@ -946,20 +975,19 @@ MasterServices, Server { } /** - * Split a server's log and expire it if we find it is one of the online - * servers. + * Expire a server if we find it is one of the online servers. * @param sn ServerName to check. + * @return true when server sn is being expired by the function. * @throws IOException */ - private void splitLogAndExpireIfOnline(final ServerName sn) + private boolean expireIfOnline(final ServerName sn) throws IOException { if (sn == null || !serverManager.isServerOnline(sn)) { - return; + return false; } - LOG.info("Forcing splitLog and expire of " + sn); - fileSystemManager.splitMetaLog(sn); - fileSystemManager.splitLog(sn); + LOG.info("Forcing expire of " + sn); serverManager.expireServer(sn); + return true; } @Override @@ -2235,6 +2263,14 @@ MasterServices, Server { return this.serverShutdownHandlerEnabled; } + /** + * Report whether this master has started initialization and is about to do meta region assignment + * @return true if master is in initialization & about to assign META regions + */ + public boolean isInitializationStartsMetaRegionAssignment() { + return this.initializationBeforeMetaAssignment; + } + @Override public AssignRegionResponse assignRegion(RpcController controller, AssignRegionRequest req) throws ServiceException { @@ -2678,4 +2714,5 @@ MasterServices, Server { String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC); return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation); } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java index 5e50046db2b..74349f72853 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java @@ -20,7 +20,9 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.NavigableMap; import java.util.Set; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -34,26 +36,29 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.ClusterId; -import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.exceptions.InvalidFamilyOperationException; import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.backup.HFileArchiver; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.exceptions.InvalidFamilyOperationException; +import org.apache.hadoop.hbase.exceptions.OrphanHLogAfterSplitException; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; -import org.apache.hadoop.hbase.exceptions.OrphanHLogAfterSplitException; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.zookeeper.KeeperException; /** * This class abstracts a bunch of operations the HMaster needs to interact with @@ -83,6 +88,7 @@ public class MasterFileSystem { private final Path tempdir; // create the split log lock final Lock splitLogLock = new ReentrantLock(); + final boolean distributedLogReplay; final boolean distributedLogSplitting; final SplitLogManager splitLogManager; private final MasterServices services; @@ -118,15 +124,14 @@ public class MasterFileSystem { FSUtils.setFsDefault(conf, new Path(this.fs.getUri())); // make sure the fs has the same conf fs.setConf(conf); - this.distributedLogSplitting = - conf.getBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, true); + this.splitLogManager = new SplitLogManager(master.getZooKeeper(), master.getConfiguration(), + master, services, master.getServerName()); + this.distributedLogSplitting = conf.getBoolean(HConstants.DISTRIBUTED_LOG_SPLITTING_KEY, true); if (this.distributedLogSplitting) { - this.splitLogManager = new SplitLogManager(master.getZooKeeper(), - master.getConfiguration(), master, services, master.getServerName()); this.splitLogManager.finishInitialization(masterRecovery); - } else { - this.splitLogManager = null; } + this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, + HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG); // setup the filesystem variable // set up the archived logs path this.oldLogDir = createInitialFileSystemLayout(); @@ -212,21 +217,23 @@ public class MasterFileSystem { } /** - * Inspect the log directory to recover any log file without - * an active region server. + * Inspect the log directory to find dead servers which need recovery work + * @return A set of ServerNames which aren't running but still have WAL files left in file system */ - void splitLogAfterStartup() { + Set getFailedServersFromLogFolders() { boolean retrySplitting = !conf.getBoolean("hbase.hlog.split.skip.errors", - HLog.SPLIT_SKIP_ERRORS_DEFAULT); + HLog.SPLIT_SKIP_ERRORS_DEFAULT); + + Set serverNames = new HashSet(); Path logsDirPath = new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME); + do { if (master.isStopped()) { - LOG.warn("Master stopped while splitting logs"); + LOG.warn("Master stopped while trying to get failed servers."); break; } - List serverNames = new ArrayList(); try { - if (!this.fs.exists(logsDirPath)) return; + if (!this.fs.exists(logsDirPath)) return serverNames; FileStatus[] logFolders = FSUtils.listStatus(this.fs, logsDirPath, null); // Get online servers after getting log folders to avoid log folder deletion of newly // checked in region servers . see HBASE-5916 @@ -235,7 +242,7 @@ public class MasterFileSystem { if (logFolders == null || logFolders.length == 0) { LOG.debug("No log files to split, proceeding..."); - return; + return serverNames; } for (FileStatus status : logFolders) { String sn = status.getPath().getName(); @@ -249,23 +256,19 @@ public class MasterFileSystem { + "to a known region server, splitting"); serverNames.add(serverName); } else { - LOG.info("Log folder " + status.getPath() - + " belongs to an existing region server"); + LOG.info("Log folder " + status.getPath() + " belongs to an existing region server"); } } - splitLog(serverNames, META_FILTER); - splitLog(serverNames, NON_META_FILTER); retrySplitting = false; } catch (IOException ioe) { - LOG.warn("Failed splitting of " + serverNames, ioe); + LOG.warn("Failed getting failed servers to be recovered.", ioe); if (!checkFileSystem()) { LOG.warn("Bad Filesystem, exiting"); Runtime.getRuntime().halt(1); } try { if (retrySplitting) { - Thread.sleep(conf.getInt( - "hbase.hlog.split.failure.retry.interval", 30 * 1000)); + Thread.sleep(conf.getInt("hbase.hlog.split.failure.retry.interval", 30 * 1000)); } } catch (InterruptedException e) { LOG.warn("Interrupted, aborting since cannot return w/o splitting"); @@ -275,10 +278,12 @@ public class MasterFileSystem { } } } while (retrySplitting); + + return serverNames; } public void splitLog(final ServerName serverName) throws IOException { - List serverNames = new ArrayList(); + Set serverNames = new HashSet(); serverNames.add(serverName); splitLog(serverNames); } @@ -290,23 +295,20 @@ public class MasterFileSystem { */ public void splitMetaLog(final ServerName serverName) throws IOException { long splitTime = 0, splitLogSize = 0; - List serverNames = new ArrayList(); + Set serverNames = new HashSet(); serverNames.add(serverName); List logDirs = getLogDirs(serverNames); - if (logDirs.isEmpty()) { - LOG.info("No meta logs to split"); - return; - } + splitLogManager.handleDeadWorkers(serverNames); splitTime = EnvironmentEdgeManager.currentTimeMillis(); - splitLogSize = splitLogManager.splitLogDistributed(logDirs, META_FILTER); + splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, META_FILTER); splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime; if (this.metricsMaster != null) { - this.metricsMaster.addSplit(splitTime, splitLogSize); + this.metricsMaster.addMetaWALSplit(splitTime, splitLogSize); } } - private List getLogDirs(final List serverNames) throws IOException { + private List getLogDirs(final Set serverNames) throws IOException { List logDirs = new ArrayList(); for (ServerName serverName: serverNames) { Path logDir = new Path(this.rootdir, HLogUtil.getHLogDirectoryName(serverName.toString())); @@ -327,30 +329,79 @@ public class MasterFileSystem { return logDirs; } - public void splitLog(final List serverNames) throws IOException { + /** + * Mark regions in recovering state when distributedLogReplay are set true + * @param serverNames Set of ServerNames to be replayed wals in order to recover changes contained + * in them + * @throws IOException + */ + public void prepareLogReplay(Set serverNames) throws IOException { + if (!this.distributedLogReplay) { + return; + } + // mark regions in recovering state + for (ServerName serverName : serverNames) { + NavigableMap regions = this.getServerUserRegions(serverName); + if (regions == null) { + continue; + } + try { + this.splitLogManager.markRegionsRecoveringInZK(serverName, regions.keySet()); + } catch (KeeperException e) { + throw new IOException(e); + } + } + } + + /** + * Mark meta regions in recovering state when distributedLogReplay are set true. The function is used + * when {@link #getServerUserRegions(ServerName)} can't be used in case meta RS is down. + * @param serverName + * @param regions + * @throws IOException + */ + public void prepareMetaLogReplay(ServerName serverName, Set regions) + throws IOException { + if (!this.distributedLogReplay || (regions == null)) { + return; + } + // mark regions in recovering state + try { + this.splitLogManager.markRegionsRecoveringInZK(serverName, regions); + } catch (KeeperException e) { + throw new IOException(e); + } + } + + public void splitLog(final Set serverNames) throws IOException { splitLog(serverNames, NON_META_FILTER); } /** - * This method is the base split method that splits HLog files matching a filter. - * Callers should pass the appropriate filter for meta and non-meta HLogs. + * Wrapper function on {@link SplitLogManager#removeStaleRecoveringRegionsFromZK(Set)} + * @param failedServers + * @throws KeeperException + */ + void removeStaleRecoveringRegionsFromZK(final Set failedServers) + throws KeeperException { + this.splitLogManager.removeStaleRecoveringRegionsFromZK(failedServers); + } + + /** + * This method is the base split method that splits HLog files matching a filter. Callers should + * pass the appropriate filter for meta and non-meta HLogs. * @param serverNames * @param filter * @throws IOException */ - public void splitLog(final List serverNames, PathFilter filter) throws IOException { + public void splitLog(final Set serverNames, PathFilter filter) throws IOException { long splitTime = 0, splitLogSize = 0; List logDirs = getLogDirs(serverNames); - if (logDirs.isEmpty()) { - LOG.info("No logs to split"); - return; - } - if (distributedLogSplitting) { splitLogManager.handleDeadWorkers(serverNames); splitTime = EnvironmentEdgeManager.currentTimeMillis(); - splitLogSize = splitLogManager.splitLogDistributed(logDirs,filter); + splitLogSize = splitLogManager.splitLogDistributed(serverNames, logDirs, filter); splitTime = EnvironmentEdgeManager.currentTimeMillis() - splitTime; } else { for(Path logDir: logDirs){ @@ -358,8 +409,8 @@ public class MasterFileSystem { // one at a time this.splitLogLock.lock(); try { - HLogSplitter splitter = HLogSplitter.createLogSplitter( - conf, rootdir, logDir, oldLogDir, this.fs); + HLogSplitter splitter = HLogSplitter.createLogSplitter(conf, rootdir, logDir, oldLogDir, + this.fs); try { // If FS is in safe mode, just wait till out of it. FSUtils.waitOnSafeMode(conf, conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 1000)); @@ -380,7 +431,11 @@ public class MasterFileSystem { } if (this.metricsMaster != null) { - this.metricsMaster.addSplit(splitTime, splitLogSize); + if (filter == this.META_FILTER) { + this.metricsMaster.addMetaWALSplit(splitTime, splitLogSize); + } else { + this.metricsMaster.addSplit(splitTime, splitLogSize); + } } } @@ -648,4 +703,18 @@ public class MasterFileSystem { this.services.getTableDescriptors().add(htd); return htd; } + + private NavigableMap getServerUserRegions(ServerName serverName) + throws IOException { + if (!this.master.isStopped()) { + try { + this.master.getCatalogTracker().waitForMeta(); + return MetaReader.getServerUserRegions(this.master.getCatalogTracker(), serverName); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted", e); + } + } + return null; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 39acd807c94..7e3faf7cc19 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -181,4 +181,9 @@ public interface MasterServices extends Server { public void dispatchMergingRegions(final HRegionInfo region_a, final HRegionInfo region_b, final boolean forcible) throws IOException; + /** + * @return true if master is initialized + */ + public boolean isInitialized(); + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java index 2a42c44551c..8f7d7c50aaf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetricsMaster.java @@ -58,6 +58,16 @@ public class MetricsMaster { masterSource.updateSplitSize(size); } + /** + * Record a single instance of a split + * @param time time that the split took + * @param size length of original HLogs that were split + */ + public synchronized void addMetaWALSplit(long time, long size) { + masterSource.updateMetaWALSplitTime(time); + masterSource.updateMetaWALSplitSize(size); + } + /** * @param inc How much to add to requests. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index da54a4293ed..ee8f34b38f2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -154,21 +154,21 @@ public class ServerManager { private Set queuedDeadServers = new HashSet(); /** - * Set of region servers which are dead and submitted to ServerShutdownHandler to - * process but not fully processed immediately. + * Set of region servers which are dead and submitted to ServerShutdownHandler to process but not + * fully processed immediately. *

- * If one server died before assignment manager finished the failover cleanup, the server - * will be added to this set and will be processed through calling + * If one server died before assignment manager finished the failover cleanup, the server will be + * added to this set and will be processed through calling * {@link ServerManager#processQueuedDeadServers()} by assignment manager. *

- * For all the region servers in this set, HLog split is already completed. + * The Boolean value indicates whether log split is needed inside ServerShutdownHandler *

- * ServerShutdownHandler processes a dead server submitted to the handler after - * the handler is enabled. It may not be able to complete the processing because meta - * is not yet online or master is currently in startup mode. In this case, the dead - * server will be parked in this set temporarily. + * ServerShutdownHandler processes a dead server submitted to the handler after the handler is + * enabled. It may not be able to complete the processing because meta is not yet online or master + * is currently in startup mode. In this case, the dead server will be parked in this set + * temporarily. */ - private Set requeuedDeadServers = new HashSet(); + private Map requeuedDeadServers = new HashMap(); /** * Constructor. @@ -513,6 +513,10 @@ public class ServerManager { } public synchronized void processDeadServer(final ServerName serverName) { + this.processDeadServer(serverName, false); + } + + public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitHlog) { // When assignment manager is cleaning up the zookeeper nodes and rebuilding the // in-memory region states, region servers could be down. Meta table can and // should be re-assigned, log splitting can be done too. However, it is better to @@ -522,13 +526,14 @@ public class ServerManager { // the handler threads and meta table could not be re-assigned in case // the corresponding server is down. So we queue them up here instead. if (!services.getAssignmentManager().isFailoverCleanupDone()) { - requeuedDeadServers.add(serverName); + requeuedDeadServers.put(serverName, shouldSplitHlog); return; } this.deadservers.add(serverName); - this.services.getExecutorService().submit(new ServerShutdownHandler( - this.master, this.services, this.deadservers, serverName, false)); + this.services.getExecutorService().submit( + new ServerShutdownHandler(this.master, this.services, this.deadservers, serverName, + shouldSplitHlog)); } /** @@ -541,18 +546,20 @@ public class ServerManager { } Iterator serverIterator = queuedDeadServers.iterator(); while (serverIterator.hasNext()) { - expireServer(serverIterator.next()); + ServerName tmpServerName = serverIterator.next(); + expireServer(tmpServerName); serverIterator.remove(); + requeuedDeadServers.remove(tmpServerName); } if (!services.getAssignmentManager().isFailoverCleanupDone()) { LOG.info("AssignmentManager hasn't finished failover cleanup"); } - serverIterator = requeuedDeadServers.iterator(); - while (serverIterator.hasNext()) { - processDeadServer(serverIterator.next()); - serverIterator.remove(); + + for(ServerName tmpServerName : requeuedDeadServers.keySet()){ + processDeadServer(tmpServerName, requeuedDeadServers.get(tmpServerName)); } + requeuedDeadServers.clear(); } /* @@ -838,6 +845,14 @@ public class ServerManager { return new HashSet(this.queuedDeadServers); } + /** + * @return A copy of the internal map of requeuedDeadServers servers and their corresponding + * splitlog need flag. + */ + Map getRequeuedDeadServers() { + return Collections.unmodifiableMap(this.requeuedDeadServers); + } + public boolean isServerOnline(ServerName serverName) { return serverName != null && onlineServers.containsKey(serverName); } @@ -851,7 +866,7 @@ public class ServerManager { public synchronized boolean isServerDead(ServerName serverName) { return serverName == null || deadservers.isDeadServer(serverName) || queuedDeadServers.contains(serverName) - || requeuedDeadServers.contains(serverName); + || requeuedDeadServers.containsKey(serverName); } public void shutdownCluster() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java index 1cd3daf9d09..621e7ae7f97 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -43,16 +44,19 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Chore; -import org.apache.hadoop.hbase.exceptions.DeserializationException; +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; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.regionserver.SplitLogWorker; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; +import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; @@ -118,6 +122,20 @@ public class SplitLogManager extends ZooKeeperListener { private long unassignedTimeout; private long lastNodeCreateTime = Long.MAX_VALUE; public boolean ignoreZKDeleteForTesting = false; + private volatile long lastRecoveringNodeCreationTime = Long.MAX_VALUE; + // When lastRecoveringNodeCreationTime is older than the following threshold, we'll check + // whether to GC stale recovering znodes + private long checkRecoveringTimeThreshold = 15000; // 15 seconds + private final Set failedRecoveringRegionDeletions = Collections + .synchronizedSet(new HashSet()); + + /** + * 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(); + + final boolean distributedLogReplay; private final ConcurrentMap tasks = new ConcurrentHashMap(); private TimeoutMonitor timeoutMonitor; @@ -181,10 +199,13 @@ public class SplitLogManager extends ZooKeeperListener { LOG.info("timeout=" + timeout + ", unassigned timeout=" + unassignedTimeout); this.serverName = serverName; - this.timeoutMonitor = - new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper); + this.timeoutMonitor = new TimeoutMonitor( + conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper); this.failedDeletions = Collections.synchronizedSet(new HashSet()); + this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, + HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG); + LOG.info("distributedLogReplay = " + this.distributedLogReplay); } public void finishInitialization(boolean masterRecovery) { @@ -244,7 +265,22 @@ public class SplitLogManager extends ZooKeeperListener { * @return cumulative size of the logfiles split */ public long splitLogDistributed(final List logDirs) throws IOException { - return splitLogDistributed(logDirs, null); + if (logDirs.isEmpty()) { + return 0; + } + Set serverNames = new HashSet(); + for (Path logDir : logDirs) { + try { + ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logDir); + if (serverName != null) { + serverNames.add(serverName); + } + } catch (IllegalArgumentException e) { + // ignore invalid format error. + LOG.warn("Cannot parse server name from " + logDir); + } + } + return splitLogDistributed(serverNames, logDirs, null); } /** @@ -258,15 +294,15 @@ public class SplitLogManager extends ZooKeeperListener { * @throws IOException If there was an error while splitting any log file * @return cumulative size of the logfiles split */ - public long splitLogDistributed(final List logDirs, PathFilter filter) - throws IOException { + public long splitLogDistributed(final Set serverNames, final List logDirs, + PathFilter filter) throws IOException { MonitoredTask status = TaskMonitor.get().createStatus( "Doing distributed log split in " + logDirs); FileStatus[] logfiles = getFileList(logDirs, filter); status.setStatus("Checking directory contents..."); LOG.debug("Scheduling batch of logs to split"); SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet(); - LOG.info("started splitting logs in " + logDirs); + LOG.info("started splitting " + logfiles.length + " logs in " + logDirs); long t = EnvironmentEdgeManager.currentTimeMillis(); long totalSize = 0; TaskBatch batch = new TaskBatch(); @@ -283,6 +319,9 @@ public class SplitLogManager extends ZooKeeperListener { } } waitForSplittingCompletion(batch, status); + // remove recovering regions from ZK + this.removeRecoveringRegionsFromZK(serverNames); + if (batch.done != batch.installed) { batch.isDead = true; SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet(); @@ -409,6 +448,171 @@ public class SplitLogManager extends ZooKeeperListener { return count; } + /** + * 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 + */ + private void removeRecoveringRegionsFromZK(final Set serverNames) { + + if (!this.distributedLogReplay) { + // the function is only used in WALEdit direct replay mode + return; + } + + int count = 0; + Set recoveredServerNameSet = new HashSet(); + if (serverNames != null) { + for (ServerName tmpServerName : serverNames) { + recoveredServerNameSet.add(tmpServerName.getServerName()); + } + } + + try { + this.recoveringRegionLock.lock(); + + List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode); + if (tasks != null) { + for (String t : tasks) { + if (!ZKSplitLog.isRescanNode(watcher, t)) { + count++; + } + } + } + if (count == 0 && this.master.isInitialized() + && !this.master.getServerManager().areDeadServersInProgress()) { + // no splitting work items left + deleteRecoveringRegionZNodes(null); + // reset lastRecoveringNodeCreationTime because we cleared all recovering znodes at + // this point. + lastRecoveringNodeCreationTime = Long.MAX_VALUE; + } else if (!recoveredServerNameSet.isEmpty()) { + // remove recovering regions which doesn't have any RS associated with it + List regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode); + if (regions != null) { + for (String region : regions) { + String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region); + List failedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath); + if (failedServers == null || failedServers.isEmpty()) { + ZKUtil.deleteNode(watcher, nodePath); + continue; + } + if (recoveredServerNameSet.containsAll(failedServers)) { + ZKUtil.deleteNodeRecursively(watcher, nodePath); + } else { + for (String failedServer : failedServers) { + 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); + if (serverNames != null && !serverNames.isEmpty()) { + this.failedRecoveringRegionDeletions.addAll(serverNames); + } + } 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 KeeperException + */ + void removeStaleRecoveringRegionsFromZK(final Set failedServers) + throws KeeperException { + + if (!this.distributedLogReplay) { + // the function is only used in distributedLogReplay mode when master is in initialization + return; + } + + Set knownFailedServers = new HashSet(); + if (failedServers != null) { + for (ServerName tmpServerName : failedServers) { + knownFailedServers.add(tmpServerName.getServerName()); + } + } + + this.recoveringRegionLock.lock(); + try { + List tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode); + if (tasks != null) { + for (String t : tasks) { + byte[] data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t)); + 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 = HLogUtil.getServerNameFromHLogDirectoryName(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 regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode); + if (regions != null) { + for (String region : regions) { + String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region); + List regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath); + if (regionFailedServers == null || regionFailedServers.isEmpty()) { + ZKUtil.deleteNode(watcher, nodePath); + continue; + } + boolean needMoreRecovery = false; + for (String tmpFailedServer : regionFailedServers) { + if (knownFailedServers.contains(tmpFailedServer)) { + needMoreRecovery = true; + break; + } + } + if (!needMoreRecovery) { + ZKUtil.deleteNode(watcher, nodePath); + } + } + } + } finally { + this.recoveringRegionLock.unlock(); + } + } + + private void deleteRecoveringRegionZNodes(List regions) { + try { + if (regions == null) { + // remove all children under /home/recovering-regions + LOG.info("Garbage collecting all recovering regions."); + ZKUtil.deleteChildrenRecursively(watcher, watcher.recoveringRegionsZNode); + } else { + for (String curRegion : regions) { + String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, curRegion); + ZKUtil.deleteNodeRecursively(watcher, nodePath); + } + } + } catch (KeeperException e) { + LOG.warn("Cannot remove recovering regions from ZooKeeper", e); + } + } + private void setDone(String path, TerminationStatus status) { Task task = tasks.get(path); if (task == null) { @@ -859,9 +1063,131 @@ public class SplitLogManager extends ZooKeeperListener { } /** - * 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. + * 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 + */ + void markRegionsRecoveringInZK(final ServerName serverName, Set userRegions) + throws KeeperException { + if (userRegions == null || !this.distributedLogReplay) { + return; + } + + try { + this.recoveringRegionLock.lock(); + // mark that we're creating recovering znodes + this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTimeMillis(); + + for (HRegionInfo region : userRegions) { + String regionEncodeName = region.getEncodedName(); + long retries = this.zkretries; + + do { + String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regionEncodeName); + long lastRecordedFlushedSequenceId = -1; + try { + long lastSequenceId = this.master.getServerManager().getLastFlushedSequenceId( + regionEncodeName.getBytes()); + + /* + * 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 = SplitLogManager.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.positionToByteArray(lastSequenceId)); + LOG.debug("Mark region " + regionEncodeName + " recovering from failed region server " + + serverName); + + // break retry loop + break; + } catch (KeeperException e) { + // ignore ZooKeeper exceptions inside retry loop + if (retries <= 1) { + throw e; + } + // wait a little bit for retry + try { + Thread.sleep(20); + } catch (Exception ignoreE) { + // ignore + } + } + } while ((--retries) > 0 && (!this.stopper.isStopped())); + } + } finally { + this.recoveringRegionLock.unlock(); + } + } + + /** + * @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.parseHLogPositionFrom(bytes); + } catch (DeserializationException e) { + lastRecordedFlushedSequenceId = -1l; + LOG.warn("Can't parse last flushed sequence Id", e); + } + return lastRecordedFlushedSequenceId; + } + + /** + * 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 id recorded in ZK of the region for serverName + * @throws IOException + */ + public static long getLastFlushedSequenceId(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. + long lastFlushedSequenceId = -1; + String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName); + nodePath = ZKUtil.joinZNode(nodePath, serverName); + try { + byte[] data = ZKUtil.getData(zkw, nodePath); + if (data != null) { + lastFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data); + } + } catch (KeeperException e) { + throw new IOException("Cannot get lastFlushedSequenceId from ZooKeeper for server=" + + serverName + "; region=" + encodedRegionName, e); + } + return lastFlushedSequenceId; + } + + /** + * 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. *

* All access is synchronized. */ @@ -944,18 +1270,14 @@ public class SplitLogManager extends ZooKeeperListener { LOG.info("dead splitlog worker " + workerName); } - void handleDeadWorkers(List serverNames) { - List workerNames = new ArrayList(serverNames.size()); - for (ServerName serverName : serverNames) { - workerNames.add(serverName); - } + void handleDeadWorkers(Set serverNames) { synchronized (deadWorkersLock) { if (deadWorkers == null) { deadWorkers = new HashSet(100); } - deadWorkers.addAll(workerNames); + deadWorkers.addAll(serverNames); } - LOG.info("dead splitlog workers " + workerNames); + LOG.info("dead splitlog workers " + serverNames); } /** @@ -1052,6 +1374,20 @@ public class SplitLogManager extends ZooKeeperListener { } failedDeletions.removeAll(tmpPaths); } + + // Garbage collect left-over /hbase/recovering-regions/... znode + long timeInterval = EnvironmentEdgeManager.currentTimeMillis() + - lastRecoveringNodeCreationTime; + if (!failedRecoveringRegionDeletions.isEmpty() + || (tot == 0 && tasks.size() == 0 && (timeInterval > checkRecoveringTimeThreshold))) { + // inside the function there have more checks before GC anything + Set previouslyFailedDeletoins = null; + if (!failedRecoveringRegionDeletions.isEmpty()) { + previouslyFailedDeletoins = new HashSet(failedRecoveringRegionDeletions); + failedRecoveringRegionDeletions.removeAll(previouslyFailedDeletoins); + } + removeRecoveringRegionsFromZK(previouslyFailedDeletoins); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java index 29560473e5d..4b376f6c6ba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/MetaServerShutdownHandler.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.master.handler; import java.io.IOException; +import java.util.HashSet; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -27,6 +29,7 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.executor.EventType; +import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.DeadServer; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.zookeeper.KeeperException; @@ -47,30 +50,55 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler { @Override public void process() throws IOException { boolean gotException = true; - try{ + try { + AssignmentManager am = this.services.getAssignmentManager(); try { - LOG.info("Splitting META logs for " + serverName); if (this.shouldSplitHlog) { - this.services.getMasterFileSystem().splitMetaLog(serverName); - } + LOG.info("Splitting META logs for " + serverName); + if(this.distributedLogReplay) { + Set regions = new HashSet(); + regions.add(HRegionInfo.FIRST_META_REGIONINFO); + this.services.getMasterFileSystem().prepareMetaLogReplay(serverName, regions); + } else { + this.services.getMasterFileSystem().splitMetaLog(serverName); + } + } } catch (IOException ioe) { this.services.getExecutorService().submit(this); this.deadServers.add(serverName); - throw new IOException("failed log splitting for " + - serverName + ", will retry", ioe); + throw new IOException("failed log splitting for " + serverName + ", will retry", ioe); } // Assign meta if we were carrying it. // Check again: region may be assigned to other where because of RIT // timeout - if (this.services.getAssignmentManager().isCarryingMeta(serverName)) { + if (am.isCarryingMeta(serverName)) { LOG.info("Server " + serverName + " was carrying META. Trying to assign."); - this.services.getAssignmentManager().regionOffline(HRegionInfo.FIRST_META_REGIONINFO); + am.regionOffline(HRegionInfo.FIRST_META_REGIONINFO); verifyAndAssignMetaWithRetries(); } else { LOG.info("META has been assigned to otherwhere, skip assigning."); } - + + try { + if (this.shouldSplitHlog && this.distributedLogReplay) { + if (!am.waitOnRegionToClearRegionsInTransition(HRegionInfo.FIRST_META_REGIONINFO, + regionAssignmentWaitTimeout)) { + throw new IOException("Region " + HRegionInfo.FIRST_META_REGIONINFO.getEncodedName() + + " didn't complete assignment in time"); + } + this.services.getMasterFileSystem().splitMetaLog(serverName); + } + } catch (Exception ex) { + if (ex instanceof IOException) { + this.services.getExecutorService().submit(this); + this.deadServers.add(serverName); + throw new IOException("failed log splitting for " + serverName + ", will retry", ex); + } else { + throw new IOException(ex); + } + } + gotException = false; } finally { if (gotException){ @@ -78,9 +106,15 @@ public class MetaServerShutdownHandler extends ServerShutdownHandler { this.deadServers.finish(serverName); } } + super.process(); } + @Override + boolean isCarryingMeta() { + return true; + } + /** * Before assign the META region, ensure it haven't * been assigned by other place diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java index 2705d90b640..b3a26850e99 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java @@ -20,13 +20,16 @@ package org.apache.hadoop.hbase.master.handler; import java.io.IOException; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; @@ -56,6 +59,8 @@ public class ServerShutdownHandler extends EventHandler { protected final MasterServices services; protected final DeadServer deadServers; protected final boolean shouldSplitHlog; // whether to split HLog or not + protected final boolean distributedLogReplay; + protected final int regionAssignmentWaitTimeout; public ServerShutdownHandler(final Server server, final MasterServices services, final DeadServer deadServers, final ServerName serverName, @@ -76,6 +81,11 @@ public class ServerShutdownHandler extends EventHandler { LOG.warn(this.serverName + " is NOT in deadservers; it should be!"); } this.shouldSplitHlog = shouldSplitHlog; + this.distributedLogReplay = server.getConfiguration().getBoolean( + HConstants.DISTRIBUTED_LOG_REPLAY_KEY, + HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG); + this.regionAssignmentWaitTimeout = server.getConfiguration().getInt( + HConstants.LOG_REPLAY_WAIT_REGION_TIMEOUT, 15000); } @Override @@ -107,21 +117,7 @@ public class ServerShutdownHandler extends EventHandler { public void process() throws IOException { final ServerName serverName = this.serverName; try { - try { - if (this.shouldSplitHlog) { - LOG.info("Splitting logs for " + serverName); - this.services.getMasterFileSystem().splitLog(serverName); - } else { - LOG.info("Skipping log splitting for " + serverName); - } - } catch (IOException ioe) { - //typecast to SSH so that we make sure that it is the SSH instance that - //gets submitted as opposed to MSSH or some other derived instance of SSH - this.services.getExecutorService().submit((ServerShutdownHandler)this); - this.deadServers.add(serverName); - throw new IOException("failed log splitting for " + - serverName + ", will retry", ioe); - } + // We don't want worker thread in the MetaServerShutdownHandler // executor pool to block by waiting availability of .META. // Otherwise, it could run into the following issue: @@ -145,7 +141,7 @@ public class ServerShutdownHandler extends EventHandler { // the dead server for further processing too. if (isCarryingMeta() // .META. || !services.getAssignmentManager().isFailoverCleanupDone()) { - this.services.getServerManager().processDeadServer(serverName); + this.services.getServerManager().processDeadServer(serverName, this.shouldSplitHlog); return; } @@ -183,6 +179,23 @@ public class ServerShutdownHandler extends EventHandler { throw new IOException("Server is stopped"); } + try { + if (this.shouldSplitHlog) { + LOG.info("Splitting logs for " + serverName + " before assignment."); + if(this.distributedLogReplay){ + Set serverNames = new HashSet(); + serverNames.add(serverName); + this.services.getMasterFileSystem().prepareLogReplay(serverNames); + } else { + this.services.getMasterFileSystem().splitLog(serverName); + } + } else { + LOG.info("Skipping log splitting for " + serverName); + } + } catch (IOException ioe) { + resubmit(serverName, ioe); + } + // Clean out anything in regions in transition. Being conservative and // doing after log splitting. Could do some states before -- OPENING? // OFFLINE? -- and then others after like CLOSING that depend on log @@ -258,18 +271,47 @@ public class ServerShutdownHandler extends EventHandler { } } } + try { am.assign(toAssignRegions); } catch (InterruptedException ie) { LOG.error("Caught " + ie + " during round-robin assignment"); throw new IOException(ie); } + + try { + if (this.shouldSplitHlog && this.distributedLogReplay) { + // wait for region assignment completes + for (HRegionInfo hri : toAssignRegions) { + if (!am.waitOnRegionToClearRegionsInTransition(hri, regionAssignmentWaitTimeout)) { + throw new IOException("Region " + hri.getEncodedName() + + " didn't complete assignment in time"); + } + } + this.services.getMasterFileSystem().splitLog(serverName); + } + } catch (Exception ex) { + if (ex instanceof IOException) { + resubmit(serverName, (IOException)ex); + } else { + throw new IOException(ex); + } + } } finally { this.deadServers.finish(serverName); } + LOG.info("Finished processing of shutdown of " + serverName); } + private void resubmit(final ServerName serverName, IOException ex) throws IOException { + // typecast to SSH so that we make sure that it is the SSH instance that + // gets submitted as opposed to MSSH or some other derived instance of SSH + this.services.getExecutorService().submit((ServerShutdownHandler) this); + this.deadServers.add(serverName); + throw new IOException("failed log splitting for " + serverName + ", will retry", ex); + } + /** * Process a dead region from a dead RS. Checks if the region is disabled or * disabling or if the region has a partially completed split. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 3f48da00065..3c958d57bfa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -95,6 +95,7 @@ import org.apache.hadoop.hbase.exceptions.DroppedSnapshotException; import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.exceptions.NotServingRegionException; +import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException; import org.apache.hadoop.hbase.exceptions.RegionTooBusyException; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.exceptions.UnknownScannerException; @@ -200,6 +201,16 @@ public class HRegion implements HeapSize { // , Writable{ protected long completeSequenceId = -1L; + /** + * Operation enum is used in {@link HRegion#startRegionOperation} to provide operation context for + * startRegionOperation to possibly invoke different checks before any region operations. Not all + * operations have to be defined here. It's only needed when a special check is need in + * startRegionOperation + */ + protected enum Operation { + ANY, GET, PUT, DELETE, SCAN, APPEND, INCREMENT, SPLIT_REGION, MERGE_REGION + } + ////////////////////////////////////////////////////////////////////////////// // Members ////////////////////////////////////////////////////////////////////////////// @@ -280,6 +291,11 @@ public class HRegion implements HeapSize { // , Writable{ private final AtomicInteger majorInProgress = new AtomicInteger(0); private final AtomicInteger minorInProgress = new AtomicInteger(0); + /** + * Min sequence id stored in store files of a region when opening the region + */ + private long minSeqIdForLogReplay = -1; + /** * @return The smallest mvcc readPoint across all the scanners in this * region. Writes older than this readPoint, are included in every @@ -641,6 +657,9 @@ public class HRegion implements HeapSize { // , Writable{ long storeSeqIdForReplay = store.getMaxSequenceId(false); maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), storeSeqIdForReplay); + if (this.minSeqIdForLogReplay == -1 || storeSeqIdForReplay < this.minSeqIdForLogReplay) { + this.minSeqIdForLogReplay = storeSeqIdForReplay; + } // Include bulk loaded files when determining seqIdForAssignment long storeSeqIdForAssignment = store.getMaxSequenceId(true); if (maxSeqId == -1 || storeSeqIdForAssignment > maxSeqId) { @@ -778,6 +797,21 @@ public class HRegion implements HeapSize { // , Writable{ return this.closing.get(); } + /** + * Reset recovering state of current region + * @param newState + */ + public void setRecovering(boolean newState) { + this.getRegionInfo().setRecovering(newState); + } + + /** + * @return True if current region is in recovering + */ + public boolean isRecovering() { + return this.getRegionInfo().isRecovering(); + } + /** @return true if region is available (not closed and not closing) */ public boolean isAvailable() { return !isClosed() && !isClosing(); @@ -1608,7 +1642,7 @@ public class HRegion implements HeapSize { // , Writable{ // look across all the HStores for this region and determine what the // closest key is across all column families, since the data may be sparse checkRow(row, "getClosestRowBefore"); - startRegionOperation(); + startRegionOperation(Operation.GET); this.readRequestsCount.increment(); try { Store store = getStore(family); @@ -1654,7 +1688,7 @@ public class HRegion implements HeapSize { // , Writable{ protected RegionScanner getScanner(Scan scan, List additionalScanners) throws IOException { - startRegionOperation(); + startRegionOperation(Operation.SCAN); try { // Verify families are all valid prepareScanner(scan); @@ -1705,7 +1739,7 @@ public class HRegion implements HeapSize { // , Writable{ throws IOException { checkReadOnly(); checkResources(); - startRegionOperation(); + startRegionOperation(Operation.DELETE); this.writeRequestsCount.increment(); try { delete.getRow(); @@ -1804,7 +1838,7 @@ public class HRegion implements HeapSize { // , Writable{ // read lock, resources may run out. For now, the thought is that this // will be extremely rare; we'll deal with it when it happens. checkResources(); - startRegionOperation(); + startRegionOperation(Operation.PUT); this.writeRequestsCount.increment(); try { // All edits for the given row (across all column families) must happen atomically. @@ -1862,13 +1896,29 @@ public class HRegion implements HeapSize { // , Writable{ */ public OperationStatus[] batchMutate( Pair[] mutationsAndLocks) throws IOException { + return batchMutate(mutationsAndLocks, false); + } + + /** + * Perform a batch of mutations. + * It supports only Put and Delete mutations and will ignore other types passed. + * @param mutationsAndLocks + * the list of mutations paired with their requested lock IDs. + * @return an array of OperationStatus which internally contains the + * OperationStatusCode and the exceptionMessage if any. + * @throws IOException + */ + OperationStatus[] batchMutate(Pair[] mutationsAndLocks, boolean isReplay) + throws IOException { BatchOperationInProgress> batchOp = new BatchOperationInProgress>(mutationsAndLocks); boolean initialized = false; while (!batchOp.isDone()) { - checkReadOnly(); + if (!isReplay) { + checkReadOnly(); + } checkResources(); long newSize; @@ -1876,11 +1926,13 @@ public class HRegion implements HeapSize { // , Writable{ try { if (!initialized) { - this.writeRequestsCount.increment(); - doPreMutationHook(batchOp); + if (!isReplay) { + this.writeRequestsCount.increment(); + doPreMutationHook(batchOp); + } initialized = true; } - long addedSize = doMiniBatchMutation(batchOp); + long addedSize = doMiniBatchMutation(batchOp, isReplay); newSize = this.addAndGetGlobalMemstoreSize(addedSize); } finally { closeRegionOperation(); @@ -1891,6 +1943,7 @@ public class HRegion implements HeapSize { // , Writable{ } return batchOp.retCodeDetails; } + private void doPreMutationHook(BatchOperationInProgress> batchOp) throws IOException { @@ -1927,10 +1980,9 @@ public class HRegion implements HeapSize { // , Writable{ } } - @SuppressWarnings("unchecked") - private long doMiniBatchMutation( - BatchOperationInProgress> batchOp) throws IOException { + private long doMiniBatchMutation(BatchOperationInProgress> batchOp, + boolean isInReplay) throws IOException { // variable to note if all Put items are for the same CF -- metrics related boolean putsCfSetConsistent = true; @@ -1941,7 +1993,7 @@ public class HRegion implements HeapSize { // , Writable{ //The set of columnFamilies first seen for Delete. Set deletesCfSet = null; - WALEdit walEdit = new WALEdit(); + WALEdit walEdit = new WALEdit(isInReplay); MultiVersionConsistencyControl.WriteEntry w = null; long txid = 0; boolean walSyncSuccessful = false; @@ -1983,7 +2035,11 @@ public class HRegion implements HeapSize { // , Writable{ try { if (isPutMutation) { // Check the families in the put. If bad, skip this one. - checkFamilies(familyMap.keySet()); + if (isInReplay) { + removeNonExistentColumnFamilyForReplay(familyMap); + } else { + checkFamilies(familyMap.keySet()); + } checkTimestamps(mutation.getFamilyMap(), now); } else { prepareDelete((Delete) mutation); @@ -2042,7 +2098,7 @@ public class HRegion implements HeapSize { // , Writable{ } } } - + // we should record the timestamp only after we have acquired the rowLock, // otherwise, newer puts/deletes are not guaranteed to have a newer timestamp now = EnvironmentEdgeManager.currentTimeMillis(); @@ -2081,9 +2137,9 @@ public class HRegion implements HeapSize { // , Writable{ w = mvcc.beginMemstoreInsert(); // calling the pre CP hook for batch mutation - if (coprocessorHost != null) { - MiniBatchOperationInProgress> miniBatchOp = - new MiniBatchOperationInProgress>(batchOp.operations, + if (!isInReplay && coprocessorHost != null) { + MiniBatchOperationInProgress> miniBatchOp = + new MiniBatchOperationInProgress>(batchOp.operations, batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive); if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L; } @@ -2168,9 +2224,9 @@ public class HRegion implements HeapSize { // , Writable{ } walSyncSuccessful = true; // calling the post CP hook for batch mutation - if (coprocessorHost != null) { - MiniBatchOperationInProgress> miniBatchOp = - new MiniBatchOperationInProgress>(batchOp.operations, + if (!isInReplay && coprocessorHost != null) { + MiniBatchOperationInProgress> miniBatchOp = + new MiniBatchOperationInProgress>(batchOp.operations, batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive); coprocessorHost.postBatchMutate(miniBatchOp); } @@ -2187,7 +2243,7 @@ public class HRegion implements HeapSize { // , Writable{ // STEP 9. Run coprocessor post hooks. This should be done after the wal is // synced so that the coprocessor contract is adhered to. // ------------------------------------ - if (coprocessorHost != null) { + if (!isInReplay && coprocessorHost != null) { for (int i = firstIndex; i < lastIndexExclusive; i++) { // only for successful puts if (batchOp.retCodeDetails[i].getOperationStatusCode() @@ -2626,6 +2682,30 @@ public class HRegion implements HeapSize { // , Writable{ } } + /** + * During replay, there could exist column families which are removed between region server + * failure and replay + */ + private void removeNonExistentColumnFamilyForReplay( + final Map> familyMap) { + List nonExistentList = null; + for (byte[] family : familyMap.keySet()) { + if (!this.htableDescriptor.hasFamily(family)) { + if (nonExistentList == null) { + nonExistentList = new ArrayList(); + } + nonExistentList.add(family); + } + } + if (nonExistentList != null) { + for (byte[] family : nonExistentList) { + // Perhaps schema was changed between crash and replay + LOG.info("No family for " + Bytes.toString(family) + " omit from reply."); + familyMap.remove(family); + } + } + } + void checkTimestamps(final Map> familyMap, long now) throws FailedSanityCheckException { if (timestampSlop == HConstants.LATEST_TIMESTAMP) { @@ -3480,7 +3560,7 @@ public class HRegion implements HeapSize { // , Writable{ "after we renewed it. Could be caused by a very slow scanner " + "or a lengthy garbage collection"); } - startRegionOperation(); + startRegionOperation(Operation.SCAN); readRequestsCount.increment(); try { @@ -4651,7 +4731,7 @@ public class HRegion implements HeapSize { // , Writable{ checkReadOnly(); // Lock row - startRegionOperation(); + startRegionOperation(Operation.APPEND); this.writeRequestsCount.increment(); WriteEntry w = null; try { @@ -4819,7 +4899,7 @@ public class HRegion implements HeapSize { // , Writable{ checkReadOnly(); // Lock row - startRegionOperation(); + startRegionOperation(Operation.INCREMENT); this.writeRequestsCount.increment(); WriteEntry w = null; try { @@ -4956,7 +5036,7 @@ public class HRegion implements HeapSize { // , Writable{ ClassSize.OBJECT + ClassSize.ARRAY + 38 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT + - (11 * Bytes.SIZEOF_LONG) + + (12 * Bytes.SIZEOF_LONG) + Bytes.SIZEOF_BOOLEAN); public static final long DEEP_OVERHEAD = FIXED_OVERHEAD + @@ -5236,6 +5316,37 @@ public class HRegion implements HeapSize { // , Writable{ */ public void startRegionOperation() throws NotServingRegionException, RegionTooBusyException, InterruptedIOException { + startRegionOperation(Operation.ANY); + } + + /** + * @param op The operation is about to be taken on the region + * @throws NotServingRegionException + * @throws RegionTooBusyException + * @throws InterruptedIOException + */ + protected void startRegionOperation(Operation op) throws NotServingRegionException, + RegionTooBusyException, InterruptedIOException { + switch (op) { + case INCREMENT: + case APPEND: + case GET: + case SCAN: + case SPLIT_REGION: + case MERGE_REGION: + // when a region is in recovering state, no read, split or merge is allowed + if (this.isRecovering()) { + throw new RegionInRecoveryException(this.getRegionNameAsString() + + " is recovering"); + } + break; + default: + break; + } + if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION) { + // split or merge region doesn't need to check the closing/closed state or lock the region + return; + } if (this.closing.get()) { throw new NotServingRegionException(getRegionNameAsString() + " is closing"); } @@ -5449,6 +5560,14 @@ public class HRegion implements HeapSize { // , Writable{ return this.openSeqNum; } + /** + * Gets the min sequence number that was read from storage when this region was opened. WAL Edits + * with smaller sequence number will be skipped from replay. + */ + public long getMinSeqIdForLogReplay() { + return this.minSeqIdForLogReplay; + } + /** * @return if a given region is in compaction now. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index acd68caa18d..52ae22ca71f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.exceptions.NotServingRegionException; import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; import org.apache.hadoop.hbase.exceptions.RegionAlreadyInTransitionException; +import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException; import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.exceptions.RegionServerRunningException; @@ -112,6 +113,7 @@ import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.ipc.RpcServerInterface; import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface; import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.master.SplitLogManager; import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; @@ -178,6 +180,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.Regio import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; +import org.apache.hadoop.hbase.regionserver.HRegion.Operation; import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler; @@ -204,6 +207,7 @@ import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker; +import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; @@ -214,6 +218,7 @@ import org.apache.hadoop.net.DNS; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; import org.cliffc.high_scale_lib.Counter; import com.google.protobuf.BlockingRpcChannel; @@ -258,6 +263,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa // catalog tracker protected CatalogTracker catalogTracker; + // Watch if a region is out of recovering state from ZooKeeper + @SuppressWarnings("unused") + private RecoveringRegionWatcher recoveringRegionWatcher; + /** * Go here to get table descriptors. */ @@ -291,6 +300,13 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa */ protected final Map 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 recoveringRegions = Collections + .synchronizedMap(new HashMap()); // Leases protected Leases leases; @@ -456,6 +472,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa /** Handle all the snapshot requests to this server */ RegionServerSnapshotManager snapshotManager; + + // configuration setting on if replay WAL edits directly to another RS + private final boolean distributedLogReplay; // Table level lock manager for locking for region operations private TableLockManager tableLockManager; @@ -547,6 +566,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa } }; this.rsHost = new RegionServerCoprocessorHost(this, this.conf); + + this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, + HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG); } /** @@ -671,6 +693,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa } this.tableLockManager = TableLockManager.createTableLockManager(conf, zooKeeper, new ServerName(isa.getHostName(), isa.getPort(), startcode)); + + // register watcher for recovering regions + if(this.distributedLogReplay) { + this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this); + } } /** @@ -1515,8 +1542,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa this.rpcServer.start(); // Create the log splitting worker and start it - this.splitLogWorker = new SplitLogWorker(this.zooKeeper, - this.getConfiguration(), this.getServerName(), this); + this.splitLogWorker = new SplitLogWorker(this.zooKeeper, this.getConfiguration(), this, this); splitLogWorker.start(); } @@ -1641,6 +1667,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa LOG.error("No sequence number found when opening " + r.getRegionNameAsString()); openSeqNum = 0; } + + // Update flushed sequence id of a recovering region in ZK + updateRecoveringRegionLastFlushedSequenceId(r); + // Update ZK, or META if (r.getRegionInfo().isMetaRegion()) { MetaRegionTracker.setMetaLocation(getZooKeeper(), @@ -1884,14 +1914,13 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa public long getLastSequenceId(byte[] region) { Long lastFlushedSequenceId = -1l; try { - GetLastFlushedSequenceIdRequest req = - RequestConverter.buildGetLastFlushedSequenceIdRequest(region); + GetLastFlushedSequenceIdRequest req = RequestConverter + .buildGetLastFlushedSequenceIdRequest(region); lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req) - .getLastFlushedSequenceId(); + .getLastFlushedSequenceId(); } catch (ServiceException e) { lastFlushedSequenceId = -1l; - LOG.warn("Unable to connect to the master to check " + - "the last flushed sequence id", e); + LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id", e); } return lastFlushedSequenceId; } @@ -1965,6 +1994,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa return this.stopping; } + public Map getRecoveringRegions() { + return this.recoveringRegions; + } + /** * * @return the configuration @@ -2651,10 +2684,12 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa try { requestCount.increment(); HRegion region = getRegion(request.getRegion()); + GetResponse.Builder builder = GetResponse.newBuilder(); ClientProtos.Get get = request.getGet(); Boolean existence = null; Result r = null; + if (request.getClosestRowBefore()) { if (get.getColumnCount() != 1) { throw new DoNotRetryIOException( @@ -3006,7 +3041,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa } List values = new ArrayList(); MultiVersionConsistencyControl.setThreadReadPoint(scanner.getMvccReadPoint()); - region.startRegionOperation(); + region.startRegionOperation(Operation.SCAN); try { int i = 0; synchronized(scanner) { @@ -3450,6 +3485,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa removeFromMovedRegions(region.getEncodedName()); if (previous == null) { + // check if the region to be opened is marked in recovering state in ZK + if (isRegionMarkedRecoveringInZK(region.getEncodedName())) { + this.recoveringRegions.put(region.getEncodedName(), null); + } // If there is no action in progress, we can submit a specific handler. // Need to pass the expected version in the constructor. if (region.isMetaRegion()) { @@ -3465,6 +3504,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa 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) { @@ -3589,6 +3631,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa checkOpen(); requestCount.increment(); HRegion region = getRegion(request.getRegion()); + region.startRegionOperation(Operation.SPLIT_REGION); LOG.info("Splitting " + region.getRegionNameAsString()); region.flushcache(); byte[] splitPoint = null; @@ -3621,6 +3664,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa HRegion regionA = getRegion(request.getRegionA()); HRegion regionB = getRegion(request.getRegionB()); boolean forcible = request.getForcible(); + regionA.startRegionOperation(Operation.MERGE_REGION); + regionB.startRegionOperation(Operation.MERGE_REGION); LOG.info("Receiving merging request for " + regionA + ", " + regionB + ",forcible=" + forcible); regionA.flushcache(); @@ -3713,9 +3758,58 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa } } + /** + * Replay the given changes when distributedLogReplay WAL edits from a failed RS. The guarantee is + * that the given mutations will be durable on the receiving RS if this method returns without any + * exception. + * @param rpcc the RPC controller + * @param request the request + * @throws ServiceException + */ + @Override + @QosPriority(priority = HConstants.REPLAY_QOS) + public MultiResponse replay(final RpcController rpcc, final MultiRequest request) + throws ServiceException { + long before = EnvironmentEdgeManager.currentTimeMillis(); + PayloadCarryingRpcController controller = (PayloadCarryingRpcController) rpcc; + CellScanner cellScanner = controller != null ? controller.cellScanner() : null; + // Clear scanner so we are not holding on to reference across call. + controller.setCellScanner(null); + try { + checkOpen(); + HRegion region = getRegion(request.getRegion()); + MultiResponse.Builder builder = MultiResponse.newBuilder(); + List mutates = new ArrayList(); + for (ClientProtos.MultiAction actionUnion : request.getActionList()) { + if (actionUnion.hasMutation()) { + MutationProto mutate = actionUnion.getMutation(); + MutationType type = mutate.getMutateType(); + switch (type) { + case PUT: + case DELETE: + mutates.add(mutate); + break; + default: + throw new DoNotRetryIOException("Unsupported mutate type: " + type.name()); + } + } else { + LOG.warn("Error: invalid action: " + actionUnion + ". " + "it must be a Mutation."); + throw new DoNotRetryIOException("Invalid action, " + "it must be a Mutation."); + } + } + if (!mutates.isEmpty()) { + doBatchOp(builder, region, mutates, cellScanner, true); + } + return builder.build(); + } catch (IOException ie) { + throw new ServiceException(ie); + } finally { + metricsRegionServer.updateReplay(EnvironmentEdgeManager.currentTimeMillis() - before); + } + } + /** * Roll the WAL writer of the region server. - * * @param controller the RPC controller * @param request the request * @throws ServiceException @@ -3841,6 +3935,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa return r; } + /** + * Execute a list of Put/Delete mutations. + */ + protected void doBatchOp(final MultiResponse.Builder builder, + final HRegion region, final List mutates, final CellScanner cells) { + doBatchOp(builder, region, mutates, cells, false); + } + /** * Execute a list of Put/Delete mutations. * @@ -3849,7 +3951,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa * @param mutations */ protected void doBatchOp(final MultiResponse.Builder builder, final HRegion region, - final List mutations, final CellScanner cells) { + final List mutations, final CellScanner cells, boolean isReplay) { @SuppressWarnings("unchecked") Pair[] mutationsWithLocks = new Pair[mutations.size()]; long before = EnvironmentEdgeManager.currentTimeMillis(); @@ -3877,7 +3979,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa cacheFlusher.reclaimMemStoreMemory(); } - OperationStatus codes[] = region.batchMutate(mutationsWithLocks); + OperationStatus codes[] = region.batchMutate(mutationsWithLocks, isReplay); for (i = 0; i < codes.length; i++) { switch (codes[i].getOperationStatusCode()) { case BAD_FAMILY: @@ -4097,4 +4199,91 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa public CompactSplitThread getCompactSplitThread() { return this.compactSplitThread; } + + /** + * check if /hbase/recovering-regions/ exists. Returns true if exists + * and set watcher as well. + * @param regionEncodedName region encode name + * @return true when /hbase/recovering-regions/ exists + * @throws KeeperException + */ + private boolean isRegionMarkedRecoveringInZK(String regionEncodedName) throws KeeperException { + boolean result = false; + String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode, regionEncodedName); + + byte[] node = ZKUtil.getDataAndWatch(this.zooKeeper, nodePath); + if (node != null) { + result = true; + } + + return result; + } + + /** + * 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(HRegion r) throws KeeperException, + IOException { + if (!r.isRecovering()) { + // return immdiately for non-recovering regions + return; + } + + HRegionInfo region = r.getRegionInfo(); + ZooKeeperWatcher zkw = getZooKeeper(); + String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName()); + long minSeqIdForLogReplay = r.getMinSeqIdForLogReplay(); + long lastRecordedFlushedSequenceId = -1; + String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode, + region.getEncodedName()); + // recovering-region level + byte[] data = ZKUtil.getData(zkw, nodePath); + if (data != null) { + lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data); + } + if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) { + ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay)); + } + if (previousRSName != null) { + // one level deeper for failed RS + nodePath = ZKUtil.joinZNode(nodePath, previousRSName); + ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay)); + LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for " + + previousRSName); + } else { + LOG.warn("Can't find failed region server for recovering region " + region.getEncodedName()); + } + } + + /** + * Return the last failed RS name under /hbase/recovering-regions/encodedRegionName + * @param encodedRegionName + * @return + * @throws IOException + * @throws KeeperException + */ + private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException { + String result = null; + long maxZxid = 0; + ZooKeeperWatcher zkw = this.getZooKeeper(); + String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName); + List 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; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LastSequenceId.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LastSequenceId.java index 9c5aac636fb..4fb8db43185 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LastSequenceId.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LastSequenceId.java @@ -26,8 +26,8 @@ import org.apache.hadoop.classification.InterfaceAudience; @InterfaceAudience.Private public interface LastSequenceId { /** - * @param regionname - * @return Last flushed sequence Id for regionname + * @param regionName Encoded region name + * @return Last flushed sequence Id for regionName or -1 if it can't be determined */ - public long getLastSequenceId(byte[] regionname); + public long getLastSequenceId(byte[] regionName); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java index b6530261763..f8a3cfa573f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java @@ -87,4 +87,8 @@ public class MetricsRegionServer { } serverSource.updateAppend(t); } + + public void updateReplay(long t){ + serverSource.updateReplay(t); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java index 2acc93e2fdd..3a6a4a1959f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.util.Map; import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.classification.InterfaceAudience; @@ -108,4 +109,9 @@ public interface RegionServerServices extends OnlineRegions { * @return The RegionServer's CatalogTracker */ public CatalogTracker getCatalogTracker(); + + /** + * @return set of recovering regions on the hosting region server + */ + public Map getRecoveringRegions(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java index 8ebfd1acf55..ce25751af2f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitLogWorker.java @@ -20,7 +20,11 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.io.InterruptedIOException; +import java.net.ConnectException; +import java.net.SocketTimeoutException; +import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.logging.Log; @@ -29,10 +33,12 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SplitLogCounters; import org.apache.hadoop.hbase.SplitLogTask; +import org.apache.hadoop.hbase.client.RetriesExhaustedException; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.exceptions.NotServingRegionException; import org.apache.hadoop.hbase.master.SplitLogManager; import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter; import org.apache.hadoop.hbase.util.CancelableProgressable; @@ -70,6 +76,7 @@ import org.apache.zookeeper.data.Stat; @InterfaceAudience.Private public class SplitLogWorker extends ZooKeeperListener implements Runnable { private static final Log LOG = LogFactory.getLog(SplitLogWorker.class); + private static final int checkInterval = 5000; // 5 seconds Thread worker; private final ServerName serverName; @@ -83,20 +90,30 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { private final Object grabTaskLock = new Object(); private boolean workerInGrabTask = false; private final int report_period; + private RegionServerServices server = null; public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf, - ServerName serverName, TaskExecutor splitTaskExecutor) { + RegionServerServices server, TaskExecutor splitTaskExecutor) { + super(watcher); + this.server = server; + this.serverName = server.getServerName(); + this.splitTaskExecutor = splitTaskExecutor; + report_period = conf.getInt("hbase.splitlog.report.period", + conf.getInt("hbase.splitlog.manager.timeout", SplitLogManager.DEFAULT_TIMEOUT) / 3); + } + + public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf, ServerName serverName, + TaskExecutor splitTaskExecutor) { super(watcher); this.serverName = serverName; this.splitTaskExecutor = splitTaskExecutor; report_period = conf.getInt("hbase.splitlog.report.period", - conf.getInt("hbase.splitlog.manager.timeout", - SplitLogManager.DEFAULT_TIMEOUT) / 2); + conf.getInt("hbase.splitlog.manager.timeout", SplitLogManager.DEFAULT_TIMEOUT) / 3); } - public SplitLogWorker(ZooKeeperWatcher watcher, final Configuration conf, - final ServerName serverName, final LastSequenceId sequenceIdChecker) { - this(watcher, conf, serverName, new TaskExecutor () { + public SplitLogWorker(final ZooKeeperWatcher watcher, final Configuration conf, + RegionServerServices server, final LastSequenceId sequenceIdChecker) { + this(watcher, conf, server, new TaskExecutor() { @Override public Status exec(String filename, CancelableProgressable p) { Path rootdir; @@ -113,7 +130,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { // encountered a bad non-retry-able persistent error. try { if (!HLogSplitter.splitLogFile(rootdir, fs.getFileStatus(new Path(rootdir, filename)), - fs, conf, p, sequenceIdChecker)) { + fs, conf, p, sequenceIdChecker, watcher)) { return Status.PREEMPTED; } } catch (InterruptedIOException iioe) { @@ -121,9 +138,18 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { return Status.RESIGNED; } catch (IOException e) { Throwable cause = e.getCause(); - if (cause instanceof InterruptedException) { + if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException + || cause instanceof ConnectException + || cause instanceof SocketTimeoutException)) { + LOG.warn("log replaying of " + filename + " can't connect to the target regionserver, " + + "resigning", e); + return Status.RESIGNED; + } else if (cause instanceof InterruptedException) { LOG.warn("log splitting of " + filename + " interrupted, resigning", e); return Status.RESIGNED; + } else if(cause instanceof KeeperException) { + LOG.warn("log splitting of " + filename + " hit ZooKeeper issue, resigning", e); + return Status.RESIGNED; } LOG.warn("log splitting of " + filename + " failed, returning error", e); return Status.ERR; @@ -204,7 +230,39 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { synchronized (taskReadyLock) { while (seq_start == taskReadySeq) { try { - taskReadyLock.wait(); + taskReadyLock.wait(checkInterval); + if (this.server != null) { + // check to see if we have stale recovering regions in our internal memory state + Map recoveringRegions = this.server.getRecoveringRegions(); + if (!recoveringRegions.isEmpty()) { + // Make a local copy to prevent ConcurrentModificationException when other threads + // modify recoveringRegions + List tmpCopy = new ArrayList(recoveringRegions.keySet()); + for (String region : tmpCopy) { + String nodePath = ZKUtil.joinZNode(this.watcher.recoveringRegionsZNode, region); + try { + if (ZKUtil.checkExists(this.watcher, nodePath) == -1) { + HRegion r = recoveringRegions.remove(region); + if (r != null) { + r.setRecovering(false); + } + LOG.debug("Mark recovering region:" + region + " up."); + } 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; + } + } + } + } } catch (InterruptedException e) { LOG.info("SplitLogWorker interrupted while waiting for task," + " exiting: " + e.toString() + (exitWorker ? "" : @@ -214,6 +272,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { } } } + } } @@ -463,9 +522,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { } } - - - @Override public void nodeDataChanged(String path) { // there will be a self generated dataChanged event every time attemptToOwnTask() @@ -510,7 +566,6 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable { return childrenPaths; } - @Override public void nodeChildrenChanged(String path) { if(path.equals(watcher.splitLogZNode)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java index f083e7b17ba..0b47a745044 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRegionHandler.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.handler; import java.io.IOException; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.logging.Log; @@ -137,6 +138,16 @@ public class OpenRegionHandler extends EventHandler { if (region == null) { return; } + + // check if we need set current region in recovering state + region.setRecovering(false); + Map recoveringRegions = this.rsServices.getRecoveringRegions(); + if (recoveringRegions != null && !recoveringRegions.isEmpty() + && recoveringRegions.containsKey(region.getRegionInfo().getEncodedName())) { + region.setRecovering(true); + recoveringRegions.put(region.getRegionInfo().getEncodedName(), region); + } + boolean failed = true; if (tickleOpening("post_region_open")) { if (updateMeta(region)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 11c000fb12c..89e63de983c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -1200,6 +1200,10 @@ class FSHLog implements HLog, Syncable { long now = EnvironmentEdgeManager.currentTimeMillis(); // coprocessor hook: if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) { + if (logEdit.isReplay()) { + // set replication scope null so that this won't be replicated + logKey.setScopes(null); + } // write to our buffer for the Hlog file. logSyncer.append(new FSHLog.Entry(logKey, logEdit)); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java index 032e2cf8f8b..b94f3607975 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java @@ -23,9 +23,11 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.net.ConnectException; 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; @@ -34,6 +36,8 @@ import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.Callable; import java.util.concurrent.CompletionService; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; @@ -41,6 +45,8 @@ 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.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.logging.Log; @@ -51,12 +57,30 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.ConnectionUtils; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RetriesExhaustedException; +import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.exceptions.OrphanHLogAfterSplitException; +import org.apache.hadoop.hbase.exceptions.RegionOpeningException; +import org.apache.hadoop.hbase.exceptions.TableNotFoundException; import org.apache.hadoop.hbase.io.HeapSize; +import org.apache.hadoop.hbase.master.RegionState; +import org.apache.hadoop.hbase.master.SplitLogManager; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface; +import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.LastSequenceId; import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry; @@ -67,9 +91,17 @@ 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; +import org.apache.hadoop.hbase.zookeeper.ZKAssign; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; +import org.apache.hadoop.hbase.zookeeper.ZKTable; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.io.MultipleIOException; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.data.Stat; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -102,6 +134,9 @@ public class HLogSplitter { OutputSink outputSink; EntryBuffers entryBuffers; + private Set disablingOrDisabledTables = new HashSet(); + private ZooKeeperWatcher watcher; + // If an exception is thrown by one of the other threads, it will be // stored here. protected AtomicReference thrown = new AtomicReference(); @@ -115,10 +150,21 @@ public class HLogSplitter { // For checking the latest flushed sequence id protected final LastSequenceId sequenceIdChecker; + protected boolean distributedLogReplay; + + // Map encodedRegionName -> lastFlushedSequenceId + Map lastFlushedSequenceIds = new ConcurrentHashMap(); + + // Number of writer threads + private final int numWriterThreads; + + // Min batch size when replay WAL edits + private final int minBatchSize; + /** * Create a new HLogSplitter using the given {@link Configuration} and the - * hbase.hlog.splitter.impl property to derived the instance - * class to use. + * hbase.hlog.splitter.impl property to derived the instance class to use. + * distributedLogReplay won't be enabled by this constructor. *

* @param conf * @param rootDir hbase directory @@ -161,17 +207,37 @@ public class HLogSplitter { public HLogSplitter(Configuration conf, Path rootDir, Path srcDir, Path oldLogDir, FileSystem fs, LastSequenceId idChecker) { + this(conf, rootDir, srcDir, oldLogDir, fs, idChecker, null); + } + + public HLogSplitter(Configuration conf, Path rootDir, Path srcDir, + Path oldLogDir, FileSystem fs, LastSequenceId idChecker, ZooKeeperWatcher zkw) { this.conf = conf; this.rootDir = rootDir; this.srcDir = srcDir; this.oldLogDir = oldLogDir; this.fs = fs; this.sequenceIdChecker = idChecker; + this.watcher = zkw; entryBuffers = new EntryBuffers( conf.getInt("hbase.regionserver.hlog.splitlog.buffersize", 128*1024*1024)); - outputSink = new OutputSink(); + + this.minBatchSize = conf.getInt("hbase.regionserver.wal.logreplay.batch.size", 512); + this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, + HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG); + + this.numWriterThreads = conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3); + if (zkw != null && this.distributedLogReplay) { + outputSink = new LogReplayOutputSink(numWriterThreads); + } else { + if (this.distributedLogReplay) { + LOG.info("ZooKeeperWatcher is passed in as NULL so disable distrubitedLogRepaly."); + } + this.distributedLogReplay = false; + outputSink = new LogRecoveredEditsOutputSink(numWriterThreads); + } } /** @@ -260,26 +326,26 @@ public class HLogSplitter { } /** - * Splits the HLog edits in the given list of logfiles (that are a mix of edits - * on multiple regions) by region and then splits them per region directories, - * in batches of (hbase.hlog.split.batch.size) + * Splits or Replays the HLog edits in the given list of logfiles (that are a mix of edits on + * multiple regions) by region and then splits(or replay when distributedLogReplay is true) them + * per region directories, in batches. *

- * This process is split into multiple threads. In the main thread, we loop - * through the logs to be split. For each log, we: + * This process is split into multiple threads. In the main thread, we loop through the logs to be + * split. For each log, we: *

    - *
  • Recover it (take and drop HDFS lease) to ensure no other process can write
  • - *
  • Read each edit (see {@link #parseHLog}
  • - *
  • Mark as "processed" or "corrupt" depending on outcome
  • + *
  • Recover it (take and drop HDFS lease) to ensure no other process can write
  • + *
  • Read each edit (see {@link #parseHLog}
  • + *
  • Mark as "processed" or "corrupt" depending on outcome
  • *
*

- * Each edit is passed into the EntryBuffers instance, which takes care of - * memory accounting and splitting the edits by region. + * Each edit is passed into the EntryBuffers instance, which takes care of memory accounting and + * splitting the edits by region. *

- * The OutputSink object then manages N other WriterThreads which pull chunks - * of edits from EntryBuffers and write them to the output region directories. + * The OutputSink object then manages N other WriterThreads which pull chunks of edits from + * EntryBuffers and write them to either recovered.edits files or replay them to newly assigned + * region servers directly *

- * After the process is complete, the log files are archived to a separate - * directory. + * After the process is complete, the log files are archived to a separate directory. */ private List splitLog(final FileStatus[] logfiles, CountDownLatch latch) throws IOException { @@ -368,8 +434,7 @@ public class HLogSplitter { /** * Splits a HLog file into region's recovered-edits directory *

- * If the log file has N regions then N recovered.edits files will be - * produced. + * If the log file has N regions then N recovered.edits files will be produced. *

* @param rootDir * @param logfile @@ -377,22 +442,23 @@ public class HLogSplitter { * @param conf * @param reporter * @param idChecker + * @param zkw ZooKeeperWatcher if it's null, we will back to the old-style log splitting where we + * dump out recoved.edits files for regions to replay on. * @return false if it is interrupted by the progress-able. * @throws IOException */ - static public boolean splitLogFile(Path rootDir, FileStatus logfile, - FileSystem fs, Configuration conf, CancelableProgressable reporter, - LastSequenceId idChecker) + static public boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs, + Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker, + ZooKeeperWatcher zkw) throws IOException { - HLogSplitter s = new HLogSplitter(conf, rootDir, null, null /* oldLogDir */, fs, idChecker); + HLogSplitter s = new HLogSplitter(conf, rootDir, null, null/* oldLogDir */, fs, idChecker, zkw); return s.splitLogFile(logfile, reporter); } /** * Splits a HLog file into region's recovered-edits directory *

- * If the log file has N regions then N recovered.edits files will be - * produced. + * If the log file has N regions then N recovered.edits files will be produced. *

* @param rootDir * @param logfile @@ -402,10 +468,10 @@ public class HLogSplitter { * @return false if it is interrupted by the progress-able. * @throws IOException */ - static public boolean splitLogFile(Path rootDir, FileStatus logfile, - FileSystem fs, Configuration conf, CancelableProgressable reporter) + static public boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs, + Configuration conf, CancelableProgressable reporter) throws IOException { - return HLogSplitter.splitLogFile(rootDir, logfile, fs, conf, reporter, null); + return HLogSplitter.splitLogFile(rootDir, logfile, fs, conf, reporter, null, null); } public boolean splitLogFile(FileStatus logfile, @@ -427,6 +493,7 @@ public class HLogSplitter { "into a temporary staging area."); long logLength = logfile.getLen(); LOG.info("Splitting hlog: " + logPath + ", length=" + logLength); + LOG.info("DistributedLogReplay = " + this.distributedLogReplay); status.setStatus("Opening log file"); if (reporter != null && !reporter.progress()) { progress_failed = true; @@ -445,25 +512,37 @@ public class HLogSplitter { LOG.warn("Nothing to split in log file " + logPath); return true; } + if(watcher != null) { + try { + disablingOrDisabledTables = ZKTable.getDisabledOrDisablingTables(watcher); + } catch (KeeperException e) { + throw new IOException("Can't get disabling/disabled tables", e); + } + } int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3); int numOpenedFilesLastCheck = 0; outputSink.setReporter(reporter); outputSink.startWriterThreads(); outputSinkStarted = true; - // Report progress every so many edits and/or files opened (opening a file - // takes a bit of time). - Map lastFlushedSequenceIds = - new TreeMap(Bytes.BYTES_COMPARATOR); Entry entry; - - while ((entry = getNextLogLine(in,logPath, skipErrors)) != null) { + Long lastFlushedSequenceId = -1L; + ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logPath); + String serverNameStr = (serverName == null) ? "" : serverName.getServerName(); + while ((entry = getNextLogLine(in, logPath, skipErrors)) != null) { byte[] region = entry.getKey().getEncodedRegionName(); - Long lastFlushedSequenceId = -1l; - if (sequenceIdChecker != null) { - lastFlushedSequenceId = lastFlushedSequenceIds.get(region); - if (lastFlushedSequenceId == null) { + String key = Bytes.toString(region); + lastFlushedSequenceId = lastFlushedSequenceIds.get(key); + if (lastFlushedSequenceId == null) { + if (this.distributedLogReplay) { + lastFlushedSequenceId = SplitLogManager.getLastFlushedSequenceId(this.watcher, + serverNameStr, key); + } else if (sequenceIdChecker != null) { lastFlushedSequenceId = sequenceIdChecker.getLastSequenceId(region); - lastFlushedSequenceIds.put(region, lastFlushedSequenceId); + } + if (lastFlushedSequenceId != null) { + lastFlushedSequenceIds.put(key, lastFlushedSequenceId); + } else { + lastFlushedSequenceId = -1L; } } if (lastFlushedSequenceId >= entry.getKey().getLogSeqNum()) { @@ -472,12 +551,13 @@ public class HLogSplitter { } entryBuffers.appendEntry(entry); editsCount++; + int moreWritersFromLastCheck = this.getNumOpenWriters() - numOpenedFilesLastCheck; // If sufficient edits have passed, check if we should report progress. if (editsCount % interval == 0 - || (outputSink.logWriters.size() - numOpenedFilesLastCheck) > numOpenedFilesBeforeReporting) { - numOpenedFilesLastCheck = outputSink.logWriters.size(); - String countsStr = (editsCount - editsSkipped) + - " edits, skipped " + editsSkipped + " edits."; + || moreWritersFromLastCheck > numOpenedFilesBeforeReporting) { + numOpenedFilesLastCheck = this.getNumOpenWriters(); + String countsStr = (editsCount - (editsSkipped + outputSink.getSkippedEdits())) + + " edits, skipped " + editsSkipped + " edits."; status.setStatus("Split " + countsStr); if (reporter != null && !reporter.progress()) { progress_failed = true; @@ -502,9 +582,8 @@ public class HLogSplitter { progress_failed = outputSink.finishWritingAndClose() == null; } String msg = "Processed " + editsCount + " edits across " - + outputSink.getOutputCounts().size() + " regions; log file=" - + logPath + " is corrupted = " + isCorrupted + " progress failed = " - + progress_failed; + + outputSink.getNumberOfRecoveredRegions() + " regions; log file=" + logPath + + " is corrupted = " + isCorrupted + " progress failed = " + progress_failed; LOG.info(msg); status.markComplete(msg); } @@ -831,6 +910,18 @@ public class HLogSplitter { return HLogFactory.createReader(fs, curLogFile, conf, reporter); } + /** + * Get current open writers + * @return + */ + private int getNumOpenWriters() { + int result = 0; + if (this.outputSink != null) { + result += this.outputSink.getNumOpenWriters(); + } + return result; + } + /** * Class which accumulates edits and separates them into a buffer per region * while simultaneously accounting RAM usage. Blocks if the RAM usage crosses @@ -880,20 +971,23 @@ public class HLogSplitter { totalBuffered += incrHeap; while (totalBuffered > maxHeapUsage && thrown.get() == null) { LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads..."); - dataAvailable.wait(3000); + dataAvailable.wait(2000); } dataAvailable.notifyAll(); } checkForErrors(); } + /** + * @return RegionEntryBuffer a buffer of edits to be written or replayed. + */ synchronized RegionEntryBuffer getChunkToWrite() { - long biggestSize=0; - byte[] biggestBufferKey=null; + long biggestSize = 0; + byte[] biggestBufferKey = null; for (Map.Entry entry : buffers.entrySet()) { long size = entry.getValue().heapSize(); - if (size > biggestSize && !currentlyWriting.contains(entry.getKey())) { + if (size > biggestSize && (!currentlyWriting.contains(entry.getKey()))) { biggestSize = size; biggestBufferKey = entry.getKey(); } @@ -968,9 +1062,11 @@ public class HLogSplitter { class WriterThread extends Thread { private volatile boolean shouldStop = false; + private OutputSink outputSink = null; - WriterThread(int i) { + WriterThread(OutputSink sink, int i) { super("WriterThread-" + i); + outputSink = sink; } public void run() { @@ -989,9 +1085,11 @@ public class HLogSplitter { if (buffer == null) { // No data currently available, wait on some more to show up synchronized (dataAvailable) { - if (shouldStop) return; + if (shouldStop && !this.outputSink.flush()) { + return; + } try { - dataAvailable.wait(1000); + dataAvailable.wait(500); } catch (InterruptedException ie) { if (!shouldStop) { throw new RuntimeException(ie); @@ -1012,39 +1110,7 @@ public class HLogSplitter { private void writeBuffer(RegionEntryBuffer buffer) throws IOException { - List entries = buffer.entryBuffer; - if (entries.isEmpty()) { - LOG.warn(this.getName() + " got an empty buffer, skipping"); - return; - } - - WriterAndPath wap = null; - - long startTime = System.nanoTime(); - try { - int editsCount = 0; - - for (Entry logEntry : entries) { - if (wap == null) { - wap = outputSink.getWriterAndPath(logEntry); - if (wap == null) { - // getWriterAndPath decided we don't need to write these edits - // Message was already logged - return; - } - } - wap.w.append(logEntry); - outputSink.updateRegionMaximumEditLogSeqNum(logEntry); - editsCount++; - } - // Pass along summary statistics - wap.incrementEdits(editsCount); - wap.incrementNanoTime(System.nanoTime() - startTime); - } catch (IOException e) { - e = RemoteExceptionHandler.checkIOException(e); - LOG.fatal(this.getName() + " Got while writing log entry to log", e); - throw e; - } + outputSink.append(buffer); } void finish() { @@ -1055,28 +1121,6 @@ public class HLogSplitter { } } - private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir, - FileSystem fs, Configuration conf) - throws IOException { - Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, true); - if (regionedits == null) { - return null; - } - if (fs.exists(regionedits)) { - LOG.warn("Found existing old edits file. It could be the " - + "result of a previous failed split attempt. Deleting " - + regionedits + ", length=" - + fs.getFileStatus(regionedits).getLen()); - if (!fs.delete(regionedits, false)) { - LOG.warn("Failed delete of old " + regionedits); - } - } - Writer w = createWriter(fs, regionedits, conf); - LOG.debug("Creating writer path=" + regionedits + " region=" - + Bytes.toStringBinary(region)); - return (new WriterAndPath(regionedits, w)); - } - Path convertRegionEditsToTemp(Path rootdir, Path edits, String tmpname) { List components = new ArrayList(10); do { @@ -1109,35 +1153,35 @@ public class HLogSplitter { } /** - * Class that manages the output streams from the log splitting process. + * 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 */ - class OutputSink { - private final Map logWriters = Collections.synchronizedMap( - new TreeMap(Bytes.BYTES_COMPARATOR)); - private final Map regionMaximumEditLogSeqNum = Collections + abstract class OutputSink { + + protected Map writers = Collections + .synchronizedMap(new TreeMap(Bytes.BYTES_COMPARATOR));; + + protected final Map regionMaximumEditLogSeqNum = Collections .synchronizedMap(new TreeMap(Bytes.BYTES_COMPARATOR)); - private final List writerThreads = Lists.newArrayList(); + + protected final List writerThreads = Lists.newArrayList(); /* Set of regions which we've decided should not output edits */ - private final Set blacklistedRegions = Collections.synchronizedSet( - new TreeSet(Bytes.BYTES_COMPARATOR)); + protected final Set blacklistedRegions = Collections + .synchronizedSet(new TreeSet(Bytes.BYTES_COMPARATOR)); - private boolean closeAndCleanCompleted = false; + protected boolean closeAndCleanCompleted = false; - private boolean logWritersClosed = false; + protected boolean writersClosed = false; - private final int numThreads; + protected final int numThreads; - private CancelableProgressable reporter = null; + protected CancelableProgressable reporter = null; - public OutputSink() { - // More threads could potentially write faster at the expense - // of causing more disk seeks as the logs are split. - // 3. After a certain setting (probably around 3) the - // process will be bound on the reader in the current - // implementation anyway. - numThreads = conf.getInt( - "hbase.regionserver.hlog.splitlog.writer.threads", 3); + protected AtomicLong skippedEdits = new AtomicLong(); + + public OutputSink(int numWriters) { + numThreads = numWriters; } void setReporter(CancelableProgressable reporter) { @@ -1145,12 +1189,11 @@ public class HLogSplitter { } /** - * Start the threads that will pump data from the entryBuffers - * to the output files. + * Start the threads that will pump data from the entryBuffers to the output files. */ synchronized void startWriterThreads() { for (int i = 0; i < numThreads; i++) { - WriterThread t = new WriterThread(i); + WriterThread t = new WriterThread(this, i); t.start(); writerThreads.add(t); } @@ -1158,66 +1201,147 @@ public class HLogSplitter { /** * + * Update region's maximum edit log SeqNum. + */ + void updateRegionMaximumEditLogSeqNum(Entry entry) { + synchronized (regionMaximumEditLogSeqNum) { + Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(entry.getKey() + .getEncodedRegionName()); + if (currentMaxSeqNum == null || entry.getKey().getLogSeqNum() > currentMaxSeqNum) { + regionMaximumEditLogSeqNum.put(entry.getKey().getEncodedRegionName(), entry.getKey() + .getLogSeqNum()); + } + } + } + + Long getRegionMaximumEditLogSeqNum(byte[] region) { + return regionMaximumEditLogSeqNum.get(region); + } + + /** + * @return the number of currently opened writers + */ + int getNumOpenWriters() { + return this.writers.size(); + } + + long getSkippedEdits() { + return this.skippedEdits.get(); + } + + /** + * Wait for writer threads to dump all info to the sink + * @return true when there is no error + * @throws IOException + */ + protected boolean finishWriting() throws IOException { + LOG.info("Waiting for split writer threads to finish"); + boolean progress_failed = false; + for (WriterThread t : writerThreads) { + t.finish(); + } + for (WriterThread t : writerThreads) { + if (!progress_failed && reporter != null && !reporter.progress()) { + progress_failed = true; + } + try { + t.join(); + } catch (InterruptedException ie) { + IOException iie = new InterruptedIOException(); + iie.initCause(ie); + throw iie; + } + checkForErrors(); + } + LOG.info("Split writers finished"); + return (!progress_failed); + } + + abstract List finishWritingAndClose() throws IOException; + + /** + * @return a map from encoded region ID to the number of edits written out for that region. + */ + abstract Map getOutputCounts(); + + /** + * @return number of regions we've recovered + */ + abstract int getNumberOfRecoveredRegions(); + + /** + * @param entry A WAL Edit Entry + * @throws IOException + */ + abstract void append(RegionEntryBuffer buffer) throws IOException; + + /** + * WriterThread call this function to help flush internal remaining edits in buffer before close + * @return true when underlying sink has something to flush + */ + protected boolean flush() throws IOException { + return false; + } + } + + /** + * Class that manages the output streams from the log splitting process. + */ + class LogRecoveredEditsOutputSink extends OutputSink { + + public LogRecoveredEditsOutputSink(int numWriters) { + // More threads could potentially write faster at the expense + // of causing more disk seeks as the logs are split. + // 3. After a certain setting (probably around 3) the + // process will be bound on the reader in the current + // implementation anyway. + super(numWriters); + } + + /** * @return null if failed to report progress * @throws IOException */ + @Override List finishWritingAndClose() throws IOException { - LOG.info("Waiting for split writer threads to finish"); - boolean progress_failed = false; + boolean isSuccessful = false; + List result = null; try { - for (WriterThread t : writerThreads) { - t.finish(); - } - for (WriterThread t : writerThreads) { - if (!progress_failed && reporter != null && !reporter.progress()) { - progress_failed = true; - } - try { - t.join(); - } catch (InterruptedException ie) { - IOException iie = new InterruptedIOException(); - iie.initCause(ie); - throw iie; - } - checkForErrors(); - } - LOG.info("Split writers finished"); - if (progress_failed) { - return null; - } - return closeStreams(); + isSuccessful = finishWriting(); } finally { + result = close(); List thrown = closeLogWriters(null); if (thrown != null && !thrown.isEmpty()) { throw MultipleIOException.createIOException(thrown); } } + return (isSuccessful) ? result : null; } /** * Close all of the output streams. * @return the list of paths written. */ - private List closeStreams() throws IOException { + private List close() throws IOException { Preconditions.checkState(!closeAndCleanCompleted); final List paths = new ArrayList(); final List thrown = Lists.newArrayList(); - ThreadPoolExecutor closeThreadPool = Threads.getBoundedCachedThreadPool( - numThreads, 30L, TimeUnit.SECONDS, new ThreadFactory() { - private int count = 1; - public Thread newThread(Runnable r) { - Thread t = new Thread(r, "split-log-closeStream-" + count++); - return t; - } - }); + ThreadPoolExecutor closeThreadPool = Threads.getBoundedCachedThreadPool(numThreads, 30L, + TimeUnit.SECONDS, new ThreadFactory() { + private int count = 1; + + public Thread newThread(Runnable r) { + Thread t = new Thread(r, "split-log-closeStream-" + count++); + return t; + } + }); CompletionService completionService = new ExecutorCompletionService( closeThreadPool); - for (final Map.Entry logWritersEntry : logWriters - .entrySet()) { + for (final Map.Entry writersEntry : writers.entrySet()) { completionService.submit(new Callable() { public Void call() throws Exception { - WriterAndPath wap = logWritersEntry.getValue(); + WriterAndPath wap = (WriterAndPath) writersEntry.getValue(); try { wap.w.close(); } catch (IOException ioe) { @@ -1225,15 +1349,25 @@ public class HLogSplitter { thrown.add(ioe); return null; } - LOG.info("Closed path " + wap.p + " (wrote " + wap.editsWritten - + " edits in " + (wap.nanosSpent / 1000 / 1000) + "ms)"); + LOG.info("Closed path " + wap.p + " (wrote " + wap.editsWritten + " edits in " + + (wap.nanosSpent / 1000 / 1000) + "ms)"); + + if (wap.editsWritten == 0) { + // just remove the empty recovered.edits file + if (fs.exists(wap.p) && !fs.delete(wap.p, false)) { + LOG.warn("Failed deleting empty " + wap.p); + throw new IOException("Failed deleting empty " + wap.p); + } + return null; + } + Path dst = getCompletedRecoveredEditsFilePath(wap.p, - regionMaximumEditLogSeqNum.get(logWritersEntry.getKey())); + regionMaximumEditLogSeqNum.get(writersEntry.getKey())); try { if (!dst.equals(wap.p) && fs.exists(dst)) { LOG.warn("Found existing old edits file. It could be the " - + "result of a previous failed split attempt. Deleting " - + dst + ", length=" + fs.getFileStatus(dst).getLen()); + + "result of a previous failed split attempt. Deleting " + dst + ", length=" + + fs.getFileStatus(dst).getLen()); if (!fs.delete(dst, false)) { LOG.warn("Failed deleting of old " + dst); throw new IOException("Failed deleting of old " + dst); @@ -1244,8 +1378,7 @@ public class HLogSplitter { // TestHLogSplit#testThreading is an example. if (fs.exists(wap.p)) { if (!fs.rename(wap.p, dst)) { - throw new IOException("Failed renaming " + wap.p + " to " - + dst); + throw new IOException("Failed renaming " + wap.p + " to " + dst); } LOG.debug("Rename " + wap.p + " to " + dst); } @@ -1262,7 +1395,7 @@ public class HLogSplitter { boolean progress_failed = false; try { - for (int i = 0, n = logWriters.size(); i < n; i++) { + for (int i = 0, n = this.writers.size(); i < n; i++) { Future future = completionService.take(); future.get(); if (!progress_failed && reporter != null && !reporter.progress()) { @@ -1282,7 +1415,7 @@ public class HLogSplitter { if (!thrown.isEmpty()) { throw MultipleIOException.createIOException(thrown); } - logWritersClosed = true; + writersClosed = true; closeAndCleanCompleted = true; if (progress_failed) { return null; @@ -1290,12 +1423,558 @@ public class HLogSplitter { return paths; } - private List closeLogWriters(List thrown) - throws IOException { - if (!logWritersClosed) { - if (thrown == null) { - thrown = Lists.newArrayList(); + private List closeLogWriters(List thrown) throws IOException { + if (writersClosed) { + return thrown; + } + + if (thrown == null) { + thrown = 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) { + WriterAndPath wap = null; + for (SinkWriter tmpWAP : writers.values()) { + try { + wap = (WriterAndPath) tmpWAP; + wap.w.close(); + } catch (IOException ioe) { + LOG.error("Couldn't close log at " + wap.p, ioe); + thrown.add(ioe); + continue; + } + LOG.info("Closed path " + wap.p + " (wrote " + wap.editsWritten + " edits in " + + (wap.nanosSpent / 1000 / 1000) + "ms)"); + } + } + writersClosed = true; + } + + return thrown; + } + + /** + * 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 WriterAndPath getWriterAndPath(Entry entry) throws IOException { + byte region[] = entry.getKey().getEncodedRegionName(); + WriterAndPath ret = (WriterAndPath) writers.get(region); + if (ret != null) { + return ret; + } + // If we already decided that this region doesn't get any output + // we don't need to check again. + if (blacklistedRegions.contains(region)) { + return null; + } + ret = createWAP(region, entry, rootDir, fs, conf); + if (ret == null) { + blacklistedRegions.add(region); + return null; + } + writers.put(region, ret); + return ret; + } + + private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir, FileSystem fs, + Configuration conf) throws IOException { + Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, true); + if (regionedits == null) { + return null; + } + if (fs.exists(regionedits)) { + LOG.warn("Found old edits file. It could be the " + + "result of a previous failed split attempt. Deleting " + regionedits + ", length=" + + fs.getFileStatus(regionedits).getLen()); + if (!fs.delete(regionedits, false)) { + LOG.warn("Failed delete of old " + regionedits); + } + } + Writer w = createWriter(fs, regionedits, conf); + LOG.debug("Creating writer path=" + regionedits + " region=" + Bytes.toStringBinary(region)); + return (new WriterAndPath(regionedits, w)); + } + + void append(RegionEntryBuffer buffer) throws IOException { + List entries = buffer.entryBuffer; + if (entries.isEmpty()) { + LOG.warn("got an empty buffer, skipping"); + return; + } + + WriterAndPath wap = null; + + long startTime = System.nanoTime(); + try { + int editsCount = 0; + + for (Entry logEntry : entries) { + if (wap == null) { + wap = getWriterAndPath(logEntry); + if (wap == null) { + // getWriterAndPath decided we don't need to write these edits + return; + } + } + wap.w.append(logEntry); + this.updateRegionMaximumEditLogSeqNum(logEntry); + editsCount++; + } + // Pass along summary statistics + wap.incrementEdits(editsCount); + wap.incrementNanoTime(System.nanoTime() - startTime); + } catch (IOException e) { + e = RemoteExceptionHandler.checkIOException(e); + LOG.fatal(" Got while writing log entry to log", e); + throw e; + } + } + + /** + * @return a map from encoded region ID to the number of edits written out for that region. + */ + Map getOutputCounts() { + TreeMap ret = new TreeMap(Bytes.BYTES_COMPARATOR); + synchronized (writers) { + for (Map.Entry entry : writers.entrySet()) { + ret.put(entry.getKey(), entry.getValue().editsWritten); + } + } + return ret; + } + + @Override + int getNumberOfRecoveredRegions() { + return writers.size(); + } + } + + /** + * Class wraps the actual writer which writes data out and related statistics + */ + private abstract static class SinkWriter { + /* Count of edits written to this path */ + long editsWritten = 0; + /* Number of nanos spent writing to this log */ + long nanosSpent = 0; + + void incrementEdits(int edits) { + editsWritten += edits; + } + + void incrementNanoTime(long nanos) { + nanosSpent += nanos; + } + } + + /** + * Private data structure that wraps a Writer and its Path, also collecting statistics about the + * data written to this output. + */ + private final static class WriterAndPath extends SinkWriter { + final Path p; + final Writer w; + + WriterAndPath(final Path p, final Writer w) { + this.p = p; + this.w = w; + } + } + + /** + * 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 long waitRegionOnlineTimeOut; + private final Set recoveredRegions = Collections.synchronizedSet(new HashSet()); + private final Map writers = + new ConcurrentHashMap(); + // online encoded region name map + private final Set onlineRegions = Collections.synchronizedSet(new HashSet()); + + private Map tableNameToHConnectionMap = Collections + .synchronizedMap(new TreeMap(Bytes.BYTES_COMPARATOR)); + /** + * Map key -> value layout + * : -> Queue + */ + private Map>> serverToBufferQueueMap = + new ConcurrentHashMap>>(); + private List 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 LogRecoveredEditsOutputSink logRecoveredEditsOutputSink; + private boolean hasEditsInDisablingOrDisabledTables = false; + + public LogReplayOutputSink(int numWriters) { + super(numWriters); + + this.waitRegionOnlineTimeOut = conf.getInt("hbase.splitlog.manager.timeout", + SplitLogManager.DEFAULT_TIMEOUT); + this.logRecoveredEditsOutputSink = new LogRecoveredEditsOutputSink(numWriters); + this.logRecoveredEditsOutputSink.setReporter(reporter); + } + + void append(RegionEntryBuffer buffer) throws IOException { + List 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 (disablingOrDisabledTables.contains(Bytes.toString(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> maxQueue = null; + synchronized (this.serverToBufferQueueMap) { + for (String key : this.serverToBufferQueueMap.keySet()) { + List> curQueue = this.serverToBufferQueueMap.get(key); + if (curQueue.size() > maxSize) { + maxSize = curQueue.size(); + maxQueue = curQueue; + maxLocKey = key; + } + } + 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 entries) throws IOException { + Set nonExistentTables = null; + Long cachedLastFlushedSequenceId = -1l; + for (HLog.Entry entry : entries) { + WALEdit edit = entry.getEdit(); + byte[] table = entry.getKey().getTablename(); + String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName()); + // skip edits of non-existent tables + if (nonExistentTables != null && nonExistentTables.contains(table)) { + this.skippedEdits.incrementAndGet(); + continue; + } + boolean needSkip = false; + Put put = null; + Delete del = null; + KeyValue lastKV = null; + HRegionLocation loc = null; + Row preRow = null; + HRegionLocation preLoc = null; + Row lastAddedRow = null; // it is not really needed here just be conservative + String preKey = null; + List kvs = edit.getKeyValues(); + HConnection hconn = this.getConnectionByTableName(table); + + for (KeyValue kv : kvs) { + // filtering HLog meta entries + // We don't handle HBASE-2231 because we may or may not replay a compaction event. + // Details at https://issues.apache.org/jira/browse/HBASE-2231?focusedCommentId=13647143& + // page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13647143 + if (kv.matchingFamily(WALEdit.METAFAMILY)) continue; + + if (lastKV == null || lastKV.getType() != kv.getType() || !lastKV.matchingRow(kv)) { + if (preRow != null) { + synchronized (serverToBufferQueueMap) { + List> queue = serverToBufferQueueMap.get(preKey); + if (queue == null) { + queue = Collections.synchronizedList(new ArrayList>()); + serverToBufferQueueMap.put(preKey, queue); + } + queue.add(new Pair(preLoc, preRow)); + lastAddedRow = preRow; + } + // store regions we have recovered so far + addToRecoveredRegions(preLoc.getRegionInfo().getEncodedName()); + } + + try { + loc = locateRegionAndRefreshLastFlushedSequenceId(hconn, table, kv.getRow()); + } catch (TableNotFoundException ex) { + // table has been deleted so skip edits of the table + LOG.info("Table " + Bytes.toString(table) + + " doesn't exist. Skip log replay for region " + encodeRegionNameStr); + lastFlushedSequenceIds.put(encodeRegionNameStr, Long.MAX_VALUE); + if (nonExistentTables == null) { + nonExistentTables = new TreeSet(Bytes.BYTES_COMPARATOR); + } + 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 HLog entry + this.skippedEdits.incrementAndGet(); + needSkip = true; + break; + } + + if (kv.isDelete()) { + del = new Delete(kv.getRow()); + del.setClusterId(entry.getKey().getClusterId()); + preRow = del; + } else { + put = new Put(kv.getRow()); + put.setClusterId(entry.getKey().getClusterId()); + preRow = put; + } + preKey = loc.getHostnamePort() + KEY_DELIMITER + Bytes.toString(table); + preLoc = loc; + } + if (kv.isDelete()) { + del.addDeleteMarker(kv); + } else { + put.add(kv); + } + lastKV = kv; + } + + // skip the edit + if(needSkip) continue; + + // add the last row + if (preRow != null && lastAddedRow != preRow) { + synchronized (serverToBufferQueueMap) { + List> queue = serverToBufferQueueMap.get(preKey); + if (queue == null) { + queue = Collections.synchronizedList(new ArrayList>()); + serverToBufferQueueMap.put(preKey, queue); + } + queue.add(new Pair(preLoc, preRow)); + } + // store regions we have recovered so far + addToRecoveredRegions(preLoc.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(HConnection hconn, + byte[] table, byte[] row) throws IOException { + HRegionLocation loc = hconn.getRegionLocation(table, row, false); + if (loc == null) { + throw new IOException("Can't locate location for row:" + Bytes.toString(row) + + " of table:" + Bytes.toString(table)); + } + if (onlineRegions.contains(loc.getRegionInfo().getEncodedName())) { + return loc; + } + + Long lastFlushedSequenceId = -1l; + loc = waitUntilRegionOnline(loc, row, this.waitRegionOnlineTimeOut); + Long cachedLastFlushedSequenceId = lastFlushedSequenceIds.get(loc.getRegionInfo() + .getEncodedName()); + + onlineRegions.add(loc.getRegionInfo().getEncodedName()); + // retrieve last flushed sequence Id from ZK. Because region postOpenDeployTasks will + // update the value for the region + lastFlushedSequenceId = SplitLogManager.getLastFlushedSequenceId(watcher, loc + .getServerName().getServerName(), loc.getRegionInfo().getEncodedName()); + if (cachedLastFlushedSequenceId == null + || lastFlushedSequenceId > cachedLastFlushedSequenceId) { + lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), lastFlushedSequenceId); + } else if (loc.getRegionInfo().isRecovering() == false) { + // 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."); + } + + return loc; + } + + private void processWorkItems(String key, List> 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 = RemoteExceptionHandler.checkIOException(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 + * @return True when region is online on the destination region server + * @throws InterruptedException + */ + private HRegionLocation waitUntilRegionOnline(HRegionLocation loc, byte[] row, + final long timeout) + throws IOException { + final long endTime = EnvironmentEdgeManager.currentTimeMillis() + timeout; + final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, + HConstants.DEFAULT_HBASE_CLIENT_PAUSE); + boolean reloadLocation = false; + byte[] tableName = loc.getRegionInfo().getTableName(); + int tries = 0; + Throwable cause = null; + while (endTime > EnvironmentEdgeManager.currentTimeMillis()) { + try { + // Try and get regioninfo from the hosting server. + HConnection hconn = getConnectionByTableName(tableName); + if(reloadLocation) { + loc = hconn.getRegionLocation(tableName, row, true); + } + BlockingInterface remoteSvr = hconn.getAdmin(loc.getServerName()); + HRegionInfo region = loc.getRegionInfo(); + if((region =ProtobufUtil.getRegionInfo(remoteSvr, region.getRegionName())) != null) { + loc.getRegionInfo().setRecovering(region.isRecovering()); + return loc; + } + } 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) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted when waiting regon " + + loc.getRegionInfo().getEncodedName() + " online.", e); + } + tries++; + } + + throw new IOException("Timeout when waiting region " + loc.getRegionInfo().getEncodedName() + + " online for " + timeout + " milliseconds.", cause); + } + + @Override + protected boolean flush() throws IOException { + String curLoc = null; + int curSize = 0; + List> curQueue = null; + synchronized (this.serverToBufferQueueMap) { + for (String locationKey : this.serverToBufferQueueMap.keySet()) { + curQueue = this.serverToBufferQueueMap.get(locationKey); + if (!curQueue.isEmpty()) { + curSize = curQueue.size(); + curLoc = locationKey; + break; + } + } + if (curSize > 0) { + this.serverToBufferQueueMap.remove(curLoc); + } + } + + if (curSize > 0) { + this.processWorkItems(curLoc, curQueue); + dataAvailable.notifyAll(); + return true; + } + return false; + } + + void addWriterError(Throwable t) { + thrown.add(t); + } + + @Override + List finishWritingAndClose() throws IOException { + List result = new ArrayList(); + try { + if (!finishWriting()) { + return null; + } + if (hasEditsInDisablingOrDisabledTables) { + result = logRecoveredEditsOutputSink.finishWritingAndClose(); + } + // returns an empty array in order to keep interface same as old way + return result; + } finally { + List thrown = closeRegionServerWriters(); + if (thrown != null && !thrown.isEmpty()) { + throw MultipleIOException.createIOException(thrown); + } + } + } + + @Override + int getNumOpenWriters() { + return this.writers.size() + this.logRecoveredEditsOutputSink.getNumOpenWriters(); + } + + private List closeRegionServerWriters() throws IOException { + List result = null; + if (!writersClosed) { + result = Lists.newArrayList(); try { for (WriterThread t : writerThreads) { while (t.isAlive()) { @@ -1311,118 +1990,122 @@ public class HLogSplitter { } } } finally { - synchronized (logWriters) { - for (WriterAndPath wap : logWriters.values()) { + synchronized (writers) { + for (String locationKey : writers.keySet()) { + RegionServerWriter tmpW = writers.get(locationKey); try { - wap.w.close(); + tmpW.close(); } catch (IOException ioe) { - LOG.error("Couldn't close log at " + wap.p, ioe); - thrown.add(ioe); - continue; + LOG.error("Couldn't close writer for region server:" + locationKey, ioe); + result.add(ioe); } - LOG.info("Closed path " + wap.p + " (wrote " + wap.editsWritten - + " edits in " + (wap.nanosSpent / 1000 / 1000) + "ms)"); } } - logWritersClosed = true; + + // close connections + synchronized (this.tableNameToHConnectionMap) { + for (byte[] tableName : this.tableNameToHConnectionMap.keySet()) { + HConnection hconn = this.tableNameToHConnectionMap.get(tableName); + try { + hconn.close(); + } catch (IOException ioe) { + result.add(ioe); + } + } + } + writersClosed = true; } } - return thrown; + return result; + } + + Map getOutputCounts() { + TreeMap ret = new TreeMap(Bytes.BYTES_COMPARATOR); + synchronized (writers) { + for (Map.Entry entry : writers.entrySet()) { + ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten); + } + } + return ret; + } + + @Override + int getNumberOfRecoveredRegions() { + return this.recoveredRegions.size(); } /** - * 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. - * + * 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 */ - WriterAndPath getWriterAndPath(Entry entry) throws IOException { - byte region[] = entry.getKey().getEncodedRegionName(); - WriterAndPath ret = logWriters.get(region); + private RegionServerWriter getRegionServerWriter(String loc) throws IOException { + RegionServerWriter ret = writers.get(loc); if (ret != null) { return ret; } - // If we already decided that this region doesn't get any output - // we don't need to check again. - if (blacklistedRegions.contains(region)) { - return null; - } - ret = createWAP(region, entry, rootDir, fs, conf); - if (ret == null) { - blacklistedRegions.add(region); - return null; - } - logWriters.put(region, ret); - return ret; - } - /** - * Update region's maximum edit log SeqNum. - */ - void updateRegionMaximumEditLogSeqNum(Entry entry) { - synchronized (regionMaximumEditLogSeqNum) { - Long currentMaxSeqNum=regionMaximumEditLogSeqNum.get(entry.getKey().getEncodedRegionName()); - if (currentMaxSeqNum == null - || entry.getKey().getLogSeqNum() > currentMaxSeqNum) { - regionMaximumEditLogSeqNum.put(entry.getKey().getEncodedRegionName(), - entry.getKey().getLogSeqNum()); - } + String tableName = getTableFromLocationStr(loc); + if(tableName.isEmpty()){ + LOG.warn("Invalid location string:" + loc + " found."); } - } - - Long getRegionMaximumEditLogSeqNum(byte[] region) { - return regionMaximumEditLogSeqNum.get(region); - } - - /** - * @return a map from encoded region ID to the number of edits written out - * for that region. - */ - private Map getOutputCounts() { - TreeMap ret = new TreeMap( - Bytes.BYTES_COMPARATOR); - synchronized (logWriters) { - for (Map.Entry entry : logWriters.entrySet()) { - ret.put(entry.getKey(), entry.getValue().editsWritten); + HConnection hconn = getConnectionByTableName(Bytes.toBytes(tableName)); + synchronized (writers) { + ret = writers.get(loc); + if (ret == null) { + ret = new RegionServerWriter(conf, Bytes.toBytes(tableName), hconn); + writers.put(loc, ret); } } return ret; } + + private HConnection getConnectionByTableName(final byte[] tableName) throws IOException { + HConnection hconn = this.tableNameToHConnectionMap.get(tableName); + if (hconn == null) { + synchronized (this.tableNameToHConnectionMap) { + hconn = this.tableNameToHConnectionMap.get(tableName); + if (hconn == null) { + hconn = HConnectionManager.createConnection(conf); + this.tableNameToHConnectionMap.put(tableName, hconn); + } + } + } + return hconn; + } + + private String getTableFromLocationStr(String loc) { + /** + * location key is in format #
+ */ + String[] splits = loc.split(KEY_DELIMITER); + if (splits.length != 2) { + return ""; + } + return splits[1]; + } } /** - * Private data structure that wraps a Writer and its Path, - * also collecting statistics about the data written to this - * output. + * Private data structure that wraps a receiving RS and collecting statistics about the data + * written to this newly assigned RS. */ - private final static class WriterAndPath { - final Path p; - final Writer w; + private final static class RegionServerWriter extends SinkWriter { + final WALEditsReplaySink sink; - /* Count of edits written to this path */ - long editsWritten = 0; - /* Number of nanos spent writing to this log */ - long nanosSpent = 0; - - WriterAndPath(final Path p, final Writer w) { - this.p = p; - this.w = w; + RegionServerWriter(final Configuration conf, final byte[] tableName, final HConnection conn) + throws IOException { + this.sink = new WALEditsReplaySink(conf, tableName, conn); } - void incrementEdits(int edits) { - editsWritten += edits; - } - - void incrementNanoTime(long nanos) { - nanosSpent += nanos; + void close() throws IOException { } } static class CorruptedLogFileException extends Exception { private static final long serialVersionUID = 1L; + CorruptedLogFileException(String s) { super(s); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java index 01e13c88813..973af645179 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Bytes; public class HLogUtil { static final Log LOG = LogFactory.getLog(HLogUtil.class); @@ -167,6 +168,37 @@ public class HLogUtil { return ServerName.parseServerName(serverName); } + /** + * This function returns region server name from a log file name which is in either format: + * hdfs:///hbase/.logs/-splitting/... or hdfs:///hbase/.logs//... + * @param logFile + * @return null if the passed in logFile isn't a valid HLog file path + */ + public static ServerName getServerNameFromHLogDirectoryName(Path logFile) { + Path logDir = logFile.getParent(); + String logDirName = logDir.getName(); + if (logDirName.equals(HConstants.HREGION_LOGDIR_NAME)) { + logDir = logFile; + logDirName = logDir.getName(); + } + ServerName serverName = null; + if (logDirName.endsWith(HLog.SPLITTING_EXT)) { + logDirName = logDirName.substring(0, logDirName.length() - HLog.SPLITTING_EXT.length()); + } + try { + serverName = ServerName.parseServerName(logDirName); + } catch (IllegalArgumentException ex) { + serverName = null; + LOG.warn("Invalid log file path=" + logFile, ex); + } + if (serverName != null && serverName.getStartcode() < 0) { + LOG.warn("Invalid log file path=" + logFile); + return null; + } + return serverName; + } + /** * Returns sorted set of edit files made by wal-log splitter, excluding files * with '.temp' suffix. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALEditsReplay.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALEditsReplay.java new file mode 100644 index 00000000000..d87ab5d91fb --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/MetricsWALEditsReplay.java @@ -0,0 +1,60 @@ +/** + * + * 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.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.util.StringUtils; + +/** + * 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 { + 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) { + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java index f5a47af914f..d8b163f02d2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java @@ -83,6 +83,7 @@ public class WALEdit implements Writable, HeapSize { static final byte[] COMPLETE_CACHE_FLUSH = Bytes.toBytes("HBASE::CACHEFLUSH"); static final byte[] COMPACTION = Bytes.toBytes("HBASE::COMPACTION"); private final int VERSION_2 = -1; + private final boolean isReplay; private final ArrayList kvs = new ArrayList(); @@ -93,6 +94,11 @@ public class WALEdit implements Writable, HeapSize { private CompressionContext compressionContext; public WALEdit() { + this(false); + } + + public WALEdit(boolean isReplay) { + this.isReplay = isReplay; } /** @@ -103,6 +109,14 @@ public class WALEdit implements Writable, HeapSize { return Bytes.equals(METAFAMILY, f); } + /** + * @return True when current WALEdit is created by log replay. Replication skips WALEdits from + * replay. + */ + public boolean isReplay() { + return this.isReplay; + } + public void setCompressionContext(final CompressionContext compressionContext) { this.compressionContext = compressionContext; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java new file mode 100644 index 00000000000..7434be37186 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEditsReplaySink.java @@ -0,0 +1,233 @@ +/** + * + * 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.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.client.Action; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.Row; +import org.apache.hadoop.hbase.client.ServerCallable; +import org.apache.hadoop.hbase.exceptions.DoNotRetryIOException; +import org.apache.hadoop.hbase.exceptions.NoSuchColumnFamilyException; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Pair; + +import com.google.protobuf.ServiceException; + +/** + * This class is responsible for replaying the edits coming from a failed region server. + *

+ * This class uses the native HBase client in order to replay WAL entries. + *

+ */ +@InterfaceAudience.Private +public class WALEditsReplaySink { + + private static final Log LOG = LogFactory.getLog(WALEditsReplaySink.class); + + private final Configuration conf; + private final HConnection conn; + private final byte[] tableName; + private final MetricsWALEditsReplay metrics; + private final AtomicLong totalReplayedEdits = new AtomicLong(); + private final boolean skipErrors; + + /** + * Create a sink for WAL log entries replay + * @param conf + * @param tableName + * @param conn + * @throws IOException + */ + public WALEditsReplaySink(Configuration conf, byte[] tableName, HConnection 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); + } + + /** + * Replay an array of actions of the same region directly into the newly assigned Region Server + * @param actions + * @throws IOException + */ + public void replayEntries(List> actions) throws IOException { + if (actions.size() == 0) { + return; + } + + int batchSize = actions.size(); + int dataSize = 0; + Map>> actionsByRegion = + new HashMap>>(); + HRegionLocation loc = null; + Row row = null; + List> regionActions = null; + // Build the action list. + for (int i = 0; i < batchSize; i++) { + loc = actions.get(i).getFirst(); + row = actions.get(i).getSecond(); + if (actionsByRegion.containsKey(loc.getRegionInfo())) { + regionActions = actionsByRegion.get(loc.getRegionInfo()); + } else { + regionActions = new ArrayList>(); + actionsByRegion.put(loc.getRegionInfo(), regionActions); + } + Action action = new Action(row, i); + regionActions.add(action); + dataSize += row.getRow().length; + } + + long startTime = EnvironmentEdgeManager.currentTimeMillis(); + + // replaying edits by region + for (HRegionInfo curRegion : actionsByRegion.keySet()) { + replayEdits(loc, curRegion, actionsByRegion.get(curRegion)); + } + + long endTime = EnvironmentEdgeManager.currentTimeMillis() - startTime; + LOG.debug("number of rows:" + actions.size() + " are sent by batch! spent " + endTime + + "(ms)!"); + + metrics.updateReplayTime(endTime); + metrics.updateReplayBatchSize(batchSize); + metrics.updateReplayDataSize(dataSize); + + 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> actions) throws IOException { + try { + ReplayServerCallable callable = new ReplayServerCallable( + this.conn, this.tableName, regionLoc, regionInfo, actions); + callable.withRetries(); + } 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 replay method call going against a single regionserver + * @param + */ + class ReplayServerCallable extends ServerCallable { + private HRegionInfo regionInfo; + private List> actions; + + private Map>>> retryActions = null; + + ReplayServerCallable(final HConnection connection, final byte [] tableName, + final HRegionLocation regionLoc, final HRegionInfo regionInfo, + final List> actions) { + super(connection, tableName, null); + this.actions = actions; + this.regionInfo = regionInfo; + this.location = regionLoc; + } + + @Override + public MultiResponse call() throws IOException { + try { + replayToServer(this.regionInfo, this.actions); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } + return null; + } + + private void replayToServer(HRegionInfo regionInfo, List> actions) + throws IOException, ServiceException { + AdminService.BlockingInterface remoteSvr = connection.getAdmin(location.getServerName()); + MultiRequest request = RequestConverter.buildMultiRequest(regionInfo.getRegionName(), + actions); + MultiResponse protoResults = remoteSvr.replay(null, request); + // check if it's a partial success + List resultList = protoResults.getResultList(); + for (int i = 0, n = resultList.size(); i < n; i++) { + ActionResult result = resultList.get(i); + if (result.hasException()) { + Throwable t = ProtobufUtil.toException(result.getException()); + if (!skipErrors) { + IOException ie = new IOException(); + ie.initCause(t); + // retry + throw ie; + } else { + LOG.warn(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + + "=true so continuing replayToServer with error:" + t.getMessage()); + return; + } + } + } + } + + @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 + for (Action action : actions) { + // use first row to relocate region because all actions are for one region + this.location = this.connection.locateRegion(tableName, action.getAction().getRow()); + break; + } + } + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java new file mode 100644 index 00000000000..a9fed572de8 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoveringRegionWatcher.java @@ -0,0 +1,95 @@ +/** + * + * 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.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +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 + */ + public void nodeDeleted(String path) { + if (this.server.isStopped() || this.server.isStopping()) { + return; + } + + String parentPath = path.substring(0, path.lastIndexOf('/')); + if (!this.watcher.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) { + return; + } + + String regionName = path.substring(parentPath.length() + 1); + HRegion region = this.server.getRecoveringRegions().remove(regionName); + if (region != null) { + region.setRecovering(false); + } + + LOG.info(path + " znode deleted. Region: " + regionName + " completes recovery."); + } + + @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.recoveringRegionsZNode.equalsIgnoreCase(parentPath)) { + return; + } + + try { + ZKUtil.getDataAndWatch(watcher, path); + } catch (KeeperException e) { + LOG.warn("Can't register watcher on znode " + path, e); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java index f41149f572d..b47ab58427e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java @@ -217,6 +217,7 @@ public class TestIOFencing { public void doTest(Class regionClass) throws Exception { Configuration c = TEST_UTIL.getConfiguration(); + c.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false); // Insert our custom region c.setClass(HConstants.REGION_IMPL, regionClass, HRegion.class); c.setBoolean("dfs.support.append", true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java index a3ea053b743..fd54d47e9ea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionServerCoprocessorExceptionWithAbort.java @@ -58,7 +58,7 @@ public class TestRegionServerCoprocessorExceptionWithAbort { conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2); // Let's fail fast. conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, BuggyRegionObserver.class.getName()); conf.set("hbase.coprocessor.abortonerror", "true"); - TEST_UTIL.startMiniCluster(2); + TEST_UTIL.startMiniCluster(3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index a60babe06d1..3067ff72893 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiGetResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest; @@ -520,4 +521,17 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { public ExecutorService getExecutorService() { return null; } + + @Override + public MultiResponse replay(RpcController controller, MultiRequest request) + throws ServiceException { + // TODO Auto-generated method stub + return null; + } + + @Override + public Map getRecoveringRegions() { + // TODO Auto-generated method stub + return null; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index 91ae3db73b3..d2ae888e070 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -331,6 +331,12 @@ public class TestCatalogJanitor { public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, boolean forcible) throws IOException { } + + @Override + public boolean isInitialized() { + // Auto-generated method stub + return false; + } } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java index d1e8832924b..5e0de324229 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java @@ -18,15 +18,25 @@ */ package org.apache.hadoop.hbase.master; -import static org.apache.hadoop.hbase.SplitLogCounters.*; +import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_wait_for_zk_delete; +import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_final_transition_failed; +import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_preempt_task; +import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_acquired; +import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_done; +import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_err; +import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_resigned; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.NavigableSet; +import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -42,7 +52,17 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.LargeTests; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.SplitLogCounters; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch; @@ -56,9 +76,12 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.ZKAssign; +import org.apache.hadoop.hbase.zookeeper.ZKTable; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -89,9 +112,14 @@ public class TestDistributedLogSplitting { HBaseTestingUtility TEST_UTIL; private void startCluster(int num_rs) throws Exception{ + conf = HBaseConfiguration.create(); + startCluster(num_rs, conf); + } + + private void startCluster(int num_rs, Configuration inConf) throws Exception { SplitLogCounters.resetCounters(); LOG.info("Starting cluster"); - conf = HBaseConfiguration.create(); + this.conf = inConf; conf.getLong("hbase.splitlog.max.resubmit", 0); // Make the failure test faster conf.setInt("zookeeper.recovery.retry", 0); @@ -111,13 +139,20 @@ public class TestDistributedLogSplitting { @After public void after() throws Exception { + for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) { + mt.getMaster().abort("closing...", new Exception("Trace info")); + } + TEST_UTIL.shutdownMiniCluster(); } @Test (timeout=300000) public void testRecoveredEdits() throws Exception { LOG.info("testRecoveredEdits"); - startCluster(NUM_RS); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false); + startCluster(NUM_RS, curConf); + final int NUM_LOG_LINES = 1000; final SplitLogManager slm = master.getMasterFileSystem().splitLogManager; // turn off load balancing to prevent regions from moving around otherwise @@ -150,8 +185,7 @@ public class TestDistributedLogSplitting { it.remove(); } } - makeHLog(hrs.getWAL(), regions, "table", - NUM_LOG_LINES, 100); + makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); slm.splitLogDistributed(logDir); @@ -172,6 +206,597 @@ public class TestDistributedLogSplitting { assertEquals(NUM_LOG_LINES, count); } + @Test(timeout = 300000) + public void testLogReplayWithNonMetaRSDown() throws Exception { + LOG.info("testLogReplayWithNonMetaRSDown"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setLong("hbase.regionserver.hlog.blocksize", 100*1024); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); + startCluster(NUM_RS, curConf); + final int NUM_REGIONS_TO_CREATE = 40; + final int NUM_LOG_LINES = 1000; + // turn off load balancing to prevent regions from moving around otherwise + // they will consume recovered.edits + master.balanceSwitch(false); + + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); + HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + + List regions = null; + HRegionServer hrs = null; + for (int i = 0; i < NUM_RS; i++) { + boolean isCarryingMeta = false; + hrs = rsts.get(i).getRegionServer(); + regions = ProtobufUtil.getOnlineRegions(hrs); + for (HRegionInfo region : regions) { + if (region.isMetaRegion()) { + isCarryingMeta = true; + break; + } + } + if (isCarryingMeta) { + continue; + } + break; + } + + LOG.info("#regions = " + regions.size()); + Iterator it = regions.iterator(); + while (it.hasNext()) { + HRegionInfo region = it.next(); + if (region.isMetaTable()) { + it.remove(); + } + } + makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + + // wait for abort completes + this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES); + ht.close(); + } + + @Test(timeout = 300000) + public void testLogReplayWithMetaRSDown() throws Exception { + LOG.info("testRecoveredEditsReplayWithMetaRSDown"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); + startCluster(NUM_RS, curConf); + final int NUM_REGIONS_TO_CREATE = 40; + final int NUM_LOG_LINES = 1000; + // turn off load balancing to prevent regions from moving around otherwise + // they will consume recovered.edits + master.balanceSwitch(false); + + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); + HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + + List regions = null; + HRegionServer hrs = null; + for (int i = 0; i < NUM_RS; i++) { + boolean isCarryingMeta = false; + hrs = rsts.get(i).getRegionServer(); + regions = ProtobufUtil.getOnlineRegions(hrs); + for (HRegionInfo region : regions) { + if (region.isMetaRegion()) { + isCarryingMeta = true; + break; + } + } + if (!isCarryingMeta) { + continue; + } + break; + } + + LOG.info("#regions = " + regions.size()); + Iterator it = regions.iterator(); + while (it.hasNext()) { + HRegionInfo region = it.next(); + if (region.isMetaTable()) { + it.remove(); + } + } + makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + + this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES); + ht.close(); + } + + private void abortRSAndVerifyRecovery(HRegionServer hrs, HTable ht, final ZooKeeperWatcher zkw, + final int numRegions, final int numofLines) throws Exception { + + abortRSAndWaitForRecovery(hrs, zkw, numRegions); + assertEquals(numofLines, TEST_UTIL.countRows(ht)); + } + + private void abortRSAndWaitForRecovery(HRegionServer hrs, final ZooKeeperWatcher zkw, + final int numRegions) throws Exception { + final MiniHBaseCluster tmpCluster = this.cluster; + + // abort RS + LOG.info("Aborting region server: " + hrs.getServerName()); + hrs.abort("testing"); + + // wait for abort completes + TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (tmpCluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1)); + } + }); + + // wait for regions come online + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (getAllOnlineRegions(tmpCluster).size() >= (numRegions + 1)); + } + }); + + // wait for all regions are fully recovered + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( + zkw.recoveringRegionsZNode, false); + return (recoveringRegions != null && recoveringRegions.size() == 0); + } + }); + } + + @Test(timeout = 300000) + public void testMasterStartsUpWithLogSplittingWork() throws Exception { + LOG.info("testMasterStartsUpWithLogSplittingWork"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false); + curConf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1); + startCluster(NUM_RS, curConf); + + final int NUM_REGIONS_TO_CREATE = 40; + final int NUM_LOG_LINES = 1000; + // turn off load balancing to prevent regions from moving around otherwise + // they will consume recovered.edits + master.balanceSwitch(false); + + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); + HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + + List regions = null; + HRegionServer hrs = null; + for (int i = 0; i < NUM_RS; i++) { + boolean isCarryingMeta = false; + hrs = rsts.get(i).getRegionServer(); + regions = ProtobufUtil.getOnlineRegions(hrs); + for (HRegionInfo region : regions) { + if (region.isMetaRegion()) { + isCarryingMeta = true; + break; + } + } + if (isCarryingMeta) { + continue; + } + break; + } + + LOG.info("#regions = " + regions.size()); + Iterator it = regions.iterator(); + while (it.hasNext()) { + HRegionInfo region = it.next(); + if (region.isMetaTable()) { + it.remove(); + } + } + makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + + // abort master + abortMaster(cluster); + + // abort RS + int numRS = cluster.getLiveRegionServerThreads().size(); + LOG.info("Aborting region server: " + hrs.getServerName()); + hrs.abort("testing"); + + // wait for abort completes + TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1)); + } + }); + + Thread.sleep(2000); + LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size()); + + startMasterAndWaitUntilLogSplit(cluster); + + // wait for abort completes + TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1)); + } + }); + + LOG.info("Current Open Regions After Master Node Starts Up:" + + getAllOnlineRegions(cluster).size()); + + assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht)); + + ht.close(); + } + + @Test(timeout = 300000) + public void testMasterStartsUpWithLogReplayWork() throws Exception { + LOG.info("testMasterStartsUpWithLogReplayWork"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); + curConf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1); + startCluster(NUM_RS, curConf); + + final int NUM_REGIONS_TO_CREATE = 40; + final int NUM_LOG_LINES = 1000; + // turn off load balancing to prevent regions from moving around otherwise + // they will consume recovered.edits + master.balanceSwitch(false); + + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); + HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + + List regions = null; + HRegionServer hrs = null; + for (int i = 0; i < NUM_RS; i++) { + boolean isCarryingMeta = false; + hrs = rsts.get(i).getRegionServer(); + regions = ProtobufUtil.getOnlineRegions(hrs); + for (HRegionInfo region : regions) { + if (region.isMetaRegion()) { + isCarryingMeta = true; + break; + } + } + if (isCarryingMeta) { + continue; + } + break; + } + + LOG.info("#regions = " + regions.size()); + Iterator it = regions.iterator(); + while (it.hasNext()) { + HRegionInfo region = it.next(); + if (region.isMetaTable()) { + it.remove(); + } + } + makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + + // abort master + abortMaster(cluster); + + // abort RS + int numRS = cluster.getLiveRegionServerThreads().size(); + LOG.info("Aborting region server: " + hrs.getServerName()); + hrs.abort("testing"); + + // wait for the RS dies + TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1)); + } + }); + + Thread.sleep(2000); + LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size()); + + startMasterAndWaitUntilLogSplit(cluster); + + // wait for all regions are fully recovered + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( + zkw.recoveringRegionsZNode, false); + return (recoveringRegions != null && recoveringRegions.size() == 0); + } + }); + + LOG.info("Current Open Regions After Master Node Starts Up:" + + getAllOnlineRegions(cluster).size()); + + assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht)); + + ht.close(); + } + + + @Test(timeout = 300000) + public void testLogReplayTwoSequentialRSDown() throws Exception { + LOG.info("testRecoveredEditsReplayTwoSequentialRSDown"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); + startCluster(NUM_RS, curConf); + final int NUM_REGIONS_TO_CREATE = 40; + final int NUM_LOG_LINES = 1000; + // turn off load balancing to prevent regions from moving around otherwise + // they will consume recovered.edits + master.balanceSwitch(false); + + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); + HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + + List regions = null; + HRegionServer hrs1 = rsts.get(0).getRegionServer(); + regions = ProtobufUtil.getOnlineRegions(hrs1); + + makeHLog(hrs1.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + + // abort RS1 + LOG.info("Aborting region server: " + hrs1.getServerName()); + hrs1.abort("testing"); + + // wait for abort completes + TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1)); + } + }); + + // wait for regions come online + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1)); + } + }); + + // sleep a little bit in order to interrupt recovering in the middle + Thread.sleep(300); + // abort second region server + rsts = cluster.getLiveRegionServerThreads(); + HRegionServer hrs2 = rsts.get(0).getRegionServer(); + LOG.info("Aborting one more region server: " + hrs2.getServerName()); + hrs2.abort("testing"); + + // wait for abort completes + TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 2)); + } + }); + + // wait for regions come online + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1)); + } + }); + + // wait for all regions are fully recovered + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( + zkw.recoveringRegionsZNode, false); + return (recoveringRegions != null && recoveringRegions.size() == 0); + } + }); + + assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht)); + ht.close(); + } + + @Test(timeout = 300000) + public void testMarkRegionsRecoveringInZK() throws Exception { + LOG.info("testMarkRegionsRecoveringInZK"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); + startCluster(NUM_RS, curConf); + master.balanceSwitch(false); + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = master.getZooKeeperWatcher(); + HTable ht = installTable(zkw, "table", "family", 40); + final SplitLogManager slm = master.getMasterFileSystem().splitLogManager; + + final HRegionServer hrs = rsts.get(0).getRegionServer(); + List regions = ProtobufUtil.getOnlineRegions(hrs); + HRegionInfo region = regions.get(0); + Set regionSet = new HashSet(); + regionSet.add(region); + slm.markRegionsRecoveringInZK(rsts.get(0).getRegionServer().getServerName(), regionSet); + slm.markRegionsRecoveringInZK(rsts.get(1).getRegionServer().getServerName(), regionSet); + + List recoveringRegions = ZKUtil.listChildrenNoWatch(zkw, + ZKUtil.joinZNode(zkw.recoveringRegionsZNode, region.getEncodedName())); + + assertEquals(recoveringRegions.size(), 2); + + // wait for splitLogWorker to mark them up because there is no WAL files recorded in ZK + TEST_UTIL.waitFor(60000, 1000, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (hrs.getRecoveringRegions().size() == 0); + } + }); + ht.close(); + } + + @Test(timeout = 300000) + public void testReplayCmd() throws Exception { + LOG.info("testReplayCmd"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); + startCluster(NUM_RS, curConf); + final int NUM_REGIONS_TO_CREATE = 40; + // turn off load balancing to prevent regions from moving around otherwise + // they will consume recovered.edits + master.balanceSwitch(false); + + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); + HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); + + List regions = null; + HRegionServer hrs = null; + for (int i = 0; i < NUM_RS; i++) { + boolean isCarryingMeta = false; + hrs = rsts.get(i).getRegionServer(); + regions = ProtobufUtil.getOnlineRegions(hrs); + for (HRegionInfo region : regions) { + if (region.isMetaRegion()) { + isCarryingMeta = true; + break; + } + } + if (isCarryingMeta) { + continue; + } + break; + } + + LOG.info("#regions = " + regions.size()); + Iterator it = regions.iterator(); + while (it.hasNext()) { + HRegionInfo region = it.next(); + if (region.isMetaTable()) { + it.remove(); + } + } + this.prepareData(ht, Bytes.toBytes("family"), Bytes.toBytes("c1")); + String originalCheckSum = TEST_UTIL.checksumRows(ht); + + // abort RA and trigger replay + abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE); + + assertEquals("Data should remain after reopening of regions", originalCheckSum, + TEST_UTIL.checksumRows(ht)); + + ht.close(); + } + + @Test(timeout = 300000) + public void testLogReplayForDisablingTable() throws Exception { + LOG.info("testLogReplayWithNonMetaRSDown"); + Configuration curConf = HBaseConfiguration.create(); + curConf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true); + startCluster(NUM_RS, curConf); + final int NUM_REGIONS_TO_CREATE = 40; + final int NUM_LOG_LINES = 1000; + // turn off load balancing to prevent regions from moving around otherwise + // they will consume recovered.edits + master.balanceSwitch(false); + + List rsts = cluster.getLiveRegionServerThreads(); + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null); + HTable disablingHT = installTable(zkw, "disableTable", "family", NUM_REGIONS_TO_CREATE); + HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE, NUM_REGIONS_TO_CREATE); + + List regions = null; + HRegionServer hrs = null; + for (int i = 0; i < NUM_RS; i++) { + boolean isCarryingMeta = false; + hrs = rsts.get(i).getRegionServer(); + regions = ProtobufUtil.getOnlineRegions(hrs); + for (HRegionInfo region : regions) { + if (region.isMetaRegion()) { + isCarryingMeta = true; + break; + } + } + if (isCarryingMeta) { + continue; + } + break; + } + + LOG.info("#regions = " + regions.size()); + Iterator it = regions.iterator(); + while (it.hasNext()) { + HRegionInfo region = it.next(); + if (region.isMetaTable()) { + it.remove(); + } + } + makeHLog(hrs.getWAL(), regions, "disableTable", "family", NUM_LOG_LINES, 100, false); + makeHLog(hrs.getWAL(), regions, "table", "family", NUM_LOG_LINES, 100); + + LOG.info("Disabling table\n"); + TEST_UTIL.getHBaseAdmin().disableTable(Bytes.toBytes("disableTable")); + + // abort RS + LOG.info("Aborting region server: " + hrs.getServerName()); + hrs.abort("testing"); + + // wait for abort completes + TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1)); + } + }); + + // wait for regions come online + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + return (getAllOnlineRegions(cluster).size() >= (NUM_REGIONS_TO_CREATE + 1)); + } + }); + + // wait for all regions are fully recovered + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( + zkw.recoveringRegionsZNode, false); + return (recoveringRegions != null && recoveringRegions.size() == 0); + } + }); + + int count = 0; + FileSystem fs = master.getMasterFileSystem().getFileSystem(); + Path rootdir = FSUtils.getRootDir(conf); + Path tdir = HTableDescriptor.getTableDir(rootdir, Bytes.toBytes("disableTable")); + for (HRegionInfo hri : regions) { + @SuppressWarnings("deprecation") + Path editsdir = + HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); + LOG.debug("checking edits dir " + editsdir); + FileStatus[] files = fs.listStatus(editsdir); + if(files != null) { + for(FileStatus file : files) { + int c = countHLog(file.getPath(), fs, conf); + count += c; + LOG.info(c + " edits in " + file.getPath()); + } + } + } + + LOG.info("Verify edits in recovered.edits files"); + assertEquals(NUM_LOG_LINES, count); + LOG.info("Verify replayed edits"); + assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht)); + + // clean up + for (HRegionInfo hri : regions) { + @SuppressWarnings("deprecation") + Path editsdir = + HLogUtil.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir, hri.getEncodedName())); + fs.delete(editsdir, true); + } + disablingHT.close(); + ht.close(); + } + /** * The original intention of this test was to force an abort of a region * server and to make sure that the failure path in the region servers is @@ -197,8 +822,9 @@ public class TestDistributedLogSplitting { installTable(new ZooKeeperWatcher(conf, "table-creation", null), "table", "family", 40); - makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs), "table", - NUM_LOG_LINES, 100); + + makeHLog(hrs.getWAL(), ProtobufUtil.getOnlineRegions(hrs), "table", "family", NUM_LOG_LINES, + 100); new Thread() { public void run() { @@ -244,7 +870,7 @@ public class TestDistributedLogSplitting { startCluster(NUM_RS); // NUM_RS=6. - ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, + final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "distributed log splitting test", null); HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE); @@ -273,6 +899,16 @@ public class TestDistributedLogSplitting { Thread.sleep(200); } + // wait for all regions are fully recovered + TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() { + @Override + public boolean evaluate() throws Exception { + List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren( + zkw.recoveringRegionsZNode, false); + return (recoveringRegions != null && recoveringRegions.size() == 0); + } + }); + assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION, TEST_UTIL.countRows(ht)); ht.close(); @@ -344,8 +980,12 @@ public class TestDistributedLogSplitting { } } - HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, - int nrs ) throws Exception { + HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception { + return installTable(zkw, tname, fname, nrs, 0); + } + + HTable installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs, + int existingRegions) throws Exception { // Create a table with regions byte [] table = Bytes.toBytes(tname); byte [] family = Bytes.toBytes(fname); @@ -367,14 +1007,14 @@ public class TestDistributedLogSplitting { for (String oregion : regions) LOG.debug("Region still online: " + oregion); } - assertEquals(1, regions.size()); + assertEquals(1 + existingRegions, regions.size()); LOG.debug("Enabling table\n"); TEST_UTIL.getHBaseAdmin().enableTable(table); LOG.debug("Waiting for no more RIT\n"); blockUntilNoRIT(zkw, master); LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n"); regions = getAllOnlineRegions(cluster); - assertEquals(numRegions + 1, regions.size()); + assertEquals(numRegions + 1 + existingRegions, regions.size()); return ht; } @@ -400,40 +1040,57 @@ public class TestDistributedLogSplitting { } } - public void makeHLog(HLog log, - List hris, String tname, + public void makeHLog(HLog log, List regions, String tname, String fname, int num_edits, int edit_size) throws IOException { + makeHLog(log, regions, tname, fname, num_edits, edit_size, true); + } + + public void makeHLog(HLog log, List regions, String tname, String fname, + int num_edits, int edit_size, boolean closeLog) throws IOException { // remove root and meta region - hris.remove(HRegionInfo.ROOT_REGIONINFO); - hris.remove(HRegionInfo.FIRST_META_REGIONINFO); + regions.remove(HRegionInfo.FIRST_META_REGIONINFO); byte[] table = Bytes.toBytes(tname); HTableDescriptor htd = new HTableDescriptor(tname); byte[] value = new byte[edit_size]; + + List hris = new ArrayList(); + for (HRegionInfo region : regions) { + if (!region.getTableNameAsString().equalsIgnoreCase(tname)) { + continue; + } + hris.add(region); + } for (int i = 0; i < edit_size; i++) { - value[i] = (byte)('a' + (i % 26)); + value[i] = (byte) ('a' + (i % 26)); } int n = hris.size(); int[] counts = new int[n]; - int j = 0; if (n > 0) { for (int i = 0; i < num_edits; i += 1) { WALEdit e = new WALEdit(); - byte [] row = Bytes.toBytes("r" + Integer.toString(i)); - byte [] family = Bytes.toBytes("f"); - byte [] qualifier = Bytes.toBytes("c" + Integer.toString(i)); - e.add(new KeyValue(row, family, qualifier, - System.currentTimeMillis(), value)); - j++; - log.append(hris.get(j % n), table, e, System.currentTimeMillis(), htd); - counts[j % n] += 1; + HRegionInfo curRegionInfo = hris.get(i % n); + byte[] startRow = curRegionInfo.getStartKey(); + if (startRow == null || startRow.length == 0) { + startRow = new byte[] { 0, 0, 0, 0, 1 }; + } + byte[] row = Bytes.incrementBytes(startRow, counts[i % n]); + row = Arrays.copyOfRange(row, 3, 8); // use last 5 bytes because + // HBaseTestingUtility.createMultiRegions use 5 bytes + // key + byte[] family = Bytes.toBytes(fname); + byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i)); + e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value)); + log.append(curRegionInfo, table, e, System.currentTimeMillis(), htd); + counts[i % n] += 1; } } log.sync(); - log.close(); + if(closeLog) { + log.close(); + } for (int i = 0; i < n; i++) { - LOG.info("region " + hris.get(i).getRegionNameAsString() + - " has " + counts[i] + " edits"); + LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits"); } return; } @@ -466,6 +1123,38 @@ public class TestDistributedLogSplitting { } } + /** + * Load table with puts and deletes with expected values so that we can verify later + */ + private void prepareData(final HTable t, final byte[] f, final byte[] column) throws IOException { + t.setAutoFlush(false); + byte[] k = new byte[3]; + + // add puts + for (byte b1 = 'a'; b1 <= 'z'; b1++) { + for (byte b2 = 'a'; b2 <= 'z'; b2++) { + for (byte b3 = 'a'; b3 <= 'z'; b3++) { + k[0] = b1; + k[1] = b2; + k[2] = b3; + Put put = new Put(k); + put.add(f, column, k); + t.put(put); + } + } + } + t.flushCommits(); + // add deletes + for (byte b3 = 'a'; b3 <= 'z'; b3++) { + k[0] = 'a'; + k[1] = 'a'; + k[2] = b3; + Delete del = new Delete(k); + t.delete(del); + } + t.flushCommits(); + } + private NavigableSet getAllOnlineRegions(MiniHBaseCluster cluster) throws IOException { NavigableSet online = new TreeSet(); @@ -493,4 +1182,27 @@ public class TestDistributedLogSplitting { assertTrue(false); } + private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException { + for (MasterThread mt : cluster.getLiveMasterThreads()) { + if (mt.getMaster().isActiveMaster()) { + mt.getMaster().abort("Aborting for tests", new Exception("Trace info")); + mt.join(); + break; + } + } + LOG.debug("Master is aborted"); + } + + private void startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster) + throws IOException, InterruptedException { + cluster.startMaster(); + HMaster master = cluster.getMaster(); + while (!master.isInitialized()) { + Thread.sleep(100); + } + ServerManager serverManager = master.getServerManager(); + while (serverManager.areDeadServersInProgress()) { + Thread.sleep(100); + } + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java index e0bb72454d8..37cac60cd12 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java @@ -783,6 +783,7 @@ public class TestMasterFailover { while (master.getServerManager().areDeadServersInProgress()) { Thread.sleep(10); } + // Failover should be completed, now wait for no RIT log("Waiting for no more RIT"); ZKAssign.blockUntilNoRIT(zkw); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java index 0781d1190da..f012e9d18f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystem.java @@ -18,14 +18,28 @@ 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.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.SplitLogTask; 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; @@ -63,4 +77,40 @@ public class TestMasterFileSystem { assertEquals(masterRoot, rootDir); } + @Test + public void testRemoveStaleRecoveringRegionsDuringMasterInitialization() throws Exception { + LOG.info("Starting testRemoveStaleRecoveringRegionsDuringMasterInitialization"); + HMaster master = UTIL.getMiniHBaseCluster().getMaster(); + MasterFileSystem fs = master.getMasterFileSystem(); + + String failedRegion = "failedRegoin1"; + String staleRegion = "staleRegion"; + ServerName inRecoveryServerName = new ServerName("mgr,1,1"); + ServerName previouselyFaildServerName = new ServerName("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.recoveringRegionsZNode, staleRegion); + ZKUtil.createWithParents(zkw, staleRegionPath); + String inRecoveringRegionPath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, failedRegion); + inRecoveringRegionPath = ZKUtil.joinZNode(inRecoveringRegionPath, + inRecoveryServerName.getServerName()); + ZKUtil.createWithParents(zkw, inRecoveringRegionPath); + Set servers = new HashSet(); + servers.add(previouselyFaildServerName); + fs.removeStaleRecoveringRegionsFromZK(servers); + + // verification + assertFalse(ZKUtil.checkExists(zkw, staleRegionPath) != -1); + assertTrue(ZKUtil.checkExists(zkw, inRecoveringRegionPath) != -1); + + ZKUtil.deleteChildrenRecursively(zkw, zkw.recoveringRegionsZNode); + ZKUtil.deleteChildrenRecursively(zkw, zkw.splitLogZNode); + zkw.close(); + } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java index e237b167a5d..ee8081480dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java @@ -355,8 +355,8 @@ public class TestMasterNoCluster { HMaster master = new HMaster(conf) { @Override - boolean assignMeta(MonitoredTask status) { - return true; + void assignMeta(MonitoredTask status) { + return; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java index 52091bdd6eb..c1d0f3daede 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSKilledWhenMasterInitializing.java @@ -101,23 +101,6 @@ public class TestRSKilledWhenMasterInitializing { KeeperException, InterruptedException { super(conf); } - - @Override - protected void splitLogAfterStartup(MasterFileSystem mfs) { - super.splitLogAfterStartup(mfs); - logSplit = true; - // If "TestingMaster.sleep" is set, sleep after log split. - if (getConfiguration().getBoolean("TestingMaster.sleep", false)) { - int duration = getConfiguration().getInt( - "TestingMaster.sleep.duration", 0); - Threads.sleep(duration); - } - } - - - public boolean isLogSplitAfterStartup() { - return logSplit; - } } @Test(timeout = 120000) @@ -163,7 +146,7 @@ public class TestRSKilledWhenMasterInitializing { /* NO.1 .META. region correctness */ // First abort master abortMaster(cluster); - TestingMaster master = startMasterAndWaitUntilLogSplit(cluster); + TestingMaster master = startMasterAndWaitTillMetaRegionAssignment(cluster); // Second kill meta server int metaServerNum = cluster.getServerWithMeta(); @@ -216,14 +199,12 @@ public class TestRSKilledWhenMasterInitializing { LOG.debug("Master is aborted"); } - private TestingMaster startMasterAndWaitUntilLogSplit(MiniHBaseCluster cluster) + private TestingMaster startMasterAndWaitTillMetaRegionAssignment(MiniHBaseCluster cluster) throws IOException, InterruptedException { TestingMaster master = (TestingMaster) cluster.startMaster().getMaster(); - while (!master.isLogSplitAfterStartup()) { + while (!master.isInitializationStartsMetaRegionAssignment()) { Thread.sleep(100); } - LOG.debug("splitted:" + master.isLogSplitAfterStartup() + ",initialized:" - + master.isInitialized()); return master; } @@ -232,7 +213,9 @@ public class TestRSKilledWhenMasterInitializing { while (!master.isInitialized()) { Thread.sleep(100); } + while (master.getServerManager().areDeadServersInProgress()) { + Thread.sleep(100); + } LOG.debug("master isInitialized"); } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java index 83ef82d66c4..40bd7a138a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java @@ -1221,7 +1221,7 @@ public class TestHLogSplit { generateHLogs(1, 10, -1); FileStatus logfile = fs.listStatus(HLOGDIR)[0]; fs.initialize(fs.getUri(), conf); - HLogSplitter.splitLogFile(HBASEDIR, logfile, fs, conf, reporter, null); + HLogSplitter.splitLogFile(HBASEDIR, logfile, fs, conf, reporter); HLogSplitter.finishSplitLogFile(HBASEDIR, OLDLOGDIR, logfile.getPath() .toString(), conf); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java index 856575b7c29..1d2c128f497 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockRegionServerServices.java @@ -191,4 +191,10 @@ public class MockRegionServerServices implements RegionServerServices { public ExecutorService getExecutorService() { return null; } + + @Override + public Map getRecoveringRegions() { + // TODO Auto-generated method stub + return null; + } }