diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index 331f2d1..39542e4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; @@ -2657,6 +2658,44 @@ public interface Admin extends Abortable, Closeable { List listReplicationPeers(Pattern pattern) throws IOException; /** + * Transit current cluster to a new state in a synchronous replication peer. + * @param peerId a short name that identifies the peer + * @param state a new state of current cluster + * @throws IOException if a remote or network exception occurs + */ + void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state) + throws IOException; + + /** + * Transit current cluster to a new state in a synchronous replication peer. But does not block + * and wait for it. + *

+ * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @param state a new state of current cluster + * @throws IOException if a remote or network exception occurs + */ + Future transitReplicationPeerSyncReplicationStateAsync(String peerId, + SyncReplicationState state) throws IOException; + + /** + * Get the current cluster state in a synchronous replication peer. + * @param peerId a short name that identifies the peer + * @return the current cluster state + * @throws IOException if a remote or network exception occurs + */ + default SyncReplicationState getReplicationPeerSyncReplicationState(String peerId) + throws IOException { + List peers = listReplicationPeers(Pattern.compile(peerId)); + if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) { + throw new IOException("Replication peer " + peerId + " does not exist"); + } + return peers.get(0).getSyncReplicationState(); + } + + /** * Mark region server(s) as decommissioned to prevent additional regions from getting * assigned to them. Optionally unload the regions on the servers. If there are multiple servers * to be decommissioned, decommissioning them at the same time can prevent wasteful region diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java index 8141e74..65cccf7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client; import com.google.protobuf.RpcChannel; +import java.io.IOException; import java.util.Collection; import java.util.EnumSet; import java.util.List; @@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter; import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.yetus.audience.InterfaceAudience; /** @@ -600,6 +602,35 @@ public interface AsyncAdmin { ReplicationPeerConfig peerConfig); /** + * Transit current cluster to a new state in a synchronous replication peer. + * @param peerId a short name that identifies the peer + * @param state a new state of current cluster + */ + CompletableFuture transitReplicationPeerSyncReplicationState(String peerId, + SyncReplicationState state); + + /** + * Get the current cluster state in a synchronous replication peer. + * @param peerId a short name that identifies the peer + * @return the current cluster state wrapped by a {@link CompletableFuture}. + */ + default CompletableFuture + getReplicationPeerSyncReplicationState(String peerId) { + CompletableFuture future = new CompletableFuture<>(); + listReplicationPeers(Pattern.compile(peerId)).whenComplete((peers, error) -> { + if (error != null) { + future.completeExceptionally(error); + } else if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) { + future.completeExceptionally( + new IOException("Replication peer " + peerId + " does not exist")); + } else { + future.complete(peers.get(0).getSyncReplicationState()); + } + }); + return future; + } + + /** * Append the replicable table-cf config of the specified peer * @param peerId a short that identifies the cluster * @param tableCfs A map from tableName to column family names diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java index 5b22668..08952cb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter; import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.yetus.audience.InterfaceAudience; /** @@ -414,6 +415,12 @@ class AsyncHBaseAdmin implements AsyncAdmin { } @Override + public CompletableFuture transitReplicationPeerSyncReplicationState(String peerId, + SyncReplicationState clusterState) { + return wrap(rawAdmin.transitReplicationPeerSyncReplicationState(peerId, clusterState)); + } + + @Override public CompletableFuture appendReplicationPeerTableCFs(String peerId, Map> tableCfs) { return wrap(rawAdmin.appendReplicationPeerTableCFs(peerId, tableCfs)); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 1176cbd..f78005f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -124,6 +124,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; @@ -1717,6 +1719,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable { MasterProtos.ClearDeadServersRequest request) throws ServiceException { return stub.clearDeadServers(controller, request); } + + @Override + public TransitReplicationPeerSyncReplicationStateResponse + transitReplicationPeerSyncReplicationState(RpcController controller, + TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException { + return stub.transitReplicationPeerSyncReplicationState(controller, request); + } }; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 07e1be4..ac8d972 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; @@ -206,6 +207,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Disab import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; @@ -4014,6 +4016,30 @@ public class HBaseAdmin implements Admin { } @Override + public void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state) + throws IOException { + get(transitReplicationPeerSyncReplicationStateAsync(peerId, state), this.syncWaitTimeout, + TimeUnit.MILLISECONDS); + } + + @Override + public Future transitReplicationPeerSyncReplicationStateAsync(String peerId, + SyncReplicationState state) throws IOException { + TransitReplicationPeerSyncReplicationStateResponse response = + executeCallable(new MasterCallable( + getConnection(), getRpcControllerFactory()) { + @Override + protected TransitReplicationPeerSyncReplicationStateResponse rpcCall() throws Exception { + return master.transitReplicationPeerSyncReplicationState(getRpcController(), + RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId, + state)); + } + }); + return new ReplicationFuture(this, peerId, response.getProcId(), + () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"); + } + + @Override public void appendReplicationPeerTableCFs(String id, Map> tableCfs) throws ReplicationException, IOException { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 0fd0e59..963cca7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.quotas.QuotaTableUtil; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; @@ -255,6 +256,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos; @@ -1613,6 +1616,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { } @Override + public CompletableFuture transitReplicationPeerSyncReplicationState(String peerId, + SyncReplicationState clusterState) { + return this + . procedureCall( + RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId, + clusterState), + (s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done), + (resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId, + () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE")); + } + + @Override public CompletableFuture appendReplicationPeerTableCFs(String id, Map> tableCfs) { if (tableCfs == null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java index 50690b4..7bb65d2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java @@ -166,6 +166,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; @@ -638,4 +640,11 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection { throws ServiceException { return stub.splitRegion(controller, request); } + + @Override + public TransitReplicationPeerSyncReplicationStateResponse + transitReplicationPeerSyncReplicationState(RpcController controller, + TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException { + return stub.transitReplicationPeerSyncReplicationState(controller, request); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java index b1c1713..331795c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -319,6 +320,9 @@ public final class ReplicationPeerConfigUtil { excludeNamespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet())); } + if (peer.hasRemoteWALDir()) { + builder.setRemoteWALDir(peer.getRemoteWALDir()); + } return builder.build(); } @@ -376,6 +380,9 @@ public final class ReplicationPeerConfigUtil { } } + if (peerConfig.getRemoteWALDir() != null) { + builder.setRemoteWALDir(peerConfig.getRemoteWALDir()); + } return builder.build(); } @@ -390,28 +397,47 @@ public final class ReplicationPeerConfigUtil { return ProtobufUtil.prependPBMagic(bytes); } - public static ReplicationPeerDescription toReplicationPeerDescription( - ReplicationProtos.ReplicationPeerDescription desc) { - boolean enabled = ReplicationProtos.ReplicationState.State.ENABLED == desc.getState() - .getState(); + public static ReplicationPeerDescription + toReplicationPeerDescription(ReplicationProtos.ReplicationPeerDescription desc) { + boolean enabled = + ReplicationProtos.ReplicationState.State.ENABLED == desc.getState().getState(); ReplicationPeerConfig config = convert(desc.getConfig()); - return new ReplicationPeerDescription(desc.getId(), enabled, config); + return new ReplicationPeerDescription(desc.getId(), enabled, config, + toSyncReplicationState(desc.getSyncReplicationState())); } - public static ReplicationProtos.ReplicationPeerDescription toProtoReplicationPeerDescription( - ReplicationPeerDescription desc) { + public static ReplicationProtos.ReplicationPeerDescription + toProtoReplicationPeerDescription(ReplicationPeerDescription desc) { ReplicationProtos.ReplicationPeerDescription.Builder builder = ReplicationProtos.ReplicationPeerDescription.newBuilder(); builder.setId(desc.getPeerId()); - ReplicationProtos.ReplicationState.Builder stateBuilder = ReplicationProtos.ReplicationState - .newBuilder(); - stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED - : ReplicationProtos.ReplicationState.State.DISABLED); + + ReplicationProtos.ReplicationState.Builder stateBuilder = + ReplicationProtos.ReplicationState.newBuilder(); + stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED : + ReplicationProtos.ReplicationState.State.DISABLED); builder.setState(stateBuilder.build()); + builder.setConfig(convert(desc.getPeerConfig())); + builder.setSyncReplicationState(toSyncReplicationState(desc.getSyncReplicationState())); + return builder.build(); } + public static ReplicationProtos.SyncReplicationState + toSyncReplicationState(SyncReplicationState state) { + ReplicationProtos.SyncReplicationState.Builder syncReplicationStateBuilder = + ReplicationProtos.SyncReplicationState.newBuilder(); + syncReplicationStateBuilder + .setState(ReplicationProtos.SyncReplicationState.State.forNumber(state.ordinal())); + return syncReplicationStateBuilder.build(); + } + + public static SyncReplicationState + toSyncReplicationState(ReplicationProtos.SyncReplicationState state) { + return SyncReplicationState.valueOf(state.getState().getNumber()); + } + public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig( Map> tableCfs, ReplicationPeerConfig peerConfig) { ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(peerConfig); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java index e0d9a4c..cc7b4bc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication; import java.util.Collection; @@ -25,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -47,6 +47,8 @@ public class ReplicationPeerConfig { private Set excludeNamespaces = null; private long bandwidth = 0; private final boolean serial; + // Used by synchronous replication + private String remoteWALDir; private ReplicationPeerConfig(ReplicationPeerConfigBuilderImpl builder) { this.clusterKey = builder.clusterKey; @@ -66,6 +68,7 @@ public class ReplicationPeerConfig { : null; this.bandwidth = builder.bandwidth; this.serial = builder.serial; + this.remoteWALDir = builder.remoteWALDir; } private Map> @@ -213,6 +216,17 @@ public class ReplicationPeerConfig { return this; } + public String getRemoteWALDir() { + return this.remoteWALDir; + } + + /** + * Use remote wal dir to decide whether a peer is sync replication peer + */ + public boolean isSyncReplication() { + return !StringUtils.isBlank(this.remoteWALDir); + } + public static ReplicationPeerConfigBuilder newBuilder() { return new ReplicationPeerConfigBuilderImpl(); } @@ -230,7 +244,8 @@ public class ReplicationPeerConfig { .setReplicateAllUserTables(peerConfig.replicateAllUserTables()) .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap()) .setExcludeNamespaces(peerConfig.getExcludeNamespaces()) - .setBandwidth(peerConfig.getBandwidth()).setSerial(peerConfig.isSerial()); + .setBandwidth(peerConfig.getBandwidth()).setSerial(peerConfig.isSerial()) + .setRemoteWALDir(peerConfig.getRemoteWALDir()); return builder; } @@ -259,6 +274,8 @@ public class ReplicationPeerConfig { private boolean serial = false; + private String remoteWALDir = null; + @Override public ReplicationPeerConfigBuilder setClusterKey(String clusterKey) { this.clusterKey = clusterKey; @@ -327,6 +344,11 @@ public class ReplicationPeerConfig { return this; } + public ReplicationPeerConfigBuilder setRemoteWALDir(String dir) { + this.remoteWALDir = dir; + return this; + } + @Override public ReplicationPeerConfig build() { // It would be nice to validate the configuration, but we have to work with "old" data @@ -357,6 +379,9 @@ public class ReplicationPeerConfig { } builder.append("bandwidth=").append(bandwidth).append(","); builder.append("serial=").append(serial); + if (this.remoteWALDir != null) { + builder.append(",remoteWALDir=").append(remoteWALDir); + } return builder.toString(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java index 4c531c5..58ff220 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java @@ -150,6 +150,13 @@ public interface ReplicationPeerConfigBuilder { ReplicationPeerConfigBuilder setSerial(boolean serial); /** + * Set the remote peer cluster's wal directory. Used by synchronous replication. + * @param dir the remote peer cluster's wal directory + * @return {@code this} + */ + ReplicationPeerConfigBuilder setRemoteWALDir(String dir); + + /** * Builds the configuration object from the current state of {@code this}. * @return A {@link ReplicationPeerConfig} instance. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java index ba97d07..b0c27bb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java @@ -20,7 +20,10 @@ package org.apache.hadoop.hbase.replication; import org.apache.yetus.audience.InterfaceAudience; /** - * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription + * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription. + *

+ * To developer, here we do not store the new sync replication state since it is just an + * intermediate state and this class is public. */ @InterfaceAudience.Public public class ReplicationPeerDescription { @@ -28,11 +31,14 @@ public class ReplicationPeerDescription { private final String id; private final boolean enabled; private final ReplicationPeerConfig config; + private final SyncReplicationState syncReplicationState; - public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config) { + public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config, + SyncReplicationState syncReplicationState) { this.id = id; this.enabled = enabled; this.config = config; + this.syncReplicationState = syncReplicationState; } public String getPeerId() { @@ -47,11 +53,16 @@ public class ReplicationPeerDescription { return this.config; } + public SyncReplicationState getSyncReplicationState() { + return this.syncReplicationState; + } + @Override public String toString() { StringBuilder builder = new StringBuilder("id : ").append(id); builder.append(", enabled : " + enabled); builder.append(", config : " + config); + builder.append(", syncReplicationState : " + syncReplicationState); return builder.toString(); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java new file mode 100644 index 0000000..448603c --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java @@ -0,0 +1,77 @@ +/** + * 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.replication; + +import java.util.Arrays; +import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; + +/** + * Used by synchronous replication. Indicate the state of the current cluster in a synchronous + * replication peer. The state may be one of {@link SyncReplicationState#ACTIVE}, + * {@link SyncReplicationState#DOWNGRADE_ACTIVE} or {@link SyncReplicationState#STANDBY}. + *

+ * For asynchronous replication, the state is {@link SyncReplicationState#NONE}. + */ +@InterfaceAudience.Public +public enum SyncReplicationState { + NONE(0), ACTIVE(1), DOWNGRADE_ACTIVE(2), STANDBY(3); + + private final byte value; + + private SyncReplicationState(int value) { + this.value = (byte) value; + } + + public static SyncReplicationState valueOf(int value) { + switch (value) { + case 0: + return NONE; + case 1: + return ACTIVE; + case 2: + return DOWNGRADE_ACTIVE; + case 3: + return STANDBY; + default: + throw new IllegalArgumentException("Unknown synchronous replication state " + value); + } + } + + public int value() { + return value & 0xFF; + } + + public static byte[] toByteArray(SyncReplicationState state) { + return ProtobufUtil + .prependPBMagic(ReplicationPeerConfigUtil.toSyncReplicationState(state).toByteArray()); + } + + public static SyncReplicationState parseFrom(byte[] bytes) throws InvalidProtocolBufferException { + if (bytes == null) { + return SyncReplicationState.NONE; + } + return ReplicationPeerConfigUtil.toSyncReplicationState(ReplicationProtos.SyncReplicationState + .parseFrom(Arrays.copyOfRange(bytes, ProtobufUtil.lengthOfPBMagic(), bytes.length))); + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java index fc037a8..db07bab 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; @@ -147,6 +148,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Enabl import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; /** @@ -1867,4 +1869,11 @@ public final class RequestConverter { } return pbServers; } + + public static TransitReplicationPeerSyncReplicationStateRequest + buildTransitReplicationPeerSyncReplicationStateRequest(String peerId, + SyncReplicationState state) { + return TransitReplicationPeerSyncReplicationStateRequest.newBuilder().setPeerId(peerId) + .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build(); + } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationState.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationState.java new file mode 100644 index 0000000..6921252 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationState.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.replication; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ClientTests.class, SmallTests.class }) +public class TestSyncReplicationState { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationState.class); + + @Test + public void testSyncReplicationStateParseFrom() throws Exception { + Assert.assertEquals(SyncReplicationState.parseFrom(null), SyncReplicationState.NONE); + for (SyncReplicationState state : SyncReplicationState.values()) { + byte[] data = SyncReplicationState.toByteArray(state); + SyncReplicationState actualState = SyncReplicationState.parseFrom(data); + Assert.assertEquals(state, actualState); + } + } +} diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java index 0487ac5b..dbb9981 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java @@ -15,16 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -// TODO: Not used yet +/** + * Indicate that a procedure wants to be rescheduled. Usually because there are something wrong but + * we do not want to fail the procedure. + *

+ * TODO: need to support scheduling after a delay. + */ @InterfaceAudience.Private @InterfaceStability.Stable public class ProcedureYieldException extends ProcedureException { + /** default constructor */ public ProcedureYieldException() { super(); diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto index 3a236c0..c2ab180 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Master.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto @@ -962,6 +962,10 @@ service MasterService { rpc ListReplicationPeers(ListReplicationPeersRequest) returns(ListReplicationPeersResponse); + /** Transit the state of current cluster in a synchronous replication peer */ + rpc TransitReplicationPeerSyncReplicationState(TransitReplicationPeerSyncReplicationStateRequest) + returns(TransitReplicationPeerSyncReplicationStateResponse); + /** Returns a list of ServerNames marked as decommissioned. */ rpc ListDecommissionedRegionServers(ListDecommissionedRegionServersRequest) returns(ListDecommissionedRegionServersResponse); diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index 39d2824..a062e9a 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -391,6 +391,21 @@ enum PeerModificationState { POST_PEER_MODIFICATION = 8; } +enum PeerSyncReplicationStateTransitionState { + PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION = 1; + SET_PEER_NEW_SYNC_REPLICATION_STATE = 2; + REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_BEGIN = 3; + REPLAY_REMOTE_WAL_IN_PEER = 4; + REMOVE_ALL_REPLICATION_QUEUES_IN_PEER = 5; + REOPEN_ALL_REGIONS_IN_PEER = 6; + TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE = 7; + REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END = 8; + SYNC_REPLICATION_SET_PEER_ENABLED = 9; + SYNC_REPLICATION_ENABLE_PEER_REFRESH_PEER_ON_RS = 10; + CREATE_DIR_FOR_REMOTE_WAL = 11; + POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 12; +} + message PeerModificationStateData { required string peer_id = 1; } @@ -401,18 +416,23 @@ enum PeerModificationType { ENABLE_PEER = 3; DISABLE_PEER = 4; UPDATE_PEER_CONFIG = 5; + TRANSIT_SYNC_REPLICATION_STATE = 6; } message RefreshPeerStateData { required string peer_id = 1; required PeerModificationType type = 2; required ServerName target_server = 3; + /** We need multiple stages for sync replication state transition **/ + optional uint32 stage = 4 [default = 0]; } message RefreshPeerParameter { required string peer_id = 1; required PeerModificationType type = 2; required ServerName target_server = 3; + /** We need multiple stages for sync replication state transition **/ + optional uint32 stage = 4 [default = 0];; } message PeerProcedureStateData { @@ -457,3 +477,43 @@ enum InitMetaState { message InitMetaStateData { } + +message TransitPeerSyncReplicationStateStateData { + /** Could be null if we fail in pre check, so optional */ + optional SyncReplicationState fromState = 1; + required SyncReplicationState toState = 2; +} + +enum RecoverStandbyState { + RENAME_SYNC_REPLICATION_WALS_DIR = 1; + REGISTER_PEER_TO_WORKER_STORAGE = 2; + DISPATCH_WALS = 3; + UNREGISTER_PEER_FROM_WORKER_STORAGE = 4; + SNAPSHOT_SYNC_REPLICATION_WALS_DIR = 5; +} + +enum SyncReplicationReplayWALState { + ASSIGN_WORKER = 1; + DISPATCH_WALS_TO_WORKER = 2; + RELEASE_WORKER = 3; +} + +message RecoverStandbyStateData { + required bool serial = 1; +} + +message SyncReplicationReplayWALStateData { + required string peer_id = 1; + repeated string wal = 2; +} + +message SyncReplicationReplayWALRemoteStateData { + required string peer_id = 1; + repeated string wal = 2; + required ServerName target_server = 3; +} + +message ReplaySyncReplicationWALParameter { + required string peer_id = 1; + repeated string wal = 2; +} diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto index 557b87c..61ba131 100644 --- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto +++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto @@ -49,6 +49,7 @@ message ReplicationPeer { repeated TableCF exclude_table_cfs = 9; repeated bytes exclude_namespaces = 10; optional bool serial = 11; + optional string remoteWALDir = 12; } /** @@ -63,12 +64,26 @@ message ReplicationState { } /** + * Indicate the state of the current cluster in a synchronous replication peer. + */ +message SyncReplicationState { + enum State { + NONE = 0; + ACTIVE = 1; + DOWNGRADE_ACTIVE = 2; + STANDBY = 3; + } + required State state = 1; +} + +/** * Used by replication. Description of the replication peer. */ message ReplicationPeerDescription { required string id = 1; required ReplicationState state = 2; required ReplicationPeer config = 3; + optional SyncReplicationState syncReplicationState = 4; } /** @@ -137,3 +152,12 @@ message ListReplicationPeersRequest { message ListReplicationPeersResponse { repeated ReplicationPeerDescription peer_desc = 1; } + +message TransitReplicationPeerSyncReplicationStateRequest { + required string peer_id = 1; + required SyncReplicationState syncReplicationState = 2; +} + +message TransitReplicationPeerSyncReplicationStateResponse { + required uint64 proc_id = 1; +} diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java index 2da3cce..0196a9a 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java @@ -54,6 +54,15 @@ public interface ReplicationPeer { PeerState getPeerState(); /** + * Returns the sync replication state of the peer by reading local cache. + *

+ * If the peer is not a synchronous replication peer, a {@link SyncReplicationState#NONE} will be + * returned. + * @return the sync replication state + */ + SyncReplicationState getSyncReplicationState(); + + /** * Test whether the peer is enabled. * @return {@code true} if enabled, otherwise {@code false}. */ diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java index d656466..22026e5 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private @@ -36,6 +37,15 @@ public class ReplicationPeerImpl implements ReplicationPeer { private volatile PeerState peerState; + // The lower 16 bits are the current sync replication state, the higher 16 bits are the new sync + // replication state. Embedded in one int so user can not get an inconsistency view of state and + // new state. + private volatile int syncReplicationStateBits; + + private static final int SHIFT = 16; + + private static final int AND_BITS = 0xFFFF; + private final List peerConfigListeners; /** @@ -45,12 +55,15 @@ public class ReplicationPeerImpl implements ReplicationPeer { * @param id string representation of this peer's identifier * @param peerConfig configuration for the replication peer */ - public ReplicationPeerImpl(Configuration conf, String id, boolean peerState, - ReplicationPeerConfig peerConfig) { + public ReplicationPeerImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig, + boolean peerState, SyncReplicationState syncReplicationState, + SyncReplicationState newSyncReplicationState) { this.conf = conf; this.id = id; this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED; this.peerConfig = peerConfig; + this.syncReplicationStateBits = + syncReplicationState.value() | (newSyncReplicationState.value() << SHIFT); this.peerConfigListeners = new ArrayList<>(); } @@ -63,6 +76,16 @@ public class ReplicationPeerImpl implements ReplicationPeer { peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig)); } + public void setNewSyncReplicationState(SyncReplicationState newState) { + this.syncReplicationStateBits = + (this.syncReplicationStateBits & AND_BITS) | (newState.value() << SHIFT); + } + + public void transitSyncReplicationState() { + this.syncReplicationStateBits = + (this.syncReplicationStateBits >>> SHIFT) | (SyncReplicationState.NONE.value() << SHIFT); + } + /** * Get the identifier of this peer * @return string representation of the id (short) @@ -77,37 +100,43 @@ public class ReplicationPeerImpl implements ReplicationPeer { return peerState; } - /** - * Get the peer config object - * @return the ReplicationPeerConfig for this peer - */ + private static SyncReplicationState getSyncReplicationState(int bits) { + return SyncReplicationState.valueOf(bits & AND_BITS); + } + + private static SyncReplicationState getNewSyncReplicationState(int bits) { + return SyncReplicationState.valueOf(bits >>> SHIFT); + } + + public Pair getSyncReplicationStateAndNewState() { + int bits = this.syncReplicationStateBits; + return Pair.newPair(getSyncReplicationState(bits), getNewSyncReplicationState(bits)); + } + + public SyncReplicationState getNewSyncReplicationState() { + return getNewSyncReplicationState(syncReplicationStateBits); + } + + @Override + public SyncReplicationState getSyncReplicationState() { + return getSyncReplicationState(syncReplicationStateBits); + } + @Override public ReplicationPeerConfig getPeerConfig() { return peerConfig; } - /** - * Get the configuration object required to communicate with this peer - * @return configuration object - */ @Override public Configuration getConfiguration() { return conf; } - /** - * Get replicable (table, cf-list) map of this peer - * @return the replicable (table, cf-list) map - */ @Override public Map> getTableCFs() { return this.peerConfig.getTableCFsMap(); } - /** - * Get replicable namespace set of this peer - * @return the replicable namespaces set - */ @Override public Set getNamespaces() { return this.peerConfig.getNamespaces(); diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java index 1adda02..f74ac37 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.replication; import java.util.List; - import org.apache.yetus.audience.InterfaceAudience; /** @@ -31,8 +30,8 @@ public interface ReplicationPeerStorage { * Add a replication peer. * @throws ReplicationException if there are errors accessing the storage service. */ - void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) - throws ReplicationException; + void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled, + SyncReplicationState syncReplicationState) throws ReplicationException; /** * Remove a replication peer. @@ -70,4 +69,32 @@ public interface ReplicationPeerStorage { * @throws ReplicationException if there are errors accessing the storage service. */ ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException; + + /** + * Set the new sync replication state that we are going to transit to. + * @throws ReplicationException if there are errors accessing the storage service. + */ + void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state) + throws ReplicationException; + + /** + * Overwrite the sync replication state with the new sync replication state which is set with the + * {@link #setPeerNewSyncReplicationState(String, SyncReplicationState)} method above, and clear + * the new sync replication state. + * @throws ReplicationException if there are errors accessing the storage service. + */ + void transitPeerSyncReplicationState(String peerId) throws ReplicationException; + + /** + * Get the sync replication state. + * @throws ReplicationException if there are errors accessing the storage service. + */ + SyncReplicationState getPeerSyncReplicationState(String peerId) throws ReplicationException; + + /** + * Get the new sync replication state. Will return {@link SyncReplicationState#NONE} if we are + * not in a transition. + * @throws ReplicationException if there are errors accessing the storage service. + */ + SyncReplicationState getPeerNewSyncReplicationState(String peerId) throws ReplicationException; } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java index 4d602ca..ba6da7a 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java @@ -80,8 +80,8 @@ public class ReplicationPeers { return true; } - public void removePeer(String peerId) { - peerCache.remove(peerId); + public ReplicationPeerImpl removePeer(String peerId) { + return peerCache.remove(peerId); } /** @@ -110,22 +110,29 @@ public class ReplicationPeers { public PeerState refreshPeerState(String peerId) throws ReplicationException { ReplicationPeerImpl peer = peerCache.get(peerId); - if (peer == null) { - throw new ReplicationException("Peer with id=" + peerId + " is not cached."); - } peer.setPeerState(peerStorage.isPeerEnabled(peerId)); return peer.getPeerState(); } public ReplicationPeerConfig refreshPeerConfig(String peerId) throws ReplicationException { ReplicationPeerImpl peer = peerCache.get(peerId); - if (peer == null) { - throw new ReplicationException("Peer with id=" + peerId + " is not cached."); - } peer.setPeerConfig(peerStorage.getPeerConfig(peerId)); return peer.getPeerConfig(); } + public SyncReplicationState refreshPeerNewSyncReplicationState(String peerId) + throws ReplicationException { + ReplicationPeerImpl peer = peerCache.get(peerId); + SyncReplicationState newState = peerStorage.getPeerNewSyncReplicationState(peerId); + peer.setNewSyncReplicationState(newState); + return newState; + } + + public void transitPeerSyncReplicationState(String peerId) { + ReplicationPeerImpl peer = peerCache.get(peerId); + peer.transitSyncReplicationState(); + } + /** * Helper method to connect to a peer * @param peerId peer's identifier @@ -134,7 +141,10 @@ public class ReplicationPeers { private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException { ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId); boolean enabled = peerStorage.isPeerEnabled(peerId); + SyncReplicationState syncReplicationState = peerStorage.getPeerSyncReplicationState(peerId); + SyncReplicationState newSyncReplicationState = + peerStorage.getPeerNewSyncReplicationState(peerId); return new ReplicationPeerImpl(ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf), - peerId, enabled, peerConfig); + peerId, peerConfig, enabled, syncReplicationState, newSyncReplicationState); } } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java index c7568bb..dc4217c 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java @@ -23,12 +23,16 @@ import java.util.List; import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Helper class for replication. @@ -36,6 +40,22 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public final class ReplicationUtils { + private static final Logger LOG = LoggerFactory.getLogger(ReplicationUtils.class); + + public static final String REPLICATION_ATTR_NAME = "__rep__"; + + public static final String REMOTE_WAL_DIR_NAME = "remoteWALs"; + + public static final String SYNC_WAL_SUFFIX = ".syncrep"; + + public static final String REMOTE_WAL_REPLAY_SUFFIX = "-replay"; + + public static final String REMOTE_WAL_SNAPSHOT_SUFFIX = "-snapshot"; + + // This is used for copying sync replication log from local to remote and overwrite the old one + // since some FileSystem implementation may not support atomic rename. + public static final String RENAME_WAL_SUFFIX = ".ren"; + private ReplicationUtils() { } @@ -171,4 +191,49 @@ public final class ReplicationUtils { return tableCFs != null && tableCFs.containsKey(tableName); } } + + public static FileSystem getRemoteWALFileSystem(Configuration conf, String remoteWALDir) + throws IOException { + return new Path(remoteWALDir).getFileSystem(conf); + } + + public static Path getPeerRemoteWALDir(String remoteWALDir, String peerId) { + return new Path(remoteWALDir, peerId); + } + + public static Path getPeerRemoteWALDir(Path remoteWALDir, String peerId) { + return new Path(remoteWALDir, peerId); + } + + public static Path getPeerReplayWALDir(Path remoteWALDir, String peerId) { + return getPeerRemoteWALDir(remoteWALDir, peerId).suffix(REMOTE_WAL_REPLAY_SUFFIX); + } + + public static Path getPeerSnapshotWALDir(String remoteWALDir, String peerId) { + return getPeerRemoteWALDir(remoteWALDir, peerId).suffix(REMOTE_WAL_SNAPSHOT_SUFFIX); + } + + public static Path getPeerSnapshotWALDir(Path remoteWALDir, String peerId) { + return getPeerRemoteWALDir(remoteWALDir, peerId).suffix(REMOTE_WAL_SNAPSHOT_SUFFIX); + } + + /** + * Do the sleeping logic + * @param msg Why we sleep + * @param sleepForRetries the base sleep time. + * @param sleepMultiplier by how many times the default sleeping time is augmented + * @param maxRetriesMultiplier the max retry multiplier + * @return True if sleepMultiplier is < maxRetriesMultiplier + */ + public static boolean sleepForRetries(String msg, long sleepForRetries, int sleepMultiplier, + int maxRetriesMultiplier) { + try { + LOG.trace("{}, sleeping {} times {}", msg, sleepForRetries, sleepMultiplier); + Thread.sleep(sleepForRetries * sleepMultiplier); + } catch (InterruptedException e) { + LOG.debug("Interrupted while sleeping between retries"); + Thread.currentThread().interrupt(); + } + return sleepMultiplier < maxRetriesMultiplier; + } } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java index bbe6549..a2cdfdf 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.replication; +import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -29,6 +30,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; + import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; @@ -51,6 +53,13 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase public static final byte[] DISABLED_ZNODE_BYTES = toByteArray(ReplicationProtos.ReplicationState.State.DISABLED); + public static final String SYNC_REPLICATION_STATE_ZNODE = "sync-rep-state"; + + public static final String NEW_SYNC_REPLICATION_STATE_ZNODE = "new-sync-rep-state"; + + public static final byte[] NONE_STATE_ZNODE_BYTES = + SyncReplicationState.toByteArray(SyncReplicationState.NONE); + /** * The name of the znode that contains the replication status of a remote slave (i.e. peer) * cluster. @@ -79,21 +88,34 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase return ZNodePaths.joinZNode(peersZNode, peerId); } + @VisibleForTesting + public String getSyncReplicationStateNode(String peerId) { + return ZNodePaths.joinZNode(getPeerNode(peerId), SYNC_REPLICATION_STATE_ZNODE); + } + + private String getNewSyncReplicationStateNode(String peerId) { + return ZNodePaths.joinZNode(getPeerNode(peerId), NEW_SYNC_REPLICATION_STATE_ZNODE); + } + @Override - public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) - throws ReplicationException { + public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled, + SyncReplicationState syncReplicationState) throws ReplicationException { + List multiOps = Arrays.asList( + ZKUtilOp.createAndFailSilent(getPeerNode(peerId), + ReplicationPeerConfigUtil.toByteArray(peerConfig)), + ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId), + enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES), + ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId), + SyncReplicationState.toByteArray(syncReplicationState)), + ZKUtilOp.createAndFailSilent(getNewSyncReplicationStateNode(peerId), NONE_STATE_ZNODE_BYTES)); try { ZKUtil.createWithParents(zookeeper, peersZNode); - ZKUtil.multiOrSequential(zookeeper, - Arrays.asList( - ZKUtilOp.createAndFailSilent(getPeerNode(peerId), - ReplicationPeerConfigUtil.toByteArray(peerConfig)), - ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId), - enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES)), - false); + ZKUtil.multiOrSequential(zookeeper, multiOps, false); } catch (KeeperException e) { - throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>" - + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e); + throw new ReplicationException( + "Could not add peer with id=" + peerId + ", peerConfig=>" + peerConfig + ", state=" + + (enabled ? "ENABLED" : "DISABLED") + ", syncReplicationState=" + syncReplicationState, + e); } } @@ -124,7 +146,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase ReplicationPeerConfigUtil.toByteArray(peerConfig)); } catch (KeeperException e) { throw new ReplicationException( - "There was a problem trying to save changes to the " + "replication peer " + peerId, e); + "There was a problem trying to save changes to the " + "replication peer " + peerId, e); } } @@ -158,13 +180,63 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase } if (data == null || data.length == 0) { throw new ReplicationException( - "Replication peer config data shouldn't be empty, peerId=" + peerId); + "Replication peer config data shouldn't be empty, peerId=" + peerId); } try { return ReplicationPeerConfigUtil.parsePeerFrom(data); } catch (DeserializationException e) { throw new ReplicationException( - "Failed to parse replication peer config for peer with id=" + peerId, e); + "Failed to parse replication peer config for peer with id=" + peerId, e); + } + } + + @Override + public void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state) + throws ReplicationException { + try { + ZKUtil.createSetData(zookeeper, getNewSyncReplicationStateNode(peerId), + SyncReplicationState.toByteArray(state)); + } catch (KeeperException e) { + throw new ReplicationException( + "Unable to set the new sync replication state for peer with id=" + peerId, e); + } + } + + @Override + public void transitPeerSyncReplicationState(String peerId) throws ReplicationException { + String newStateNode = getNewSyncReplicationStateNode(peerId); + try { + byte[] data = ZKUtil.getData(zookeeper, newStateNode); + ZKUtil.multiOrSequential(zookeeper, + Arrays.asList(ZKUtilOp.setData(newStateNode, NONE_STATE_ZNODE_BYTES), + ZKUtilOp.setData(getSyncReplicationStateNode(peerId), data)), + false); + } catch (KeeperException | InterruptedException e) { + throw new ReplicationException( + "Error transiting sync replication state for peer with id=" + peerId, e); + } + } + + private SyncReplicationState getSyncReplicationState(String peerId, String path) + throws ReplicationException { + try { + byte[] data = ZKUtil.getData(zookeeper, path); + return SyncReplicationState.parseFrom(data); + } catch (KeeperException | InterruptedException | IOException e) { + throw new ReplicationException( + "Error getting sync replication state of path " + path + " for peer with id=" + peerId, e); } } + + @Override + public SyncReplicationState getPeerNewSyncReplicationState(String peerId) + throws ReplicationException { + return getSyncReplicationState(peerId, getNewSyncReplicationStateNode(peerId)); + } + + @Override + public SyncReplicationState getPeerSyncReplicationState(String peerId) + throws ReplicationException { + return getSyncReplicationState(peerId, getSyncReplicationStateNode(peerId)); + } } diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java index 437804c..4a2c3cd 100644 --- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java @@ -166,7 +166,8 @@ public abstract class TestReplicationStateBasic { assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty()); assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size()); rp.getPeerStorage().addPeer(ID_ONE, - ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true); + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true, + SyncReplicationState.NONE); rqs.addPeerToHFileRefs(ID_ONE); rqs.addHFileRefs(ID_ONE, files1); assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size()); @@ -189,10 +190,12 @@ public abstract class TestReplicationStateBasic { public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException { rp.init(); rp.getPeerStorage().addPeer(ID_ONE, - ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true); + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true, + SyncReplicationState.NONE); rqs.addPeerToHFileRefs(ID_ONE); rp.getPeerStorage().addPeer(ID_TWO, - ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true); + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true, + SyncReplicationState.NONE); rqs.addPeerToHFileRefs(ID_TWO); List> files1 = new ArrayList<>(3); @@ -241,9 +244,13 @@ public abstract class TestReplicationStateBasic { assertNumberOfPeers(0); // Add some peers - rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true); + rp.getPeerStorage().addPeer(ID_ONE, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true, + SyncReplicationState.NONE); assertNumberOfPeers(1); - rp.getPeerStorage().addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), true); + rp.getPeerStorage().addPeer(ID_TWO, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true, + SyncReplicationState.NONE); assertNumberOfPeers(2); assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationUtils @@ -253,7 +260,9 @@ public abstract class TestReplicationStateBasic { assertNumberOfPeers(1); // Add one peer - rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true); + rp.getPeerStorage().addPeer(ID_ONE, + ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true, + SyncReplicationState.NONE); rp.addPeer(ID_ONE); assertNumberOfPeers(2); assertTrue(rp.getPeer(ID_ONE).isPeerEnabled()); @@ -365,7 +374,7 @@ public abstract class TestReplicationStateBasic { // Add peers for the corresponding queues so they are not orphans rp.getPeerStorage().addPeer("qId" + i, ReplicationPeerConfig.newBuilder().setClusterKey("localhost:2818:/bogus" + i).build(), - true); + true, SyncReplicationState.NONE); } } } diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java index 3290fb0..1258695 100644 --- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java @@ -87,8 +87,9 @@ public class TestZKReplicationPeerStorage { Random rand = new Random(seed); return ReplicationPeerConfig.newBuilder().setClusterKey(Long.toHexString(rand.nextLong())) .setReplicationEndpointImpl(Long.toHexString(rand.nextLong())) - .setNamespaces(randNamespaces(rand)).setExcludeNamespaces(randNamespaces(rand)) - .setTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean()) + .setRemoteWALDir(Long.toHexString(rand.nextLong())).setNamespaces(randNamespaces(rand)) + .setExcludeNamespaces(randNamespaces(rand)).setTableCFsMap(randTableCFs(rand)) + .setExcludeTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean()) .setBandwidth(rand.nextInt(1000)).build(); } @@ -139,7 +140,8 @@ public class TestZKReplicationPeerStorage { public void test() throws ReplicationException { int peerCount = 10; for (int i = 0; i < peerCount; i++) { - STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0); + STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0, + SyncReplicationState.valueOf(i % 4)); } List peerIds = STORAGE.listPeerIds(); assertEquals(peerCount, peerIds.size()); @@ -163,6 +165,10 @@ public class TestZKReplicationPeerStorage { for (int i = 0; i < peerCount; i++) { assertEquals(i % 2 != 0, STORAGE.isPeerEnabled(Integer.toString(i))); } + for (int i = 0; i < peerCount; i++) { + assertEquals(SyncReplicationState.valueOf(i % 4), + STORAGE.getPeerSyncReplicationState(Integer.toString(i))); + } String toRemove = Integer.toString(peerCount / 2); STORAGE.removePeer(toRemove); peerIds = STORAGE.listPeerIds(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java index 3175af3..573ac7a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -1391,6 +1392,29 @@ public interface MasterObserver { String regex) throws IOException {} /** + * Called before transit current cluster state for the specified synchronous replication peer + * @param ctx the environment to interact with the framework and master + * @param peerId a short name that identifies the peer + * @param state the new state + */ + default void preTransitReplicationPeerSyncReplicationState( + final ObserverContext ctx, String peerId, + SyncReplicationState state) throws IOException { + } + + /** + * Called after transit current cluster state for the specified synchronous replication peer + * @param ctx the environment to interact with the framework and master + * @param peerId a short name that identifies the peer + * @param from the old state + * @param to the new state + */ + default void postTransitReplicationPeerSyncReplicationState( + final ObserverContext ctx, String peerId, + SyncReplicationState from, SyncReplicationState to) throws IOException { + } + + /** * Called before new LockProcedure is queued. * @param ctx the environment to interact with the framework and master */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java index 922deb8..ad38d1c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java @@ -281,7 +281,14 @@ public enum EventType { * * RS_REFRESH_PEER */ - RS_REFRESH_PEER (84, ExecutorType.RS_REFRESH_PEER); + RS_REFRESH_PEER(84, ExecutorType.RS_REFRESH_PEER), + + /** + * RS replay sync replication wal.
+ * + * RS_REPLAY_SYNC_REPLICATION_WAL + */ + RS_REPLAY_SYNC_REPLICATION_WAL(85, ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL); private final int code; private final ExecutorType executor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java index 7f130d1..ea97354 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java @@ -47,7 +47,8 @@ public enum ExecutorType { RS_REGION_REPLICA_FLUSH_OPS (28), RS_COMPACTED_FILES_DISCHARGER (29), RS_OPEN_PRIORITY_REGION (30), - RS_REFRESH_PEER (31); + RS_REFRESH_PEER(31), + RS_REPLAY_SYNC_REPLICATION_WAL(32); ExecutorType(int value) { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java index 1645d68..7ffd3da 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java @@ -22,9 +22,9 @@ import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHel import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.completeFile; import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.endFileLease; import static org.apache.hadoop.hbase.io.asyncfs.FanOutOneBlockAsyncDFSOutputHelper.getStatus; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY; import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.READER_IDLE; import static org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleState.WRITER_IDLE; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY; import java.io.IOException; import java.io.InterruptedIOException; @@ -40,7 +40,6 @@ import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.Encryptor; import org.apache.hadoop.fs.Path; 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 530ed17..e55ee06 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 @@ -133,12 +133,14 @@ import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure; import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure; +import org.apache.hadoop.hbase.master.replication.AbstractPeerProcedure; import org.apache.hadoop.hbase.master.replication.AddPeerProcedure; import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure; import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure; -import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure; import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure; import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; +import org.apache.hadoop.hbase.master.replication.SyncReplicationReplayWALManager; +import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure; import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer; @@ -175,6 +177,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner; import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner; import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader; @@ -339,6 +342,8 @@ public class HMaster extends HRegionServer implements MasterServices { // manager of replication private ReplicationPeerManager replicationPeerManager; + private SyncReplicationReplayWALManager syncReplicationReplayWALManager; + // buffer for "fatal error" notices from region servers // in the cluster. This is only used for assisting // operations/debugging. @@ -748,6 +753,7 @@ public class HMaster extends HRegionServer implements MasterServices { this.splitOrMergeTracker.start(); this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf); + this.syncReplicationReplayWALManager = new SyncReplicationReplayWALManager(this); this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager); this.drainingServerTracker.start(); @@ -3489,7 +3495,7 @@ public class HMaster extends HRegionServer implements MasterServices { return favoredNodesManager; } - private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException { + private long executePeerProcedure(AbstractPeerProcedure procedure) throws IOException { long procId = procedureExecutor.submitProcedure(procedure); procedure.getLatch().await(); return procId; @@ -3560,6 +3566,16 @@ public class HMaster extends HRegionServer implements MasterServices { return peers; } + @Override + public long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state) + throws ReplicationException, IOException { + LOG.info( + getClientIdAuditPrefix() + + " transit current cluster state to {} in a synchronous replication peer id={}", + state, peerId); + return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId, state)); + } + /** * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0 @@ -3718,4 +3734,9 @@ public class HMaster extends HRegionServer implements MasterServices { public SnapshotQuotaObserverChore getSnapshotQuotaObserverChore() { return this.snapshotQuotaChore; } + + @Override + public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() { + return this.syncReplicationReplayWALManager; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java index e563cd4..019c64f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java @@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -1606,6 +1607,26 @@ public class MasterCoprocessorHost }); } + public void preTransitReplicationPeerSyncReplicationState(String peerId, + SyncReplicationState state) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() { + @Override + public void call(MasterObserver observer) throws IOException { + observer.preTransitReplicationPeerSyncReplicationState(this, peerId, state); + } + }); + } + + public void postTransitReplicationPeerSyncReplicationState(String peerId, + SyncReplicationState from, SyncReplicationState to) throws IOException { + execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() { + @Override + public void call(MasterObserver observer) throws IOException { + observer.postTransitReplicationPeerSyncReplicationState(this, peerId, from, to); + } + }); + } + public void preRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos, LockType type, String description) throws IOException { execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() { 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 864be02..7ccbd71 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 @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; @@ -133,7 +134,6 @@ public class MasterFileSystem { * Idempotent. */ private void createInitialFileSystemLayout() throws IOException { - final String[] protectedSubDirs = new String[] { HConstants.BASE_NAMESPACE_DIR, HConstants.HFILE_ARCHIVE_DIRECTORY, @@ -145,7 +145,8 @@ public class MasterFileSystem { HConstants.HREGION_LOGDIR_NAME, HConstants.HREGION_OLDLOGDIR_NAME, HConstants.CORRUPT_DIR_NAME, - WALProcedureStore.MASTER_PROCEDURE_LOGDIR + WALProcedureStore.MASTER_PROCEDURE_LOGDIR, + ReplicationUtils.REMOTE_WAL_DIR_NAME }; // check if the root directory exists checkRootDir(this.rootdir, conf, this.fs); @@ -192,7 +193,9 @@ public class MasterFileSystem { return this.fs; } - protected FileSystem getWALFileSystem() { return this.walFs; } + public FileSystem getWALFileSystem() { + return this.walFs; + } public Configuration getConfiguration() { return this.conf; @@ -234,13 +237,9 @@ public class MasterFileSystem { } /** - * Get the rootdir. Make sure its wholesome and exists before returning. - * @param rd - * @param c - * @param fs - * @return hbase.rootdir (after checks for existence and bootstrapping if - * needed populating the directory with necessary bootup files). - * @throws IOException + * Get the rootdir. Make sure its wholesome and exists before returning. + * @return hbase.rootdir (after checks for existence and bootstrapping if needed populating the + * directory with necessary bootup files). */ private Path checkRootDir(final Path rd, final Configuration c, final FileSystem fs) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 8efbfe5..fbc5589 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -291,6 +291,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; @@ -1948,6 +1950,20 @@ public class MasterRpcServices extends RSRpcServices } @Override + public TransitReplicationPeerSyncReplicationStateResponse + transitReplicationPeerSyncReplicationState(RpcController controller, + TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException { + try { + long procId = master.transitReplicationPeerSyncReplicationState(request.getPeerId(), + ReplicationPeerConfigUtil.toSyncReplicationState(request.getSyncReplicationState())); + return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId) + .build(); + } catch (ReplicationException | IOException e) { + throw new ServiceException(e); + } + } + + @Override public ListReplicationPeersResponse listReplicationPeers(RpcController controller, ListReplicationPeersRequest request) throws ServiceException { ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder(); 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 ac521d5..7b0c56a 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 @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.master.locking.LockManager; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; +import org.apache.hadoop.hbase.master.replication.SyncReplicationReplayWALManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost; import org.apache.hadoop.hbase.procedure2.LockedResource; @@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -460,6 +462,11 @@ public interface MasterServices extends Server { ReplicationPeerManager getReplicationPeerManager(); /** + * Returns the {@link SyncReplicationReplayWALManager}. + */ + SyncReplicationReplayWALManager getSyncReplicationReplayWALManager(); + + /** * Update the peerConfig for the specified peer * @param peerId a short name that identifies the peer * @param peerConfig new config for the peer @@ -476,6 +483,14 @@ public interface MasterServices extends Server { IOException; /** + * Set current cluster state for a synchronous replication peer. + * @param peerId a short name that identifies the peer + * @param clusterState state of current cluster + */ + long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState clusterState) + throws ReplicationException, IOException; + + /** * @return {@link LockManager} to lock namespaces/tables/regions. */ LockManager getLockManager(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java index 0ec932c..2c01b16 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.conf.ConfigurationObserver; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; +import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; @@ -139,6 +140,10 @@ public class MasterProcedureEnv implements ConfigurationObserver { return master.getReplicationPeerManager(); } + public MasterFileSystem getMasterFileSystem() { + return master.getMasterFileSystem(); + } + public boolean isRunning() { if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false; return master.getMasterProcedureExecutor().isRunning(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index 1420986..8a28b84 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java @@ -207,7 +207,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { // check if the next procedure is still a child. // if not, remove the rq from the fairq and go back to the xlock state Procedure nextProc = rq.peek(); - if (nextProc != null && !Procedure.haveSameParent(nextProc, pollResult)) { + if (nextProc != null && !Procedure.haveSameParent(nextProc, pollResult) + && nextProc.getRootProcId() != pollResult.getRootProcId()) { removeFromRunQueue(fairq, rq); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java index 399bcd7..0195ab9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java @@ -23,7 +23,8 @@ import org.apache.yetus.audience.InterfaceAudience; public interface PeerProcedureInterface { enum PeerOperationType { - ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH + ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH, TRANSIT_SYNC_REPLICATION_STATE, + RECOVER_STANDBY, SYNC_REPLICATION_REPLAY_WAL, SYNC_REPLICATION_REPLAY_WAL_REMOTE } String getPeerId(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java index 1ae0c2f..86d8e43 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerQueue.java @@ -49,6 +49,8 @@ class PeerQueue extends Queue { } private static boolean requirePeerExclusiveLock(PeerProcedureInterface proc) { - return proc.getPeerOperationType() != PeerOperationType.REFRESH; + return proc.getPeerOperationType() != PeerOperationType.REFRESH + && proc.getPeerOperationType() != PeerOperationType.SYNC_REPLICATION_REPLAY_WAL + && proc.getPeerOperationType() != PeerOperationType.SYNC_REPLICATION_REPLAY_WAL_REMOTE; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java index 0ad8a63..458e073 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java @@ -46,7 +46,7 @@ public abstract class AbstractPeerProcedure protected AbstractPeerProcedure(String peerId) { this.peerId = peerId; - this.latch = ProcedurePrepareLatch.createLatch(2, 0); + this.latch = ProcedurePrepareLatch.createLatch(2, 1); } public ProcedurePrepareLatch getLatch() { @@ -94,4 +94,20 @@ public abstract class AbstractPeerProcedure super.deserializeStateData(serializer); peerId = serializer.deserialize(PeerProcedureStateData.class).getPeerId(); } + + @Override + protected void rollbackState(MasterProcedureEnv env, TState state) + throws IOException, InterruptedException { + if (state == getInitialState()) { + // actually the peer related operations has no rollback, but if we haven't done any + // modifications on the peer storage yet, we can just return. + return; + } + throw new UnsupportedOperationException(); + } + + protected final void refreshPeer(MasterProcedureEnv env, PeerOperationType type) { + addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream() + .map(sn -> new RefreshPeerProcedure(peerId, type, sn)).toArray(RefreshPeerProcedure[]::new)); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java index 6fd8aa3..ad4df61 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java @@ -109,12 +109,6 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure new RefreshPeerProcedure(peerId, type, sn)) - .toArray(RefreshPeerProcedure[]::new)); - } - protected ReplicationPeerConfig getOldPeerConfig() { return null; } @@ -328,17 +322,6 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(RecoverStandbyProcedure.class); + + private boolean serial; + + public RecoverStandbyProcedure() { + } + + public RecoverStandbyProcedure(String peerId, boolean serial) { + super(peerId); + this.serial = serial; + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, RecoverStandbyState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + SyncReplicationReplayWALManager syncReplicationReplayWALManager = + env.getMasterServices().getSyncReplicationReplayWALManager(); + switch (state) { + case RENAME_SYNC_REPLICATION_WALS_DIR: + try { + syncReplicationReplayWALManager.renameToPeerReplayWALDir(peerId); + } catch (IOException e) { + LOG.warn("Failed to rename remote wal dir for peer id={}", peerId, e); + setFailure("master-recover-standby", e); + return Flow.NO_MORE_STATE; + } + setNextState(RecoverStandbyState.REGISTER_PEER_TO_WORKER_STORAGE); + return Flow.HAS_MORE_STATE; + case REGISTER_PEER_TO_WORKER_STORAGE: + try { + syncReplicationReplayWALManager.registerPeer(peerId); + } catch (ReplicationException e) { + LOG.warn("Failed to register peer to worker storage for peer id={}, retry", peerId, e); + throw new ProcedureYieldException(); + } + setNextState(RecoverStandbyState.DISPATCH_WALS); + return Flow.HAS_MORE_STATE; + case DISPATCH_WALS: + dispathWals(syncReplicationReplayWALManager); + setNextState(RecoverStandbyState.UNREGISTER_PEER_FROM_WORKER_STORAGE); + return Flow.HAS_MORE_STATE; + case UNREGISTER_PEER_FROM_WORKER_STORAGE: + try { + syncReplicationReplayWALManager.unregisterPeer(peerId); + } catch (ReplicationException e) { + LOG.warn("Failed to unregister peer from worker storage for peer id={}, retry", peerId, + e); + throw new ProcedureYieldException(); + } + setNextState(RecoverStandbyState.SNAPSHOT_SYNC_REPLICATION_WALS_DIR); + return Flow.HAS_MORE_STATE; + case SNAPSHOT_SYNC_REPLICATION_WALS_DIR: + try { + syncReplicationReplayWALManager.renameToPeerSnapshotWALDir(peerId); + } catch (IOException e) { + LOG.warn("Failed to cleanup replay wals dir for peer id={}, , retry", peerId, e); + throw new ProcedureYieldException(); + } + return Flow.NO_MORE_STATE; + default: + throw new UnsupportedOperationException("unhandled state=" + state); + } + } + + // TODO: dispatch wals by region server when serial is true and sort wals + private void dispathWals(SyncReplicationReplayWALManager syncReplicationReplayWALManager) + throws ProcedureYieldException { + try { + List wals = syncReplicationReplayWALManager.getReplayWALsAndCleanUpUnusedFiles(peerId); + addChildProcedure(wals.stream().map(wal -> new SyncReplicationReplayWALProcedure(peerId, + Arrays.asList(syncReplicationReplayWALManager.removeWALRootPath(wal)))) + .toArray(SyncReplicationReplayWALProcedure[]::new)); + } catch (IOException e) { + LOG.warn("Failed to get replay wals for peer id={}, , retry", peerId, e); + throw new ProcedureYieldException(); + } + } + + @Override + protected RecoverStandbyState getState(int stateId) { + return RecoverStandbyState.forNumber(stateId); + } + + @Override + protected int getStateId(RecoverStandbyState state) { + return state.getNumber(); + } + + @Override + protected RecoverStandbyState getInitialState() { + return RecoverStandbyState.RENAME_SYNC_REPLICATION_WALS_DIR; + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.RECOVER_STANDBY; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + serializer.serialize(RecoverStandbyStateData.newBuilder().setSerial(serial).build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + RecoverStandbyStateData data = serializer.deserialize(RecoverStandbyStateData.class); + serial = data.getSerial(); + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java index 10e16e9..8502edf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java @@ -55,6 +55,8 @@ public class RefreshPeerProcedure extends Procedure justification = "Will never change after construction") private ServerName targetServer; + private int stage; + private boolean dispatched; private ProcedureEvent event; @@ -65,9 +67,15 @@ public class RefreshPeerProcedure extends Procedure } public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer) { + this(peerId, type, targetServer, 0); + } + + public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer, + int stage) { this.peerId = peerId; this.type = type; this.targetServer = targetServer; + this.stage = stage; } @Override @@ -92,6 +100,8 @@ public class RefreshPeerProcedure extends Procedure return PeerModificationType.DISABLE_PEER; case UPDATE_CONFIG: return PeerModificationType.UPDATE_PEER_CONFIG; + case TRANSIT_SYNC_REPLICATION_STATE: + return PeerModificationType.TRANSIT_SYNC_REPLICATION_STATE; default: throw new IllegalArgumentException("Unknown type: " + type); } @@ -109,6 +119,8 @@ public class RefreshPeerProcedure extends Procedure return PeerOperationType.DISABLE; case UPDATE_PEER_CONFIG: return PeerOperationType.UPDATE_CONFIG; + case TRANSIT_SYNC_REPLICATION_STATE: + return PeerOperationType.TRANSIT_SYNC_REPLICATION_STATE; default: throw new IllegalArgumentException("Unknown type: " + type); } @@ -119,7 +131,8 @@ public class RefreshPeerProcedure extends Procedure assert targetServer.equals(remote); return new ServerOperation(this, getProcId(), RefreshPeerCallable.class, RefreshPeerParameter.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type)) - .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray()); + .setTargetServer(ProtobufUtil.toServerName(remote)).setStage(stage).build() + .toByteArray()); } private void complete(MasterProcedureEnv env, Throwable error) { @@ -196,7 +209,7 @@ public class RefreshPeerProcedure extends Procedure protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { serializer.serialize( RefreshPeerStateData.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type)) - .setTargetServer(ProtobufUtil.toServerName(targetServer)).build()); + .setTargetServer(ProtobufUtil.toServerName(targetServer)).setStage(stage).build()); } @Override @@ -205,5 +218,6 @@ public class RefreshPeerProcedure extends Procedure peerId = data.getPeerId(); type = toPeerOperationType(data.getType()); targetServer = ProtobufUtil.toServerName(data.getTargetServer()); + stage = data.getStage(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java index 82dc07e..4b77c8d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java @@ -66,9 +66,15 @@ public class RemovePeerProcedure extends ModifyPeerProcedure { env.getReplicationPeerManager().removePeer(peerId); } - @Override - protected void postPeerModification(MasterProcedureEnv env) + private void removeRemoteWALs(MasterProcedureEnv env) throws IOException { + env.getMasterServices().getSyncReplicationReplayWALManager().removePeerRemoteWALs(peerId); + } + + @Override protected void postPeerModification(MasterProcedureEnv env) throws IOException, ReplicationException { + if (peerConfig.isSyncReplication()) { + removeRemoteWALs(env); + } env.getReplicationPeerManager().removeAllQueuesAndHFileRefs(peerId); if (peerConfig.isSerial()) { env.getReplicationPeerManager().removeAllLastPushedSeqIds(peerId); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java index 87d0111..8e49137 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java @@ -18,8 +18,10 @@ package org.apache.hadoop.hbase.master.replication; import java.io.IOException; +import java.net.URI; import java.util.ArrayList; import java.util.Collection; +import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -30,6 +32,7 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -43,10 +46,14 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; +import org.apache.hbase.thirdparty.com.google.common.collect.Maps; + /** * Manages and performs all replication admin operations. *

@@ -61,6 +68,13 @@ public class ReplicationPeerManager { private final ConcurrentMap peers; + private final ImmutableMap> + allowedTransition = Maps.immutableEnumMap(ImmutableMap.of(SyncReplicationState.ACTIVE, + EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE, SyncReplicationState.STANDBY), + SyncReplicationState.STANDBY, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE), + SyncReplicationState.DOWNGRADE_ACTIVE, + EnumSet.of(SyncReplicationState.STANDBY, SyncReplicationState.ACTIVE))); + ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage, ConcurrentMap peers) { this.peerStorage = peerStorage; @@ -109,8 +123,20 @@ public class ReplicationPeerManager { return desc; } + private void checkPeerInDAStateIfSyncReplication(String peerId) throws DoNotRetryIOException { + ReplicationPeerDescription desc = peers.get(peerId); + if (desc != null && desc.getPeerConfig().isSyncReplication() + && !SyncReplicationState.DOWNGRADE_ACTIVE.equals(desc.getSyncReplicationState())) { + throw new DoNotRetryIOException("Couldn't remove synchronous replication peer with state=" + + desc.getSyncReplicationState() + + ", Transit the synchronous replication state to be DOWNGRADE_ACTIVE firstly."); + } + } + ReplicationPeerConfig preRemovePeer(String peerId) throws DoNotRetryIOException { - return checkPeerExists(peerId).getPeerConfig(); + ReplicationPeerDescription pd = checkPeerExists(peerId); + checkPeerInDAStateIfSyncReplication(peerId); + return pd.getPeerConfig(); } void preEnablePeer(String peerId) throws DoNotRetryIOException { @@ -149,6 +175,36 @@ public class ReplicationPeerManager { oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId + " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'"); } + + if (!isStringEquals(peerConfig.getRemoteWALDir(), oldPeerConfig.getRemoteWALDir())) { + throw new DoNotRetryIOException( + "Changing the remote wal dir on an existing peer is not allowed. Existing remote wal " + + "dir '" + oldPeerConfig.getRemoteWALDir() + "' for peer " + peerId + + " does not match new remote wal dir '" + peerConfig.getRemoteWALDir() + "'"); + } + + if (oldPeerConfig.isSyncReplication()) { + if (!ReplicationUtils.isNamespacesAndTableCFsEqual(oldPeerConfig, peerConfig)) { + throw new DoNotRetryIOException( + "Changing the replicated namespace/table config on a synchronous replication " + + "peer(peerId: " + peerId + ") is not allowed."); + } + } + return desc; + } + + /** + * @return the old desciption of the peer + */ + ReplicationPeerDescription preTransitPeerSyncReplicationState(String peerId, + SyncReplicationState state) throws DoNotRetryIOException { + ReplicationPeerDescription desc = checkPeerExists(peerId); + SyncReplicationState fromState = desc.getSyncReplicationState(); + EnumSet allowedToStates = allowedTransition.get(fromState); + if (allowedToStates == null || !allowedToStates.contains(state)) { + throw new DoNotRetryIOException("Can not transit current cluster state from " + fromState + + " to " + state + " for peer id=" + peerId); + } return desc; } @@ -159,8 +215,12 @@ public class ReplicationPeerManager { return; } ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build(); - peerStorage.addPeer(peerId, copiedPeerConfig, enabled); - peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig)); + SyncReplicationState syncReplicationState = + copiedPeerConfig.isSyncReplication() ? SyncReplicationState.DOWNGRADE_ACTIVE + : SyncReplicationState.NONE; + peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState); + peers.put(peerId, + new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState)); } public void removePeer(String peerId) throws ReplicationException { @@ -179,7 +239,8 @@ public class ReplicationPeerManager { return; } peerStorage.setPeerState(peerId, enabled); - peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig())); + peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig(), + desc.getSyncReplicationState())); } public void enablePeer(String peerId) throws ReplicationException { @@ -196,7 +257,7 @@ public class ReplicationPeerManager { ReplicationPeerDescription desc = peers.get(peerId); ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig(); ReplicationPeerConfigBuilder newPeerConfigBuilder = - ReplicationPeerConfig.newBuilder(peerConfig); + ReplicationPeerConfig.newBuilder(peerConfig); // we need to use the new conf to overwrite the old one. newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration()); newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration()); @@ -204,7 +265,8 @@ public class ReplicationPeerManager { newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration()); ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build(); peerStorage.updatePeerConfig(peerId, newPeerConfig); - peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig)); + peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig, + desc.getSyncReplicationState())); } public List listPeers(Pattern pattern) { @@ -212,7 +274,7 @@ public class ReplicationPeerManager { return new ArrayList<>(peers.values()); } return peers.values().stream().filter(r -> pattern.matcher(r.getPeerId()).matches()) - .collect(Collectors.toList()); + .collect(Collectors.toList()); } public Optional getPeerConfig(String peerId) { @@ -224,7 +286,26 @@ public class ReplicationPeerManager { queueStorage.removeLastSequenceIds(peerId); } - void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException { + public void setPeerNewSyncReplicationState(String peerId, SyncReplicationState state) + throws ReplicationException { + peerStorage.setPeerNewSyncReplicationState(peerId, state); + } + + public void transitPeerSyncReplicationState(String peerId, SyncReplicationState newState) + throws ReplicationException { + if (peerStorage.getPeerNewSyncReplicationState(peerId) != SyncReplicationState.NONE) { + // Only transit if this is not a retry + peerStorage.transitPeerSyncReplicationState(peerId); + } + ReplicationPeerDescription desc = peers.get(peerId); + if (desc.getSyncReplicationState() != newState) { + // Only recreate the desc if this is not a retry + peers.put(peerId, + new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), newState)); + } + } + + public void removeAllQueues(String peerId) throws ReplicationException { // Here we need two passes to address the problem of claimQueue. Maybe a claimQueue is still // on-going when the refresh peer config procedure is done, if a RS which has already been // scanned claims the queue of a RS which has not been scanned yet, we will miss that queue in @@ -238,6 +319,10 @@ public class ReplicationPeerManager { // unless it has already been removed by others. ReplicationUtils.removeAllQueues(queueStorage, peerId); ReplicationUtils.removeAllQueues(queueStorage, peerId); + } + + public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException { + removeAllQueues(peerId); queueStorage.removePeerFromHFileRefs(peerId); } @@ -248,10 +333,10 @@ public class ReplicationPeerManager { // If replicate_all flag is true, it means all user tables will be replicated to peer cluster. // Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer // cluster. - if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) - || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { - throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " - + "when you want replicate all cluster"); + if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) || + (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { + throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " + + "when you want replicate all cluster"); } checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(), peerConfig.getExcludeTableCFsMap()); @@ -259,21 +344,59 @@ public class ReplicationPeerManager { // If replicate_all flag is false, it means all user tables can't be replicated to peer // cluster. Then allow to config namespaces or table-cfs which will be replicated to peer // cluster. - if ((peerConfig.getExcludeNamespaces() != null - && !peerConfig.getExcludeNamespaces().isEmpty()) - || (peerConfig.getExcludeTableCFsMap() != null - && !peerConfig.getExcludeTableCFsMap().isEmpty())) { + if ((peerConfig.getExcludeNamespaces() != null && + !peerConfig.getExcludeNamespaces().isEmpty()) || + (peerConfig.getExcludeTableCFsMap() != null && + !peerConfig.getExcludeTableCFsMap().isEmpty())) { throw new DoNotRetryIOException( - "Need clean exclude-namespaces or exclude-table-cfs config firstly" - + " when replicate_all flag is false"); + "Need clean exclude-namespaces or exclude-table-cfs config firstly" + + " when replicate_all flag is false"); } checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(), peerConfig.getTableCFsMap()); } + if (peerConfig.isSyncReplication()) { + checkPeerConfigForSyncReplication(peerConfig); + } + checkConfiguredWALEntryFilters(peerConfig); } + private void checkPeerConfigForSyncReplication(ReplicationPeerConfig peerConfig) + throws DoNotRetryIOException { + // This is used to reduce the difficulty for implementing the sync replication state transition + // as we need to reopen all the related regions. + // TODO: Add namespace, replicat_all flag back + if (peerConfig.replicateAllUserTables()) { + throw new DoNotRetryIOException( + "Only support replicated table config for sync replication peer"); + } + if (peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) { + throw new DoNotRetryIOException( + "Only support replicated table config for sync replication peer"); + } + if (peerConfig.getTableCFsMap() == null || peerConfig.getTableCFsMap().isEmpty()) { + throw new DoNotRetryIOException("Need config replicated tables for sync replication peer"); + } + for (List cfs : peerConfig.getTableCFsMap().values()) { + if (cfs != null && !cfs.isEmpty()) { + throw new DoNotRetryIOException( + "Only support replicated table config for sync replication peer"); + } + } + Path remoteWALDir = new Path(peerConfig.getRemoteWALDir()); + if (!remoteWALDir.isAbsolute()) { + throw new DoNotRetryIOException( + "The remote WAL directory " + peerConfig.getRemoteWALDir() + " is not absolute"); + } + URI remoteWALDirUri = remoteWALDir.toUri(); + if (remoteWALDirUri.getScheme() == null || remoteWALDirUri.getAuthority() == null) { + throw new DoNotRetryIOException("The remote WAL directory " + peerConfig.getRemoteWALDir() + + " is not qualified, you must provide scheme and authority"); + } + } + /** * Set a namespace in the peer config means that all tables in this namespace will be replicated * to the peer cluster. @@ -313,7 +436,7 @@ public class ReplicationPeerManager { private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) throws DoNotRetryIOException { String filterCSV = peerConfig.getConfiguration() - .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY); + .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY); if (filterCSV != null && !filterCSV.isEmpty()) { String[] filters = filterCSV.split(","); for (String filter : filters) { @@ -353,7 +476,8 @@ public class ReplicationPeerManager { for (String peerId : peerStorage.listPeerIds()) { ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId); boolean enabled = peerStorage.isPeerEnabled(peerId); - peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig)); + SyncReplicationState state = peerStorage.getPeerSyncReplicationState(peerId); + peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig, state)); } return new ReplicationPeerManager(peerStorage, ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALManager.java new file mode 100644 index 0000000..377c9f1 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALManager.java @@ -0,0 +1,218 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.replication; + +import static org.apache.hadoop.hbase.replication.ReplicationUtils.REMOTE_WAL_REPLAY_SUFFIX; +import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerRemoteWALDir; +import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerReplayWALDir; +import static org.apache.hadoop.hbase.replication.ReplicationUtils.getPeerSnapshotWALDir; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.master.MasterServices; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + +@InterfaceAudience.Private +public class SyncReplicationReplayWALManager { + + private static final Logger LOG = + LoggerFactory.getLogger(SyncReplicationReplayWALManager.class); + + private final MasterServices services; + + private final FileSystem fs; + + private final Path walRootDir; + + private final Path remoteWALDir; + + private final ZKSyncReplicationReplayWALWorkerStorage workerStorage; + + private final Map> workers = new HashMap<>(); + + private final Object workerLock = new Object(); + + public SyncReplicationReplayWALManager(MasterServices services) + throws IOException, ReplicationException { + this.services = services; + this.fs = services.getMasterFileSystem().getWALFileSystem(); + this.walRootDir = services.getMasterFileSystem().getWALRootDir(); + this.remoteWALDir = new Path(this.walRootDir, ReplicationUtils.REMOTE_WAL_DIR_NAME); + this.workerStorage = new ZKSyncReplicationReplayWALWorkerStorage(services.getZooKeeper(), + services.getConfiguration()); + checkReplayingWALDir(); + } + + private void checkReplayingWALDir() throws IOException, ReplicationException { + FileStatus[] files = fs.listStatus(remoteWALDir); + for (FileStatus file : files) { + String name = file.getPath().getName(); + if (name.endsWith(REMOTE_WAL_REPLAY_SUFFIX)) { + String peerId = name.substring(0, name.length() - REMOTE_WAL_REPLAY_SUFFIX.length()); + workers.put(peerId, workerStorage.getPeerWorkers(peerId)); + } + } + } + + public void registerPeer(String peerId) throws ReplicationException { + workers.put(peerId, new HashSet<>()); + workerStorage.addPeer(peerId); + } + + public void unregisterPeer(String peerId) throws ReplicationException { + workers.remove(peerId); + workerStorage.removePeer(peerId); + } + + public ServerName getPeerWorker(String peerId) throws ReplicationException { + Optional worker = Optional.empty(); + ServerName workerServer = null; + synchronized (workerLock) { + worker = services.getServerManager().getOnlineServers().keySet().stream() + .filter(server -> !workers.get(peerId).contains(server)).findFirst(); + if (worker.isPresent()) { + workerServer = worker.get(); + workers.get(peerId).add(workerServer); + } + } + if (workerServer != null) { + workerStorage.addPeerWorker(peerId, workerServer); + } + return workerServer; + } + + public void removePeerWorker(String peerId, ServerName worker) throws ReplicationException { + synchronized (workerLock) { + workers.get(peerId).remove(worker); + } + workerStorage.removePeerWorker(peerId, worker); + } + public void createPeerRemoteWALDir(String peerId) throws IOException { + Path peerRemoteWALDir = getPeerRemoteWALDir(remoteWALDir, peerId); + if (!fs.exists(peerRemoteWALDir) && !fs.mkdirs(peerRemoteWALDir)) { + throw new IOException("Unable to mkdir " + peerRemoteWALDir); + } + } + + private void rename(Path src, Path dst, String peerId) throws IOException { + if (fs.exists(src)) { + deleteDir(dst, peerId); + if (!fs.rename(src, dst)) { + throw new IOException( + "Failed to rename dir from " + src + " to " + dst + " for peer id=" + peerId); + } + LOG.info("Renamed dir from {} to {} for peer id={}", src, dst, peerId); + } else if (!fs.exists(dst)) { + throw new IOException( + "Want to rename from " + src + " to " + dst + ", but they both do not exist"); + } + } + + public void renameToPeerReplayWALDir(String peerId) throws IOException { + rename(getPeerRemoteWALDir(remoteWALDir, peerId), getPeerReplayWALDir(remoteWALDir, peerId), + peerId); + } + + public void renameToPeerSnapshotWALDir(String peerId) throws IOException { + rename(getPeerReplayWALDir(remoteWALDir, peerId), getPeerSnapshotWALDir(remoteWALDir, peerId), + peerId); + } + + public List getReplayWALsAndCleanUpUnusedFiles(String peerId) throws IOException { + Path peerReplayWALDir = getPeerReplayWALDir(remoteWALDir, peerId); + for (FileStatus status : fs.listStatus(peerReplayWALDir, + p -> p.getName().endsWith(ReplicationUtils.RENAME_WAL_SUFFIX))) { + Path src = status.getPath(); + String srcName = src.getName(); + String dstName = + srcName.substring(0, srcName.length() - ReplicationUtils.RENAME_WAL_SUFFIX.length()); + FSUtils.renameFile(fs, src, new Path(src.getParent(), dstName)); + } + List wals = new ArrayList<>(); + for (FileStatus status : fs.listStatus(peerReplayWALDir)) { + Path path = status.getPath(); + if (path.getName().endsWith(ReplicationUtils.SYNC_WAL_SUFFIX)) { + wals.add(path); + } else { + if (!fs.delete(path, true)) { + LOG.warn("Can not delete unused file: " + path); + } + } + } + return wals; + } + + public void snapshotPeerReplayWALDir(String peerId) throws IOException { + Path peerReplayWALDir = getPeerReplayWALDir(remoteWALDir, peerId); + if (fs.exists(peerReplayWALDir) && !fs.delete(peerReplayWALDir, true)) { + throw new IOException( + "Failed to remove replay wals dir " + peerReplayWALDir + " for peer id=" + peerId); + } + } + + private void deleteDir(Path dir, String peerId) throws IOException { + if (!fs.delete(dir, true) && fs.exists(dir)) { + throw new IOException("Failed to remove dir " + dir + " for peer id=" + peerId); + } + } + + public void removePeerRemoteWALs(String peerId) throws IOException { + deleteDir(getPeerRemoteWALDir(remoteWALDir, peerId), peerId); + deleteDir(getPeerReplayWALDir(remoteWALDir, peerId), peerId); + deleteDir(getPeerSnapshotWALDir(remoteWALDir, peerId), peerId); + } + + public String removeWALRootPath(Path path) { + String pathStr = path.toString(); + // remove the "/" too. + return pathStr.substring(walRootDir.toString().length() + 1); + } + + public void finishReplayWAL(String wal) throws IOException { + Path walPath = new Path(walRootDir, wal); + fs.truncate(walPath, 0); + } + + public boolean isReplayWALFinished(String wal) throws IOException { + Path walPath = new Path(walRootDir, wal); + return fs.getFileStatus(walPath).getLen() == 0; + } + + @VisibleForTesting + public Path getRemoteWALDir() { + return remoteWALDir; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALProcedure.java new file mode 100644 index 0000000..26d6a3f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALProcedure.java @@ -0,0 +1,164 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.replication; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SyncReplicationReplayWALState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SyncReplicationReplayWALStateData; + +@InterfaceAudience.Private +public class SyncReplicationReplayWALProcedure + extends StateMachineProcedure + implements PeerProcedureInterface { + + private static final Logger LOG = + LoggerFactory.getLogger(SyncReplicationReplayWALProcedure.class); + + private String peerId; + + private ServerName worker = null; + + private List wals; + + public SyncReplicationReplayWALProcedure() { + } + + public SyncReplicationReplayWALProcedure(String peerId, List wals) { + this.peerId = peerId; + this.wals = wals; + } + + @Override protected Flow executeFromState(MasterProcedureEnv env, + SyncReplicationReplayWALState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + SyncReplicationReplayWALManager syncReplicationReplayWALManager = + env.getMasterServices().getSyncReplicationReplayWALManager(); + switch (state) { + case ASSIGN_WORKER: + try { + worker = syncReplicationReplayWALManager.getPeerWorker(peerId); + } catch (ReplicationException e) { + LOG.info("Failed to get worker to replay wals {} for peer id={}, retry", wals, peerId); + throw new ProcedureYieldException(); + } + if (worker == null) { + LOG.info("No worker to replay wals {} for peer id={}, retry", wals, peerId); + setNextState(SyncReplicationReplayWALState.ASSIGN_WORKER); + } else { + setNextState(SyncReplicationReplayWALState.DISPATCH_WALS_TO_WORKER); + } + return Flow.HAS_MORE_STATE; + case DISPATCH_WALS_TO_WORKER: + addChildProcedure(new SyncReplicationReplayWALRemoteProcedure(peerId, wals, worker)); + setNextState(SyncReplicationReplayWALState.RELEASE_WORKER); + return Flow.HAS_MORE_STATE; + case RELEASE_WORKER: + boolean finished = false; + try { + finished = syncReplicationReplayWALManager.isReplayWALFinished(wals.get(0)); + } catch (IOException e) { + LOG.info("Failed to check whether replay wals {} finished for peer id={}", wals, peerId); + throw new ProcedureYieldException(); + } + try { + syncReplicationReplayWALManager.removePeerWorker(peerId, worker); + } catch (ReplicationException e) { + LOG.info("Failed to remove worker for peer id={}, retry", peerId); + throw new ProcedureYieldException(); + } + if (!finished) { + LOG.info("Failed to replay wals {} for peer id={}, retry", wals, peerId); + setNextState(SyncReplicationReplayWALState.ASSIGN_WORKER); + return Flow.HAS_MORE_STATE; + } + return Flow.NO_MORE_STATE; + default: + throw new UnsupportedOperationException("unhandled state=" + state); + } + } + + @Override + protected void rollbackState(MasterProcedureEnv env, + SyncReplicationReplayWALState state) + throws IOException, InterruptedException { + if (state == getInitialState()) { + return; + } + throw new UnsupportedOperationException(); + } + + @Override + protected SyncReplicationReplayWALState getState(int state) { + return SyncReplicationReplayWALState.forNumber(state); + } + + @Override + protected int getStateId( + SyncReplicationReplayWALState state) { + return state.getNumber(); + } + + @Override + protected SyncReplicationReplayWALState getInitialState() { + return SyncReplicationReplayWALState.ASSIGN_WORKER; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) + throws IOException { + SyncReplicationReplayWALStateData.Builder builder = + SyncReplicationReplayWALStateData.newBuilder(); + builder.setPeerId(peerId); + wals.stream().forEach(builder::addWal); + serializer.serialize(builder.build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + SyncReplicationReplayWALStateData data = + serializer.deserialize(SyncReplicationReplayWALStateData.class); + peerId = data.getPeerId(); + wals = new ArrayList<>(); + data.getWalList().forEach(wals::add); + } + + @Override + public String getPeerId() { + return peerId; + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.SYNC_REPLICATION_REPLAY_WAL; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALRemoteProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALRemoteProcedure.java new file mode 100644 index 0000000..9f4f330 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/SyncReplicationReplayWALRemoteProcedure.java @@ -0,0 +1,213 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.replication; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface; +import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.ServerOperation; +import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException; +import org.apache.hadoop.hbase.procedure2.Procedure; +import org.apache.hadoop.hbase.procedure2.ProcedureEvent; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure; +import org.apache.hadoop.hbase.procedure2.RemoteProcedureException; +import org.apache.hadoop.hbase.replication.regionserver.ReplaySyncReplicationWALCallable; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ReplaySyncReplicationWALParameter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SyncReplicationReplayWALRemoteStateData; + +@InterfaceAudience.Private +public class SyncReplicationReplayWALRemoteProcedure extends Procedure + implements RemoteProcedure, PeerProcedureInterface { + + private static final Logger LOG = + LoggerFactory.getLogger(SyncReplicationReplayWALRemoteProcedure.class); + + private String peerId; + + private ServerName targetServer; + + private List wals; + + private boolean dispatched; + + private ProcedureEvent event; + + private boolean succ; + + public SyncReplicationReplayWALRemoteProcedure() { + } + + public SyncReplicationReplayWALRemoteProcedure(String peerId, List wals, + ServerName targetServer) { + this.peerId = peerId; + this.wals = wals; + this.targetServer = targetServer; + } + + @Override + public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) { + ReplaySyncReplicationWALParameter.Builder builder = + ReplaySyncReplicationWALParameter.newBuilder(); + builder.setPeerId(peerId); + wals.stream().forEach(builder::addWal); + return new ServerOperation(this, getProcId(), ReplaySyncReplicationWALCallable.class, + builder.build().toByteArray()); + } + + @Override + public void remoteCallFailed(MasterProcedureEnv env, ServerName remote, IOException exception) { + complete(env, exception); + } + + @Override + public void remoteOperationCompleted(MasterProcedureEnv env) { + complete(env, null); + } + + @Override + public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) { + complete(env, error); + } + + private void complete(MasterProcedureEnv env, Throwable error) { + if (event == null) { + LOG.warn("procedure event for {} is null, maybe the procedure is created when recovery", + getProcId()); + return; + } + if (error != null) { + LOG.warn("Replay wals {} on {} failed for peer id={}", wals, targetServer, peerId, error); + this.succ = false; + } else { + truncateWALs(env); + LOG.info("Replay wals {} on {} succeed for peer id={}", wals, targetServer, peerId); + this.succ = true; + } + event.wake(env.getProcedureScheduler()); + event = null; + } + + /** + * Only truncate wals one by one when task succeed. The parent procedure will check the first + * wal length to know whether this task succeed. + */ + private void truncateWALs(MasterProcedureEnv env) { + String firstWal = wals.get(0); + try { + env.getMasterServices().getSyncReplicationReplayWALManager().finishReplayWAL(firstWal); + } catch (IOException e) { + // As it is idempotent to rerun this task. Just ignore this exception and return. + LOG.warn("Failed to truncate wal {} for peer id={}", firstWal, peerId, e); + return; + } + for (int i = 1; i < wals.size(); i++) { + String wal = wals.get(i); + try { + env.getMasterServices().getSyncReplicationReplayWALManager().finishReplayWAL(wal); + } catch (IOException e1) { + try { + // retry + env.getMasterServices().getSyncReplicationReplayWALManager().finishReplayWAL(wal); + } catch (IOException e2) { + // As the parent procedure only check the first wal length. Just ignore this exception. + LOG.warn("Failed to truncate wal {} for peer id={}", wal, peerId, e2); + } + } + } + } + + @Override + protected Procedure[] execute(MasterProcedureEnv env) + throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException { + if (dispatched) { + if (succ) { + return null; + } + // retry + dispatched = false; + } + + // Dispatch task to target server + try { + env.getRemoteDispatcher().addOperationToNode(targetServer, this); + } catch (FailedRemoteDispatchException e) { + LOG.warn( + "Can not add remote operation for replay wals {} on {} for peer id={}, " + + "this usually because the server is already dead, retry", + wals, targetServer, peerId); + throw new ProcedureYieldException(); + } + dispatched = true; + event = new ProcedureEvent<>(this); + event.suspendIfNotReady(this); + throw new ProcedureSuspendedException(); + } + + @Override + protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException { + throw new UnsupportedOperationException(); + } + + @Override + protected boolean abort(MasterProcedureEnv env) { + return false; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) + throws IOException { + SyncReplicationReplayWALRemoteStateData.Builder builder = + SyncReplicationReplayWALRemoteStateData.newBuilder().setPeerId(peerId) + .setTargetServer(ProtobufUtil.toServerName(targetServer)); + wals.stream().forEach(builder::addWal); + serializer.serialize(builder.build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + SyncReplicationReplayWALRemoteStateData data = + serializer.deserialize(SyncReplicationReplayWALRemoteStateData.class); + peerId = data.getPeerId(); + wals = new ArrayList<>(); + data.getWalList().forEach(wals::add); + targetServer = ProtobufUtil.toServerName(data.getTargetServer()); + } + + @Override + public String getPeerId() { + return peerId; + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.SYNC_REPLICATION_REPLAY_WAL_REMOTE; + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java new file mode 100644 index 0000000..c650974 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java @@ -0,0 +1,313 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.replication; + +import java.io.IOException; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; +import org.apache.hadoop.hbase.master.MasterCoprocessorHost; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerSyncReplicationStateTransitionState; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData; + +/** + * The procedure for transit current sync replication state for a synchronous replication peer. + */ +@InterfaceAudience.Private +public class TransitPeerSyncReplicationStateProcedure + extends AbstractPeerProcedure { + + private static final Logger LOG = + LoggerFactory.getLogger(TransitPeerSyncReplicationStateProcedure.class); + + private SyncReplicationState fromState; + + private SyncReplicationState toState; + + private boolean enabled; + + public TransitPeerSyncReplicationStateProcedure() { + } + + public TransitPeerSyncReplicationStateProcedure(String peerId, SyncReplicationState state) { + super(peerId); + this.toState = state; + } + + @Override + public PeerOperationType getPeerOperationType() { + return PeerOperationType.TRANSIT_SYNC_REPLICATION_STATE; + } + + @Override + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.serializeStateData(serializer); + TransitPeerSyncReplicationStateStateData.Builder builder = + TransitPeerSyncReplicationStateStateData.newBuilder() + .setToState(ReplicationPeerConfigUtil.toSyncReplicationState(toState)); + if (fromState != null) { + builder.setFromState(ReplicationPeerConfigUtil.toSyncReplicationState(fromState)); + } + serializer.serialize(builder.build()); + } + + @Override + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { + super.deserializeStateData(serializer); + TransitPeerSyncReplicationStateStateData data = + serializer.deserialize(TransitPeerSyncReplicationStateStateData.class); + toState = ReplicationPeerConfigUtil.toSyncReplicationState(data.getToState()); + if (data.hasFromState()) { + fromState = ReplicationPeerConfigUtil.toSyncReplicationState(data.getFromState()); + } + } + + @Override + protected PeerSyncReplicationStateTransitionState getState(int stateId) { + return PeerSyncReplicationStateTransitionState.forNumber(stateId); + } + + @Override + protected int getStateId(PeerSyncReplicationStateTransitionState state) { + return state.getNumber(); + } + + @Override + protected PeerSyncReplicationStateTransitionState getInitialState() { + return PeerSyncReplicationStateTransitionState.PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION; + } + + private void preTransit(MasterProcedureEnv env) throws IOException { + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + cpHost.preTransitReplicationPeerSyncReplicationState(peerId, toState); + } + ReplicationPeerDescription desc = + env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, toState); + if (toState == SyncReplicationState.ACTIVE) { + Path remoteWALDirForPeer = + ReplicationUtils.getPeerRemoteWALDir(desc.getPeerConfig().getRemoteWALDir(), peerId); + // check whether the remote wal directory is present + if (!remoteWALDirForPeer.getFileSystem(env.getMasterConfiguration()) + .exists(remoteWALDirForPeer)) { + throw new DoNotRetryIOException( + "The remote WAL directory " + remoteWALDirForPeer + " does not exist"); + } + } + fromState = desc.getSyncReplicationState(); + enabled = desc.isEnabled(); + } + + private void postTransit(MasterProcedureEnv env) throws IOException { + LOG.info( + "Successfully transit current cluster state from {} to {} for sync replication peer {}", + fromState, toState, peerId); + MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); + if (cpHost != null) { + env.getMasterCoprocessorHost().postTransitReplicationPeerSyncReplicationState(peerId, + fromState, toState); + } + } + + private void reopenRegions(MasterProcedureEnv env) { + addChildProcedure( + env.getReplicationPeerManager().getPeerConfig(peerId).get().getTableCFsMap().keySet().stream() + .map(ReopenTableRegionsProcedure::new).toArray(ReopenTableRegionsProcedure[]::new)); + } + + private void createDirForRemoteWAL(MasterProcedureEnv env) + throws ProcedureYieldException, IOException { + MasterFileSystem mfs = env.getMasterFileSystem(); + Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME); + Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId); + FileSystem walFs = mfs.getWALFileSystem(); + if (walFs.exists(remoteWALDirForPeer)) { + LOG.warn("Wal dir {} already exists, usually this should not happen, continue anyway", + remoteWALDirForPeer); + } else if (!walFs.mkdirs(remoteWALDirForPeer)) { + LOG.warn("Can not create remote wal dir {}", remoteWALDirForPeer); + throw new ProcedureYieldException(); + } + } + + private void setNextStateAfterRefreshBegin() { + if (fromState.equals(SyncReplicationState.ACTIVE)) { + setNextState(toState.equals(SyncReplicationState.STANDBY) + ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER + : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER); + } else if (fromState.equals(SyncReplicationState.DOWNGRADE_ACTIVE)) { + setNextState(toState.equals(SyncReplicationState.STANDBY) + ? PeerSyncReplicationStateTransitionState.REMOVE_ALL_REPLICATION_QUEUES_IN_PEER + : PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER); + } else { + assert toState.equals(SyncReplicationState.DOWNGRADE_ACTIVE); + setNextState(PeerSyncReplicationStateTransitionState.REPLAY_REMOTE_WAL_IN_PEER); + } + } + + private void setNextStateAfterRefreshEnd() { + if (toState == SyncReplicationState.STANDBY) { + setNextState( + enabled ? PeerSyncReplicationStateTransitionState.SYNC_REPLICATION_SET_PEER_ENABLED + : PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL); + } else { + setNextState( + PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION); + } + } + + private void replayRemoteWAL(boolean serial) { + addChildProcedure(new RecoverStandbyProcedure(peerId, serial)); + } + + @Override + protected Flow executeFromState(MasterProcedureEnv env, + PeerSyncReplicationStateTransitionState state) + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { + switch (state) { + case PRE_PEER_SYNC_REPLICATION_STATE_TRANSITION: + try { + preTransit(env); + } catch (IOException e) { + LOG.warn("Failed to call pre CP hook or the pre check is failed for peer {} " + + "when transiting sync replication peer state to {}, " + + "mark the procedure as failure and give up", peerId, toState, e); + setFailure("master-transit-peer-sync-replication-state", e); + return Flow.NO_MORE_STATE; + } + setNextState(PeerSyncReplicationStateTransitionState.SET_PEER_NEW_SYNC_REPLICATION_STATE); + return Flow.HAS_MORE_STATE; + case SET_PEER_NEW_SYNC_REPLICATION_STATE: + try { + env.getReplicationPeerManager().setPeerNewSyncReplicationState(peerId, toState); + if (toState.equals(SyncReplicationState.STANDBY) && enabled) { + // disable the peer if we are going to transit to STANDBY state, as we need to remove + // all the pending replication files. If we do not disable the peer and delete the wal + // queues on zk directly, RS will get NoNode exception when updating the wal position + // and crash. + env.getReplicationPeerManager().disablePeer(peerId); + } + } catch (ReplicationException e) { + LOG.warn("Failed to update peer storage for peer {} when starting transiting sync " + + "replication peer state from {} to {}, retry", peerId, fromState, toState, e); + throw new ProcedureYieldException(); + } + setNextState( + PeerSyncReplicationStateTransitionState.REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_BEGIN); + return Flow.HAS_MORE_STATE; + case REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_BEGIN: + addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream() + .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 0)) + .toArray(RefreshPeerProcedure[]::new)); + setNextStateAfterRefreshBegin(); + return Flow.HAS_MORE_STATE; + case REPLAY_REMOTE_WAL_IN_PEER: + replayRemoteWAL(env.getReplicationPeerManager().getPeerConfig(peerId).get().isSerial()); + setNextState( + PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE); + return Flow.HAS_MORE_STATE; + case REMOVE_ALL_REPLICATION_QUEUES_IN_PEER: + try { + env.getReplicationPeerManager().removeAllQueues(peerId); + } catch (ReplicationException e) { + LOG.warn("Failed to remove all replication queues peer {} when starting transiting" + + " sync replication peer state from {} to {}, retry", peerId, fromState, toState, e); + throw new ProcedureYieldException(); + } + setNextState(fromState.equals(SyncReplicationState.ACTIVE) + ? PeerSyncReplicationStateTransitionState.REOPEN_ALL_REGIONS_IN_PEER + : PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE); + return Flow.HAS_MORE_STATE; + case REOPEN_ALL_REGIONS_IN_PEER: + reopenRegions(env); + setNextState( + PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE); + return Flow.HAS_MORE_STATE; + case TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE: + try { + env.getReplicationPeerManager().transitPeerSyncReplicationState(peerId, toState); + } catch (ReplicationException e) { + LOG.warn("Failed to update peer storage for peer {} when ending transiting sync " + + "replication peer state from {} to {}, retry", peerId, fromState, toState, e); + throw new ProcedureYieldException(); + } + setNextState( + PeerSyncReplicationStateTransitionState.REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END); + return Flow.HAS_MORE_STATE; + case REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END: + addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream() + .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn, 1)) + .toArray(RefreshPeerProcedure[]::new)); + setNextStateAfterRefreshEnd(); + return Flow.HAS_MORE_STATE; + case SYNC_REPLICATION_SET_PEER_ENABLED: + try { + env.getReplicationPeerManager().enablePeer(peerId); + } catch (ReplicationException e) { + LOG.warn("Failed to set peer enabled for peer {} when transiting sync replication peer " + + "state from {} to {}, retry", peerId, fromState, toState, e); + throw new ProcedureYieldException(); + } + setNextState( + PeerSyncReplicationStateTransitionState.SYNC_REPLICATION_ENABLE_PEER_REFRESH_PEER_ON_RS); + return Flow.HAS_MORE_STATE; + case SYNC_REPLICATION_ENABLE_PEER_REFRESH_PEER_ON_RS: + refreshPeer(env, PeerOperationType.ENABLE); + setNextState(PeerSyncReplicationStateTransitionState.CREATE_DIR_FOR_REMOTE_WAL); + return Flow.HAS_MORE_STATE; + case CREATE_DIR_FOR_REMOTE_WAL: + try { + createDirForRemoteWAL(env); + } catch (IOException e) { + LOG.warn("Failed to create remote wal dir for peer {} when transiting sync replication " + + "peer state from {} to {}, retry", peerId, fromState, toState, e); + throw new ProcedureYieldException(); + } + setNextState( + PeerSyncReplicationStateTransitionState.POST_PEER_SYNC_REPLICATION_STATE_TRANSITION); + return Flow.HAS_MORE_STATE; + case POST_PEER_SYNC_REPLICATION_STATE_TRANSITION: + try { + postTransit(env); + } catch (IOException e) { + LOG.warn( + "Failed to call post CP hook for peer {} when transiting sync replication " + + "peer state from {} to {}, ignore since the procedure has already done", + peerId, fromState, toState, e); + } + return Flow.NO_MORE_STATE; + default: + throw new UnsupportedOperationException("unhandled state=" + state); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ZKSyncReplicationReplayWALWorkerStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ZKSyncReplicationReplayWALWorkerStorage.java new file mode 100644 index 0000000..5991cf0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ZKSyncReplicationReplayWALWorkerStorage.java @@ -0,0 +1,108 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.replication; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.replication.ReplicationException; +import org.apache.hadoop.hbase.replication.ZKReplicationStorageBase; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.hadoop.hbase.zookeeper.ZNodePaths; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; + +@InterfaceAudience.Private +public class ZKSyncReplicationReplayWALWorkerStorage extends ZKReplicationStorageBase { + + public static final String WORKERS_ZNODE = "zookeeper.znode.sync.replication.replaywal.workers"; + + public static final String WORKERS_ZNODE_DEFAULT = "replaywal-workers"; + + /** + * The name of the znode that contains a list of workers to replay wal. + */ + private final String workersZNode; + + public ZKSyncReplicationReplayWALWorkerStorage(ZKWatcher zookeeper, Configuration conf) { + super(zookeeper, conf); + String workersZNodeName = conf.get(WORKERS_ZNODE, WORKERS_ZNODE_DEFAULT); + workersZNode = ZNodePaths.joinZNode(replicationZNode, workersZNodeName); + } + + private String getPeerNode(String peerId) { + return ZNodePaths.joinZNode(workersZNode, peerId); + } + + public void addPeer(String peerId) throws ReplicationException { + try { + ZKUtil.createWithParents(zookeeper, getPeerNode(peerId)); + } catch (KeeperException e) { + throw new ReplicationException( + "Failed to add peer id=" + peerId + " to replaywal-workers storage", e); + } + } + + public void removePeer(String peerId) throws ReplicationException { + try { + ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId)); + } catch (KeeperException e) { + throw new ReplicationException( + "Failed to remove peer id=" + peerId + " to replaywal-workers storage", e); + } + } + + private String getPeerWorkerNode(String peerId, ServerName worker) { + return ZNodePaths.joinZNode(getPeerNode(peerId), worker.getServerName()); + } + + public void addPeerWorker(String peerId, ServerName worker) throws ReplicationException { + try { + ZKUtil.createWithParents(zookeeper, getPeerWorkerNode(peerId, worker)); + } catch (KeeperException e) { + throw new ReplicationException("Failed to add worker=" + worker + " for peer id=" + peerId, + e); + } + } + + public void removePeerWorker(String peerId, ServerName worker) throws ReplicationException { + try { + ZKUtil.deleteNode(zookeeper, getPeerWorkerNode(peerId, worker)); + } catch (KeeperException e) { + throw new ReplicationException("Failed to remove worker=" + worker + " for peer id=" + peerId, + e); + } + } + + public Set getPeerWorkers(String peerId) throws ReplicationException { + try { + List children = ZKUtil.listChildrenNoWatch(zookeeper, getPeerNode(peerId)); + if (children == null) { + return new HashSet<>(); + } + return children.stream().map(ServerName::valueOf).collect(Collectors.toSet()); + } catch (KeeperException e) { + throw new ReplicationException("Failed to list workers for peer id=" + peerId, e); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java index 157ad1b..c1b3911 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java @@ -16,7 +16,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.protobuf; @@ -24,25 +23,25 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.PrivateCellUtil; -import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.io.SizedCellScanner; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.HBaseRpcControllerImpl; -import org.apache.hadoop.hbase.wal.WALEdit; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; -import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; +import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; + @InterfaceAudience.Private public class ReplicationProtbufUtil { /** 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 d71e988..250b163 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 @@ -144,6 +144,7 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; +import org.apache.hadoop.hbase.regionserver.compactions.ForbidMajorCompactionChecker; import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector; @@ -1992,6 +1993,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return compact(compaction, store, throughputController, null); } + private boolean shouldForbidMajorCompaction() { + if (rsServices != null && rsServices.getReplicationSourceService() != null) { + return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider() + .checkState(getRegionInfo().getTable(), ForbidMajorCompactionChecker.get()); + } + return false; + } + public boolean compact(CompactionContext compaction, HStore store, ThroughputController throughputController, User user) throws IOException { assert compaction != null && compaction.hasSelection(); @@ -2001,6 +2010,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi store.cancelRequestedCompaction(compaction); return false; } + + if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) { + LOG.warn("Skipping major compaction on " + this + + " because this cluster is transiting sync replication state" + + " from STANDBY to DOWNGRADE_ACTIVE"); + store.cancelRequestedCompaction(compaction); + return false; + } + MonitoredTask status = null; boolean requestNeedsCancellation = true; /* @@ -4337,12 +4355,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi /** * Add updates first to the wal and then add values to memstore. + *

* Warning: Assumption is caller has lock on passed in row. * @param edits Cell updates by column - * @throws IOException */ - void put(final byte [] row, byte [] family, List edits) - throws IOException { + void put(final byte[] row, byte[] family, List edits) throws IOException { NavigableMap> familyMap; familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); 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 054bebd..9509ea7 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 @@ -1802,26 +1802,30 @@ public class HRegionServer extends HasThread implements * be hooked up to WAL. */ private void setupWALAndReplication() throws IOException { - WALFactory factory = new WALFactory(conf, serverName.toString()); - - // TODO Replication make assumptions here based on the default filesystem impl - Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME); - String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString()); - - Path logDir = new Path(walRootDir, logName); - LOG.debug("logDir={}", logDir); - if (this.walFs.exists(logDir)) { - throw new RegionServerRunningException( - "Region server has already created directory at " + this.serverName.toString()); - } - // Always create wal directory as now we need this when master restarts to find out the live - // region servers. - if (!this.walFs.mkdirs(logDir)) { - throw new IOException("Can not create wal directory " + logDir); - } - // Instantiate replication if replication enabled. Pass it the log directories. - createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir, - factory.getWALProvider()); + boolean isMasterNoTableOrSystemTableOnly = this instanceof HMaster && + (!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf)); + WALFactory factory = + new WALFactory(conf, serverName.toString(), !isMasterNoTableOrSystemTableOnly); + if (!isMasterNoTableOrSystemTableOnly) { + // TODO Replication make assumptions here based on the default filesystem impl + Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME); + String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString()); + + Path logDir = new Path(walRootDir, logName); + LOG.debug("logDir={}", logDir); + if (this.walFs.exists(logDir)) { + throw new RegionServerRunningException( + "Region server has already created directory at " + this.serverName.toString()); + } + // Always create wal directory as now we need this when master restarts to find out the live + // region servers. + if (!this.walFs.mkdirs(logDir)) { + throw new IOException("Can not create wal directory " + logDir); + } + // Instantiate replication if replication enabled. Pass it the log directories. + createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir, + factory.getWALProvider()); + } this.walFactory = factory; } @@ -1925,6 +1929,8 @@ public class HRegionServer extends HasThread implements } this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER, conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2)); + this.executorService.startExecutorService(ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL, + conf.getInt("hbase.regionserver.executor.replay.sync.replication.wal.threads", 1)); Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller", uncaughtExceptionHandler); @@ -1957,8 +1963,7 @@ public class HRegionServer extends HasThread implements sinkConf.setInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER, 1); if (this.csm != null) { // SplitLogWorker needs csm. If none, don't start this. - this.splitLogWorker = new SplitLogWorker(this, sinkConf, this, - this, walFactory); + this.splitLogWorker = new SplitLogWorker(sinkConf, this, this, walFactory); splitLogWorker.start(); } else { LOG.warn("SplitLogWorker Service NOT started; CoordinatedStateManager is null"); @@ -2469,16 +2474,15 @@ public class HRegionServer extends HasThread implements * @return Return the object that implements the replication * source executorService. */ - @VisibleForTesting + @Override public ReplicationSourceService getReplicationSourceService() { return replicationSourceHandler; } /** - * @return Return the object that implements the replication - * sink executorService. + * @return Return the object that implements the replication sink executorService. */ - ReplicationSinkService getReplicationSinkService() { + public ReplicationSinkService getReplicationSinkService() { return replicationSinkHandler; } @@ -2871,14 +2875,14 @@ public class HRegionServer extends HasThread implements /** * @return Return the walRootDir. */ - protected Path getWALRootDir() { + public Path getWALRootDir() { return walRootDir; } /** * @return Return the walFs. */ - protected FileSystem getWALFileSystem() { + public FileSystem getWALFileSystem() { return walFs; } @@ -2943,11 +2947,6 @@ public class HRegionServer extends HasThread implements */ private static void createNewReplicationInstance(Configuration conf, HRegionServer server, FileSystem walFs, Path walDir, Path oldWALDir, WALProvider walProvider) throws IOException { - if ((server instanceof HMaster) && - (!LoadBalancer.isTablesOnMaster(conf) || LoadBalancer.isSystemTablesOnlyOnMaster(conf))) { - return; - } - // read in the name of the source replication class from the config file. String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME, HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java index 55c5219..05a8fdf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/LogRoller.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver; import java.io.Closeable; import java.io.IOException; +import java.util.Iterator; +import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -30,6 +32,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; +import org.apache.hadoop.hbase.regionserver.wal.WALClosedException; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HasThread; import org.apache.hadoop.hbase.wal.WAL; @@ -177,17 +180,24 @@ public class LogRoller extends HasThread implements Closeable { rollLock.lock(); // FindBugs UL_UNRELEASED_LOCK_EXCEPTION_PATH try { this.lastrolltime = now; - for (Entry entry : walNeedsRoll.entrySet()) { + for (Iterator> iter = walNeedsRoll.entrySet().iterator(); iter + .hasNext();) { + Entry entry = iter.next(); final WAL wal = entry.getKey(); // Force the roll if the logroll.period is elapsed or if a roll was requested. // The returned value is an array of actual region names. - final byte [][] regionsToFlush = wal.rollWriter(periodic || - entry.getValue().booleanValue()); - walNeedsRoll.put(wal, Boolean.FALSE); - if (regionsToFlush != null) { - for (byte[] r : regionsToFlush) { - scheduleFlush(r); + try { + final byte[][] regionsToFlush = + wal.rollWriter(periodic || entry.getValue().booleanValue()); + walNeedsRoll.put(wal, Boolean.FALSE); + if (regionsToFlush != null) { + for (byte[] r : regionsToFlush) { + scheduleFlush(r); + } } + } catch (WALClosedException e) { + LOG.warn("WAL has been closed. Skipping rolling of writer and just remove it", e); + iter.remove(); } } } catch (FailedLogCloseException e) { @@ -234,10 +244,8 @@ public class LogRoller extends HasThread implements Closeable { } /** - * For testing only * @return true if all WAL roll finished */ - @VisibleForTesting public boolean walRollFinished() { for (boolean needRoll : walNeedsRoll.values()) { if (needRoll) { @@ -247,9 +255,23 @@ public class LogRoller extends HasThread implements Closeable { return true; } + /** + * Wait until all wals have been rolled after calling {@link #requestRollAll()}. + */ + public void waitUntilWalRollFinished() throws InterruptedException { + while (!walRollFinished()) { + Thread.sleep(100); + } + } + @Override public void close() { running = false; interrupt(); } + + @VisibleForTesting + Map getWalNeedsRoll() { + return this.walNeedsRoll; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 8828a22..bdb86d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -120,6 +120,9 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler; import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.regionserver.RejectReplicationRequestStateChecker; +import org.apache.hadoop.hbase.replication.regionserver.RejectRequestsFromClientStateChecker; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessChecker; @@ -2202,9 +2205,26 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } + private void checkShouldRejectReplicationRequest(List entries) throws IOException { + ReplicationSourceService replicationSource = regionServer.getReplicationSourceService(); + if (replicationSource == null || entries.isEmpty()) { + return; + } + // We can ensure that all entries are for one peer, so only need to check one entry's + // table name. if the table hit sync replication at peer side and the peer cluster + // is (or is transiting to) state ACTIVE or DOWNGRADE_ACTIVE, we should reject to apply + // those entries according to the design doc. + TableName table = TableName.valueOf(entries.get(0).getKey().getTableName().toByteArray()); + if (replicationSource.getSyncReplicationPeerInfoProvider().checkState(table, + RejectReplicationRequestStateChecker.get())) { + throw new DoNotRetryIOException( + "Reject to apply to sink cluster because sync replication state of sink cluster " + + "is ACTIVE or DOWNGRADE_ACTIVE, table: " + table); + } + } + /** * Replicate WAL entries on the region server. - * * @param controller the RPC controller * @param request the request * @throws ServiceException @@ -2218,7 +2238,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, if (regionServer.replicationSinkHandler != null) { requestCount.increment(); List entries = request.getEntryList(); - CellScanner cellScanner = ((HBaseRpcController)controller).cellScanner(); + checkShouldRejectReplicationRequest(entries); + CellScanner cellScanner = ((HBaseRpcController) controller).cellScanner(); regionServer.getRegionServerCoprocessorHost().preReplicateLogEntries(); regionServer.replicationSinkHandler.replicateLogEntries(entries, cellScanner, request.getReplicationClusterId(), request.getSourceBaseNamespaceDirPath(), @@ -2431,6 +2452,18 @@ public class RSRpcServices implements HBaseRPCErrorHandler, return region.execService(execController, serviceCall); } + private boolean shouldRejectRequestsFromClient(HRegion region) { + return regionServer.getReplicationSourceService().getSyncReplicationPeerInfoProvider() + .checkState(region.getRegionInfo().getTable(), RejectRequestsFromClientStateChecker.get()); + } + + private void rejectIfInStandByState(HRegion region) throws DoNotRetryIOException { + if (shouldRejectRequestsFromClient(region)) { + throw new DoNotRetryIOException( + region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state."); + } + } + /** * Get data from a table. * @@ -2439,8 +2472,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @throws ServiceException */ @Override - public GetResponse get(final RpcController controller, - final GetRequest request) throws ServiceException { + public GetResponse get(final RpcController controller, final GetRequest request) + throws ServiceException { long before = EnvironmentEdgeManager.currentTime(); OperationQuota quota = null; HRegion region = null; @@ -2449,6 +2482,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, requestCount.increment(); rpcGetRequestCount.increment(); region = getRegion(request.getRegion()); + rejectIfInStandByState(region); GetResponse.Builder builder = GetResponse.newBuilder(); ClientProtos.Get get = request.getGet(); @@ -2587,16 +2621,45 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } } + private void failRegionAction(MultiResponse.Builder responseBuilder, + RegionActionResult.Builder regionActionResultBuilder, RegionAction regionAction, + CellScanner cellScanner, Throwable error) { + rpcServer.getMetrics().exception(error); + regionActionResultBuilder.setException(ResponseConverter.buildException(error)); + responseBuilder.addRegionActionResult(regionActionResultBuilder.build()); + // All Mutations in this RegionAction not executed as we can not see the Region online here + // in this RS. Will be retried from Client. Skipping all the Cells in CellScanner + // corresponding to these Mutations. + if (cellScanner != null) { + skipCellsForMutations(regionAction.getActionList(), cellScanner); + } + } + + private boolean isReplicationRequest(Action action) { + // replication request can only be put or delete. + if (!action.hasMutation()) { + return false; + } + MutationProto mutation = action.getMutation(); + MutationType type = mutation.getMutateType(); + if (type != MutationType.PUT && type != MutationType.DELETE) { + return false; + } + // replication will set a special attribute so we can make use of it to decide whether a request + // is for replication. + return mutation.getAttributeList().stream().map(p -> p.getName()) + .filter(n -> n.equals(ReplicationUtils.REPLICATION_ATTR_NAME)).findAny().isPresent(); + } + /** * Execute multiple actions on a table: get, mutate, and/or execCoprocessor - * * @param rpcc the RPC controller * @param request the multi request * @throws ServiceException */ @Override public MultiResponse multi(final RpcController rpcc, final MultiRequest request) - throws ServiceException { + throws ServiceException { try { checkOpen(); } catch (IOException ie) { @@ -2636,17 +2699,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler, region = getRegion(regionSpecifier); quota = getRpcQuotaManager().checkQuota(region, regionAction.getActionList()); } catch (IOException e) { - rpcServer.getMetrics().exception(e); - regionActionResultBuilder.setException(ResponseConverter.buildException(e)); - responseBuilder.addRegionActionResult(regionActionResultBuilder.build()); - // All Mutations in this RegionAction not executed as we can not see the Region online here - // in this RS. Will be retried from Client. Skipping all the Cells in CellScanner - // corresponding to these Mutations. - skipCellsForMutations(regionAction.getActionList(), cellScanner); + failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, e); continue; // For this region it's a failure. } - + boolean rejectIfFromClient = shouldRejectRequestsFromClient(region); if (regionAction.hasAtomic() && regionAction.getAtomic()) { + // We only allow replication in standby state and it will not set the atomic flag. + if (rejectIfFromClient) { + failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, + new DoNotRetryIOException( + region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state")); + quota.close(); + continue; + } // How does this call happen? It may need some work to play well w/ the surroundings. // Need to return an item per Action along w/ Action index. TODO. try { @@ -2677,6 +2742,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler, regionActionResultBuilder.setException(ResponseConverter.buildException(e)); } } else { + if (rejectIfFromClient && regionAction.getActionCount() > 0 && + !isReplicationRequest(regionAction.getAction(0))) { + // fail if it is not a replication request + failRegionAction(responseBuilder, regionActionResultBuilder, regionAction, cellScanner, + new DoNotRetryIOException( + region.getRegionInfo().getRegionNameAsString() + " is in STANDBY state")); + quota.close(); + continue; + } // doNonAtomicRegionMutation manages the exception internally if (context != null && closeCallBack == null) { // An RpcCallBack that creates a list of scanners that needs to perform callBack @@ -2692,7 +2766,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, responseBuilder.addRegionActionResult(regionActionResultBuilder.build()); quota.close(); ClientProtos.RegionLoadStats regionLoadStats = region.getLoadStatistics(); - if(regionLoadStats != null) { + if (regionLoadStats != null) { regionStats.put(regionSpecifier, regionLoadStats); } } @@ -2751,8 +2825,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler, * @param request the mutate request */ @Override - public MutateResponse mutate(final RpcController rpcc, - final MutateRequest request) throws ServiceException { + public MutateResponse mutate(final RpcController rpcc, final MutateRequest request) + throws ServiceException { // rpc controller is how we bring in data via the back door; it is unprotobuf'ed data. // It is also the conduit via which we pass back data. HBaseRpcController controller = (HBaseRpcController)rpcc; @@ -2772,6 +2846,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, requestCount.increment(); rpcMutateRequestCount.increment(); region = getRegion(request.getRegion()); + rejectIfInStandByState(region); MutateResponse.Builder builder = MutateResponse.newBuilder(); MutationProto mutation = request.getMutation(); if (!region.getRegionInfo().isMetaRegion()) { @@ -2941,6 +3016,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, "'hbase.client.scanner.timeout.period' configuration."); } } + rejectIfInStandByState(rsh.r); RegionInfo hri = rsh.s.getRegionInfo(); // Yes, should be the same instance if (regionServer.getOnlineRegion(hri.getRegionName()) != rsh.r) { @@ -2967,6 +3043,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler, private RegionScannerHolder newRegionScanner(ScanRequest request, ScanResponse.Builder builder) throws IOException { HRegion region = getRegion(request.getRegion()); + rejectIfInStandByState(region); ClientProtos.Scan protoScan = request.getScan(); boolean isLoadingCfsOnDemandSet = protoScan.hasLoadColumnFamiliesOnDemand(); Scan scan = ProtobufUtil.toScan(protoScan); 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 00f4077..7c4362c 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 @@ -262,4 +262,9 @@ public interface RegionServerServices extends Server, MutableOnlineRegions, Favo * @return True if cluster is up; false if cluster is not up (we are shutting down). */ boolean isClusterUp(); + + /** + * @return Return the object that implements the replication source executorService. + */ + ReplicationSourceService getReplicationSourceService(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java index 23ba773..09ec477 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java @@ -17,19 +17,30 @@ */ package org.apache.hadoop.hbase.regionserver; +import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler; +import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider; import org.apache.yetus.audience.InterfaceAudience; /** - * A source for a replication stream has to expose this service. - * This service allows an application to hook into the - * regionserver and watch for new transactions. + * A source for a replication stream has to expose this service. This service allows an application + * to hook into the regionserver and watch for new transactions. */ @InterfaceAudience.Private public interface ReplicationSourceService extends ReplicationService { /** + * Returns an info provider for sync replication peer. + */ + SyncReplicationPeerInfoProvider getSyncReplicationPeerInfoProvider(); + + /** * Returns a Handler to handle peer procedures. */ PeerProcedureHandler getPeerProcedureHandler(); + + /** + * Return the replication peers. + */ + ReplicationPeers getReplicationPeers(); } 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 a1c2030..4a9712c 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 @@ -23,22 +23,31 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.net.ConnectException; import java.net.SocketTimeoutException; - -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import java.util.Optional; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.client.RetriesExhaustedException; import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALSplitter; +import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status; +import org.apache.hadoop.hbase.replication.ReplicationPeerImpl; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.util.CancelableProgressable; import org.apache.hadoop.hbase.util.ExceptionUtil; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALSplitter; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -67,67 +76,133 @@ public class SplitLogWorker implements Runnable { Thread worker; // thread pool which executes recovery work private SplitLogWorkerCoordination coordination; - private Configuration conf; private RegionServerServices server; public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server, TaskExecutor splitTaskExecutor) { this.server = server; - this.conf = conf; this.coordination = hserver.getCoordinatedStateManager().getSplitLogWorkerCoordination(); coordination.init(server, conf, splitTaskExecutor, this); } - public SplitLogWorker(final Server hserver, final Configuration conf, - final RegionServerServices server, final LastSequenceId sequenceIdChecker, - final WALFactory factory) { - this(hserver, conf, server, new TaskExecutor() { - @Override - public Status exec(String filename, CancelableProgressable p) { - Path walDir; - FileSystem fs; - try { - walDir = FSUtils.getWALRootDir(conf); - fs = walDir.getFileSystem(conf); - } catch (IOException e) { - LOG.warn("could not find root dir or fs", e); - return Status.RESIGNED; - } - // TODO have to correctly figure out when log splitting has been - // interrupted or has encountered a transient error and when it has - // encountered a bad non-retry-able persistent error. - try { - if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)), - fs, conf, p, sequenceIdChecker, - server.getCoordinatedStateManager().getSplitLogWorkerCoordination(), factory)) { - return Status.PREEMPTED; - } - } catch (InterruptedIOException iioe) { - LOG.warn("log splitting of " + filename + " interrupted, resigning", iioe); - return Status.RESIGNED; - } catch (IOException e) { - if (e instanceof FileNotFoundException) { - // A wal file may not exist anymore. Nothing can be recovered so move on - LOG.warn("WAL {} does not exist anymore", filename, e); - return Status.DONE; - } - Throwable cause = e.getCause(); - 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; - } - LOG.warn("log splitting of " + filename + " failed, returning error", e); - return Status.ERR; - } + public SplitLogWorker(Configuration conf, RegionServerServices server, + LastSequenceId sequenceIdChecker, WALFactory factory) { + this(server, conf, server, (f, p) -> splitLog(f, p, conf, server, sequenceIdChecker, factory)); + } + + // returns whether we need to continue the split work + private static boolean processSyncReplicationWAL(String name, Configuration conf, + RegionServerServices server, FileSystem fs, Path walDir) throws IOException { + Path walFile = new Path(walDir, name); + String filename = walFile.getName(); + Optional optSyncPeerId = + SyncReplicationWALProvider.getSyncReplicationPeerIdFromWALName(filename); + if (!optSyncPeerId.isPresent()) { + return true; + } + String peerId = optSyncPeerId.get(); + ReplicationPeerImpl peer = + server.getReplicationSourceService().getReplicationPeers().getPeer(peerId); + if (peer == null || !peer.getPeerConfig().isSyncReplication()) { + return true; + } + Pair stateAndNewState = + peer.getSyncReplicationStateAndNewState(); + if (stateAndNewState.getFirst().equals(SyncReplicationState.ACTIVE) && + stateAndNewState.getSecond().equals(SyncReplicationState.NONE)) { + // copy the file to remote and overwrite the previous one + String remoteWALDir = peer.getPeerConfig().getRemoteWALDir(); + Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId); + Path tmpRemoteWAL = new Path(remoteWALDirForPeer, filename + ".tmp"); + FileSystem remoteFs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir); + try (FSDataInputStream in = fs.open(walFile); @SuppressWarnings("deprecation") + FSDataOutputStream out = remoteFs.createNonRecursive(tmpRemoteWAL, true, + FSUtils.getDefaultBufferSize(remoteFs), remoteFs.getDefaultReplication(tmpRemoteWAL), + remoteFs.getDefaultBlockSize(tmpRemoteWAL), null)) { + IOUtils.copy(in, out); + } + Path toCommitRemoteWAL = + new Path(remoteWALDirForPeer, filename + ReplicationUtils.RENAME_WAL_SUFFIX); + // Some FileSystem implementations may not support atomic rename so we need to do it in two + // phases + FSUtils.renameFile(remoteFs, tmpRemoteWAL, toCommitRemoteWAL); + FSUtils.renameFile(remoteFs, toCommitRemoteWAL, new Path(remoteWALDirForPeer, filename)); + } else if ((stateAndNewState.getFirst().equals(SyncReplicationState.ACTIVE) && + stateAndNewState.getSecond().equals(SyncReplicationState.STANDBY)) || + stateAndNewState.getFirst().equals(SyncReplicationState.STANDBY)) { + // check whether we still need to process this file + // actually we only write wal file which name is ended with .syncrep in A state, and after + // transiting to a state other than A, we will reopen all the regions so the data in the wal + // will be flushed so the wal file will be archived soon. But it is still possible that there + // is a server crash when we are transiting from A to S, to simplify the logic of the transit + // procedure, here we will also check the remote snapshot directory in state S, so that we do + // not need wait until all the wal files with .syncrep suffix to be archived before finishing + // the procedure. + String remoteWALDir = peer.getPeerConfig().getRemoteWALDir(); + Path remoteSnapshotDirForPeer = ReplicationUtils.getPeerSnapshotWALDir(remoteWALDir, peerId); + FileSystem remoteFs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir); + if (remoteFs.exists(new Path(remoteSnapshotDirForPeer, filename))) { + // the file has been replayed when the remote cluster was transited from S to DA, the + // content will be replicated back to us so give up split it. + LOG.warn("Giveup splitting {} since it has been replayed in the remote cluster and " + + "the content will be replicated back", filename); + return false; + } + } + return true; + } + + private static Status splitLog(String name, CancelableProgressable p, Configuration conf, + RegionServerServices server, LastSequenceId sequenceIdChecker, WALFactory factory) { + Path walDir; + FileSystem fs; + try { + walDir = FSUtils.getWALRootDir(conf); + fs = walDir.getFileSystem(conf); + } catch (IOException e) { + LOG.warn("could not find root dir or fs", e); + return Status.RESIGNED; + } + try { + if (!processSyncReplicationWAL(name, conf, server, fs, walDir)) { return Status.DONE; } - }); + } catch (IOException e) { + LOG.warn("failed to process sync replication wal {}", name, e); + return Status.RESIGNED; + } + // TODO have to correctly figure out when log splitting has been + // interrupted or has encountered a transient error and when it has + // encountered a bad non-retry-able persistent error. + try { + if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, name)), fs, conf, + p, sequenceIdChecker, server.getCoordinatedStateManager().getSplitLogWorkerCoordination(), + factory)) { + return Status.PREEMPTED; + } + } catch (InterruptedIOException iioe) { + LOG.warn("log splitting of " + name + " interrupted, resigning", iioe); + return Status.RESIGNED; + } catch (IOException e) { + if (e instanceof FileNotFoundException) { + // A wal file may not exist anymore. Nothing can be recovered so move on + LOG.warn("WAL {} does not exist anymore", name, e); + return Status.DONE; + } + Throwable cause = e.getCause(); + if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException || + cause instanceof ConnectException || cause instanceof SocketTimeoutException)) { + LOG.warn("log replaying of " + name + " can't connect to the target regionserver, " + + "resigning", e); + return Status.RESIGNED; + } else if (cause instanceof InterruptedException) { + LOG.warn("log splitting of " + name + " interrupted, resigning", e); + return Status.RESIGNED; + } + LOG.warn("log splitting of " + name + " failed, returning error", e); + return Status.ERR; + } + return Status.DONE; } @Override @@ -191,6 +266,7 @@ public class SplitLogWorker implements Runnable { * {@link org.apache.hadoop.hbase.master.SplitLogManager} commit the work in * SplitLogManager.TaskFinisher */ + @FunctionalInterface public interface TaskExecutor { enum Status { DONE(), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ForbidMajorCompactionChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ForbidMajorCompactionChecker.java new file mode 100644 index 0000000..eecc780 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ForbidMajorCompactionChecker.java @@ -0,0 +1,44 @@ +/** + * 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.compactions; + +import java.util.function.BiPredicate; + +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Check whether forbid major compaction for region. + */ +@InterfaceAudience.Private +public class ForbidMajorCompactionChecker + implements BiPredicate { + + private static final ForbidMajorCompactionChecker INST = new ForbidMajorCompactionChecker(); + + @Override + public boolean test(SyncReplicationState state, SyncReplicationState newState) { + // Forbid major compaction when cluster transit sync replication state from S to DA + return state == SyncReplicationState.STANDBY + || newState == SyncReplicationState.DOWNGRADE_ACTIVE; + } + + public static ForbidMajorCompactionChecker get() { + return INST; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 825ad17..72ad8b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -434,6 +434,13 @@ public abstract class AbstractFSWAL implements WAL { this.implClassName = getClass().getSimpleName(); } + /** + * Used to initialize the WAL. Usually just call rollWriter to create the first log writer. + */ + public void init() throws IOException { + rollWriter(); + } + @Override public void registerWALActionsListener(WALActionsListener listener) { this.listeners.add(listener); @@ -747,15 +754,14 @@ public abstract class AbstractFSWAL implements WAL { public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException { rollWriterLock.lock(); try { + if (this.closed) { + throw new WALClosedException("WAL has been closed"); + } // Return if nothing to flush. if (!force && this.writer != null && this.numEntries.get() <= 0) { return null; } byte[][] regionsToFlush = null; - if (this.closed) { - LOG.debug("WAL closed. Skipping rolling of writer"); - return regionsToFlush; - } try (TraceScope scope = TraceUtil.createTrace("FSHLog.rollWriter")) { Path oldPath = getOldPath(); Path newPath = getNewPath(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java index d032d83..7f3e30b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java @@ -52,12 +52,12 @@ import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.htrace.core.TraceScope; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.EventLoop; @@ -248,7 +248,6 @@ public class AsyncFSWAL extends AbstractFSWAL { batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE); waitOnShutdownInSeconds = conf.getInt(ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS, DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS); - rollWriter(); } private static boolean waitingRoll(int epochAndState) { @@ -471,6 +470,44 @@ public class AsyncFSWAL extends AbstractFSWAL { // whether to issue a sync in the caller method. } + private void drainNonMarkerEditsAndFailSyncs() { + if (toWriteAppends.isEmpty()) { + return; + } + boolean hasNonMarkerEdits = false; + Iterator iter = toWriteAppends.descendingIterator(); + while (iter.hasNext()) { + FSWALEntry entry = iter.next(); + if (!entry.getEdit().isMetaEdit()) { + hasNonMarkerEdits = true; + break; + } + } + if (hasNonMarkerEdits) { + for (;;) { + iter.remove(); + if (!iter.hasNext()) { + break; + } + iter.next(); + } + unackedAppends.clear(); + // fail the sync futures which are under the txid of the first remaining edit, if none, fail + // all the sync futures. + long txid = toWriteAppends.isEmpty() ? Long.MAX_VALUE : toWriteAppends.peek().getTxid(); + IOException error = new IOException("WAL is closing, only marker edit is allowed"); + for (Iterator syncIter = syncFutures.iterator(); syncIter.hasNext();) { + SyncFuture future = syncIter.next(); + if (future.getTxid() < txid) { + future.done(future.getTxid(), error); + syncIter.remove(); + } else { + break; + } + } + } + } + private void consume() { consumeLock.lock(); try { @@ -513,6 +550,9 @@ public class AsyncFSWAL extends AbstractFSWAL { } waitingConsumePayloadsGatingSequence.set(nextCursor); } + if (markerEditOnly()) { + drainNonMarkerEditsAndFailSyncs(); + } appendAndSync(); if (hasConsumerTask.get()) { return; @@ -554,9 +594,18 @@ public class AsyncFSWAL extends AbstractFSWAL { return consumerScheduled.compareAndSet(false, true); } + // This is used by sync replication, where we are going to close the wal soon after we reopen all + // the regions. Will be overridden by sub classes. + protected boolean markerEditOnly() { + return false; + } + @Override public long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore) throws IOException { + if (markerEditOnly() && !edits.isMetaEdit()) { + throw new IOException("WAL is closing, only marker edit is allowed"); + } long txid = stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, waitingConsumePayloads); if (shouldScheduleConsumer()) { @@ -607,10 +656,14 @@ public class AsyncFSWAL extends AbstractFSWAL { } } + protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException { + return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, this.blocksize, + eventLoopGroup, channelClass); + } + @Override protected AsyncWriter createWriterInstance(Path path) throws IOException { - return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, - this.blocksize, eventLoopGroup, channelClass); + return createAsyncWriter(fs, path); } private void waitForSafePoint() { @@ -632,13 +685,12 @@ public class AsyncFSWAL extends AbstractFSWAL { } } - private long closeWriter() { - AsyncWriter oldWriter = this.writer; - if (oldWriter != null) { - long fileLength = oldWriter.getLength(); + protected final long closeWriter(AsyncWriter writer) { + if (writer != null) { + long fileLength = writer.getLength(); closeExecutor.execute(() -> { try { - oldWriter.close(); + writer.close(); } catch (IOException e) { LOG.warn("close old writer failed", e); } @@ -654,7 +706,7 @@ public class AsyncFSWAL extends AbstractFSWAL { throws IOException { Preconditions.checkNotNull(nextWriter); waitForSafePoint(); - long oldFileLen = closeWriter(); + long oldFileLen = closeWriter(this.writer); logRollAndSetupWalProps(oldPath, newPath, oldFileLen); this.writer = nextWriter; if (nextWriter instanceof AsyncProtobufLogWriter) { @@ -679,7 +731,8 @@ public class AsyncFSWAL extends AbstractFSWAL { @Override protected void doShutdown() throws IOException { waitForSafePoint(); - closeWriter(); + closeWriter(this.writer); + this.writer = null; closeExecutor.shutdown(); try { if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java new file mode 100644 index 0000000..4301ae7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java @@ -0,0 +1,94 @@ +/** + * 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.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; + +/** + * An {@link AsyncWriter} wrapper which writes data to a set of {@link AsyncWriter} instances. + */ +@InterfaceAudience.Private +public final class CombinedAsyncWriter implements AsyncWriter { + + private static final Logger LOG = LoggerFactory.getLogger(CombinedAsyncWriter.class); + + private final ImmutableList writers; + + private CombinedAsyncWriter(ImmutableList writers) { + this.writers = writers; + } + + @Override + public long getLength() { + return writers.get(0).getLength(); + } + + @Override + public void close() throws IOException { + Exception error = null; + for (AsyncWriter writer : writers) { + try { + writer.close(); + } catch (Exception e) { + LOG.warn("close writer failed", e); + if (error == null) { + error = e; + } + } + } + if (error != null) { + throw new IOException("Failed to close at least one writer, please see the warn log above. " + + "The cause is the first exception occured", error); + } + } + + @Override + public void append(Entry entry) { + writers.forEach(w -> w.append(entry)); + } + + @Override + public CompletableFuture sync() { + CompletableFuture future = new CompletableFuture<>(); + AtomicInteger remaining = new AtomicInteger(writers.size()); + writers.forEach(w -> w.sync().whenComplete((length, error) -> { + if (error != null) { + future.completeExceptionally(error); + return; + } + if (remaining.decrementAndGet() == 0) { + future.complete(length); + } + })); + return future; + } + + public static CombinedAsyncWriter create(AsyncWriter writer, AsyncWriter... writers) { + return new CombinedAsyncWriter( + ImmutableList. builder().add(writer).add(writers).build()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java new file mode 100644 index 0000000..bf5b96d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java @@ -0,0 +1,121 @@ +/** + * 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.io.InterruptedIOException; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.client.ConnectionUtils; +import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; + +/** + * An AsyncFSWAL which writes data to two filesystems. + */ +@InterfaceAudience.Private +public class DualAsyncFSWAL extends AsyncFSWAL { + + private static final Logger LOG = LoggerFactory.getLogger(DualAsyncFSWAL.class); + + private final FileSystem remoteFs; + + private final Path remoteWALDir; + + private volatile boolean skipRemoteWAL = false; + + private volatile boolean markerEditOnly = false; + + public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteWALDir, + String logDir, String archiveDir, Configuration conf, List listeners, + boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup, + Class channelClass) throws FailedLogCloseException, IOException { + super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix, + eventLoopGroup, channelClass); + this.remoteFs = remoteFs; + this.remoteWALDir = remoteWALDir; + } + + // will be overridden in testcase + @VisibleForTesting + protected AsyncWriter createCombinedAsyncWriter(AsyncWriter localWriter, + AsyncWriter remoteWriter) { + return CombinedAsyncWriter.create(remoteWriter, localWriter); + } + + @Override + protected AsyncWriter createWriterInstance(Path path) throws IOException { + AsyncWriter localWriter = super.createWriterInstance(path); + // retry forever if we can not create the remote writer to prevent aborting the RS due to log + // rolling error, unless the skipRemoteWal is set to true. + // TODO: since for now we only have one thread doing log rolling, this may block the rolling for + // other wals + Path remoteWAL = new Path(remoteWALDir, path.getName()); + for (int retry = 0;; retry++) { + if (skipRemoteWAL) { + return localWriter; + } + AsyncWriter remoteWriter; + try { + remoteWriter = createAsyncWriter(remoteFs, remoteWAL); + } catch (IOException e) { + LOG.warn("create remote writer {} failed, retry = {}", remoteWAL, retry, e); + try { + Thread.sleep(ConnectionUtils.getPauseTime(100, retry)); + } catch (InterruptedException ie) { + // restore the interrupt state + Thread.currentThread().interrupt(); + Closeables.close(localWriter, true); + throw (IOException) new InterruptedIOException().initCause(ie); + } + continue; + } + return createCombinedAsyncWriter(localWriter, remoteWriter); + } + } + + @Override + protected boolean markerEditOnly() { + return markerEditOnly; + } + + // Allow temporarily skipping the creation of remote writer. When failing to write to the remote + // dfs cluster, we need to reopen the regions and switch to use the original wal writer. But we + // need to write a close marker when closing a region, and if it fails, the whole rs will abort. + // So here we need to skip the creation of remote writer and make it possible to write the region + // close marker. + // Setting markerEdit only to true is for transiting from A to S, where we need to give up writing + // any pending wal entries as they will be discarded. The remote cluster will replicated the + // correct data back later. We still need to allow writing marker edits such as close region event + // to allow closing a region. + public void skipRemoteWAL(boolean markerEditOnly) { + if (markerEditOnly) { + this.markerEditOnly = true; + } + this.skipRemoteWAL = true; + } +} 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 61b9cfb..baa87a4 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 @@ -214,12 +214,8 @@ public class FSHLog extends AbstractFSWAL { this.lowReplicationRollLimit = conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit", 5); this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2); - this.useHsync = conf.getBoolean(HRegion.WAL_HSYNC_CONF_KEY, HRegion.DEFAULT_WAL_HSYNC); - // rollWriter sets this.hdfs_out if it can. - rollWriter(); - // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is // put on the ring buffer. String hostingThreadName = Thread.currentThread().getName(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALClosedException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALClosedException.java new file mode 100644 index 0000000..ac6aad0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALClosedException.java @@ -0,0 +1,47 @@ +/** + * 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 org.apache.hadoop.hbase.regionserver.LogRoller; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Thrown when {@link LogRoller} try to roll writer but the WAL already was closed. This may + * happened when peer's sync replication state was transited from + * {@link SyncReplicationState#ACTIVE} to {@link SyncReplicationState#DOWNGRADE_ACTIVE} and the + * region's WAL was changed to a new one. But the old WAL was still left in {@link LogRoller}. + */ +@InterfaceAudience.Private +public class WALClosedException extends IOException { + + private static final long serialVersionUID = -3183198896865290678L; + + public WALClosedException() { + super(); + } + + /** + * @param msg exception message + */ + public WALClosedException(String msg) { + super(msg); + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java index 1b17adc..3b18253 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALUtil.java @@ -20,11 +20,13 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.IOException; +import java.util.ArrayList; import java.util.NavigableMap; - +import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.util.CommonFSUtils; @@ -34,7 +36,9 @@ import org.apache.hadoop.hbase.wal.WALKeyImpl; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; + import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor; @@ -179,4 +183,23 @@ public class WALUtil { return conf.getLong("hbase.regionserver.hlog.blocksize", CommonFSUtils.getDefaultBlockSize(fs, dir) * 2); } + + public static void filterCells(WALEdit edit, Function mapper) { + ArrayList cells = edit.getCells(); + int size = cells.size(); + int newSize = 0; + for (int i = 0; i < size; i++) { + Cell cell = mapper.apply(cells.get(i)); + if (cell != null) { + cells.set(newSize, cell); + newSize++; + } + } + for (int i = size - 1; i >= newSize; i--) { + cells.remove(i); + } + if (newSize < size / 2) { + cells.trimToSize(); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java index 6f2c764..2bb9811 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ChainWALEntryFilter.java @@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.replication; import java.util.ArrayList; import java.util.Collections; import java.util.List; - -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.yetus.audience.InterfaceAudience; /** * A {@link WALEntryFilter} which contains multiple filters and applies them @@ -82,22 +82,16 @@ public class ChainWALEntryFilter implements WALEntryFilter { if (entry == null || cellFilters.length == 0) { return; } - ArrayList cells = entry.getEdit().getCells(); - int size = cells.size(); - for (int i = size - 1; i >= 0; i--) { - Cell cell = cells.get(i); - for (WALCellFilter filter : cellFilters) { - cell = filter.filterCell(entry, cell); - if (cell != null) { - cells.set(i, cell); - } else { - cells.remove(i); - break; - } + WALUtil.filterCells(entry.getEdit(), c -> filterCell(entry, c)); + } + + private Cell filterCell(Entry entry, Cell cell) { + for (WALCellFilter filter : cellFilters) { + cell = filter.filterCell(entry, cell); + if (cell == null) { + break; } } - if (cells.size() < size / 2) { - cells.trimToSize(); - } + return cell; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java new file mode 100644 index 0000000..efafd09 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java @@ -0,0 +1,35 @@ +/** + * 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.replication.regionserver; + +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Get notification for replication peer events. Mainly used for telling the + * {@link org.apache.hadoop.hbase.wal.SyncReplicationWALProvider} to close some WAL if not used any + * more. + */ +@InterfaceAudience.Private +public interface PeerActionListener { + + static final PeerActionListener DUMMY = new PeerActionListener() {}; + + default void peerSyncReplicationStateChange(String peerId, SyncReplicationState from, + SyncReplicationState to, int stage) {} +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java index 65da9af..52b604b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java @@ -15,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; - +import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.yetus.audience.InterfaceAudience; @@ -29,13 +28,16 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public interface PeerProcedureHandler { - public void addPeer(String peerId) throws ReplicationException, IOException; + void addPeer(String peerId) throws ReplicationException, IOException; + + void removePeer(String peerId) throws ReplicationException, IOException; - public void removePeer(String peerId) throws ReplicationException, IOException; + void disablePeer(String peerId) throws ReplicationException, IOException; - public void disablePeer(String peerId) throws ReplicationException, IOException; + void enablePeer(String peerId) throws ReplicationException, IOException; - public void enablePeer(String peerId) throws ReplicationException, IOException; + void updatePeerConfig(String peerId) throws ReplicationException, IOException; - public void updatePeerConfig(String peerId) throws ReplicationException, IOException; + void transitSyncReplicationPeerState(String peerId, int stage, HRegionServer rs) + throws ReplicationException, IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java index 78c1977..d01b130 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java @@ -18,24 +18,35 @@ package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; +import java.io.InterruptedIOException; import java.util.concurrent.locks.Lock; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.LogRoller; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerImpl; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.util.KeyLocker; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; @InterfaceAudience.Private public class PeerProcedureHandlerImpl implements PeerProcedureHandler { + private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class); + private final ReplicationSourceManager replicationSourceManager; + private final PeerActionListener peerActionListener; private final KeyLocker peersLock = new KeyLocker<>(); - public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) { + public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager, + PeerActionListener peerActionListener) { this.replicationSourceManager = replicationSourceManager; + this.peerActionListener = peerActionListener; } @Override @@ -61,7 +72,6 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler { } private void refreshPeerState(String peerId) throws ReplicationException, IOException { - PeerState newState; Lock peerLock = peersLock.acquireLock(peerId); ReplicationPeerImpl peer = null; PeerState oldState = null; @@ -72,7 +82,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler { throw new ReplicationException("Peer with id=" + peerId + " is not cached."); } oldState = peer.getPeerState(); - newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); + PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); // RS need to start work with the new replication state change if (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED)) { replicationSourceManager.refreshSources(peerId); @@ -132,4 +142,83 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler { peerLock.unlock(); } } + + @Override + public void transitSyncReplicationPeerState(String peerId, int stage, HRegionServer rs) + throws ReplicationException, IOException { + ReplicationPeers replicationPeers = replicationSourceManager.getReplicationPeers(); + Lock peerLock = peersLock.acquireLock(peerId); + try { + ReplicationPeerImpl peer = replicationPeers.getPeer(peerId); + if (peer == null) { + throw new ReplicationException("Peer with id=" + peerId + " is not cached."); + } + if (!peer.getPeerConfig().isSyncReplication()) { + throw new ReplicationException("Peer with id=" + peerId + " is not synchronous."); + } + SyncReplicationState newSyncReplicationState = peer.getNewSyncReplicationState(); + if (stage == 0) { + if (newSyncReplicationState != SyncReplicationState.NONE) { + LOG.warn("The new sync replication state for peer {} has already been set to {}, " + + "this should be a retry, give up", peerId, newSyncReplicationState); + return; + } + // refresh the peer state first, as when we transit to STANDBY, we may need to disable the + // peer before processing the sync replication state. + PeerState oldState = peer.getPeerState(); + boolean success = false; + try { + PeerState newState = replicationPeers.refreshPeerState(peerId); + if (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED)) { + replicationSourceManager.refreshSources(peerId); + } + success = true; + } finally { + if (!success) { + peer.setPeerState(oldState.equals(PeerState.ENABLED)); + } + } + newSyncReplicationState = replicationPeers.refreshPeerNewSyncReplicationState(peerId); + SyncReplicationState oldSyncReplicationState = peer.getSyncReplicationState(); + peerActionListener.peerSyncReplicationStateChange(peerId, oldSyncReplicationState, + newSyncReplicationState, stage); + } else { + if (newSyncReplicationState == SyncReplicationState.NONE) { + LOG.warn( + "The new sync replication state for peer {} has already been clear, and the " + + "current state is {}, this should be a retry, give up", + peerId, newSyncReplicationState); + return; + } + if (newSyncReplicationState == SyncReplicationState.STANDBY) { + replicationSourceManager.drainSources(peerId); + // Need to roll the wals and make the ReplicationSource for this peer track the new file. + // If we do not do this, there will be two problems that can not be addressed at the same + // time. First, if we just throw away the current wal file, and later when we transit the + // peer to DA, and the wal has not been rolled yet, then the new data written to the wal + // file will not be replicated and cause data inconsistency. But if we just track the + // current wal file without rolling, it may contains some data before we transit the peer + // to S, later if we transit the peer to DA, the data will also be replicated and cause + // data inconsistency. So here we need to roll the wal, and let the ReplicationSource + // track the new wal file, and throw the old wal files away. + LogRoller roller = rs.getWalRoller(); + roller.requestRollAll(); + try { + roller.waitUntilWalRollFinished(); + } catch (InterruptedException e) { + // reset the interrupted flag + Thread.currentThread().interrupt(); + throw (IOException) new InterruptedIOException( + "Interrupted while waiting for wal roll finish").initCause(e); + } + } + SyncReplicationState oldState = peer.getSyncReplicationState(); + peerActionListener.peerSyncReplicationStateChange(peerId, oldState, newSyncReplicationState, + stage); + peer.transitSyncReplicationState(); + } + } finally { + peerLock.unlock(); + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java index a21ca44..f1bb538 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.replication.ReplicationPeer; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -144,15 +143,9 @@ public class RecoveredReplicationSource extends ReplicationSource { } void tryFinish() { - // use synchronize to make sure one last thread will clean the queue - synchronized (workerThreads) { - Threads.sleep(100);// wait a short while for other worker thread to fully exit - boolean allTasksDone = workerThreads.values().stream().allMatch(w -> w.isFinished()); - if (allTasksDone) { - this.getSourceMetrics().clear(); - manager.removeRecoveredSource(this); - LOG.info("Finished recovering queue {} with the following stats: {}", queueId, getStats()); - } + if (workerThreads.isEmpty()) { + this.getSourceMetrics().clear(); + manager.finishRecoveredSource(this); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java index 91109cf..b0d4db0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java @@ -48,13 +48,6 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper } @Override - protected void noMoreData() { - LOG.debug("Finished recovering queue for group {} of peer {}", walGroupId, source.getQueueId()); - source.getSourceMetrics().incrCompletedRecoveryQueue(); - setWorkerState(WorkerState.FINISHED); - } - - @Override protected void postFinish() { source.tryFinish(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java index 7ada24b..8fe16bc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java @@ -35,12 +35,15 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.R public class RefreshPeerCallable implements RSProcedureCallable { private static final Logger LOG = Logger.getLogger(RefreshPeerCallable.class); + private HRegionServer rs; private String peerId; private PeerModificationType type; + private int stage; + private Exception initError; @Override @@ -67,6 +70,9 @@ public class RefreshPeerCallable implements RSProcedureCallable { case UPDATE_PEER_CONFIG: handler.updatePeerConfig(this.peerId); break; + case TRANSIT_SYNC_REPLICATION_STATE: + handler.transitSyncReplicationPeerState(peerId, stage, rs); + break; default: throw new IllegalArgumentException("Unknown peer modification type: " + type); } @@ -80,6 +86,7 @@ public class RefreshPeerCallable implements RSProcedureCallable { RefreshPeerParameter param = RefreshPeerParameter.parseFrom(parameter); this.peerId = param.getPeerId(); this.type = param.getType(); + this.stage = param.getStage(); } catch (InvalidProtocolBufferException e) { initError = e; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.java new file mode 100644 index 0000000..9ad0af2 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectReplicationRequestStateChecker.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.replication.regionserver; + +import java.util.function.BiPredicate; + +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Check whether we need to reject the replication request from source cluster. + */ +@InterfaceAudience.Private +public class RejectReplicationRequestStateChecker + implements BiPredicate { + + private static final RejectReplicationRequestStateChecker INST = + new RejectReplicationRequestStateChecker(); + + @Override + public boolean test(SyncReplicationState state, SyncReplicationState newState) { + return state == SyncReplicationState.ACTIVE || state == SyncReplicationState.DOWNGRADE_ACTIVE + || newState == SyncReplicationState.ACTIVE + || newState == SyncReplicationState.DOWNGRADE_ACTIVE; + } + + public static RejectReplicationRequestStateChecker get() { + return INST; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectRequestsFromClientStateChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectRequestsFromClientStateChecker.java new file mode 100644 index 0000000..8e68f0f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RejectRequestsFromClientStateChecker.java @@ -0,0 +1,44 @@ +/** + * 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.replication.regionserver; + +import java.util.function.BiPredicate; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Check whether we need to reject the request from client. + */ +@InterfaceAudience.Private +public class RejectRequestsFromClientStateChecker + implements BiPredicate { + + private static final RejectRequestsFromClientStateChecker INST = + new RejectRequestsFromClientStateChecker(); + + @Override + public boolean test(SyncReplicationState state, SyncReplicationState newState) { + // reject requests from client if we are in standby state, or we are going to transit to standby + // state. + return state == SyncReplicationState.STANDBY || newState == SyncReplicationState.STANDBY; + } + + public static RejectRequestsFromClientStateChecker get() { + return INST; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java new file mode 100644 index 0000000..24963f1 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplaySyncReplicationWALCallable.java @@ -0,0 +1,182 @@ +/** + * 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.replication.regionserver; + +import java.io.EOFException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.locks.Lock; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellScanner; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.executor.EventType; +import org.apache.hadoop.hbase.procedure2.RSProcedureCallable; +import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.KeyLocker; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WAL.Reader; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ReplaySyncReplicationWALParameter; + +/** + * This callable executed at RS side to replay sync replication wal. + */ +@InterfaceAudience.Private +public class ReplaySyncReplicationWALCallable implements RSProcedureCallable { + + private static final Logger LOG = LoggerFactory.getLogger(ReplaySyncReplicationWALCallable.class); + + private static final String REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE = + "hbase.replay.sync.replication.wal.batch.size"; + + private static final long DEFAULT_REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE = 8 * 1024 * 1024; + + private HRegionServer rs; + + private FileSystem fs; + + private Configuration conf; + + private String peerId; + + private List wals = new ArrayList<>(); + + private Exception initError; + + private long batchSize; + + private final KeyLocker peersLock = new KeyLocker<>(); + + @Override + public Void call() throws Exception { + if (initError != null) { + throw initError; + } + LOG.info("Received a replay sync replication wals {} event, peerId={}", wals, peerId); + if (rs.getReplicationSinkService() != null) { + Lock peerLock = peersLock.acquireLock(wals.get(0)); + try { + for (String wal : wals) { + replayWAL(wal); + } + } finally { + peerLock.unlock(); + } + } + return null; + } + + @Override + public void init(byte[] parameter, HRegionServer rs) { + this.rs = rs; + this.fs = rs.getWALFileSystem(); + this.conf = rs.getConfiguration(); + try { + ReplaySyncReplicationWALParameter param = + ReplaySyncReplicationWALParameter.parseFrom(parameter); + this.peerId = param.getPeerId(); + param.getWalList().forEach(this.wals::add); + this.batchSize = rs.getConfiguration().getLong(REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE, + DEFAULT_REPLAY_SYNC_REPLICATION_WAL_BATCH_SIZE); + } catch (InvalidProtocolBufferException e) { + initError = e; + } + } + + @Override + public EventType getEventType() { + return EventType.RS_REPLAY_SYNC_REPLICATION_WAL; + } + + private void replayWAL(String wal) throws IOException { + try (Reader reader = getReader(wal)) { + List entries = readWALEntries(reader); + while (!entries.isEmpty()) { + Pair pair = ReplicationProtbufUtil + .buildReplicateWALEntryRequest(entries.toArray(new Entry[entries.size()])); + ReplicateWALEntryRequest request = pair.getFirst(); + rs.getReplicationSinkService().replicateLogEntries(request.getEntryList(), + pair.getSecond(), request.getReplicationClusterId(), + request.getSourceBaseNamespaceDirPath(), request.getSourceHFileArchiveDirPath()); + // Read next entries. + entries = readWALEntries(reader); + } + } + } + + private Reader getReader(String wal) throws IOException { + Path path = new Path(rs.getWALRootDir(), wal); + long length = rs.getWALFileSystem().getFileStatus(path).getLen(); + try { + FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf); + return WALFactory.createReader(rs.getWALFileSystem(), path, rs.getConfiguration()); + } catch (EOFException e) { + if (length <= 0) { + LOG.warn("File is empty. Could not open {} for reading because {}", path, e); + return null; + } + throw e; + } + } + + // return whether we should include this entry. + private boolean filter(Entry entry) { + WALEdit edit = entry.getEdit(); + WALUtil.filterCells(edit, c -> CellUtil.matchingFamily(c, WALEdit.METAFAMILY) ? null : c); + return !edit.isEmpty(); + } + + private List readWALEntries(Reader reader) throws IOException { + List entries = new ArrayList<>(); + if (reader == null) { + return entries; + } + long size = 0; + for (;;) { + Entry entry = reader.next(); + if (entry == null) { + break; + } + if (filter(entry)) { + entries.add(entry); + size += entry.getEdit().heapSize(); + if (size > batchSize) { + break; + } + } + } + return entries; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java index 8290ac3..b04f0cb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.ReplicationTracker; import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider; import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.yetus.audience.InterfaceAudience; @@ -66,6 +67,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer private ReplicationTracker replicationTracker; private Configuration conf; private ReplicationSink replicationSink; + private SyncReplicationPeerInfoProvider syncReplicationPeerInfoProvider; // Hosting server private Server server; /** Statistics thread schedule pool */ @@ -120,19 +122,30 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer } catch (KeeperException ke) { throw new IOException("Could not read cluster id", ke); } + SyncReplicationPeerMappingManager mapping = new SyncReplicationPeerMappingManager(); this.replicationManager = new ReplicationSourceManager(queueStorage, replicationPeers, replicationTracker, conf, this.server, fs, logDir, oldLogDir, clusterId, - walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty()); + walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty(), + mapping); + this.syncReplicationPeerInfoProvider = + new SyncReplicationPeerInfoProviderImpl(replicationPeers, mapping); + PeerActionListener peerActionListener = PeerActionListener.DUMMY; if (walProvider != null) { walProvider .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager)); + if (walProvider instanceof SyncReplicationWALProvider) { + SyncReplicationWALProvider syncWALProvider = (SyncReplicationWALProvider) walProvider; + peerActionListener = syncWALProvider; + syncWALProvider.setPeerInfoProvider(syncReplicationPeerInfoProvider); + } } this.statsThreadPeriod = this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60); LOG.debug("Replication stats-in-log period={} seconds", this.statsThreadPeriod); this.replicationLoad = new ReplicationLoad(); - this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager); + this.peerProcedureHandler = + new PeerProcedureHandlerImpl(replicationManager, peerActionListener); } @Override @@ -262,7 +275,7 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer List oldSources = this.replicationManager.getOldSources(); for (ReplicationSourceInterface source : oldSources) { if (source instanceof ReplicationSource) { - sourceMetricsList.add(((ReplicationSource) source).getSourceMetrics()); + sourceMetricsList.add(source.getSourceMetrics()); } } @@ -270,4 +283,14 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer MetricsSink sinkMetrics = this.replicationSink.getSinkMetrics(); this.replicationLoad.buildReplicationLoad(sourceMetricsList, sinkMetrics); } + + @Override + public SyncReplicationPeerInfoProvider getSyncReplicationPeerInfoProvider() { + return syncReplicationPeerInfoProvider; + } + + @Override + public ReplicationPeers getReplicationPeers() { + return replicationPeers; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java index fb4e0f9..a334b16 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -41,9 +40,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Delete; @@ -52,13 +48,18 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; -import org.apache.hadoop.hbase.wal.WALEdit; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; /** *

@@ -82,10 +83,10 @@ public class ReplicationSink { private final Configuration conf; // Volatile because of note in here -- look for double-checked locking: // http://www.oracle.com/technetwork/articles/javase/bloch-effective-08-qa-140880.html - private volatile Connection sharedHtableCon; + private volatile Connection sharedConn; private final MetricsSink metrics; private final AtomicLong totalReplicatedEdits = new AtomicLong(); - private final Object sharedHtableConLock = new Object(); + private final Object sharedConnLock = new Object(); // Number of hfiles that we successfully replicated private long hfilesReplicated = 0; private SourceFSConfigurationProvider provider; @@ -93,9 +94,8 @@ public class ReplicationSink { /** * Create a sink for replication - * - * @param conf conf object - * @param stopper boolean to tell this thread to stop + * @param conf conf object + * @param stopper boolean to tell this thread to stop * @throws IOException thrown when HDFS goes bad or bad file name */ public ReplicationSink(Configuration conf, Stoppable stopper) @@ -104,16 +104,15 @@ public class ReplicationSink { decorateConf(); this.metrics = new MetricsSink(); this.walEntrySinkFilter = setupWALEntrySinkFilter(); - String className = - conf.get("hbase.replication.source.fs.conf.provider", - DefaultSourceFSConfigurationProvider.class.getCanonicalName()); + String className = conf.get("hbase.replication.source.fs.conf.provider", + DefaultSourceFSConfigurationProvider.class.getCanonicalName()); try { - @SuppressWarnings("rawtypes") - Class c = Class.forName(className); - this.provider = (SourceFSConfigurationProvider) c.getDeclaredConstructor().newInstance(); + Class c = + Class.forName(className).asSubclass(SourceFSConfigurationProvider.class); + this.provider = c.getDeclaredConstructor().newInstance(); } catch (Exception e) { - throw new IllegalArgumentException("Configured source fs configuration provider class " - + className + " throws error.", e); + throw new IllegalArgumentException( + "Configured source fs configuration provider class " + className + " throws error.", e); } } @@ -178,8 +177,7 @@ public class ReplicationSink { Map>>> bulkLoadHFileMap = null; for (WALEntry entry : entries) { - TableName table = - TableName.valueOf(entry.getKey().getTableName().toByteArray()); + TableName table = TableName.valueOf(entry.getKey().getTableName().toByteArray()); if (this.walEntrySinkFilter != null) { if (this.walEntrySinkFilter.filter(table, entry.getKey().getWriteTime())) { // Skip Cells in CellScanner associated with this entry. @@ -221,6 +219,8 @@ public class ReplicationSink { clusterIds.add(toUUID(clusterId)); } mutation.setClusterIds(clusterIds); + mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME, + HConstants.EMPTY_BYTE_ARRAY); addToHashMultiMap(rowMap, table, clusterIds, mutation); } if (CellUtil.isDelete(cell)) { @@ -374,11 +374,11 @@ public class ReplicationSink { */ public void stopReplicationSinkServices() { try { - if (this.sharedHtableCon != null) { - synchronized (sharedHtableConLock) { - if (this.sharedHtableCon != null) { - this.sharedHtableCon.close(); - this.sharedHtableCon = null; + if (this.sharedConn != null) { + synchronized (sharedConnLock) { + if (this.sharedConn != null) { + this.sharedConn.close(); + this.sharedConn = null; } } } @@ -394,14 +394,12 @@ public class ReplicationSink { * @param allRows list of actions * @throws IOException */ - protected void batch(TableName tableName, Collection> allRows) throws IOException { + private void batch(TableName tableName, Collection> allRows) throws IOException { if (allRows.isEmpty()) { return; } - Table table = null; - try { - Connection connection = getConnection(); - table = connection.getTable(tableName); + Connection connection = getConnection(); + try (Table table = connection.getTable(tableName)) { for (List rows : allRows) { table.batch(rows, null); } @@ -414,21 +412,18 @@ public class ReplicationSink { throw rewde; } catch (InterruptedException ix) { throw (InterruptedIOException) new InterruptedIOException().initCause(ix); - } finally { - if (table != null) { - table.close(); - } } } private Connection getConnection() throws IOException { // See https://en.wikipedia.org/wiki/Double-checked_locking - Connection connection = sharedHtableCon; + Connection connection = sharedConn; if (connection == null) { - synchronized (sharedHtableConLock) { - connection = sharedHtableCon; + synchronized (sharedConnLock) { + connection = sharedConn; if (connection == null) { - connection = sharedHtableCon = ConnectionFactory.createConnection(conf); + connection = ConnectionFactory.createConnection(conf); + sharedConn = connection; } } } @@ -441,9 +436,10 @@ public class ReplicationSink { * of the last edit that was applied */ public String getStats() { - return this.totalReplicatedEdits.get() == 0 ? "" : "Sink: " + - "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() + - ", total replicated edits: " + this.totalReplicatedEdits; + long total = this.totalReplicatedEdits.get(); + return total == 0 ? "" + : "Sink: " + "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp() + + ", total replicated edits: " + total; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java index b63712b..a0d8321 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java @@ -62,6 +62,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; /** @@ -89,8 +90,6 @@ public class ReplicationSource implements ReplicationSourceInterface { protected Configuration conf; protected ReplicationQueueInfo replicationQueueInfo; - // id of the peer cluster this source replicates to - private String peerId; // The manager of all sources to which we ping back our progress protected ReplicationSourceManager manager; @@ -122,6 +121,7 @@ public class ReplicationSource implements ReplicationSourceInterface { private long defaultBandwidth; private long currentBandwidth; private WALFileLengthProvider walFileLengthProvider; + @VisibleForTesting protected final ConcurrentHashMap workerThreads = new ConcurrentHashMap<>(); @@ -168,8 +168,6 @@ public class ReplicationSource implements ReplicationSourceInterface { this.queueId = queueId; this.replicationQueueInfo = new ReplicationQueueInfo(queueId); - // ReplicationQueueInfo parses the peerId out of the znode for us - this.peerId = this.replicationQueueInfo.getPeerId(); this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2); defaultBandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0); @@ -177,8 +175,8 @@ public class ReplicationSource implements ReplicationSourceInterface { this.throttler = new ReplicationThrottler((double) currentBandwidth / 10.0); this.totalBufferUsed = manager.getTotalBufferUsed(); this.walFileLengthProvider = walFileLengthProvider; - LOG.info("queueId=" + queueId + ", ReplicationSource : " + peerId - + ", currentBandwidth=" + this.currentBandwidth); + LOG.info("queueId={}, ReplicationSource : {}, currentBandwidth={}", queueId, + replicationPeer.getId(), this.currentBandwidth); } private void decorateConf() { @@ -194,6 +192,9 @@ public class ReplicationSource implements ReplicationSourceInterface { PriorityBlockingQueue queue = queues.get(logPrefix); if (queue == null) { queue = new PriorityBlockingQueue<>(queueSizePerGroup, new LogsComparator()); + // make sure that we do not use an empty queue when setting up a ReplicationSource, otherwise + // the shipper may quit immediately + queue.put(log); queues.put(logPrefix, queue); if (this.isSourceActive() && this.walEntryFilter != null) { // new wal group observed after source startup, start a new worker thread to track it @@ -201,8 +202,10 @@ public class ReplicationSource implements ReplicationSourceInterface { // still not launched, so it's necessary to check workerThreads before start the worker tryStartNewShipper(logPrefix, queue); } + } else { + queue.put(log); } - queue.put(log); + this.metrics.incrSizeOfLogQueue(); // This will log a warning for each new log that gets created above the warn threshold int queueSize = queue.size(); @@ -215,6 +218,7 @@ public class ReplicationSource implements ReplicationSourceInterface { @Override public void addHFileRefs(TableName tableName, byte[] family, List> pairs) throws ReplicationException { + String peerId = replicationPeer.getId(); Map> tableCFMap = replicationPeer.getTableCFs(); if (tableCFMap != null) { List tableCfs = tableCFMap.get(tableName); @@ -274,8 +278,8 @@ public class ReplicationSource implements ReplicationSourceInterface { tableDescriptors = ((HRegionServer) server).getTableDescriptors(); } replicationEndpoint - .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs, peerId, - clusterId, replicationPeer, metrics, tableDescriptors, server)); + .init(new ReplicationEndpoint.Context(conf, replicationPeer.getConfiguration(), fs, + replicationPeer.getId(), clusterId, replicationPeer, metrics, tableDescriptors, server)); replicationEndpoint.start(); replicationEndpoint.awaitRunning(waitOnEndpointSeconds, TimeUnit.SECONDS); } @@ -357,8 +361,8 @@ public class ReplicationSource implements ReplicationSourceInterface { if (peerBandwidth != currentBandwidth) { currentBandwidth = peerBandwidth; throttler.setBandwidth((double) currentBandwidth / 10.0); - LOG.info("ReplicationSource : " + peerId - + " bandwidth throttling changed, currentBandWidth=" + currentBandwidth); + LOG.info("ReplicationSource : {} bandwidth throttling changed, currentBandWidth={}", + replicationPeer.getId(), currentBandwidth); } } @@ -387,15 +391,6 @@ public class ReplicationSource implements ReplicationSourceInterface { return sleepMultiplier < maxRetriesMultiplier; } - /** - * check whether the peer is enabled or not - * @return true if the peer is enabled, otherwise false - */ - @Override - public boolean isPeerEnabled() { - return replicationPeer.isPeerEnabled(); - } - private void initialize() { int sleepMultiplier = 1; while (this.isSourceActive()) { @@ -525,6 +520,7 @@ public class ReplicationSource implements ReplicationSourceInterface { if (this.replicationEndpoint != null) { this.replicationEndpoint.stop(); } + metrics.clear(); if (join) { for (ReplicationSourceShipper worker : workers) { Threads.shutdown(worker, this.sleepForRetries); @@ -549,11 +545,6 @@ public class ReplicationSource implements ReplicationSourceInterface { } @Override - public String getPeerId() { - return this.peerId; - } - - @Override public Path getCurrentPath() { // only for testing for (ReplicationSourceShipper worker : workerThreads.values()) { @@ -580,14 +571,17 @@ public class ReplicationSource implements ReplicationSourceInterface { } /** + *

* Split a path to get the start time + *

+ *

* For example: 10.20.20.171%3A60020.1277499063250 + *

* @param p path to split * @return start time */ private static long getTS(Path p) { - int tsIndex = p.getName().lastIndexOf('.') + 1; - return Long.parseLong(p.getName().substring(tsIndex)); + return AbstractFSWALProvider.getWALStartTimeFromWALName(p.getName()); } } @@ -637,6 +631,11 @@ public class ReplicationSource implements ReplicationSourceInterface { return server.getServerName(); } + @Override + public ReplicationPeer getPeer() { + return replicationPeer; + } + Server getServer() { return server; } @@ -644,4 +643,8 @@ public class ReplicationSource implements ReplicationSourceInterface { ReplicationQueueStorage getQueueStorage() { return queueStorage; } + + void removeWorker(ReplicationSourceShipper worker) { + workerThreads.remove(worker.walGroupId, worker); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java index 090b465..3ce5bfe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java @@ -104,10 +104,17 @@ public interface ReplicationSourceInterface { /** * Get the id that the source is replicating to. - * * @return peer id */ - String getPeerId(); + default String getPeerId() { + return getPeer().getId(); + } + + /** + * Get the replication peer instance. + * @return the replication peer instance + */ + ReplicationPeer getPeer(); /** * Get a string representation of the current statistics @@ -119,9 +126,17 @@ public interface ReplicationSourceInterface { /** * @return peer enabled or not */ - boolean isPeerEnabled(); + default boolean isPeerEnabled() { + return getPeer().isPeerEnabled(); + } /** + * @return whether this is sync replication peer. + */ + default boolean isSyncReplication() { + return getPeer().getPeerConfig().isSyncReplication(); + } + /** * @return active or not */ boolean isSourceActive(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java index a370867..428ec98 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hbase.replication.regionserver; +import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -51,13 +53,17 @@ import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationListener; import org.apache.hadoop.hbase.replication.ReplicationPeer; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerImpl; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationTracker; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -85,20 +91,20 @@ import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFacto * operations. *
  • Need synchronized on {@link #walsById}. There are four methods which modify it, * {@link #addPeer(String)}, {@link #removePeer(String)}, - * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and {@link #preLogRoll(Path)}. + * {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} and {@link #preLogRoll(Path)}. * {@link #walsById} is a ConcurrentHashMap and there is a Lock for peer id in * {@link PeerProcedureHandlerImpl}. So there is no race between {@link #addPeer(String)} and - * {@link #removePeer(String)}. {@link #cleanOldLogs(NavigableSet, String, boolean, String)} is - * called by {@link ReplicationSourceInterface}. So no race with {@link #addPeer(String)}. + * {@link #removePeer(String)}. {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} + * is called by {@link ReplicationSourceInterface}. So no race with {@link #addPeer(String)}. * {@link #removePeer(String)} will terminate the {@link ReplicationSourceInterface} firstly, then * remove the wals from {@link #walsById}. So no race with {@link #removePeer(String)}. The only - * case need synchronized is {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and + * case need synchronized is {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} and * {@link #preLogRoll(Path)}.
  • *
  • No need synchronized on {@link #walsByIdRecoveredQueues}. There are three methods which * modify it, {@link #removePeer(String)} , - * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and + * {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} and * {@link ReplicationSourceManager.NodeFailoverWorker#run()}. - * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} is called by + * {@link #cleanOldLogs(String, boolean, ReplicationSourceInterface)} is called by * {@link ReplicationSourceInterface}. {@link #removePeer(String)} will terminate the * {@link ReplicationSourceInterface} firstly, then remove the wals from * {@link #walsByIdRecoveredQueues}. And {@link ReplicationSourceManager.NodeFailoverWorker#run()} @@ -136,6 +142,8 @@ public class ReplicationSourceManager implements ReplicationListener { // For recovered source, the queue id's format is peer_id-servername-* private final ConcurrentMap>> walsByIdRecoveredQueues; + private final SyncReplicationPeerMappingManager syncReplicationPeerMappingManager; + private final Configuration conf; private final FileSystem fs; // The paths to the latest log of each wal group, for new coming peers @@ -152,9 +160,15 @@ public class ReplicationSourceManager implements ReplicationListener { private final boolean replicationForBulkLoadDataEnabled; - private AtomicLong totalBufferUsed = new AtomicLong(); + // How long should we sleep for each retry when deleting remote wal files for sync replication + // peer. + private final long sleepForRetries; + // Maximum number of retries before taking bold actions when deleting remote wal files for sync + // replication peer. + private final int maxRetriesMultiplier; + /** * Creates a replication manager and sets the watch on all the other registered region servers * @param queueStorage the interface for manipulating replication queues @@ -170,9 +184,8 @@ public class ReplicationSourceManager implements ReplicationListener { public ReplicationSourceManager(ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf, Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId, - WALFileLengthProvider walFileLengthProvider) throws IOException { - // CopyOnWriteArrayList is thread-safe. - // Generally, reading is more than modifying. + WALFileLengthProvider walFileLengthProvider, + SyncReplicationPeerMappingManager syncReplicationPeerMappingManager) throws IOException { this.sources = new ConcurrentHashMap<>(); this.queueStorage = queueStorage; this.replicationPeers = replicationPeers; @@ -185,10 +198,11 @@ public class ReplicationSourceManager implements ReplicationListener { this.fs = fs; this.logDir = logDir; this.oldLogDir = oldLogDir; - this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 30000); // 30 - // seconds + // 30 seconds + this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 30000); this.clusterId = clusterId; this.walFileLengthProvider = walFileLengthProvider; + this.syncReplicationPeerMappingManager = syncReplicationPeerMappingManager; this.replicationTracker.registerListener(this); // It's preferable to failover 1 RS at a time, but with good zk servers // more could be processed at the same time. @@ -202,8 +216,11 @@ public class ReplicationSourceManager implements ReplicationListener { tfb.setDaemon(true); this.executor.setThreadFactory(tfb.build()); this.latestPaths = new HashSet(); - replicationForBulkLoadDataEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, - HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT); + this.replicationForBulkLoadDataEnabled = conf.getBoolean( + HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT); + this.sleepForRetries = this.conf.getLong("replication.source.sync.sleepforretries", 1000); + this.maxRetriesMultiplier = + this.conf.getInt("replication.source.sync.maxretriesmultiplier", 60); } /** @@ -249,8 +266,11 @@ public class ReplicationSourceManager implements ReplicationListener { } /** - * 1. Add peer to replicationPeers 2. Add the normal source and related replication queue 3. Add - * HFile Refs + *
      + *
    1. Add peer to replicationPeers
    2. + *
    3. Add the normal source and related replication queue
    4. + *
    5. Add HFile Refs
    6. + *
    * @param peerId the id of replication peer */ public void addPeer(String peerId) throws IOException { @@ -269,13 +289,16 @@ public class ReplicationSourceManager implements ReplicationListener { } /** - * 1. Remove peer for replicationPeers 2. Remove all the recovered sources for the specified id - * and related replication queues 3. Remove the normal source and related replication queue 4. - * Remove HFile Refs + *
      + *
    1. Remove peer for replicationPeers
    2. + *
    3. Remove all the recovered sources for the specified id and related replication queues
    4. + *
    5. Remove the normal source and related replication queue
    6. + *
    7. Remove HFile Refs
    8. + *
    * @param peerId the id of the replication peer */ public void removePeer(String peerId) { - replicationPeers.removePeer(peerId); + ReplicationPeer peer = replicationPeers.removePeer(peerId); String terminateMessage = "Replication stream was removed by a user"; List oldSourcesToDelete = new ArrayList<>(); // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer @@ -306,7 +329,10 @@ public class ReplicationSourceManager implements ReplicationListener { deleteQueue(peerId); this.walsById.remove(peerId); } - + ReplicationPeerConfig peerConfig = peer.getPeerConfig(); + if (peerConfig.isSyncReplication()) { + syncReplicationPeerMappingManager.remove(peerId, peerConfig); + } // Remove HFile Refs abortWhenFail(() -> this.queueStorage.removePeerFromHFileRefs(peerId)); } @@ -358,16 +384,92 @@ public class ReplicationSourceManager implements ReplicationListener { } } } + ReplicationPeerConfig peerConfig = peer.getPeerConfig(); + if (peerConfig.isSyncReplication()) { + syncReplicationPeerMappingManager.add(peer.getId(), peerConfig); + } src.startup(); return src; } /** + *

    + * This is used when we transit a sync replication peer to {@link SyncReplicationState#STANDBY}. + *

    + *

    + * When transiting to {@link SyncReplicationState#STANDBY}, we can remove all the pending wal + * files for a replication peer as we do not need to replicate them any more. And this is + * necessary, otherwise when we transit back to {@link SyncReplicationState#DOWNGRADE_ACTIVE} + * later, the stale data will be replicated again and cause inconsistency. + *

    + *

    + * See HBASE-20426 for more details. + *

    + * @param peerId the id of the sync replication peer + */ + public void drainSources(String peerId) throws IOException, ReplicationException { + String terminateMessage = "Sync replication peer " + peerId + + " is transiting to STANDBY. Will close the previous replication source and open a new one"; + ReplicationPeer peer = replicationPeers.getPeer(peerId); + assert peer.getPeerConfig().isSyncReplication(); + ReplicationSourceInterface src = createSource(peerId, peer); + // synchronized here to avoid race with preLogRoll where we add new log to source and also + // walsById. + ReplicationSourceInterface toRemove; + Map> wals = new HashMap<>(); + synchronized (latestPaths) { + toRemove = sources.put(peerId, src); + if (toRemove != null) { + LOG.info("Terminate replication source for " + toRemove.getPeerId()); + toRemove.terminate(terminateMessage); + toRemove.getSourceMetrics().clear(); + } + // Here we make a copy of all the remaining wal files and then delete them from the + // replication queue storage after releasing the lock. It is not safe to just remove the old + // map from walsById since later we may fail to delete them from the replication queue + // storage, and when we retry next time, we can not know the wal files that need to be deleted + // from the replication queue storage. + walsById.get(peerId).forEach((k, v) -> wals.put(k, new TreeSet<>(v))); + } + LOG.info("Startup replication source for " + src.getPeerId()); + src.startup(); + for (NavigableSet walsByGroup : wals.values()) { + for (String wal : walsByGroup) { + queueStorage.removeWAL(server.getServerName(), peerId, wal); + } + } + synchronized (walsById) { + Map> oldWals = walsById.get(peerId); + wals.forEach((k, v) -> { + NavigableSet walsByGroup = oldWals.get(k); + if (walsByGroup != null) { + walsByGroup.removeAll(v); + } + }); + } + // synchronized on oldsources to avoid race with NodeFailoverWorker. Since NodeFailoverWorker is + // a background task, we will delete the file from replication queue storage under the lock to + // simplify the logic. + synchronized (this.oldsources) { + for (Iterator iter = oldsources.iterator(); iter.hasNext();) { + ReplicationSourceInterface oldSource = iter.next(); + if (oldSource.getPeerId().equals(peerId)) { + String queueId = oldSource.getQueueId(); + oldSource.terminate(terminateMessage); + oldSource.getSourceMetrics().clear(); + queueStorage.removeQueue(server.getServerName(), queueId); + walsByIdRecoveredQueues.remove(queueId); + iter.remove(); + } + } + } + } + + /** * Close the previous replication sources of this peer id and open new sources to trigger the new * replication state changes or new replication config changes. Here we don't need to change * replication queue storage and only to enqueue all logs to the new replication source * @param peerId the id of the replication peer - * @throws IOException */ public void refreshSources(String peerId) throws IOException { String terminateMessage = "Peer " + peerId + @@ -381,7 +483,7 @@ public class ReplicationSourceManager implements ReplicationListener { LOG.info("Terminate replication source for " + toRemove.getPeerId()); toRemove.terminate(terminateMessage); } - for (SortedSet walsByGroup : walsById.get(peerId).values()) { + for (NavigableSet walsByGroup : walsById.get(peerId).values()) { walsByGroup.forEach(wal -> src.enqueueLog(new Path(this.logDir, wal))); } } @@ -417,12 +519,25 @@ public class ReplicationSourceManager implements ReplicationListener { * Clear the metrics and related replication queue of the specified old source * @param src source to clear */ - void removeRecoveredSource(ReplicationSourceInterface src) { - LOG.info("Done with the recovered queue " + src.getQueueId()); - this.oldsources.remove(src); + private boolean removeRecoveredSource(ReplicationSourceInterface src) { + if (!this.oldsources.remove(src)) { + return false; + } + LOG.info("Done with the recovered queue {}", src.getQueueId()); // Delete queue from storage and memory deleteQueue(src.getQueueId()); this.walsByIdRecoveredQueues.remove(src.getQueueId()); + return true; + } + + void finishRecoveredSource(ReplicationSourceInterface src) { + synchronized (oldsources) { + if (!removeRecoveredSource(src)) { + return; + } + } + LOG.info("Finished recovering queue {} with the following stats: {}", src.getQueueId(), + src.getStats()); } /** @@ -435,6 +550,7 @@ public class ReplicationSourceManager implements ReplicationListener { // Delete queue from storage and memory deleteQueue(src.getQueueId()); this.walsById.remove(src.getQueueId()); + } /** @@ -496,17 +612,15 @@ public class ReplicationSourceManager implements ReplicationListener { /** * This method will log the current position to storage. And also clean old logs from the * replication queue. - * @param queueId id of the replication queue - * @param queueRecovered indicates if this queue comes from another region server + * @param source the replication source * @param entryBatch the wal entry batch we just shipped */ - public void logPositionAndCleanOldLogs(String queueId, boolean queueRecovered, + public void logPositionAndCleanOldLogs(ReplicationSourceInterface source, WALEntryBatch entryBatch) { String fileName = entryBatch.getLastWalPath().getName(); - interruptOrAbortWhenFail(() -> this.queueStorage - .setWALPosition(server.getServerName(), queueId, fileName, entryBatch.getLastWalPosition(), - entryBatch.getLastSeqIds())); - cleanOldLogs(fileName, entryBatch.isEndOfFile(), queueId, queueRecovered); + interruptOrAbortWhenFail(() -> this.queueStorage.setWALPosition(server.getServerName(), + source.getQueueId(), fileName, entryBatch.getLastWalPosition(), entryBatch.getLastSeqIds())); + cleanOldLogs(fileName, entryBatch.isEndOfFile(), source); } /** @@ -514,38 +628,109 @@ public class ReplicationSourceManager implements ReplicationListener { * file is closed and has no more entries. * @param log Path to the log * @param inclusive whether we should also remove the given log file - * @param queueId id of the replication queue - * @param queueRecovered Whether this is a recovered queue + * @param source the replication source */ @VisibleForTesting - void cleanOldLogs(String log, boolean inclusive, String queueId, boolean queueRecovered) { + void cleanOldLogs(String log, boolean inclusive, ReplicationSourceInterface source) { String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log); - if (queueRecovered) { - NavigableSet wals = walsByIdRecoveredQueues.get(queueId).get(logPrefix); + if (source.isRecovered()) { + NavigableSet wals = walsByIdRecoveredQueues.get(source.getQueueId()).get(logPrefix); if (wals != null) { - cleanOldLogs(wals, log, inclusive, queueId); + NavigableSet walsToRemove = wals.headSet(log, inclusive); + if (walsToRemove.isEmpty()) { + return; + } + cleanOldLogs(walsToRemove, source); + walsToRemove.clear(); } } else { + NavigableSet wals; + NavigableSet walsToRemove; // synchronized on walsById to avoid race with preLogRoll synchronized (this.walsById) { - NavigableSet wals = walsById.get(queueId).get(logPrefix); - if (wals != null) { - cleanOldLogs(wals, log, inclusive, queueId); + wals = walsById.get(source.getQueueId()).get(logPrefix); + if (wals == null) { + return; + } + walsToRemove = wals.headSet(log, inclusive); + if (walsToRemove.isEmpty()) { + return; } + walsToRemove = new TreeSet<>(walsToRemove); + } + // cleanOldLogs may spend some time, especially for sync replication where we may want to + // remove remote wals as the remote cluster may have already been down, so we do it outside + // the lock to avoid block preLogRoll + cleanOldLogs(walsToRemove, source); + // now let's remove the files in the set + synchronized (this.walsById) { + wals.removeAll(walsToRemove); } } } - private void cleanOldLogs(NavigableSet wals, String key, boolean inclusive, String id) { - NavigableSet walSet = wals.headSet(key, inclusive); - if (walSet.isEmpty()) { - return; + private void removeRemoteWALs(String peerId, String remoteWALDir, Collection wals) + throws IOException { + Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId); + FileSystem fs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir); + for (String wal : wals) { + Path walFile = new Path(remoteWALDirForPeer, wal); + try { + if (!fs.delete(walFile, false) && fs.exists(walFile)) { + throw new IOException("Can not delete " + walFile); + } + } catch (FileNotFoundException e) { + // Just ignore since this means the file has already been deleted. + // The javadoc of the FileSystem.delete methods does not specify the behavior of deleting an + // inexistent file, so here we deal with both, i.e, check the return value of the + // FileSystem.delete, and also catch FNFE. + LOG.debug("The remote wal {} has already been deleted?", walFile, e); + } } - LOG.debug("Removing {} logs in the list: {}", walSet.size(), walSet); - for (String wal : walSet) { - interruptOrAbortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal)); + } + + private void cleanOldLogs(NavigableSet wals, ReplicationSourceInterface source) { + LOG.debug("Removing {} logs in the list: {}", wals.size(), wals); + // The intention here is that, we want to delete the remote wal files ASAP as it may effect the + // failover time if you want to transit the remote cluster from S to A. And the infinite retry + // is not a problem, as if we can not contact with the remote HDFS cluster, then usually we can + // not contact with the HBase cluster either, so the replication will be blocked either. + if (source.isSyncReplication()) { + String peerId = source.getPeerId(); + String remoteWALDir = source.getPeer().getPeerConfig().getRemoteWALDir(); + // Filter out the wals need to be removed from the remote directory. Its name should be the + // special format, and also, the peer id in its name should match the peer id for the + // replication source. + List remoteWals = wals.stream().filter(w -> SyncReplicationWALProvider + .getSyncReplicationPeerIdFromWALName(w).map(peerId::equals).orElse(false)) + .collect(Collectors.toList()); + LOG.debug("Removing {} logs from remote dir {} in the list: {}", remoteWals.size(), + remoteWALDir, remoteWals); + if (!remoteWals.isEmpty()) { + for (int sleepMultiplier = 0;;) { + try { + removeRemoteWALs(peerId, remoteWALDir, remoteWals); + break; + } catch (IOException e) { + LOG.warn("Failed to delete remote wals from remote dir {} for peer {}", remoteWALDir, + peerId); + } + if (!source.isSourceActive()) { + // skip the following operations + return; + } + if (ReplicationUtils.sleepForRetries("Failed to delete remote wals", sleepForRetries, + sleepMultiplier, maxRetriesMultiplier)) { + sleepMultiplier++; + } + } + } + } + String queueId = source.getQueueId(); + for (String wal : wals) { + interruptOrAbortWhenFail( + () -> this.queueStorage.removeWAL(server.getServerName(), queueId, wal)); } - walSet.clear(); } // public because of we call it in TestReplicationEmptyWALRecovery @@ -738,18 +923,6 @@ public class ReplicationSourceManager implements ReplicationListener { actualPeerId); continue; } - // track sources in walsByIdRecoveredQueues - Map> walsByGroup = new HashMap<>(); - walsByIdRecoveredQueues.put(queueId, walsByGroup); - for (String wal : walsSet) { - String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal); - NavigableSet wals = walsByGroup.get(walPrefix); - if (wals == null) { - wals = new TreeSet<>(); - walsByGroup.put(walPrefix, wals); - } - wals.add(wal); - } ReplicationSourceInterface src = createSource(queueId, peer); // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer @@ -757,9 +930,36 @@ public class ReplicationSourceManager implements ReplicationListener { peer = replicationPeers.getPeer(src.getPeerId()); if (peer == null || !isOldPeer(src.getPeerId(), peer)) { src.terminate("Recovered queue doesn't belong to any current peer"); - removeRecoveredSource(src); + deleteQueue(queueId); continue; } + // Do not setup recovered queue if a sync replication peer is in STANDBY state, or is + // transiting to STANDBY state. The only exception is we are in STANDBY state and + // transiting to DA, under this state we will replay the remote WAL and they need to be + // replicated back. + if (peer.getPeerConfig().isSyncReplication()) { + Pair stateAndNewState = + peer.getSyncReplicationStateAndNewState(); + if ((stateAndNewState.getFirst().equals(SyncReplicationState.STANDBY) && + stateAndNewState.getSecond().equals(SyncReplicationState.NONE)) || + stateAndNewState.getSecond().equals(SyncReplicationState.STANDBY)) { + src.terminate("Sync replication peer is in STANDBY state"); + deleteQueue(queueId); + continue; + } + } + // track sources in walsByIdRecoveredQueues + Map> walsByGroup = new HashMap<>(); + walsByIdRecoveredQueues.put(queueId, walsByGroup); + for (String wal : walsSet) { + String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal); + NavigableSet wals = walsByGroup.get(walPrefix); + if (wals == null) { + wals = new TreeSet<>(); + walsByGroup.put(walPrefix, wals); + } + wals.add(wal); + } oldsources.add(src); for (String wal : walsSet) { src.enqueueLog(new Path(oldLogDir, wal)); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java index 51df46a..30696d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.replication.regionserver; +import static org.apache.hadoop.hbase.replication.ReplicationUtils.sleepForRetries; + import java.io.IOException; import java.util.List; import java.util.concurrent.PriorityBlockingQueue; @@ -49,13 +51,13 @@ public class ReplicationSourceShipper extends Thread { public enum WorkerState { RUNNING, STOPPED, - FINISHED, // The worker is done processing a recovered queue + FINISHED, // The worker is done processing a queue } private final Configuration conf; protected final String walGroupId; protected final PriorityBlockingQueue queue; - private final ReplicationSourceInterface source; + private final ReplicationSource source; // Last position in the log that we sent to ZooKeeper // It will be accessed by the stats thread so make it volatile @@ -72,7 +74,7 @@ public class ReplicationSourceShipper extends Thread { protected final int maxRetriesMultiplier; public ReplicationSourceShipper(Configuration conf, String walGroupId, - PriorityBlockingQueue queue, ReplicationSourceInterface source) { + PriorityBlockingQueue queue, ReplicationSource source) { this.conf = conf; this.walGroupId = walGroupId; this.queue = queue; @@ -92,12 +94,12 @@ public class ReplicationSourceShipper extends Thread { if (!source.isPeerEnabled()) { // The peer enabled check is in memory, not expensive, so do not need to increase the // sleep interval as it may cause a long lag when we enable the peer. - sleepForRetries("Replication is disabled", 1); + sleepForRetries("Replication is disabled", sleepForRetries, 1, maxRetriesMultiplier); continue; } try { WALEntryBatch entryBatch = entryReader.take(); - // the NO_MORE_DATA instance has no path so do not all shipEdits + // the NO_MORE_DATA instance has no path so do not call shipEdits if (entryBatch == WALEntryBatch.NO_MORE_DATA) { noMoreData(); } else { @@ -112,12 +114,20 @@ public class ReplicationSourceShipper extends Thread { if (!isFinished()) { setWorkerState(WorkerState.STOPPED); } else { + source.removeWorker(this); postFinish(); } } - // To be implemented by recovered shipper - protected void noMoreData() { + private void noMoreData() { + if (source.isRecovered()) { + LOG.debug("Finished recovering queue for group {} of peer {}", walGroupId, + source.getQueueId()); + source.getSourceMetrics().incrCompletedRecoveryQueue(); + } else { + LOG.debug("Finished queue for group {} of peer {}", walGroupId, source.getQueueId()); + } + setWorkerState(WorkerState.FINISHED); } // To be implemented by recovered shipper @@ -211,7 +221,8 @@ public class ReplicationSourceShipper extends Thread { } catch (Exception ex) { LOG.warn("{} threw unknown exception:", source.getReplicationEndpoint().getClass().getName(), ex); - if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) { + if (sleepForRetries("ReplicationEndpoint threw exception", sleepForRetries, sleepMultiplier, + maxRetriesMultiplier)) { sleepMultiplier++; } } @@ -250,8 +261,7 @@ public class ReplicationSourceShipper extends Thread { // position and the file will be removed soon in cleanOldLogs. if (batch.isEndOfFile() || !batch.getLastWalPath().equals(currentPath) || batch.getLastWalPosition() != currentPosition) { - source.getSourceManager().logPositionAndCleanOldLogs(source.getQueueId(), - source.isRecovered(), batch); + source.getSourceManager().logPositionAndCleanOldLogs(source, batch); updated = true; } // if end of file is true, then we can just skip to the next file in queue. @@ -304,21 +314,4 @@ public class ReplicationSourceShipper extends Thread { public boolean isFinished() { return state == WorkerState.FINISHED; } - - /** - * Do the sleeping logic - * @param msg Why we sleep - * @param sleepMultiplier by how many times the default sleeping time is augmented - * @return True if sleepMultiplier is < maxRetriesMultiplier - */ - public boolean sleepForRetries(String msg, int sleepMultiplier) { - try { - LOG.trace("{}, sleeping {} times {}", msg, sleepForRetries, sleepMultiplier); - Thread.sleep(this.sleepForRetries * sleepMultiplier); - } catch (InterruptedException e) { - LOG.debug("Interrupted while sleeping between retries"); - Thread.currentThread().interrupt(); - } - return sleepMultiplier < maxRetriesMultiplier; - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java index f685a9b..666ee2a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java @@ -143,7 +143,7 @@ class ReplicationSourceWALReader extends Thread { entryBatchQueue.put(batch); sleepMultiplier = 1; } else { // got no entries and didn't advance position in WAL - handleEmptyWALEntryBatch(entryStream.getCurrentPath()); + handleEmptyWALEntryBatch(); entryStream.reset(); // reuse stream } } @@ -227,10 +227,11 @@ class ReplicationSourceWALReader extends Thread { return batch; } - private void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException { + private void handleEmptyWALEntryBatch() throws InterruptedException { LOG.trace("Didn't read any new entries from WAL"); - if (source.isRecovered()) { - // we're done with queue recovery, shut ourself down + if (logQueue.isEmpty()) { + // we're done with current queue, either this is a recovered queue, or it is the special group + // for a sync replication peer and the peer has been transited to DA or S state. setReaderRunning(false); // shuts down shipper thread immediately entryBatchQueue.put(WALEntryBatch.NO_MORE_DATA); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java new file mode 100644 index 0000000..cfe525a --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProvider.java @@ -0,0 +1,48 @@ +/** + * 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.replication.regionserver; + +import java.util.Optional; +import java.util.function.BiPredicate; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Get the information for a sync replication peer. + */ +@InterfaceAudience.Private +public interface SyncReplicationPeerInfoProvider { + + /** + * Return the peer id and remote WAL directory if the table is synchronously replicated and the + * state is {@link SyncReplicationState#ACTIVE}. + */ + Optional> getPeerIdAndRemoteWALDir(TableName table); + + /** + * Check whether the given table is contained in a sync replication peer which can pass the state + * checker. + *

    + * Will call the checker with current sync replication state and new sync replication state. + */ + boolean checkState(TableName table, + BiPredicate checker); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java new file mode 100644 index 0000000..170441b --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerInfoProviderImpl.java @@ -0,0 +1,83 @@ +/** + * 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.replication.regionserver; + +import java.util.Optional; +import java.util.function.BiPredicate; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.replication.ReplicationPeerImpl; +import org.apache.hadoop.hbase.replication.ReplicationPeers; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +class SyncReplicationPeerInfoProviderImpl implements SyncReplicationPeerInfoProvider { + + private final ReplicationPeers replicationPeers; + + private final SyncReplicationPeerMappingManager mapping; + + SyncReplicationPeerInfoProviderImpl(ReplicationPeers replicationPeers, + SyncReplicationPeerMappingManager mapping) { + this.replicationPeers = replicationPeers; + this.mapping = mapping; + } + + @Override + public Optional> getPeerIdAndRemoteWALDir(TableName table) { + if (table == null) { + return Optional.empty(); + } + String peerId = mapping.getPeerId(table); + if (peerId == null) { + return Optional.empty(); + } + ReplicationPeerImpl peer = replicationPeers.getPeer(peerId); + if (peer == null) { + return Optional.empty(); + } + Pair states = + peer.getSyncReplicationStateAndNewState(); + if ((states.getFirst() == SyncReplicationState.ACTIVE && + states.getSecond() == SyncReplicationState.NONE) || + (states.getFirst() == SyncReplicationState.DOWNGRADE_ACTIVE && + states.getSecond() == SyncReplicationState.ACTIVE)) { + return Optional.of(Pair.newPair(peerId, peer.getPeerConfig().getRemoteWALDir())); + } else { + return Optional.empty(); + } + } + + @Override + public boolean checkState(TableName table, + BiPredicate checker) { + String peerId = mapping.getPeerId(table); + if (peerId == null) { + return false; + } + ReplicationPeerImpl peer = replicationPeers.getPeer(peerId); + if (peer == null) { + return false; + } + Pair states = + peer.getSyncReplicationStateAndNewState(); + return checker.test(states.getFirst(), states.getSecond()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java new file mode 100644 index 0000000..5d19f72 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerMappingManager.java @@ -0,0 +1,47 @@ +/** + * 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.replication.regionserver; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Used to map region to sync replication peer id. + *

    + * TODO: now only support include table options. + */ +@InterfaceAudience.Private +class SyncReplicationPeerMappingManager { + + private final ConcurrentMap table2PeerId = new ConcurrentHashMap<>(); + + void add(String peerId, ReplicationPeerConfig peerConfig) { + peerConfig.getTableCFsMap().keySet().forEach(tn -> table2PeerId.put(tn, peerId)); + } + + void remove(String peerId, ReplicationPeerConfig peerConfig) { + peerConfig.getTableCFsMap().keySet().forEach(table2PeerId::remove); + } + + String getPeerId(TableName tableName) { + return table2PeerId.get(tableName); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java index b2c199e..0393af4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java @@ -304,7 +304,8 @@ class WALEntryStream implements Closeable { return true; } } else { - // no more files in queue, this could only happen for recovered queue. + // no more files in queue, this could happen for recovered queue, or for a wal group of a sync + // replication peer which has already been transited to DA or S. setCurrentPath(null); } return false; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 2758c7e..52b7a92 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -112,6 +112,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.replication.ReplicationEndpoint; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; @@ -2503,6 +2504,13 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor, } @Override + public void preTransitReplicationPeerSyncReplicationState( + final ObserverContext ctx, String peerId, + SyncReplicationState clusterState) throws IOException { + requirePermission(ctx, "transitSyncReplicationPeerState", Action.ADMIN); + } + + @Override public void preListReplicationPeers(final ObserverContext ctx, String regex) throws IOException { requirePermission(ctx, "listReplicationPeers", Action.ADMIN); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java index 301d158..a49ee02 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java @@ -28,9 +28,9 @@ import java.util.Collection; import java.util.HashSet; import java.util.Map; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException; @@ -155,12 +155,16 @@ public class FSHDFSUtils extends FSUtils { * Recover the lease from HDFS, retrying multiple times. */ @Override - public void recoverFileLease(final FileSystem fs, final Path p, - Configuration conf, CancelableProgressable reporter) - throws IOException { + public void recoverFileLease(FileSystem fs, Path p, Configuration conf, + CancelableProgressable reporter) throws IOException { + if (fs instanceof FilterFileSystem) { + fs = ((FilterFileSystem) fs).getRawFileSystem(); + } // lease recovery not needed for local file system case. - if (!(fs instanceof DistributedFileSystem)) return; - recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter); + if (!(fs instanceof DistributedFileSystem)) { + return; + } + recoverDFSFileLease((DistributedFileSystem) fs, p, conf, reporter); } /* diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index 53db140..5b968db 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -843,6 +843,15 @@ public abstract class FSUtils extends CommonFSUtils { return frags; } + public static void renameFile(FileSystem fs, Path src, Path dst) throws IOException { + if (fs.exists(dst) && !fs.delete(dst, false)) { + throw new IOException("Can not delete " + dst); + } + if (!fs.rename(src, dst)) { + throw new IOException("Can not rename from " + src + " to " + dst); + } + } + /** * A {@link PathFilter} that returns only regular files. */ @@ -939,6 +948,11 @@ public abstract class FSUtils extends CommonFSUtils { } } + public void recoverFileLease(final FileSystem fs, final Path p, Configuration conf) + throws IOException { + recoverFileLease(fs, p, conf, null); + } + /** * Recover file lease. Used when a file might be suspect * to be had been left open by another process. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java index 231afd5..ccdc95f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; @@ -136,6 +137,15 @@ public abstract class AbstractFSWALProvider> implemen walCopy = wal; if (walCopy == null) { walCopy = createWAL(); + boolean succ = false; + try { + walCopy.init(); + succ = true; + } finally { + if (!succ) { + walCopy.close(); + } + } wal = walCopy; } } @@ -238,26 +248,30 @@ public abstract class AbstractFSWALProvider> implemen if (walName == null) { throw new IllegalArgumentException("The WAL path couldn't be null"); } - final String[] walPathStrs = walName.toString().split("\\" + WAL_FILE_NAME_DELIMITER); - return Long.parseLong(walPathStrs[walPathStrs.length - (isMetaFile(walName) ? 2 : 1)]); + Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(walName.getName()); + if (matcher.matches()) { + return Long.parseLong(matcher.group(2)); + } else { + throw new IllegalArgumentException(walName.getName() + " is not a valid wal file name"); + } } /** * Pattern used to validate a WAL file name see {@link #validateWALFilename(String)} for * description. */ - private static final Pattern pattern = - Pattern.compile(".*\\.\\d*(" + META_WAL_PROVIDER_ID + ")*"); + private static final Pattern WAL_FILE_NAME_PATTERN = + Pattern.compile("(.+)\\.(\\d+)(\\.[0-9A-Za-z]+)?"); /** * A WAL file name is of the format: <wal-name>{@link #WAL_FILE_NAME_DELIMITER} - * <file-creation-timestamp>[.meta]. provider-name is usually made up of a server-name and a - * provider-id + * <file-creation-timestamp>[.<suffix>]. provider-name is usually made up of a + * server-name and a provider-id * @param filename name of the file to validate * @return true if the filename matches an WAL, false otherwise */ public static boolean validateWALFilename(String filename) { - return pattern.matcher(filename).matches(); + return WAL_FILE_NAME_PATTERN.matcher(filename).matches(); } /** @@ -503,15 +517,27 @@ public abstract class AbstractFSWALProvider> implemen listeners.add(listener); } + private static String getWALNameGroupFromWALName(String name, int group) { + Matcher matcher = WAL_FILE_NAME_PATTERN.matcher(name); + if (matcher.matches()) { + return matcher.group(group); + } else { + throw new IllegalArgumentException(name + " is not a valid wal file name"); + } + } /** * Get prefix of the log from its name, assuming WAL name in format of * log_prefix.filenumber.log_suffix * @param name Name of the WAL to parse * @return prefix of the log + * @throws IllegalArgumentException if the name passed in is not a valid wal file name * @see AbstractFSWAL#getCurrentFileName() */ public static String getWALPrefixFromWALName(String name) { - int endIndex = name.replaceAll(META_WAL_PROVIDER_ID, "").lastIndexOf("."); - return name.substring(0, endIndex); + return getWALNameGroupFromWALName(name, 1); + } + + public static long getWALStartTimeFromWALName(String name) { + return Long.parseLong(getWALNameGroupFromWALName(name, 2)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java index c920279..56edb75 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.wal; import java.io.IOException; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -32,12 +31,10 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.base.Throwables; import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; -import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; -import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory; /** * A WAL provider that use {@link AsyncFSWAL}. @@ -62,6 +59,7 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider { private EventLoopGroup eventLoopGroup; private Class channelClass; + @Override protected AsyncFSWAL createWAL() throws IOException { return new AsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf), @@ -74,15 +72,9 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider { @Override protected void doInit(Configuration conf) throws IOException { Pair> eventLoopGroupAndChannelClass = - NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf); - if (eventLoopGroupAndChannelClass != null) { - eventLoopGroup = eventLoopGroupAndChannelClass.getFirst(); - channelClass = eventLoopGroupAndChannelClass.getSecond(); - } else { - eventLoopGroup = new NioEventLoopGroup(1, - new DefaultThreadFactory("AsyncFSWAL", true, Thread.MAX_PRIORITY)); - channelClass = NioSocketChannel.class; - } + NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf); + eventLoopGroup = eventLoopGroupAndChannelClass.getFirst(); + channelClass = eventLoopGroupAndChannelClass.getSecond(); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java index 12b63f5..7f33eda 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java @@ -27,6 +27,9 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; +import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; +import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory; /** * Helper class for passing netty event loop config to {@link AsyncFSWALProvider}. @@ -57,7 +60,10 @@ public final class NettyAsyncFSWALConfigHelper { static Pair> getEventLoopConfig(Configuration conf) { String name = conf.get(EVENT_LOOP_CONFIG); if (StringUtils.isBlank(name)) { - return null; + // create new event loop group if config is empty + return Pair.> newPair( + new NioEventLoopGroup(0, new DefaultThreadFactory("AsyncFSWAL", true, Thread.MAX_PRIORITY)), + NioSocketChannel.class); } return EVENT_LOOP_CONFIG_MAP.get(name); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java index 28817e9..0b7b8da 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java @@ -31,6 +31,7 @@ import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL; // imports for classes still in regionserver.wal import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.util.Bytes; @@ -132,6 +133,7 @@ public class RegionGroupingProvider implements WALProvider { private RegionGroupingStrategy strategy; private WALFactory factory; + private Configuration conf; private List listeners = new ArrayList<>(); private String providerId; private Class providerClass; @@ -141,6 +143,7 @@ public class RegionGroupingProvider implements WALProvider { if (null != strategy) { throw new IllegalStateException("WALProvider.init should only be called once."); } + this.conf = conf; this.factory = factory; StringBuilder sb = new StringBuilder().append(factory.factoryId); if (providerId != null) { @@ -156,11 +159,11 @@ public class RegionGroupingProvider implements WALProvider { } private WALProvider createProvider(String group) throws IOException { - if (META_WAL_PROVIDER_ID.equals(providerId)) { - return factory.createProvider(providerClass, META_WAL_PROVIDER_ID); - } else { - return factory.createProvider(providerClass, group); - } + WALProvider provider = WALFactory.createProvider(providerClass); + provider.init(factory, conf, + META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : group); + provider.addWALActionsListener(new MetricsWAL()); + return provider; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java new file mode 100644 index 0000000..b9fffcf --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java @@ -0,0 +1,346 @@ +/** + * 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.wal; + +import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDirectoryName; +import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.function.BiPredicate; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL; +import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener; +import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.KeyLocker; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.common.base.Throwables; +import org.apache.hbase.thirdparty.com.google.common.collect.Streams; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; + +/** + * The special {@link WALProvider} for synchronous replication. + *

    + * It works like an interceptor, when getting WAL, first it will check if the given region should be + * replicated synchronously, if so it will return a special WAL for it, otherwise it will delegate + * the request to the normal {@link WALProvider}. + */ +@InterfaceAudience.Private +public class SyncReplicationWALProvider implements WALProvider, PeerActionListener { + + private static final Logger LOG = LoggerFactory.getLogger(SyncReplicationWALProvider.class); + + // only for injecting errors for testcase, do not use it for other purpose. + @VisibleForTesting + public static final String DUAL_WAL_IMPL = "hbase.wal.sync.impl"; + + private final WALProvider provider; + + private SyncReplicationPeerInfoProvider peerInfoProvider = + new DefaultSyncReplicationPeerInfoProvider(); + + private WALFactory factory; + + private Configuration conf; + + private List listeners = new ArrayList<>(); + + private EventLoopGroup eventLoopGroup; + + private Class channelClass; + + private AtomicBoolean initialized = new AtomicBoolean(false); + + // when switching from A to DA, we will put a Optional.empty into this map if there is no WAL for + // the peer yet. When getting WAL from this map the caller should know that it should not use + // DualAsyncFSWAL any more. + private final ConcurrentMap> peerId2WAL = + new ConcurrentHashMap<>(); + + private final KeyLocker createLock = new KeyLocker<>(); + + SyncReplicationWALProvider(WALProvider provider) { + this.provider = provider; + } + + public void setPeerInfoProvider(SyncReplicationPeerInfoProvider peerInfoProvider) { + this.peerInfoProvider = peerInfoProvider; + } + + @Override + public void init(WALFactory factory, Configuration conf, String providerId) throws IOException { + if (!initialized.compareAndSet(false, true)) { + throw new IllegalStateException("WALProvider.init should only be called once."); + } + provider.init(factory, conf, providerId); + this.conf = conf; + this.factory = factory; + Pair> eventLoopGroupAndChannelClass = + NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf); + eventLoopGroup = eventLoopGroupAndChannelClass.getFirst(); + channelClass = eventLoopGroupAndChannelClass.getSecond(); + } + + // Use a timestamp to make it identical. That means, after we transit the peer to DA/S and then + // back to A, the log prefix will be changed. This is used to simplify the implementation for + // replication source, where we do not need to consider that a terminated shipper could be added + // back. + private String getLogPrefix(String peerId) { + return factory.factoryId + "-" + EnvironmentEdgeManager.currentTime() + "-" + peerId; + } + + private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException { + Class clazz = + conf.getClass(DUAL_WAL_IMPL, DualAsyncFSWAL.class, DualAsyncFSWAL.class); + try { + Constructor constructor = null; + for (Constructor c : clazz.getDeclaredConstructors()) { + if (c.getParameterCount() > 0) { + constructor = c; + break; + } + } + if (constructor == null) { + throw new IllegalArgumentException("No valid constructor provided for class " + clazz); + } + constructor.setAccessible(true); + return (DualAsyncFSWAL) constructor.newInstance( + CommonFSUtils.getWALFileSystem(conf), + ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir), + CommonFSUtils.getWALRootDir(conf), + ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId), + getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId), + conf, listeners, true, getLogPrefix(peerId), ReplicationUtils.SYNC_WAL_SUFFIX, + eventLoopGroup, channelClass); + } catch (InstantiationException | IllegalAccessException e) { + throw new RuntimeException(e); + } catch (InvocationTargetException e) { + Throwable cause = e.getTargetException(); + Throwables.propagateIfPossible(cause, IOException.class); + throw new RuntimeException(cause); + } + } + + private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException { + Optional opt = peerId2WAL.get(peerId); + if (opt != null) { + return opt.orElse(null); + } + Lock lock = createLock.acquireLock(peerId); + try { + opt = peerId2WAL.get(peerId); + if (opt != null) { + return opt.orElse(null); + } + DualAsyncFSWAL wal = createWAL(peerId, remoteWALDir); + boolean succ = false; + try { + wal.init(); + succ = true; + } finally { + if (!succ) { + wal.close(); + } + } + peerId2WAL.put(peerId, Optional.of(wal)); + return wal; + } finally { + lock.unlock(); + } + } + + @Override + public WAL getWAL(RegionInfo region) throws IOException { + if (region == null) { + return provider.getWAL(null); + } + WAL wal = null; + Optional> peerIdAndRemoteWALDir = + peerInfoProvider.getPeerIdAndRemoteWALDir(region.getTable()); + if (peerIdAndRemoteWALDir.isPresent()) { + Pair pair = peerIdAndRemoteWALDir.get(); + wal = getWAL(pair.getFirst(), pair.getSecond()); + } + return wal != null ? wal : provider.getWAL(region); + } + + private Stream getWALStream() { + return Streams.concat( + peerId2WAL.values().stream().filter(Optional::isPresent).map(Optional::get), + provider.getWALs().stream()); + } + + @Override + public List getWALs() { + return getWALStream().collect(Collectors.toList()); + } + + @Override + public void shutdown() throws IOException { + // save the last exception and rethrow + IOException failure = null; + for (Optional wal : peerId2WAL.values()) { + if (wal.isPresent()) { + try { + wal.get().shutdown(); + } catch (IOException e) { + LOG.error("Shutdown WAL failed", e); + failure = e; + } + } + } + provider.shutdown(); + if (failure != null) { + throw failure; + } + } + + @Override + public void close() throws IOException { + // save the last exception and rethrow + IOException failure = null; + for (Optional wal : peerId2WAL.values()) { + if (wal.isPresent()) { + try { + wal.get().close(); + } catch (IOException e) { + LOG.error("Close WAL failed", e); + failure = e; + } + } + } + provider.close(); + if (failure != null) { + throw failure; + } + } + + @Override + public long getNumLogFiles() { + return peerId2WAL.size() + provider.getNumLogFiles(); + } + + @Override + public long getLogFileSize() { + return peerId2WAL.values().stream().filter(Optional::isPresent).map(Optional::get) + .mapToLong(DualAsyncFSWAL::getLogFileSize).sum() + provider.getLogFileSize(); + } + + private void safeClose(WAL wal) { + if (wal != null) { + try { + wal.close(); + } catch (IOException e) { + LOG.error("Close WAL failed", e); + } + } + } + + @Override + public void addWALActionsListener(WALActionsListener listener) { + listeners.add(listener); + provider.addWALActionsListener(listener); + } + + @Override + public void peerSyncReplicationStateChange(String peerId, SyncReplicationState from, + SyncReplicationState to, int stage) { + if (from == SyncReplicationState.ACTIVE) { + if (stage == 0) { + Lock lock = createLock.acquireLock(peerId); + try { + Optional opt = peerId2WAL.get(peerId); + if (opt != null) { + opt.ifPresent(w -> w.skipRemoteWAL(to == SyncReplicationState.STANDBY)); + } else { + // add a place holder to tell the getWAL caller do not use DualAsyncFSWAL any more. + peerId2WAL.put(peerId, Optional.empty()); + } + } finally { + lock.unlock(); + } + } else if (stage == 1) { + peerId2WAL.remove(peerId).ifPresent(this::safeClose); + } + } + } + + private static class DefaultSyncReplicationPeerInfoProvider + implements SyncReplicationPeerInfoProvider { + + @Override + public Optional> getPeerIdAndRemoteWALDir(TableName table) { + return Optional.empty(); + } + + @Override + public boolean checkState(TableName table, + BiPredicate checker) { + return false; + } + } + + private static final Pattern LOG_PREFIX_PATTERN = Pattern.compile(".*-\\d+-(.+)"); + + /** + *

    + * Returns the peer id if the wal file name is in the special group for a sync replication peer. + *

    + *

    + * The prefix format is <factoryId>-<ts>-<peerId>. + *

    + */ + public static Optional getSyncReplicationPeerIdFromWALName(String name) { + if (!name.endsWith(ReplicationUtils.SYNC_WAL_SUFFIX)) { + // fast path to return earlier if the name is not for a sync replication peer. + return Optional.empty(); + } + String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(name); + Matcher matcher = LOG_PREFIX_PATTERN.matcher(logPrefix); + if (matcher.matches()) { + return Optional.of(matcher.group(1)); + } else { + return Optional.empty(); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java index 1d4dc1b..cd0e52e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALEdit.java @@ -20,12 +20,11 @@ package org.apache.hadoop.hbase.wal; import java.io.IOException; import java.util.ArrayList; - import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.codec.Codec; import org.apache.hadoop.hbase.io.HeapSize; @@ -33,9 +32,9 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; + import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor; @@ -54,7 +53,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDe @InterfaceAudience.LimitedPrivate({ HBaseInterfaceAudience.REPLICATION, HBaseInterfaceAudience.COPROC }) public class WALEdit implements HeapSize { - private static final Logger LOG = LoggerFactory.getLogger(WALEdit.class); // TODO: Get rid of this; see HBASE-8457 public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java index 24604d9..4f6a898 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java @@ -132,13 +132,10 @@ public class WALFactory { } } - WALProvider createProvider(Class clazz, String providerId) - throws IOException { - LOG.info("Instantiating WALProvider of type " + clazz); + static WALProvider createProvider(Class clazz) throws IOException { + LOG.info("Instantiating WALProvider of type {}", clazz); try { - final WALProvider result = clazz.getDeclaredConstructor().newInstance(); - result.init(this, conf, providerId); - return result; + return clazz.getDeclaredConstructor().newInstance(); } catch (Exception e) { LOG.error("couldn't set up WALProvider, the configured class is " + clazz); LOG.debug("Exception details for failure to load WALProvider.", e); @@ -147,14 +144,15 @@ public class WALFactory { } /** - * instantiate a provider from a config property. requires conf to have already been set (as well - * as anything the provider might need to read). + * @param conf must not be null, will keep a reference to read params in later reader/writer + * instances. + * @param factoryId a unique identifier for this factory. used i.e. by filesystem implementations + * to make a directory */ - WALProvider getProvider(String key, String defaultValue, String providerId) throws IOException { - Class clazz = getProviderClass(key, defaultValue); - WALProvider provider = createProvider(clazz, providerId); - provider.addWALActionsListener(new MetricsWAL()); - return provider; + public WALFactory(Configuration conf, String factoryId) throws IOException { + // default enableSyncReplicationWALProvider is true, only disable SyncReplicationWALProvider + // for HMaster or HRegionServer which take system table only. See HBASE-19999 + this(conf, factoryId, true); } /** @@ -162,8 +160,11 @@ public class WALFactory { * instances. * @param factoryId a unique identifier for this factory. used i.e. by filesystem implementations * to make a directory + * @param enableSyncReplicationWALProvider whether wrap the wal provider to a + * {@link SyncReplicationWALProvider} */ - public WALFactory(Configuration conf, String factoryId) throws IOException { + public WALFactory(Configuration conf, String factoryId, boolean enableSyncReplicationWALProvider) + throws IOException { // until we've moved reader/writer construction down into providers, this initialization must // happen prior to provider initialization, in case they need to instantiate a reader/writer. timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000); @@ -174,7 +175,13 @@ public class WALFactory { this.factoryId = factoryId; // end required early initialization if (conf.getBoolean("hbase.regionserver.hlog.enabled", true)) { - provider = getProvider(WAL_PROVIDER, DEFAULT_WAL_PROVIDER, null); + WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER)); + if (enableSyncReplicationWALProvider) { + provider = new SyncReplicationWALProvider(provider); + } + provider.init(this, conf, null); + provider.addWALActionsListener(new MetricsWAL()); + this.provider = provider; } else { // special handling of existing configuration behavior. LOG.warn("Running with WAL disabled."); @@ -231,8 +238,9 @@ public class WALFactory { if (provider != null) { return provider; } - provider = getProvider(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER, - AbstractFSWALProvider.META_WAL_PROVIDER_ID); + provider = createProvider(getProviderClass(META_WAL_PROVIDER, DEFAULT_META_WAL_PROVIDER)); + provider.init(this, conf, AbstractFSWALProvider.META_WAL_PROVIDER_ID); + provider.addWALActionsListener(new MetricsWAL()); if (metaProvider.compareAndSet(null, provider)) { return provider; } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java index 71cbaf4..953243c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java @@ -127,13 +127,21 @@ public class WALKeyImpl implements WALKey { } @VisibleForTesting - public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, - long logSeqNum, + public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum, final long now, UUID clusterId) { List clusterIds = new ArrayList<>(1); clusterIds.add(clusterId); - init(encodedRegionName, tablename, logSeqNum, now, clusterIds, - HConstants.NO_NONCE, HConstants.NO_NONCE, null, null); + init(encodedRegionName, tablename, logSeqNum, now, clusterIds, HConstants.NO_NONCE, + HConstants.NO_NONCE, null, null); + } + + @VisibleForTesting + public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum, + final long now, UUID clusterId, MultiVersionConcurrencyControl mvcc) { + List clusterIds = new ArrayList<>(1); + clusterIds.add(clusterId); + init(encodedRegionName, tablename, logSeqNum, now, clusterIds, HConstants.NO_NONCE, + HConstants.NO_NONCE, mvcc, null); } // TODO: Fix being able to pass in sequenceid. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java index b6202fd..a550bf5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.regionserver.MetricsRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.hadoop.hbase.regionserver.ReplicationSourceService; import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager; import org.apache.hadoop.hbase.regionserver.ServerNonceManager; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester; @@ -353,4 +354,9 @@ public class MockRegionServerServices implements RegionServerServices { public boolean isClusterUp() { return true; } + + @Override + public ReplicationSourceService getReplicationSourceService() { + return null; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java index 685c560..6462234 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java @@ -26,6 +26,7 @@ import static org.junit.Assert.fail; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -34,6 +35,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -47,10 +49,13 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterReplicationEndpointForTest; import org.apache.hadoop.hbase.replication.regionserver.TestReplicator.ReplicationEndpointForTest; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -251,6 +256,64 @@ public class TestReplicationAdmin { } @Test + public void testRemovePeerWithNonDAState() throws Exception { + TableName tableName = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(tableName, Bytes.toBytes("family")); + ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(); + + Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("remoteWAL"); + TEST_UTIL.getTestFileSystem().mkdirs(new Path(rootDir, ID_ONE)); + builder.setClusterKey(KEY_ONE); + builder.setRemoteWALDir(rootDir.makeQualified(TEST_UTIL.getTestFileSystem().getUri(), + TEST_UTIL.getTestFileSystem().getWorkingDirectory()).toString()); + builder.setReplicateAllUserTables(false); + Map> tableCfs = new HashMap<>(); + tableCfs.put(tableName, new ArrayList<>()); + builder.setTableCFsMap(tableCfs); + hbaseAdmin.addReplicationPeer(ID_ONE, builder.build()); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE)); + + // Transit sync replication state to ACTIVE. + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE, SyncReplicationState.ACTIVE); + assertEquals(SyncReplicationState.ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE)); + + try { + hbaseAdmin.removeReplicationPeer(ID_ONE); + fail("Can't remove a synchronous replication peer with state=ACTIVE"); + } catch (IOException e) { + // OK + } + + // Transit sync replication state to DA + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE, + SyncReplicationState.DOWNGRADE_ACTIVE); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE)); + // Transit sync replication state to STANDBY + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE, SyncReplicationState.STANDBY); + assertEquals(SyncReplicationState.STANDBY, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE)); + + try { + hbaseAdmin.removeReplicationPeer(ID_ONE); + fail("Can't remove a synchronous replication peer with state=STANDBY"); + } catch (IOException e) { + // OK + } + + // Transit sync replication state to DA + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE, + SyncReplicationState.DOWNGRADE_ACTIVE); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE)); + + hbaseAdmin.removeReplicationPeer(ID_ONE); + assertEquals(0, hbaseAdmin.listReplicationPeers().size()); + } + + @Test public void testAddPeerWithState() throws Exception { ReplicationPeerConfig rpc1 = new ReplicationPeerConfig(); rpc1.setClusterKey(KEY_ONE); @@ -906,4 +969,217 @@ public class TestReplicationAdmin { // OK } } + + @Test + public void testPeerRemoteWALDir() throws Exception { + TableName tableName = TableName.valueOf(name.getMethodName()); + + String rootDir = "hdfs://srv1:9999/hbase"; + ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(); + builder.setClusterKey(KEY_ONE); + hbaseAdmin.addReplicationPeer(ID_ONE, builder.build()); + + ReplicationPeerConfig rpc = hbaseAdmin.getReplicationPeerConfig(ID_ONE); + assertNull(rpc.getRemoteWALDir()); + + builder.setRemoteWALDir("hdfs://srv2:8888/hbase"); + try { + hbaseAdmin.updateReplicationPeerConfig(ID_ONE, builder.build()); + fail("Change remote wal dir is not allowed"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + builder = ReplicationPeerConfig.newBuilder(); + builder.setClusterKey(KEY_SECOND); + builder.setRemoteWALDir("whatever"); + + try { + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + fail("Only support replicated table config for sync replication"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + builder.setReplicateAllUserTables(false); + Set namespaces = new HashSet(); + namespaces.add("ns1"); + builder.setNamespaces(namespaces); + try { + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + fail("Only support replicated table config for sync replication"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + builder.setNamespaces(null); + try { + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + fail("Only support replicated table config for sync replication, and tables can't be empty"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + Map> tableCfs = new HashMap<>(); + tableCfs.put(tableName, Arrays.asList("cf1")); + builder.setTableCFsMap(tableCfs); + try { + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + fail("Only support replicated table config for sync replication"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + tableCfs = new HashMap<>(); + tableCfs.put(tableName, new ArrayList<>()); + builder.setTableCFsMap(tableCfs); + try { + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + fail("The remote WAL dir must be absolute"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + builder.setRemoteWALDir("/hbase/remoteWALs"); + try { + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + fail("The remote WAL dir must be qualified"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + builder.setRemoteWALDir(rootDir); + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + rpc = hbaseAdmin.getReplicationPeerConfig(ID_SECOND); + assertEquals(rootDir, rpc.getRemoteWALDir()); + + try { + builder.setRemoteWALDir("hdfs://srv2:8888/hbase"); + hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build()); + fail("Change remote wal dir is not allowed"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + try { + builder.setRemoteWALDir(null); + hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build()); + fail("Change remote wal dir is not allowed"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + try { + builder = ReplicationPeerConfig.newBuilder(rpc); + tableCfs = new HashMap<>(); + tableCfs.put(TableName.valueOf("ns1:" + name.getMethodName()), new ArrayList<>()); + builder.setTableCFsMap(tableCfs); + hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build()); + fail( + "Change replicated table config on an existing synchronous peer is not allowed"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + } + + @Test + public void testTransitSyncReplicationPeerState() throws Exception { + TableName tableName = TableName.valueOf(name.getMethodName()); + TEST_UTIL.createTable(tableName, Bytes.toBytes("family")); + ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(); + builder.setClusterKey(KEY_ONE); + builder.setReplicateAllUserTables(false); + hbaseAdmin.addReplicationPeer(ID_ONE, builder.build()); + assertEquals(SyncReplicationState.NONE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE)); + + try { + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE, + SyncReplicationState.DOWNGRADE_ACTIVE); + fail("Can't transit sync replication state if replication peer don't config remote wal dir"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + + Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("remoteWAL"); + builder = ReplicationPeerConfig.newBuilder(); + builder.setClusterKey(KEY_SECOND); + builder.setRemoteWALDir(rootDir.makeQualified(TEST_UTIL.getTestFileSystem().getUri(), + TEST_UTIL.getTestFileSystem().getWorkingDirectory()).toString()); + builder.setReplicateAllUserTables(false); + Map> tableCfs = new HashMap<>(); + tableCfs.put(tableName, new ArrayList<>()); + builder.setTableCFsMap(tableCfs); + hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build()); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + + // Disable and enable peer don't affect SyncReplicationState + hbaseAdmin.disableReplicationPeer(ID_SECOND); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + hbaseAdmin.enableReplicationPeer(ID_SECOND); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + + try { + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE); + fail("Can't transit sync replication state to ACTIVE if remote wal dir does not exist"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + TEST_UTIL.getTestFileSystem() + .mkdirs(ReplicationUtils.getPeerRemoteWALDir(rootDir, ID_SECOND)); + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE); + assertEquals(SyncReplicationState.ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.STANDBY); + assertEquals(SyncReplicationState.STANDBY, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, + SyncReplicationState.DOWNGRADE_ACTIVE); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE); + assertEquals(SyncReplicationState.ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, + SyncReplicationState.DOWNGRADE_ACTIVE); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.STANDBY); + assertEquals(SyncReplicationState.STANDBY, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + try { + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE); + fail("Can't transit sync replication state from STANDBY to ACTIVE"); + } catch (Exception e) { + // OK + LOG.info("Expected error:", e); + } + hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, + SyncReplicationState.DOWNGRADE_ACTIVE); + assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE, + hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND)); + hbaseAdmin.removeReplicationPeer(ID_ONE); + hbaseAdmin.removeReplicationPeer(ID_SECOND); + assertEquals(0, hbaseAdmin.listReplicationPeers().size()); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java index 34570d3..ac20dbd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java @@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.locking.LockManager; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; +import org.apache.hadoop.hbase.master.replication.SyncReplicationReplayWALManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost; import org.apache.hadoop.hbase.procedure2.LockedResource; @@ -52,6 +53,7 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -467,4 +469,14 @@ public class MockNoopMasterServices implements MasterServices { public boolean isClusterUp() { return true; } -} + + public long transitReplicationPeerSyncReplicationState(String peerId, + SyncReplicationState clusterState) throws ReplicationException, IOException { + return 0; + } + + @Override + public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() { + return null; + } +} \ No newline at end of file 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 35ad7eb..1c4e72a 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 @@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.regionserver.MetricsRegionServer; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerServices; +import org.apache.hadoop.hbase.regionserver.ReplicationSourceService; import org.apache.hadoop.hbase.regionserver.SecureBulkLoadManager; import org.apache.hadoop.hbase.regionserver.ServerNonceManager; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester; @@ -696,4 +697,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { public boolean isClusterUp() { return true; } + + @Override + public ReplicationSourceService getReplicationSourceService() { + return null; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java index 08dd428..24b930c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -109,7 +110,8 @@ public class TestReplicationHFileCleaner { public void setup() throws ReplicationException, IOException { root = TEST_UTIL.getDataTestDirOnTestFS(); rp.getPeerStorage().addPeer(peerId, - ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true); + ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true, + SyncReplicationState.NONE); rq.addPeerToHFileRefs(peerId); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java index ca4b227..939f35c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java @@ -100,6 +100,7 @@ public class TestCompactionPolicy { HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); hlog = new FSHLog(fs, basedir, logName, conf); + hlog.init(); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); region = HRegion.createHRegion(info, basedir, conf, htd, hlog); region.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java index 3cf06d4..1490653 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java @@ -102,65 +102,64 @@ public class TestFailedAppendAndSync { return name.getMethodName(); } - /** - * Reproduce locking up that happens when we get an exceptions appending and syncing. - * See HBASE-14317. - * First I need to set up some mocks for Server and RegionServerServices. I also need to - * set up a dodgy WAL that will throw an exception when we go to append to it. - */ - @Test - public void testLockupAroundBadAssignSync() throws IOException { + // Dodgy WAL. Will throw exceptions when flags set. + class DodgyFSLog extends FSHLog { + volatile boolean throwSyncException = false; + volatile boolean throwAppendException = false; final AtomicLong rolls = new AtomicLong(0); - // Dodgy WAL. Will throw exceptions when flags set. - class DodgyFSLog extends FSHLog { - volatile boolean throwSyncException = false; - volatile boolean throwAppendException = false; - public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf) - throws IOException { - super(fs, root, logDir, conf); - } - - @Override - public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException { - byte [][] regions = super.rollWriter(force); - rolls.getAndIncrement(); - return regions; - } + public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf) + throws IOException { + super(fs, root, logDir, conf); + } - @Override - protected Writer createWriterInstance(Path path) throws IOException { - final Writer w = super.createWriterInstance(path); - return new Writer() { - @Override - public void close() throws IOException { - w.close(); - } + @Override + public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException { + byte[][] regions = super.rollWriter(force); + rolls.getAndIncrement(); + return regions; + } - @Override - public void sync(boolean forceSync) throws IOException { - if (throwSyncException) { - throw new IOException("FAKE! Failed to replace a bad datanode..."); - } - w.sync(forceSync); - } + @Override + protected Writer createWriterInstance(Path path) throws IOException { + final Writer w = super.createWriterInstance(path); + return new Writer() { + @Override + public void close() throws IOException { + w.close(); + } - @Override - public void append(Entry entry) throws IOException { - if (throwAppendException) { - throw new IOException("FAKE! Failed to replace a bad datanode..."); - } - w.append(entry); - } + @Override + public void sync(boolean forceSync) throws IOException { + if (throwSyncException) { + throw new IOException("FAKE! Failed to replace a bad datanode..."); + } + w.sync(forceSync); + } - @Override - public long getLength() { - return w.getLength(); - } - }; + @Override + public void append(Entry entry) throws IOException { + if (throwAppendException) { + throw new IOException("FAKE! Failed to replace a bad datanode..."); } - } + w.append(entry); + } + @Override + public long getLength() { + return w.getLength(); + } + }; + } + } + /** + * Reproduce locking up that happens when we get an exceptions appending and syncing. + * See HBASE-14317. + * First I need to set up some mocks for Server and RegionServerServices. I also need to + * set up a dodgy WAL that will throw an exception when we go to append to it. + */ + @Test + public void testLockupAroundBadAssignSync() throws IOException { // Make up mocked server and services. Server server = mock(Server.class); when(server.getConfiguration()).thenReturn(CONF); @@ -172,6 +171,7 @@ public class TestFailedAppendAndSync { FileSystem fs = FileSystem.get(CONF); Path rootDir = new Path(dir + getName()); DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF); + dodgyWAL.init(); LogRoller logRoller = new LogRoller(server, services); logRoller.addWAL(dodgyWAL); logRoller.start(); @@ -192,7 +192,7 @@ public class TestFailedAppendAndSync { } catch (IOException ioe) { fail(); } - long rollsCount = rolls.get(); + long rollsCount = dodgyWAL.rolls.get(); try { dodgyWAL.throwAppendException = true; dodgyWAL.throwSyncException = false; @@ -202,8 +202,10 @@ public class TestFailedAppendAndSync { } catch (IOException ioe) { threwOnAppend = true; } - while (rollsCount == rolls.get()) Threads.sleep(100); - rollsCount = rolls.get(); + while (rollsCount == dodgyWAL.rolls.get()) { + Threads.sleep(100); + } + rollsCount = dodgyWAL.rolls.get(); // When we get to here.. we should be ok. A new WAL has been put in place. There were no // appends to sync. We should be able to continue. @@ -217,14 +219,16 @@ public class TestFailedAppendAndSync { } catch (IOException ioe) { threwOnBoth = true; } - while (rollsCount == rolls.get()) Threads.sleep(100); + while (rollsCount == dodgyWAL.rolls.get()) { + Threads.sleep(100); + } // Again, all should be good. New WAL and no outstanding unsync'd edits so we should be able // to just continue. // So, should be no abort at this stage. Verify. - Mockito.verify(server, Mockito.atLeast(0)). - abort(Mockito.anyString(), (Throwable)Mockito.anyObject()); + Mockito.verify(server, Mockito.atLeast(0)).abort(Mockito.anyString(), + Mockito.any(Throwable.class)); try { dodgyWAL.throwAppendException = false; dodgyWAL.throwSyncException = true; @@ -239,8 +243,8 @@ public class TestFailedAppendAndSync { // happens. If it don't we'll timeout the whole test. That is fine. while (true) { try { - Mockito.verify(server, Mockito.atLeast(1)). - abort(Mockito.anyString(), (Throwable)Mockito.anyObject()); + Mockito.verify(server, Mockito.atLeast(1)).abort(Mockito.anyString(), + Mockito.any(Throwable.class)); break; } catch (WantedButNotInvoked t) { Threads.sleep(1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 1ff6b27..e95639f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -217,7 +217,6 @@ public class TestHRegion { protected static HBaseTestingUtility TEST_UTIL; public static Configuration CONF ; private String dir; - private static FileSystem FILESYSTEM; private final int MAX_VERSIONS = 2; // Test names @@ -239,7 +238,6 @@ public class TestHRegion { @Before public void setup() throws IOException { TEST_UTIL = HBaseTestingUtility.createLocalHTU(); - FILESYSTEM = TEST_UTIL.getTestFileSystem(); CONF = TEST_UTIL.getConfiguration(); dir = TEST_UTIL.getDataTestDir("TestHRegion").toString(); method = name.getMethodName(); @@ -342,6 +340,7 @@ public class TestHRegion { FileSystem fs = FileSystem.get(CONF); Path rootDir = new Path(dir + "testMemstoreSnapshotSize"); MyFaultyFSLog faultyLog = new MyFaultyFSLog(fs, rootDir, "testMemstoreSnapshotSize", CONF); + faultyLog.init(); HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, faultyLog, COLUMN_FAMILY_BYTES); @@ -353,7 +352,6 @@ public class TestHRegion { Put put = new Put(value); put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value); faultyLog.setFailureType(FaultyFSLog.FailureType.SYNC); - boolean threwIOE = false; try { region.put(put); @@ -390,6 +388,7 @@ public class TestHRegion { FileSystem fs = FileSystem.get(CONF); Path rootDir = new Path(dir + testName); FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF); + hLog.init(); HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES); @@ -1165,6 +1164,7 @@ public class TestHRegion { FailAppendFlushMarkerWAL wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf), method, walConf); + wal.init(); this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family); try { @@ -1196,7 +1196,7 @@ public class TestHRegion { wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH}; wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf), method, walConf); - + wal.init(); this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family); region.put(put); @@ -2448,6 +2448,7 @@ public class TestHRegion { FileSystem fs = FileSystem.get(CONF); Path rootDir = new Path(dir + "testDataInMemoryWithoutWAL"); FSHLog hLog = new FSHLog(fs, rootDir, "testDataInMemoryWithoutWAL", CONF); + hLog.init(); // This chunk creation is done throughout the code base. Do we want to move it into core? // It is missing from this test. W/o it we NPE. ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); @@ -2500,9 +2501,9 @@ public class TestHRegion { RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class); // Because the preBatchMutate returns void, we can't do usual Mockito when...then form. Must // do below format (from Mockito doc). - Mockito.doAnswer(new Answer() { + Mockito.doAnswer(new Answer() { @Override - public Object answer(InvocationOnMock invocation) throws Throwable { + public Void answer(InvocationOnMock invocation) throws Throwable { MiniBatchOperationInProgress mb = invocation.getArgument(0); mb.addOperationsFromCP(0, new Mutation[]{addPut}); return null; @@ -3796,9 +3797,12 @@ public class TestHRegion { boolean previousEmpty = res.isEmpty(); res.clear(); - InternalScanner scanner = region.getScanner(scan); - while (scanner.next(res)) - ; + try (InternalScanner scanner = region.getScanner(scan)) { + boolean moreRows; + do { + moreRows = scanner.next(res); + } while (moreRows); + } if (!res.isEmpty() || !previousEmpty || i > compactInterval) { assertEquals("i=" + i, expectedCount, res.size()); long timestamp = res.get(0).getTimestamp(); @@ -3894,7 +3898,7 @@ public class TestHRegion { region.put(put); numPutsFinished++; if (numPutsFinished > 0 && numPutsFinished % 47 == 0) { - System.out.println("put iteration = " + numPutsFinished); + LOG.debug("put iteration = {}", numPutsFinished); Delete delete = new Delete(row, (long) numPutsFinished - 30); region.delete(delete); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java index ce83326..84f7973 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java @@ -27,25 +27,18 @@ import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.wal.WAL; import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * A test similar to TestHRegion, but with in-memory flush families. * Also checks wal truncation after in-memory compaction. */ @Category({VerySlowRegionServerTests.class, LargeTests.class}) -@SuppressWarnings("deprecation") public class TestHRegionWithInMemoryFlush extends TestHRegion { @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestHRegionWithInMemoryFlush.class); - // Do not spin up clusters in here. If you need to spin up a cluster, do it - // over in TestHRegionOnCluster. - private static final Logger LOG = LoggerFactory.getLogger(TestHRegionWithInMemoryFlush.class); - /** * @return A region on which you must call * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java new file mode 100644 index 0000000..e8c9423 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestLogRoller.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.regionserver.wal.FSHLog; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({RegionServerTests.class, MediumTests.class}) +public class TestLogRoller { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestLogRoller.class); + + private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + private static final int logRollPeriod = 20 * 1000; + + @Before + public void setup() throws Exception { + TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.period", logRollPeriod); + TEST_UTIL.startMiniCluster(1); + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testRemoveClosedWAL() throws Exception { + HRegionServer rs = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0); + Configuration conf = rs.getConfiguration(); + LogRoller logRoller = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).getWalRoller(); + int originalSize = logRoller.getWalNeedsRoll().size(); + FSHLog wal1 = new FSHLog(rs.getWALFileSystem(), rs.getWALRootDir(), + AbstractFSWALProvider.getWALDirectoryName(rs.getServerName().getServerName()), conf); + logRoller.addWAL(wal1); + FSHLog wal2 = new FSHLog(rs.getWALFileSystem(), rs.getWALRootDir(), + AbstractFSWALProvider.getWALDirectoryName(rs.getServerName().getServerName()), conf); + logRoller.addWAL(wal2); + FSHLog wal3 = new FSHLog(rs.getWALFileSystem(), rs.getWALRootDir(), + AbstractFSWALProvider.getWALDirectoryName(rs.getServerName().getServerName()), conf); + logRoller.addWAL(wal3); + + assertEquals(originalSize + 3, logRoller.getWalNeedsRoll().size()); + assertTrue(logRoller.getWalNeedsRoll().containsKey(wal1)); + + wal1.close(); + Thread.sleep(2 * logRollPeriod); + + assertEquals(originalSize + 2, logRoller.getWalNeedsRoll().size()); + assertFalse(logRoller.getWalNeedsRoll().containsKey(wal1)); + + wal2.close(); + wal3.close(); + Thread.sleep(2 * logRollPeriod); + + assertEquals(originalSize, logRoller.getWalNeedsRoll().size()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java index 8b96fa7..e5006ea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Scan; @@ -36,7 +37,6 @@ import org.apache.hadoop.hbase.client.TestIncrementsFromClientSide; import org.apache.hadoop.hbase.regionserver.wal.FSHLog; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.wal.WAL; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; @@ -81,12 +81,12 @@ public class TestRegionIncrement { } private HRegion getRegion(final Configuration conf, final String tableName) throws IOException { - WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(), - TEST_UTIL.getDataTestDir().toString(), conf); + FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(), + TEST_UTIL.getDataTestDir().toString(), conf); + wal.init(); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); - return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName), - HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, conf, - false, Durability.SKIP_WAL, wal, INCREMENT_BYTES); + return TEST_UTIL.createLocalHRegion(TableName.valueOf(tableName), HConstants.EMPTY_BYTE_ARRAY, + HConstants.EMPTY_BYTE_ARRAY, false, Durability.SKIP_WAL, wal, INCREMENT_BYTES); } private void closeRegion(final HRegion region) throws IOException { @@ -170,8 +170,6 @@ public class TestRegionIncrement { /** * Have each thread update its own Cell. Avoid contention with another thread. - * @throws IOException - * @throws InterruptedException */ @Test public void testUnContendedSingleCellIncrement() @@ -209,13 +207,9 @@ public class TestRegionIncrement { /** * Have each thread update its own Cell. Avoid contention with another thread. - * This is - * @throws IOException - * @throws InterruptedException */ @Test - public void testContendedAcrossCellsIncrement() - throws IOException, InterruptedException { + public void testContendedAcrossCellsIncrement() throws IOException, InterruptedException { final HRegion region = getRegion(TEST_UTIL.getConfiguration(), TestIncrementsFromClientSide.filterStringSoTableNameSafe(this.name.getMethodName())); long startTime = System.currentTimeMillis(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java index 29a75b8..84b8d6c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java @@ -215,6 +215,7 @@ public class TestWALLockup { FileSystem fs = FileSystem.get(CONF); Path rootDir = new Path(dir + getName()); DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF); + dodgyWAL.init(); Path originalWAL = dodgyWAL.getCurrentFileName(); // I need a log roller running. LogRoller logRoller = new LogRoller(server, services); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java index 5336963..aa0e6b5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestFSWAL.java @@ -473,4 +473,13 @@ public abstract class AbstractTestFSWAL { assertNull(key.getWriteEntry()); } } + + @Test(expected = WALClosedException.class) + public void testRollWriterForClosedWAL() throws IOException { + String testName = currentTest.getMethodName(); + AbstractFSWAL wal = newWAL(FS, CommonFSUtils.getWALRootDir(CONF), DIR.toString(), testName, + CONF, null, true, null, null); + wal.close(); + wal.rollWriter(); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java index c3f3277..5098609 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java @@ -18,33 +18,15 @@ */ package org.apache.hadoop.hbase.regionserver.wal; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.io.Closeable; import java.io.IOException; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.coprocessor.SampleRegionWALCoprocessor; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.apache.hadoop.hbase.wal.WALProvider; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -56,8 +38,8 @@ import org.junit.rules.TestName; /** * WAL tests that can be reused across providers. */ -public abstract class AbstractTestProtobufLog { - protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); +public abstract class AbstractTestProtobufLog { + protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); protected FileSystem fs; protected Path dir; @@ -93,14 +75,7 @@ public abstract class AbstractTestProtobufLog { TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000); // faster failover with cluster.shutdown();fs.close() idiom - TEST_UTIL.getConfiguration() - .setInt("hbase.ipc.client.connect.max.retries", 1); - TEST_UTIL.getConfiguration().setInt( - "dfs.client.block.recovery.retries", 1); - TEST_UTIL.getConfiguration().setInt( - "hbase.ipc.client.connection.maxidletime", 500); - TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, - SampleRegionWALCoprocessor.class.getName()); + TEST_UTIL.getConfiguration().setInt("dfs.client.block.recovery.retries", 1); TEST_UTIL.startMiniDFSCluster(3); } @@ -131,77 +106,24 @@ public abstract class AbstractTestProtobufLog { * @throws IOException */ private void doRead(boolean withTrailer) throws IOException { - final int columnCount = 5; - final int recordCount = 5; - final TableName tableName = - TableName.valueOf("tablename"); - final byte[] row = Bytes.toBytes("row"); + int columnCount = 5; + int recordCount = 5; + TableName tableName = TableName.valueOf("tablename"); + byte[] row = Bytes.toBytes("row"); long timestamp = System.currentTimeMillis(); Path path = new Path(dir, "tempwal"); // delete the log if already exists, for test only fs.delete(path, true); - W writer = null; - ProtobufLogReader reader = null; - try { - HRegionInfo hri = new HRegionInfo(tableName, - HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW); - HTableDescriptor htd = new HTableDescriptor(tableName); - fs.mkdirs(dir); - // Write log in pb format. - writer = createWriter(path); - for (int i = 0; i < recordCount; ++i) { - WALKeyImpl key = new WALKeyImpl( - hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID); - WALEdit edit = new WALEdit(); - for (int j = 0; j < columnCount; ++j) { - if (i == 0) { - htd.addFamily(new HColumnDescriptor("column" + j)); - } - String value = i + "" + j; - edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value))); - } - append(writer, new WAL.Entry(key, edit)); - } - sync(writer); - if (withTrailer) writer.close(); - - // Now read the log using standard means. - reader = (ProtobufLogReader) wals.createReader(fs, path); - if (withTrailer) { - assertNotNull(reader.trailer); - } else { - assertNull(reader.trailer); - } - for (int i = 0; i < recordCount; ++i) { - WAL.Entry entry = reader.next(); - assertNotNull(entry); - assertEquals(columnCount, entry.getEdit().size()); - assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName()); - assertEquals(tableName, entry.getKey().getTableName()); - int idx = 0; - for (Cell val : entry.getEdit().getCells()) { - assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(), - val.getRowLength())); - String value = i + "" + idx; - assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val)); - idx++; - } - } - WAL.Entry entry = reader.next(); - assertNull(entry); - } finally { - if (writer != null) { - writer.close(); - } - if (reader != null) { - reader.close(); + fs.mkdirs(dir); + try (WALProvider.Writer writer = createWriter(path)) { + ProtobufLogTestHelper.doWrite(writer, withTrailer, tableName, columnCount, recordCount, row, + timestamp); + try (ProtobufLogReader reader = (ProtobufLogReader) wals.createReader(fs, path)) { + ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row, + timestamp); } } } - protected abstract W createWriter(Path path) throws IOException; - - protected abstract void append(W writer, WAL.Entry entry) throws IOException; - - protected abstract void sync(W writer) throws IOException; + protected abstract WALProvider.Writer createWriter(Path path) throws IOException; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java index e7cdf1f..93c379c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java @@ -1097,6 +1097,7 @@ public abstract class AbstractTestWALReplay { private MockWAL createMockWAL() throws IOException { MockWAL wal = new MockWAL(fs, hbaseRootDir, logName, conf); + wal.init(); // Set down maximum recovery so we dfsclient doesn't linger retrying something // long gone. HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java new file mode 100644 index 0000000..420585f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java @@ -0,0 +1,121 @@ +/** + * 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 static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.stream.IntStream; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.apache.hadoop.hbase.wal.WALProvider; + +/** + * Helper class for testing protobuf log. + */ +public final class ProtobufLogTestHelper { + + private ProtobufLogTestHelper() { + } + + private static byte[] toValue(int prefix, int suffix) { + return Bytes.toBytes(prefix + "-" + suffix); + } + + private static RegionInfo toRegionInfo(TableName tableName) { + return RegionInfoBuilder.newBuilder(tableName).setRegionId(1024).build(); + } + + private static WAL.Entry generateEdit(int i, RegionInfo hri, TableName tableName, byte[] row, + int columnCount, long timestamp, MultiVersionConcurrencyControl mvcc) { + WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp, + HConstants.DEFAULT_CLUSTER_ID, mvcc); + WALEdit edit = new WALEdit(); + int prefix = i; + IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j)) + .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add); + return new WAL.Entry(key, edit); + } + + public static void doWrite(WALProvider.Writer writer, boolean withTrailer, TableName tableName, + int columnCount, int recordCount, byte[] row, long timestamp) throws IOException { + RegionInfo hri = toRegionInfo(tableName); + for (int i = 0; i < recordCount; i++) { + writer.append(generateEdit(i, hri, tableName, row, columnCount, timestamp, null)); + } + writer.sync(false); + if (withTrailer) { + writer.close(); + } + } + + public static void doWrite(WAL wal, RegionInfo hri, TableName tableName, int columnCount, + int recordCount, byte[] row, long timestamp, MultiVersionConcurrencyControl mvcc) + throws IOException { + for (int i = 0; i < recordCount; i++) { + WAL.Entry entry = generateEdit(i, hri, tableName, row, columnCount, timestamp, mvcc); + wal.append(hri, entry.getKey(), entry.getEdit(), true); + } + wal.sync(); + } + + public static void doRead(ProtobufLogReader reader, boolean withTrailer, RegionInfo hri, + TableName tableName, int columnCount, int recordCount, byte[] row, long timestamp) + throws IOException { + if (withTrailer) { + assertNotNull(reader.trailer); + } else { + assertNull(reader.trailer); + } + for (int i = 0; i < recordCount; ++i) { + WAL.Entry entry = reader.next(); + assertNotNull(entry); + assertEquals(columnCount, entry.getEdit().size()); + assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName()); + assertEquals(tableName, entry.getKey().getTableName()); + int idx = 0; + for (Cell val : entry.getEdit().getCells()) { + assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(), + val.getRowLength())); + assertArrayEquals(toValue(i, idx), CellUtil.cloneValue(val)); + idx++; + } + } + assertNull(reader.next()); + } + + public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName, + int columnCount, int recordCount, byte[] row, long timestamp) throws IOException { + doRead(reader, withTrailer, toRegionInfo(tableName), tableName, columnCount, recordCount, row, + timestamp); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java index 450c01b..5f0f77c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java @@ -67,8 +67,10 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL { protected AbstractFSWAL newWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir, Configuration conf, List listeners, boolean failIfWALExists, String prefix, String suffix) throws IOException { - return new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, - suffix, GROUP, CHANNEL_CLASS); + AsyncFSWAL wal = new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, + failIfWALExists, prefix, suffix, GROUP, CHANNEL_CLASS); + wal.init(); + return wal; } @Override @@ -76,15 +78,16 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL { String archiveDir, Configuration conf, List listeners, boolean failIfWALExists, String prefix, String suffix, final Runnable action) throws IOException { - return new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, - suffix, GROUP, CHANNEL_CLASS) { + AsyncFSWAL wal = new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, + failIfWALExists, prefix, suffix, GROUP, CHANNEL_CLASS) { @Override void atHeadOfRingBufferEventHandlerAppend() { action.run(); super.atHeadOfRingBufferEventHandlerAppend(); } - }; + wal.init(); + return wal; } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java index 0ea75b6..7626dcf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java @@ -18,29 +18,24 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.IOException; -import java.io.InterruptedIOException; -import java.util.concurrent.ExecutionException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.AsyncFSWALProvider; -import org.apache.hadoop.hbase.wal.WAL.Entry; -import org.apache.hadoop.hbase.wal.WALProvider; -import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; +import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.experimental.categories.Category; -import org.apache.hbase.thirdparty.com.google.common.base.Throwables; import org.apache.hbase.thirdparty.io.netty.channel.Channel; import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup; import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel; @Category({ RegionServerTests.class, MediumTests.class }) -public class TestAsyncProtobufLog extends AbstractTestProtobufLog { +public class TestAsyncProtobufLog extends AbstractTestProtobufLog { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -64,25 +59,8 @@ public class TestAsyncProtobufLog extends AbstractTestProtobufLog CHANNEL_CLASS; + + private static WALFactory WALS; + + @Rule + public final TestName name = new TestName(); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + EVENT_LOOP_GROUP = new NioEventLoopGroup(); + CHANNEL_CLASS = NioSocketChannel.class; + UTIL.startMiniDFSCluster(3); + UTIL.getTestFileSystem().mkdirs(UTIL.getDataTestDirOnTestFS()); + WALS = new WALFactory(UTIL.getConfiguration(), TestCombinedAsyncWriter.class.getSimpleName()); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + if (WALS != null) { + WALS.close(); + } + EVENT_LOOP_GROUP.shutdownGracefully().syncUninterruptibly(); + UTIL.shutdownMiniDFSCluster(); + } + + @Test + public void testWithTrailer() throws IOException { + doTest(true); + } + + @Test + public void testWithoutTrailer() throws IOException { + doTest(false); + } + + private Path getPath(int index) throws IOException { + String methodName = name.getMethodName().replaceAll("[^A-Za-z0-9_-]", "_"); + return new Path(UTIL.getDataTestDirOnTestFS(), methodName + "-" + index); + } + + private void doTest(boolean withTrailer) throws IOException { + int columnCount = 5; + int recordCount = 5; + TableName tableName = TableName.valueOf("tablename"); + byte[] row = Bytes.toBytes("row"); + long timestamp = System.currentTimeMillis(); + Path path1 = getPath(1); + Path path2 = getPath(2); + FileSystem fs = UTIL.getTestFileSystem(); + Configuration conf = UTIL.getConfiguration(); + try ( + AsyncWriter writer1 = AsyncFSWALProvider.createAsyncWriter(conf, fs, path1, false, + EVENT_LOOP_GROUP.next(), CHANNEL_CLASS); + AsyncWriter writer2 = AsyncFSWALProvider.createAsyncWriter(conf, fs, path2, false, + EVENT_LOOP_GROUP.next(), CHANNEL_CLASS); + CombinedAsyncWriter writer = CombinedAsyncWriter.create(writer1, writer2)) { + ProtobufLogTestHelper.doWrite(new WriterOverAsyncWriter(writer), withTrailer, tableName, + columnCount, recordCount, row, timestamp); + try (ProtobufLogReader reader = (ProtobufLogReader) WALS.createReader(fs, path1)) { + ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row, + timestamp); + } + try (ProtobufLogReader reader = (ProtobufLogReader) WALS.createReader(fs, path2)) { + ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row, + timestamp); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java index 7baaa6c..f288f74 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java @@ -74,8 +74,10 @@ public class TestFSHLog extends AbstractTestFSWAL { protected AbstractFSWAL newWAL(FileSystem fs, Path rootDir, String walDir, String archiveDir, Configuration conf, List listeners, boolean failIfWALExists, String prefix, String suffix) throws IOException { - return new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix, - suffix); + FSHLog wal = + new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix); + wal.init(); + return wal; } @Override @@ -83,8 +85,8 @@ public class TestFSHLog extends AbstractTestFSWAL { String archiveDir, Configuration conf, List listeners, boolean failIfWALExists, String prefix, String suffix, final Runnable action) throws IOException { - return new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix, - suffix) { + FSHLog wal = new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, + prefix, suffix) { @Override void atHeadOfRingBufferEventHandlerAppend() { @@ -92,6 +94,8 @@ public class TestFSHLog extends AbstractTestFSWAL { super.atHeadOfRingBufferEventHandlerAppend(); } }; + wal.init(); + return wal; } @Test @@ -100,6 +104,7 @@ public class TestFSHLog extends AbstractTestFSWAL { final String name = this.name.getMethodName(); FSHLog log = new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null); + log.init(); try { Field ringBufferEventHandlerField = FSHLog.class.getDeclaredField("ringBufferEventHandler"); ringBufferEventHandlerField.setAccessible(true); @@ -142,7 +147,7 @@ public class TestFSHLog extends AbstractTestFSWAL { try (FSHLog log = new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME, CONF, null, true, null, null)) { - + log.init(); log.registerWALActionsListener(new WALActionsListener() { @Override public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java index 2d938d4..d429a01 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java @@ -23,14 +23,12 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.wal.FSHLogProvider; -import org.apache.hadoop.hbase.wal.WAL.Entry; -import org.apache.hadoop.hbase.wal.WALProvider; import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.junit.ClassRule; import org.junit.experimental.categories.Category; @Category({ RegionServerTests.class, MediumTests.class }) -public class TestProtobufLog extends AbstractTestProtobufLog { +public class TestProtobufLog extends AbstractTestProtobufLog { @ClassRule public static final HBaseClassTestRule CLASS_RULE = @@ -40,14 +38,4 @@ public class TestProtobufLog extends AbstractTestProtobufLog protected Writer createWriter(Path path) throws IOException { return FSHLogProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false); } - - @Override - protected void append(Writer writer, Entry entry) throws IOException { - writer.append(entry); - } - - @Override - protected void sync(Writer writer) throws IOException { - writer.sync(false); - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java index 17f24e8..c446306 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALDurability.java @@ -104,6 +104,7 @@ public class TestWALDurability { FileSystem fs = FileSystem.get(conf); Path rootDir = new Path(dir + getName()); CustomFSLog customFSLog = new CustomFSLog(fs, rootDir, getName(), conf); + customFSLog.init(); HRegion region = initHRegion(tableName, null, null, customFSLog); byte[] bytes = Bytes.toBytes(getName()); Put put = new Put(bytes); @@ -118,6 +119,7 @@ public class TestWALDurability { conf.set(HRegion.WAL_HSYNC_CONF_KEY, "true"); fs = FileSystem.get(conf); customFSLog = new CustomFSLog(fs, rootDir, getName(), conf); + customFSLog.init(); region = initHRegion(tableName, null, null, customFSLog); customFSLog.resetSyncFlag(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index 649e981..66e19a8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -48,6 +48,7 @@ public class TestWALReplay extends AbstractTestWALReplay { @Override protected WAL createWAL(Configuration c, Path hbaseRootDir, String logName) throws IOException { FSHLog wal = new FSHLog(FileSystem.get(c), hbaseRootDir, logName, c); + wal.init(); // Set down maximum recovery so we dfsclient doesn't linger retrying something // long gone. HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java new file mode 100644 index 0000000..9d938b0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java @@ -0,0 +1,63 @@ +/** + * 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.io.InterruptedIOException; +import java.util.concurrent.ExecutionException; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALProvider; +import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; + +import org.apache.hbase.thirdparty.com.google.common.base.Throwables; + +class WriterOverAsyncWriter implements WALProvider.Writer { + + private final WALProvider.AsyncWriter asyncWriter; + + public WriterOverAsyncWriter(AsyncWriter asyncWriter) { + this.asyncWriter = asyncWriter; + } + + @Override + public void close() throws IOException { + asyncWriter.close(); + } + + @Override + public long getLength() { + return asyncWriter.getLength(); + } + + @Override + public void append(Entry entry) throws IOException { + asyncWriter.append(entry); + } + + @Override + public void sync(boolean forceSync) throws IOException { + try { + asyncWriter.sync().get(); + } catch (InterruptedException e) { + throw new InterruptedIOException(); + } catch (ExecutionException e) { + Throwables.propagateIfPossible(e.getCause(), IOException.class); + throw new IOException(e.getCause()); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java new file mode 100644 index 0000000..62000b4 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/DualAsyncFSWALForTest.java @@ -0,0 +1,149 @@ +/** + * 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.replication; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL; +import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; +import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; +import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; + +import org.apache.hbase.thirdparty.com.google.common.io.Closeables; +import org.apache.hbase.thirdparty.io.netty.channel.Channel; +import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup; + +class DualAsyncFSWALForTest extends DualAsyncFSWAL { + + private boolean localBroken; + + private boolean remoteBroken; + + private CountDownLatch arrive; + + private CountDownLatch resume; + + private final class MyCombinedAsyncWriter implements AsyncWriter { + + private final AsyncWriter localWriter; + + private final AsyncWriter remoteWriter; + + public MyCombinedAsyncWriter(AsyncWriter localWriter, AsyncWriter remoteWriter) { + this.localWriter = localWriter; + this.remoteWriter = remoteWriter; + } + + @Override + public long getLength() { + return localWriter.getLength(); + } + + @Override + public void close() throws IOException { + Closeables.close(localWriter, true); + Closeables.close(remoteWriter, true); + } + + @Override + public CompletableFuture sync() { + CompletableFuture localFuture; + CompletableFuture remoteFuture; + if (!localBroken) { + localFuture = localWriter.sync(); + } else { + localFuture = new CompletableFuture<>(); + localFuture.completeExceptionally(new IOException("Inject error")); + } + if (!remoteBroken) { + remoteFuture = remoteWriter.sync(); + } else { + remoteFuture = new CompletableFuture<>(); + remoteFuture.completeExceptionally(new IOException("Inject error")); + } + return CompletableFuture.allOf(localFuture, remoteFuture).thenApply(v -> { + return localFuture.getNow(0L); + }); + } + + @Override + public void append(Entry entry) { + if (!localBroken) { + localWriter.append(entry); + } + if (!remoteBroken) { + remoteWriter.append(entry); + } + } + } + + public DualAsyncFSWALForTest(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteWALDir, + String logDir, String archiveDir, Configuration conf, List listeners, + boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup, + Class channelClass) throws FailedLogCloseException, IOException { + super(fs, remoteFs, rootDir, remoteWALDir, logDir, archiveDir, conf, listeners, failIfWALExists, + prefix, suffix, eventLoopGroup, channelClass); + } + + @Override + protected AsyncWriter createCombinedAsyncWriter(AsyncWriter localWriter, + AsyncWriter remoteWriter) { + return new MyCombinedAsyncWriter(localWriter, remoteWriter); + } + + @Override + protected AsyncWriter createWriterInstance(Path path) throws IOException { + if (arrive != null) { + arrive.countDown(); + try { + resume.await(); + } catch (InterruptedException e) { + } + } + if (localBroken || remoteBroken) { + throw new IOException("WAL broken"); + } + return super.createWriterInstance(path); + } + + public void setLocalBroken() { + this.localBroken = true; + } + + public void setRemoteBroken() { + this.remoteBroken = true; + } + + public void suspendLogRoll() { + arrive = new CountDownLatch(1); + resume = new CountDownLatch(1); + } + + public void waitUntilArrive() throws InterruptedException { + arrive.await(); + } + + public void resumeLogRoll() { + resume.countDown(); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java index ec6ec96..67f793d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.List; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -40,12 +39,13 @@ import org.apache.hadoop.hbase.wal.WAL.Entry; */ public class ReplicationSourceDummy implements ReplicationSourceInterface { - ReplicationSourceManager manager; - String peerClusterId; - Path currentPath; - MetricsSource metrics; - WALFileLengthProvider walFileLengthProvider; - AtomicBoolean startup = new AtomicBoolean(false); + private ReplicationSourceManager manager; + private ReplicationPeer replicationPeer; + private String peerClusterId; + private Path currentPath; + private MetricsSource metrics; + private WALFileLengthProvider walFileLengthProvider; + private AtomicBoolean startup = new AtomicBoolean(false); @Override public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager, @@ -56,6 +56,7 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface { this.peerClusterId = peerClusterId; this.metrics = metrics; this.walFileLengthProvider = walFileLengthProvider; + this.replicationPeer = rp; } @Override @@ -153,4 +154,9 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface { public ServerName getServerWALsBelongTo() { return null; } + + @Override + public ReplicationPeer getPeer() { + return replicationPeer; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java new file mode 100644 index 0000000..f765139 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SyncReplicationTestBase.java @@ -0,0 +1,270 @@ +/** + * 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.replication; + +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.regex.Pattern; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HBaseZKTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; + +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; + +/** + * Base class for testing sync replication. + */ +public class SyncReplicationTestBase { + + protected static final HBaseZKTestingUtility ZK_UTIL = new HBaseZKTestingUtility(); + + protected static final HBaseTestingUtility UTIL1 = new HBaseTestingUtility(); + + protected static final HBaseTestingUtility UTIL2 = new HBaseTestingUtility(); + + protected static TableName TABLE_NAME = TableName.valueOf("SyncRep"); + + protected static byte[] CF = Bytes.toBytes("cf"); + + protected static byte[] CQ = Bytes.toBytes("cq"); + + protected static String PEER_ID = "1"; + + protected static Path REMOTE_WAL_DIR1; + + protected static Path REMOTE_WAL_DIR2; + + protected static void initTestingUtility(HBaseTestingUtility util, String zkParent) { + util.setZkCluster(ZK_UTIL.getZkCluster()); + Configuration conf = util.getConfiguration(); + conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, zkParent); + conf.setInt("replication.source.size.capacity", 102400); + conf.setLong("replication.source.sleepforretries", 100); + conf.setInt("hbase.regionserver.maxlogs", 10); + conf.setLong("hbase.master.logcleaner.ttl", 10); + conf.setInt("zookeeper.recovery.retry", 1); + conf.setInt("zookeeper.recovery.retry.intervalmill", 10); + conf.setLong(HConstants.THREAD_WAKE_FREQUENCY, 100); + conf.setInt("replication.stats.thread.period.seconds", 5); + conf.setBoolean("hbase.tests.use.shortcircuit.reads", false); + conf.setLong("replication.sleep.before.failover", 2000); + conf.setInt("replication.source.maxretriesmultiplier", 10); + conf.setFloat("replication.source.ratio", 1.0f); + conf.setBoolean("replication.source.eof.autorecovery", true); + } + + @BeforeClass + public static void setUp() throws Exception { + ZK_UTIL.startMiniZKCluster(); + initTestingUtility(UTIL1, "/cluster1"); + initTestingUtility(UTIL2, "/cluster2"); + UTIL1.startMiniCluster(2,3); + UTIL2.startMiniCluster(2,3); + TableDescriptor td = + TableDescriptorBuilder.newBuilder(TABLE_NAME).setColumnFamily(ColumnFamilyDescriptorBuilder + .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build(); + UTIL1.getAdmin().createTable(td); + UTIL2.getAdmin().createTable(td); + FileSystem fs1 = UTIL1.getTestFileSystem(); + FileSystem fs2 = UTIL2.getTestFileSystem(); + REMOTE_WAL_DIR1 = + new Path(UTIL1.getMiniHBaseCluster().getMaster().getMasterFileSystem().getWALRootDir(), + "remoteWALs").makeQualified(fs1.getUri(), fs1.getWorkingDirectory()); + REMOTE_WAL_DIR2 = + new Path(UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem().getWALRootDir(), + "remoteWALs").makeQualified(fs2.getUri(), fs2.getWorkingDirectory()); + UTIL1.getAdmin().addReplicationPeer(PEER_ID, + ReplicationPeerConfig.newBuilder().setClusterKey(UTIL2.getClusterKey()) + .setReplicateAllUserTables(false) + .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>())) + .setRemoteWALDir(REMOTE_WAL_DIR2.toUri().toString()).build()); + UTIL2.getAdmin().addReplicationPeer(PEER_ID, + ReplicationPeerConfig.newBuilder().setClusterKey(UTIL1.getClusterKey()) + .setReplicateAllUserTables(false) + .setTableCFsMap(ImmutableMap.of(TABLE_NAME, new ArrayList<>())) + .setRemoteWALDir(REMOTE_WAL_DIR1.toUri().toString()).build()); + } + + private static void shutdown(HBaseTestingUtility util) throws Exception { + if (util.getHBaseCluster() == null) { + return; + } + Admin admin = util.getAdmin(); + if (!admin.listReplicationPeers(Pattern.compile(PEER_ID)).isEmpty()) { + if (admin + .getReplicationPeerSyncReplicationState(PEER_ID) != SyncReplicationState.DOWNGRADE_ACTIVE) { + admin.transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + } + admin.removeReplicationPeer(PEER_ID); + } + util.shutdownMiniCluster(); + } + + @AfterClass + public static void tearDown() throws Exception { + shutdown(UTIL1); + shutdown(UTIL2); + ZK_UTIL.shutdownMiniZKCluster(); + } + + protected final void write(HBaseTestingUtility util, int start, int end) throws IOException { + try (Table table = util.getConnection().getTable(TABLE_NAME)) { + for (int i = start; i < end; i++) { + table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); + } + } + } + + protected final void verify(HBaseTestingUtility util, int start, int end) throws IOException { + try (Table table = util.getConnection().getTable(TABLE_NAME)) { + for (int i = start; i < end; i++) { + assertEquals(i, Bytes.toInt(table.get(new Get(Bytes.toBytes(i))).getValue(CF, CQ))); + } + } + } + + protected final void verifyThroughRegion(HBaseTestingUtility util, int start, int end) + throws IOException { + HRegion region = util.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0); + for (int i = start; i < end; i++) { + assertEquals(i, Bytes.toInt(region.get(new Get(Bytes.toBytes(i))).getValue(CF, CQ))); + } + } + + protected final void verifyNotReplicatedThroughRegion(HBaseTestingUtility util, int start, + int end) throws IOException { + HRegion region = util.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0); + for (int i = start; i < end; i++) { + assertTrue(region.get(new Get(Bytes.toBytes(i))).isEmpty()); + } + } + + protected final void waitUntilReplicationDone(HBaseTestingUtility util, int end) + throws Exception { + // The reject check is in RSRpcService so we can still read through HRegion + HRegion region = util.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0); + util.waitFor(30000, new ExplainingPredicate() { + + @Override + public boolean evaluate() throws Exception { + return !region.get(new Get(Bytes.toBytes(end - 1))).isEmpty(); + } + + @Override + public String explainFailure() throws Exception { + return "Replication has not been catched up yet"; + } + }); + } + + protected final void writeAndVerifyReplication(HBaseTestingUtility util1, + HBaseTestingUtility util2, int start, int end) throws Exception { + write(util1, start, end); + waitUntilReplicationDone(util2, end); + verifyThroughRegion(util2, start, end); + } + + protected final Path getRemoteWALDir(MasterFileSystem mfs, String peerId) { + Path remoteWALDir = new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME); + return getRemoteWALDir(remoteWALDir, peerId); + } + + protected Path getRemoteWALDir(Path remoteWALDir, String peerId) { + return new Path(remoteWALDir, peerId); + } + + protected Path getReplayRemoteWALs(Path remoteWALDir, String peerId) { + return new Path(remoteWALDir, peerId + "-replay"); + } + + protected void verifyRemovedPeer(String peerId, Path remoteWALDir, HBaseTestingUtility utility) + throws Exception { + ReplicationPeerStorage rps = ReplicationStorageFactory + .getReplicationPeerStorage(utility.getZooKeeperWatcher(), utility.getConfiguration()); + try { + rps.getPeerSyncReplicationState(peerId); + fail("Should throw exception when get the sync replication state of a removed peer."); + } catch (NullPointerException e) { + // ignore. + } + try { + rps.getPeerNewSyncReplicationState(peerId); + fail("Should throw exception when get the new sync replication state of a removed peer"); + } catch (NullPointerException e) { + // ignore. + } + try (FileSystem fs = utility.getTestFileSystem()) { + Assert.assertFalse(fs.exists(getRemoteWALDir(remoteWALDir, peerId))); + Assert.assertFalse(fs.exists(getReplayRemoteWALs(remoteWALDir, peerId))); + } + } + + protected void verifyReplicationRequestRejection(HBaseTestingUtility utility, + boolean expectedRejection) throws Exception { + HRegionServer regionServer = utility.getRSForFirstRegionInTable(TABLE_NAME); + ClusterConnection connection = regionServer.getClusterConnection(); + Entry[] entries = new Entry[10]; + for (int i = 0; i < entries.length; i++) { + entries[i] = + new Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TABLE_NAME, 0), new WALEdit()); + } + if (!expectedRejection) { + ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()), + entries, null, null, null); + } else { + try { + ReplicationProtbufUtil.replicateWALEntry(connection.getAdmin(regionServer.getServerName()), + entries, null, null, null); + Assert.fail("Should throw IOException when sync-replication state is in A or DA"); + } catch (DoNotRetryIOException e) { + Assert.assertTrue(e.getMessage().contains("Reject to apply to sink cluster")); + Assert.assertTrue(e.getMessage().contains(TABLE_NAME.toString())); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java index f96dbe5..cd84293 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationBase.java @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -28,6 +27,8 @@ import java.util.List; import java.util.NavigableMap; import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; @@ -58,6 +59,9 @@ import org.junit.BeforeClass; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; + /** * This class is only a base for other integration-level replication tests. * Do not add tests here. @@ -99,6 +103,10 @@ public class TestReplicationBase { return false; } + protected boolean isSyncPeer() { + return false; + } + protected final void cleanUp() throws IOException, InterruptedException { // Starting and stopping replication can make us miss new logs, // rolling like this makes sure the most recent one gets added to the queue @@ -245,9 +253,19 @@ public class TestReplicationBase { @Before public void setUpBase() throws Exception { if (!peerExist(PEER_ID2)) { - ReplicationPeerConfig rpc = ReplicationPeerConfig.newBuilder() - .setClusterKey(utility2.getClusterKey()).setSerial(isSerialPeer()).build(); - hbaseAdmin.addReplicationPeer(PEER_ID2, rpc); + ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder() + .setClusterKey(utility2.getClusterKey()).setSerial(isSerialPeer()); + if (isSyncPeer()) { + FileSystem fs2 = utility2.getTestFileSystem(); + // The remote wal dir is not important as we do not use it in DA state, here we only need to + // confirm that a sync peer in DA state can still replicate data to remote cluster + // asynchronously. + builder.setReplicateAllUserTables(false) + .setTableCFsMap(ImmutableMap.of(tableName, ImmutableList.of())) + .setRemoteWALDir(new Path("/RemoteWAL") + .makeQualified(fs2.getUri(), fs2.getWorkingDirectory()).toUri().toString()); + } + hbaseAdmin.addReplicationPeer(PEER_ID2, builder.build()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java index b94b443..5c96742 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationChangingPeerRegionservers.java @@ -62,22 +62,28 @@ public class TestReplicationChangingPeerRegionservers extends TestReplicationBas private static final Logger LOG = LoggerFactory.getLogger(TestReplicationChangingPeerRegionservers.class); - @Parameter + @Parameter(0) public boolean serialPeer; + @Parameter(1) + public boolean syncPeer; + @Override protected boolean isSerialPeer() { return serialPeer; } - @Parameters(name = "{index}: serialPeer={0}") - public static List parameters() { - return ImmutableList.of(true, false); + @Override + protected boolean isSyncPeer() { + return syncPeer; + } + + @Parameters(name = "{index}: serialPeer={0}, syncPeer={1}") + public static List parameters() { + return ImmutableList.of(new Object[] { false, false }, new Object[] { false, true }, + new Object[] { true, false }, new Object[] { true, true }); } - /** - * @throws java.lang.Exception - */ @Before public void setUp() throws Exception { // Starting and stopping replication can make us miss new logs, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTestsSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTestsSync.java new file mode 100644 index 0000000..9ca0044 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSmallTestsSync.java @@ -0,0 +1,40 @@ +/** + * 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.replication; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.junit.ClassRule; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestReplicationSmallTestsSync extends TestReplicationSmallTests { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestReplicationSmallTestsSync.class); + + @Override + protected boolean isSyncPeer() { + return true; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java index 65eac4a..2419095 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java @@ -155,11 +155,13 @@ public class TestReplicationTrackerZKImpl { public void testPeerNameControl() throws Exception { int exists = 0; rp.getPeerStorage().addPeer("6", - ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true); + ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true, + SyncReplicationState.NONE); try { rp.getPeerStorage().addPeer("6", - ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true); + ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true, + SyncReplicationState.NONE); } catch (ReplicationException e) { if (e.getCause() instanceof KeeperException.NodeExistsException) { exists++; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java new file mode 100644 index 0000000..42adab6 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationActive.java @@ -0,0 +1,137 @@ +/** + * 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.replication; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.client.AsyncConnection; +import org.apache.hadoop.hbase.client.AsyncTable; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WAL.Reader; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationActive extends SyncReplicationTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationActive.class); + + @Test + public void testActive() throws Exception { + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + + // confirm that peer with state A will reject replication request. + verifyReplicationRequestRejection(UTIL1, true); + verifyReplicationRequestRejection(UTIL2, false); + + UTIL1.getAdmin().disableReplicationPeer(PEER_ID); + write(UTIL1, 0, 100); + Thread.sleep(2000); + // peer is disabled so no data have been replicated + verifyNotReplicatedThroughRegion(UTIL2, 0, 100); + + // Ensure that there's no cluster id in remote log entries. + verifyNoClusterIdInRemoteLog(UTIL2, REMOTE_WAL_DIR2, PEER_ID); + + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + // confirm that peer with state DA will reject replication request. + verifyReplicationRequestRejection(UTIL2, true); + // confirm that the data is there after we convert the peer to DA + verify(UTIL2, 0, 100); + + try (AsyncConnection conn = + ConnectionFactory.createAsyncConnection(UTIL1.getConfiguration()).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setMaxAttempts(1).build(); + CompletableFuture future = + table.put(new Put(Bytes.toBytes(1000)).addColumn(CF, CQ, Bytes.toBytes(1000))); + Thread.sleep(2000); + // should hang on rolling + assertFalse(future.isDone()); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + try { + future.get(); + fail("should fail because of the wal is closing"); + } catch (ExecutionException e) { + // expected + assertThat(e.getCause().getMessage(), containsString("only marker edit is allowed")); + } + } + // confirm that the data has not been persisted + HRegion region = UTIL1.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0); + assertTrue(region.get(new Get(Bytes.toBytes(1000))).isEmpty()); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + + writeAndVerifyReplication(UTIL2, UTIL1, 100, 200); + + // shutdown the cluster completely + UTIL1.shutdownMiniCluster(); + // confirm that we can convert to DA even if the remote slave cluster is down + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + // confirm that peer with state DA will reject replication request. + verifyReplicationRequestRejection(UTIL2, true); + write(UTIL2, 200, 300); + } + + private void verifyNoClusterIdInRemoteLog(HBaseTestingUtility utility, Path remoteDir, + String peerId) throws Exception { + FileSystem fs2 = utility.getTestFileSystem(); + FileStatus[] files = fs2.listStatus(new Path(remoteDir, peerId)); + Assert.assertTrue(files.length > 0); + for (FileStatus file : files) { + try ( + Reader reader = WALFactory.createReader(fs2, file.getPath(), utility.getConfiguration())) { + Entry entry = reader.next(); + Assert.assertTrue(entry != null); + while (entry != null) { + Assert.assertEquals(entry.getKey().getClusterIds().size(), 0); + entry = reader.next(); + } + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalCopyToRemote.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalCopyToRemote.java new file mode 100644 index 0000000..cf8993b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalCopyToRemote.java @@ -0,0 +1,108 @@ +/** + * 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.replication; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.util.concurrent.ExecutionException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; +import org.apache.hadoop.hbase.client.AsyncConnection; +import org.apache.hadoop.hbase.client.AsyncTable; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationMoreLogsInLocalCopyToRemote extends SyncReplicationTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationMoreLogsInLocalCopyToRemote.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestSyncReplicationMoreLogsInLocalCopyToRemote.class); + + @BeforeClass + public static void setUp() throws Exception { + UTIL1.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL, + DualAsyncFSWALForTest.class, DualAsyncFSWAL.class); + UTIL2.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL, + DualAsyncFSWALForTest.class, DualAsyncFSWAL.class); + SyncReplicationTestBase.setUp(); + } + + @Test + public void testSplitLog() throws Exception { + UTIL1.getAdmin().disableReplicationPeer(PEER_ID); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME); + DualAsyncFSWALForTest wal = + (DualAsyncFSWALForTest) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build()); + wal.setRemoteBroken(); + try (AsyncConnection conn = + ConnectionFactory.createAsyncConnection(UTIL1.getConfiguration()).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setMaxAttempts(1).build(); + try { + table.put(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, Bytes.toBytes(0))).get(); + fail("Should fail since the rs will crash and we will not retry"); + } catch (ExecutionException e) { + // expected + LOG.info("Expected error:", e); + } + } + UTIL1.waitFor(60000, new ExplainingPredicate() { + + @Override + public boolean evaluate() throws Exception { + try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) { + return table.exists(new Get(Bytes.toBytes(0))); + } + } + + @Override + public String explainFailure() throws Exception { + return "The row is still not available"; + } + }); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + // We should have copied the local log to remote, so we should be able to get the value + try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) { + assertEquals(0, Bytes.toInt(table.get(new Get(Bytes.toBytes(0))).getValue(CF, CQ))); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalGiveUpSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalGiveUpSplitting.java new file mode 100644 index 0000000..9a6d242 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationMoreLogsInLocalGiveUpSplitting.java @@ -0,0 +1,128 @@ +/** + * 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.replication; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.AsyncConnection; +import org.apache.hadoop.hbase.client.AsyncTable; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.RetriesExhaustedException; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.SyncReplicationWALProvider; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationMoreLogsInLocalGiveUpSplitting extends SyncReplicationTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationMoreLogsInLocalGiveUpSplitting.class); + + private static final Logger LOG = + LoggerFactory.getLogger(TestSyncReplicationMoreLogsInLocalGiveUpSplitting.class); + + @BeforeClass + public static void setUp() throws Exception { + UTIL1.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL, + DualAsyncFSWALForTest.class, DualAsyncFSWAL.class); + UTIL2.getConfiguration().setClass(SyncReplicationWALProvider.DUAL_WAL_IMPL, + DualAsyncFSWALForTest.class, DualAsyncFSWAL.class); + SyncReplicationTestBase.setUp(); + } + + @Test + public void testSplitLog() throws Exception { + UTIL1.getAdmin().disableReplicationPeer(PEER_ID); + UTIL2.getAdmin().disableReplicationPeer(PEER_ID); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) { + table.put(new Put(Bytes.toBytes(0)).addColumn(CF, CQ, Bytes.toBytes(0))); + } + HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME); + DualAsyncFSWALForTest wal = + (DualAsyncFSWALForTest) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build()); + wal.setRemoteBroken(); + wal.suspendLogRoll(); + try (AsyncConnection conn = + ConnectionFactory.createAsyncConnection(UTIL1.getConfiguration()).get()) { + AsyncTable table = conn.getTableBuilder(TABLE_NAME).setMaxAttempts(1) + .setWriteRpcTimeout(5, TimeUnit.SECONDS).build(); + try { + table.put(new Put(Bytes.toBytes(1)).addColumn(CF, CQ, Bytes.toBytes(1))).get(); + fail("Should fail since the rs will hang and we will get a rpc timeout"); + } catch (ExecutionException e) { + // expected + LOG.info("Expected error:", e); + } + } + wal.waitUntilArrive(); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + wal.resumeLogRoll(); + try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) { + assertEquals(0, Bytes.toInt(table.get(new Get(Bytes.toBytes(0))).getValue(CF, CQ))); + // we failed to write this entry to remote so it should not exist + assertFalse(table.exists(new Get(Bytes.toBytes(1)))); + } + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + // make sure that the region is online. We can not use waitTableAvailable since the table in + // stand by state can not be read from client. + try (Table table = UTIL1.getConnection().getTable(TABLE_NAME)) { + try { + table.exists(new Get(Bytes.toBytes(0))); + } catch (DoNotRetryIOException | RetriesExhaustedException e) { + // expected + assertThat(e.getMessage(), containsString("STANDBY")); + } + } + HRegion region = UTIL1.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0); + // we give up splitting the whole wal file so this record will also be gone. + assertTrue(region.get(new Get(Bytes.toBytes(0))).isEmpty()); + UTIL2.getAdmin().enableReplicationPeer(PEER_ID); + // finally it should be replicated back + waitUntilReplicationDone(UTIL1, 1); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java new file mode 100644 index 0000000..0cd1846 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationRemoveRemoteWAL.java @@ -0,0 +1,100 @@ +/** + * 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.replication; + +import static org.hamcrest.CoreMatchers.endsWith; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationRemoveRemoteWAL extends SyncReplicationTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationRemoveRemoteWAL.class); + + private void waitUntilDeleted(Path remoteWAL) throws Exception { + MasterFileSystem mfs = UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem(); + UTIL1.waitFor(30000, new ExplainingPredicate() { + + @Override + public boolean evaluate() throws Exception { + return !mfs.getWALFileSystem().exists(remoteWAL); + } + + @Override + public String explainFailure() throws Exception { + return remoteWAL + " has not been deleted yet"; + } + }); + } + + @Test + public void testRemoveRemoteWAL() throws Exception { + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + + MasterFileSystem mfs = UTIL2.getMiniHBaseCluster().getMaster().getMasterFileSystem(); + Path remoteWALDir = ReplicationUtils.getPeerRemoteWALDir( + new Path(mfs.getWALRootDir(), ReplicationUtils.REMOTE_WAL_DIR_NAME), PEER_ID); + FileStatus[] remoteWALStatus = mfs.getWALFileSystem().listStatus(remoteWALDir); + assertEquals(1, remoteWALStatus.length); + Path remoteWAL = remoteWALStatus[0].getPath(); + assertThat(remoteWAL.getName(), endsWith(ReplicationUtils.SYNC_WAL_SUFFIX)); + writeAndVerifyReplication(UTIL1, UTIL2, 0, 100); + + HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME); + rs.getWalRoller().requestRollAll(); + // The replicated wal file should be deleted finally + waitUntilDeleted(remoteWAL); + remoteWALStatus = mfs.getWALFileSystem().listStatus(remoteWALDir); + assertEquals(1, remoteWALStatus.length); + remoteWAL = remoteWALStatus[0].getPath(); + assertThat(remoteWAL.getName(), endsWith(ReplicationUtils.SYNC_WAL_SUFFIX)); + + UTIL1.getAdmin().disableReplicationPeer(PEER_ID); + write(UTIL1, 100, 200); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + + // should still be there since the peer is disabled and we haven't replicated the data yet + assertTrue(mfs.getWALFileSystem().exists(remoteWAL)); + + UTIL1.getAdmin().enableReplicationPeer(PEER_ID); + waitUntilReplicationDone(UTIL2, 200); + verifyThroughRegion(UTIL2, 100, 200); + + // Confirm that we will also remove the remote wal files in DA state + waitUntilDeleted(remoteWAL); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java new file mode 100644 index 0000000..de409fc --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandBy.java @@ -0,0 +1,121 @@ +/** + * 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.replication; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.Arrays; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.Append; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RetriesExhaustedException; +import org.apache.hadoop.hbase.client.RowMutations; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationStandBy extends SyncReplicationTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationStandBy.class); + + @FunctionalInterface + private interface TableAction { + + void call(Table table) throws IOException; + } + + private void assertDisallow(Table table, TableAction action) throws IOException { + try { + action.call(table); + } catch (DoNotRetryIOException | RetriesExhaustedException e) { + // expected + assertThat(e.getMessage(), containsString("STANDBY")); + } + } + + @Test + public void testStandby() throws Exception { + MasterFileSystem mfs = UTIL2.getHBaseCluster().getMaster().getMasterFileSystem(); + Path remoteWALDir = getRemoteWALDir(mfs, PEER_ID); + assertFalse(mfs.getWALFileSystem().exists(remoteWALDir)); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + assertTrue(mfs.getWALFileSystem().exists(remoteWALDir)); + try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) { + assertDisallow(table, t -> t.get(new Get(Bytes.toBytes("row")))); + assertDisallow(table, + t -> t.put(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")))); + assertDisallow(table, t -> t.delete(new Delete(Bytes.toBytes("row")))); + assertDisallow(table, t -> t.incrementColumnValue(Bytes.toBytes("row"), CF, CQ, 1)); + assertDisallow(table, + t -> t.append(new Append(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")))); + assertDisallow(table, + t -> t.get(Arrays.asList(new Get(Bytes.toBytes("row")), new Get(Bytes.toBytes("row1"))))); + assertDisallow(table, + t -> t.put( + Arrays.asList(new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row")), + new Put(Bytes.toBytes("row1")).addColumn(CF, CQ, Bytes.toBytes("row1"))))); + assertDisallow(table, t -> t.mutateRow(new RowMutations(Bytes.toBytes("row")) + .add((Mutation) new Put(Bytes.toBytes("row")).addColumn(CF, CQ, Bytes.toBytes("row"))))); + } + // We should still allow replication writes + writeAndVerifyReplication(UTIL1, UTIL2, 0, 100); + + // Remove the peers in ACTIVE & STANDBY cluster. + FileSystem fs2 = REMOTE_WAL_DIR2.getFileSystem(UTIL2.getConfiguration()); + Assert.assertTrue(fs2.exists(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID))); + + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + Assert.assertFalse(fs2.exists(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID))); + Assert.assertFalse(fs2.exists(getReplayRemoteWALs(REMOTE_WAL_DIR2, PEER_ID))); + + UTIL1.getAdmin().removeReplicationPeer(PEER_ID); + verifyRemovedPeer(PEER_ID, REMOTE_WAL_DIR1, UTIL1); + + // Peer remoteWAL dir will be renamed to replay WAL dir when transit from S to DA, and the + // replay WAL dir will be removed after replaying all WALs, so create a emtpy dir here to test + // whether the removeReplicationPeer would remove the remoteWAL dir. + fs2.create(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID)); + fs2.create(getReplayRemoteWALs(REMOTE_WAL_DIR2, PEER_ID)); + Assert.assertTrue(fs2.exists(getRemoteWALDir(REMOTE_WAL_DIR2, PEER_ID))); + Assert.assertTrue(fs2.exists(getReplayRemoteWALs(REMOTE_WAL_DIR2, PEER_ID))); + UTIL2.getAdmin().removeReplicationPeer(PEER_ID); + verifyRemovedPeer(PEER_ID, REMOTE_WAL_DIR2, UTIL2); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandbyKillMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandbyKillMaster.java new file mode 100644 index 0000000..6265f5c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandbyKillMaster.java @@ -0,0 +1,88 @@ +/** + * 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.replication; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationStandbyKillMaster extends SyncReplicationTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestSyncReplicationStandbyKillMaster.class); + + private final long SLEEP_TIME = 2000; + + private final int COUNT = 1000; + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationStandbyKillMaster.class); + + @Test + public void testStandbyKillMaster() throws Exception { + MasterFileSystem mfs = UTIL2.getHBaseCluster().getMaster().getMasterFileSystem(); + Path remoteWALDir = getRemoteWALDir(mfs, PEER_ID); + assertFalse(mfs.getWALFileSystem().exists(remoteWALDir)); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + assertTrue(mfs.getWALFileSystem().exists(remoteWALDir)); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + + // Disable async replication and write data, then shutdown + UTIL1.getAdmin().disableReplicationPeer(PEER_ID); + write(UTIL1, 0, COUNT); + UTIL1.shutdownMiniCluster(); + + Thread t = new Thread(() -> { + try { + Thread.sleep(SLEEP_TIME); + UTIL2.getMiniHBaseCluster().getMaster().stop("Stop master for test"); + } catch (Exception e) { + LOG.error("Failed to stop master", e); + } + }); + t.start(); + + // Transit standby to DA to replay logs + try { + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + } catch (Exception e) { + LOG.error("Failed to transit standby cluster to " + SyncReplicationState.DOWNGRADE_ACTIVE); + } + + while (UTIL2.getAdmin().getReplicationPeerSyncReplicationState(PEER_ID) + != SyncReplicationState.DOWNGRADE_ACTIVE) { + Thread.sleep(SLEEP_TIME); + } + verify(UTIL2, 0, COUNT); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandbyKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandbyKillRS.java new file mode 100644 index 0000000..3c9724f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSyncReplicationStandbyKillRS.java @@ -0,0 +1,119 @@ +/** + * 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.replication; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.List; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.master.MasterFileSystem; +import org.apache.hadoop.hbase.master.ServerManager; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationStandbyKillRS extends SyncReplicationTestBase { + + private static final Logger LOG = + LoggerFactory.getLogger(TestSyncReplicationStandbyKillRS.class); + + private final long SLEEP_TIME = 1000; + + private final int COUNT = 1000; + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationStandbyKillRS.class); + + @Test + public void testStandbyKillRegionServer() throws Exception { + MasterFileSystem mfs = UTIL2.getHBaseCluster().getMaster().getMasterFileSystem(); + Path remoteWALDir = getRemoteWALDir(mfs, PEER_ID); + assertFalse(mfs.getWALFileSystem().exists(remoteWALDir)); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + assertTrue(mfs.getWALFileSystem().exists(remoteWALDir)); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + + // Disable async replication and write data, then shutdown + UTIL1.getAdmin().disableReplicationPeer(PEER_ID); + write(UTIL1, 0, COUNT); + UTIL1.shutdownMiniCluster(); + + JVMClusterUtil.MasterThread activeMaster = UTIL2.getMiniHBaseCluster().getMasterThread(); + Thread t = new Thread(() -> { + try { + List regionServers = + UTIL2.getMiniHBaseCluster().getLiveRegionServerThreads(); + for (JVMClusterUtil.RegionServerThread rst : regionServers) { + ServerName serverName = rst.getRegionServer().getServerName(); + rst.getRegionServer().stop("Stop RS for test"); + waitForRSShutdownToStartAndFinish(activeMaster, serverName); + JVMClusterUtil.RegionServerThread restarted = + UTIL2.getMiniHBaseCluster().startRegionServer(); + restarted.waitForServerOnline(); + } + } catch (Exception e) { + LOG.error("Failed to kill RS", e); + } + }); + t.start(); + + // Transit standby to DA to replay logs + try { + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + } catch (Exception e) { + LOG.error("Failed to transit standby cluster to " + SyncReplicationState.DOWNGRADE_ACTIVE); + } + + while (UTIL2.getAdmin().getReplicationPeerSyncReplicationState(PEER_ID) + != SyncReplicationState.DOWNGRADE_ACTIVE) { + Thread.sleep(SLEEP_TIME); + } + verify(UTIL2, 0, COUNT); + } + + private void waitForRSShutdownToStartAndFinish(JVMClusterUtil.MasterThread activeMaster, + ServerName serverName) throws InterruptedException { + ServerManager sm = activeMaster.getMaster().getServerManager(); + // First wait for it to be in dead list + while (!sm.getDeadServers().isDeadServer(serverName)) { + LOG.debug("Waiting for [" + serverName + "] to be listed as dead in master"); + Thread.sleep(SLEEP_TIME); + } + LOG.debug("Server [" + serverName + "] marked as dead, waiting for it to " + + "finish dead processing"); + while (sm.areDeadServersInProgress()) { + LOG.debug("Server [" + serverName + "] still being processed, waiting"); + Thread.sleep(SLEEP_TIME); + } + LOG.debug("Server [" + serverName + "] done with server shutdown processing"); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java new file mode 100644 index 0000000..d01a0ac --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/master/TestRecoverStandbyProcedure.java @@ -0,0 +1,186 @@ +/** + * 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.replication.master; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.replication.RecoverStandbyProcedure; +import org.apache.hadoop.hbase.master.replication.SyncReplicationReplayWALManager; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; +import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; +import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter; +import org.apache.hadoop.hbase.regionserver.wal.WALUtil; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MasterTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException; +import org.apache.hadoop.hbase.wal.WAL.Entry; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALKeyImpl; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({MasterTests.class, LargeTests.class}) +public class TestRecoverStandbyProcedure { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestRecoverStandbyProcedure.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestRecoverStandbyProcedure.class); + + private static final TableName tableName = TableName.valueOf("TestRecoverStandbyProcedure"); + + private static final RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tableName).build(); + + private static final byte[] family = Bytes.toBytes("CF"); + + private static final byte[] qualifier = Bytes.toBytes("q"); + + private static final long timestamp = System.currentTimeMillis(); + + private static final int ROW_COUNT = 1000; + + private static final int WAL_NUMBER = 10; + + private static final int RS_NUMBER = 3; + + private static final String PEER_ID = "1"; + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + private static SyncReplicationReplayWALManager syncReplicationReplayWALManager; + + private static ProcedureExecutor procExec; + + private static FileSystem fs; + + private static Configuration conf; + + @BeforeClass + public static void setupCluster() throws Exception { + UTIL.startMiniCluster(RS_NUMBER); + UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); + conf = UTIL.getConfiguration(); + HMaster master = UTIL.getHBaseCluster().getMaster(); + fs = master.getMasterFileSystem().getWALFileSystem(); + syncReplicationReplayWALManager = master.getSyncReplicationReplayWALManager(); + procExec = master.getMasterProcedureExecutor(); + } + + @AfterClass + public static void cleanupTest() throws Exception { + try { + UTIL.shutdownMiniCluster(); + } catch (Exception e) { + LOG.warn("failure shutting down cluster", e); + } + } + + @Before + public void setupBeforeTest() throws IOException { + UTIL.createTable(tableName, family); + } + + @After + public void tearDownAfterTest() throws IOException { + try (Admin admin = UTIL.getAdmin()) { + if (admin.isTableEnabled(tableName)) { + admin.disableTable(tableName); + } + admin.deleteTable(tableName); + } + } + + @Test + public void testRecoverStandby() throws IOException, StreamLacksCapabilityException { + setupSyncReplicationWALs(); + long procId = procExec.submitProcedure(new RecoverStandbyProcedure(PEER_ID, false)); + ProcedureTestingUtility.waitProcedure(procExec, procId); + ProcedureTestingUtility.assertProcNotFailed(procExec, procId); + + try (Table table = UTIL.getConnection().getTable(tableName)) { + for (int i = 0; i < WAL_NUMBER * ROW_COUNT; i++) { + Result result = table.get(new Get(Bytes.toBytes(i)).setTimestamp(timestamp)); + assertNotNull(result); + assertEquals(i, Bytes.toInt(result.getValue(family, qualifier))); + } + } + } + + private void setupSyncReplicationWALs() throws IOException, StreamLacksCapabilityException { + Path peerRemoteWALDir = ReplicationUtils + .getPeerRemoteWALDir(syncReplicationReplayWALManager.getRemoteWALDir(), PEER_ID); + if (!fs.exists(peerRemoteWALDir)) { + fs.mkdirs(peerRemoteWALDir); + } + for (int i = 0; i < WAL_NUMBER; i++) { + try (ProtobufLogWriter writer = new ProtobufLogWriter()) { + Path wal = new Path(peerRemoteWALDir, "srv1,8888." + i + ".syncrep"); + writer.init(fs, wal, conf, true, WALUtil.getWALBlockSize(conf, fs, peerRemoteWALDir)); + List entries = setupWALEntries(i * ROW_COUNT, (i + 1) * ROW_COUNT); + for (Entry entry : entries) { + writer.append(entry); + } + writer.sync(false); + LOG.info("Created wal {} to replay for peer id={}", wal, PEER_ID); + } + } + } + + private List setupWALEntries(int startRow, int endRow) { + return IntStream.range(startRow, endRow) + .mapToObj(i -> createWALEntry(Bytes.toBytes(i), Bytes.toBytes(i))) + .collect(Collectors.toList()); + } + + private Entry createWALEntry(byte[] row, byte[] value) { + WALKeyImpl key = new WALKeyImpl(regionInfo.getEncodedNameAsBytes(), tableName, 1); + WALEdit edit = new WALEdit(); + edit.add(new KeyValue(row, family, qualifier, timestamp, value)); + return new Entry(key, edit); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDrainReplicationQueuesForStandBy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDrainReplicationQueuesForStandBy.java new file mode 100644 index 0000000..5da7870 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestDrainReplicationQueuesForStandBy.java @@ -0,0 +1,118 @@ +/** + * 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.replication.regionserver; + +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.hadoop.hbase.replication.SyncReplicationTestBase; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ReplicationTests.class, MediumTests.class }) +public class TestDrainReplicationQueuesForStandBy extends SyncReplicationTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestDrainReplicationQueuesForStandBy.class); + + @Test + public void test() throws Exception { + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + UTIL1.getAdmin().disableReplicationPeer(PEER_ID); + write(UTIL1, 0, 100); + + HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME); + String walGroupId = AbstractFSWALProvider.getWALPrefixFromWALName( + ((AbstractFSWAL) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build())) + .getCurrentFileName().getName()); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + // transit cluster2 to DA and cluster 1 to S + verify(UTIL2, 0, 100); + + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + // delete the original value, and then major compact + try (Table table = UTIL2.getConnection().getTable(TABLE_NAME)) { + for (int i = 0; i < 100; i++) { + table.delete(new Delete(Bytes.toBytes(i))); + } + } + UTIL2.flush(TABLE_NAME); + UTIL2.compact(TABLE_NAME, true); + // wait until the new values are replicated back to cluster1 + HRegion region = rs.getRegions(TABLE_NAME).get(0); + UTIL1.waitFor(30000, new ExplainingPredicate() { + + @Override + public boolean evaluate() throws Exception { + return region.get(new Get(Bytes.toBytes(99))).isEmpty(); + } + + @Override + public String explainFailure() throws Exception { + return "Replication has not been catched up yet"; + } + }); + // transit cluster1 to DA and cluster2 to S, then we will start replicating from cluster1 to + // cluster2 + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + UTIL1.getAdmin().enableReplicationPeer(PEER_ID); + + // confirm that we will not replicate the old data which causes inconsistency + ReplicationSource source = (ReplicationSource) ((Replication) rs.getReplicationSourceService()) + .getReplicationManager().getSource(PEER_ID); + UTIL1.waitFor(30000, new ExplainingPredicate() { + + @Override + public boolean evaluate() throws Exception { + return !source.workerThreads.containsKey(walGroupId); + } + + @Override + public String explainFailure() throws Exception { + return "Replication has not been catched up yet"; + } + }); + HRegion region2 = UTIL2.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0); + for (int i = 0; i < 100; i++) { + assertTrue(region2.get(new Get(Bytes.toBytes(i))).isEmpty()); + } + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java index aa6c39c..2d6c28f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSink.java @@ -129,8 +129,7 @@ public class TestReplicationSink { TestSourceFSConfigurationProvider.class.getCanonicalName()); TEST_UTIL.startMiniCluster(3); - SINK = - new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE); + SINK = new ReplicationSink(new Configuration(TEST_UTIL.getConfiguration()), STOPPABLE); table1 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAME1); table2 = TEST_UTIL.createTable(TABLE_NAME2, FAM_NAME2); Path rootDir = FSUtils.getRootDir(TEST_UTIL.getConfiguration()); @@ -419,7 +418,7 @@ public class TestReplicationSink { return builder.build(); } - private WALEntry.Builder createWALEntryBuilder(TableName table) { + public static WALEntry.Builder createWALEntryBuilder(TableName table) { WALEntry.Builder builder = WALEntry.newBuilder(); builder.setAssociatedCellCount(1); WALKey.Builder keyBuilder = WALKey.newBuilder(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index eb46cd7..febe764 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.io.IOException; import java.lang.reflect.Field; @@ -49,19 +51,19 @@ import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.replication.ReplicationFactory; @@ -71,11 +73,14 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationSourceDummy; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.ReplicationUtils; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage; import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; 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; import org.apache.hadoop.hbase.util.Pair; @@ -132,9 +137,9 @@ public abstract class TestReplicationSourceManager { protected static ZKWatcher zkw; - protected static HTableDescriptor htd; + protected static TableDescriptor htd; - protected static HRegionInfo hri; + protected static RegionInfo hri; protected static final byte[] r1 = Bytes.toBytes("r1"); @@ -155,6 +160,8 @@ public abstract class TestReplicationSourceManager { protected static Path logDir; + protected static Path remoteLogDir; + protected static CountDownLatch latch; protected static List files = new ArrayList<>(); @@ -172,16 +179,21 @@ public abstract class TestReplicationSourceManager { ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state"); ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES); + ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/sync-rep-state"); + ZKUtil.setData(zkw, "/hbase/replication/peers/1/sync-rep-state", + ZKReplicationPeerStorage.NONE_STATE_ZNODE_BYTES); + ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/new-sync-rep-state"); + ZKUtil.setData(zkw, "/hbase/replication/peers/1/new-sync-rep-state", + ZKReplicationPeerStorage.NONE_STATE_ZNODE_BYTES); ZKUtil.createWithParents(zkw, "/hbase/replication/state"); ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES); ZKClusterId.setClusterId(zkw, new ClusterId()); FSUtils.setRootDir(utility.getConfiguration(), utility.getDataTestDir()); fs = FileSystem.get(conf); - oldLogDir = new Path(utility.getDataTestDir(), - HConstants.HREGION_OLDLOGDIR_NAME); - logDir = new Path(utility.getDataTestDir(), - HConstants.HREGION_LOGDIR_NAME); + oldLogDir = utility.getDataTestDir(HConstants.HREGION_OLDLOGDIR_NAME); + logDir = utility.getDataTestDir(HConstants.HREGION_LOGDIR_NAME); + remoteLogDir = utility.getDataTestDir(ReplicationUtils.REMOTE_WAL_DIR_NAME); replication = new Replication(); replication.initialize(new DummyServer(), fs, logDir, oldLogDir, null); managerOfCluster = getManagerFromCluster(); @@ -198,19 +210,16 @@ public abstract class TestReplicationSourceManager { } waitPeer(slaveId, manager, true); - htd = new HTableDescriptor(test); - HColumnDescriptor col = new HColumnDescriptor(f1); - col.setScope(HConstants.REPLICATION_SCOPE_GLOBAL); - htd.addFamily(col); - col = new HColumnDescriptor(f2); - col.setScope(HConstants.REPLICATION_SCOPE_LOCAL); - htd.addFamily(col); + htd = TableDescriptorBuilder.newBuilder(test) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(f1) + .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(f2)).build(); scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + for(byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } - hri = new HRegionInfo(htd.getTableName(), r1, r2); + hri = RegionInfoBuilder.newBuilder(htd.getTableName()).setStartKey(r1).setEndKey(r2).build(); } private static ReplicationSourceManager getManagerFromCluster() { @@ -241,6 +250,7 @@ public abstract class TestReplicationSourceManager { private void cleanLogDir() throws IOException { fs.delete(logDir, true); fs.delete(oldLogDir, true); + fs.delete(remoteLogDir, true); } @Before @@ -279,10 +289,10 @@ public abstract class TestReplicationSourceManager { .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager)); final WAL wal = wals.getWAL(hri); manager.init(); - HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame")); - htd.addFamily(new HColumnDescriptor(f1)); + TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf("tableame")) + .setColumnFamily(ColumnFamilyDescriptorBuilder.of(f1)).build(); NavigableMap scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR); - for(byte[] fam : htd.getFamiliesKeys()) { + for(byte[] fam : htd.getColumnFamilyNames()) { scopes.put(fam, 0); } // Testing normal log rolling every 20 @@ -322,7 +332,11 @@ public abstract class TestReplicationSourceManager { wal.rollWriter(); - manager.logPositionAndCleanOldLogs("1", false, + ReplicationSourceInterface source = mock(ReplicationSourceInterface.class); + when(source.getQueueId()).thenReturn("1"); + when(source.isRecovered()).thenReturn(false); + when(source.isSyncReplication()).thenReturn(false); + manager.logPositionAndCleanOldLogs(source, new WALEntryBatch(0, manager.getSources().get(0).getCurrentPath())); wal.append(hri, @@ -380,8 +394,8 @@ public abstract class TestReplicationSourceManager { // populate some znodes in the peer znode SortedSet files = new TreeSet<>(); String group = "testgroup"; - String file1 = group + ".log1"; - String file2 = group + ".log2"; + String file1 = group + "." + EnvironmentEdgeManager.currentTime() + ".log1"; + String file2 = group + "." + EnvironmentEdgeManager.currentTime() + ".log2"; files.add(file1); files.add(file2); for (String file : files) { @@ -397,7 +411,11 @@ public abstract class TestReplicationSourceManager { assertEquals(1, manager.getWalsByIdRecoveredQueues().size()); String id = "1-" + server.getServerName().getServerName(); assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id).get(group)); - manager.cleanOldLogs(file2, false, id, true); + ReplicationSourceInterface source = mock(ReplicationSourceInterface.class); + when(source.getQueueId()).thenReturn(id); + when(source.isRecovered()).thenReturn(true); + when(source.isSyncReplication()).thenReturn(false); + manager.cleanOldLogs(file2, false, source); // log1 should be deleted assertEquals(Sets.newHashSet(file2), manager.getWalsByIdRecoveredQueues().get(id).get(group)); } @@ -481,14 +499,13 @@ public abstract class TestReplicationSourceManager { * corresponding ReplicationSourceInterface correctly cleans up the corresponding * replication queue and ReplicationPeer. * See HBASE-16096. - * @throws Exception */ @Test public void testPeerRemovalCleanup() throws Exception{ String replicationSourceImplName = conf.get("replication.replicationsource.implementation"); final String peerId = "FakePeer"; - final ReplicationPeerConfig peerConfig = new ReplicationPeerConfig() - .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase"); + final ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() + .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build(); try { DummyServer server = new DummyServer(); ReplicationQueueStorage rq = ReplicationStorageFactory @@ -497,7 +514,7 @@ public abstract class TestReplicationSourceManager { // initialization to throw an exception. conf.set("replication.replicationsource.implementation", FailInitializeDummyReplicationSource.class.getName()); - final ReplicationPeers rp = manager.getReplicationPeers(); + manager.getReplicationPeers(); // Set up the znode and ReplicationPeer for the fake peer // Don't wait for replication source to initialize, we know it won't. addPeerAndWait(peerId, peerConfig, false); @@ -542,8 +559,8 @@ public abstract class TestReplicationSourceManager { @Test public void testRemovePeerMetricsCleanup() throws Exception { final String peerId = "DummyPeer"; - final ReplicationPeerConfig peerConfig = new ReplicationPeerConfig() - .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase"); + final ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder() + .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build(); try { MetricsReplicationSourceSource globalSource = getGlobalSource(); final int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue(); @@ -575,17 +592,68 @@ public abstract class TestReplicationSourceManager { } } + private ReplicationSourceInterface mockReplicationSource(String peerId) { + ReplicationSourceInterface source = mock(ReplicationSourceInterface.class); + when(source.getPeerId()).thenReturn(peerId); + when(source.getQueueId()).thenReturn(peerId); + when(source.isRecovered()).thenReturn(false); + when(source.isSyncReplication()).thenReturn(true); + ReplicationPeerConfig config = mock(ReplicationPeerConfig.class); + when(config.getRemoteWALDir()) + .thenReturn(remoteLogDir.makeQualified(fs.getUri(), fs.getWorkingDirectory()).toString()); + ReplicationPeer peer = mock(ReplicationPeer.class); + when(peer.getPeerConfig()).thenReturn(config); + when(source.getPeer()).thenReturn(peer); + return source; + } + + @Test + public void testRemoveRemoteWALs() throws Exception { + String peerId2 = slaveId + "_2"; + addPeerAndWait(peerId2, + ReplicationPeerConfig.newBuilder() + .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase").build(), + true); + try { + // make sure that we can deal with files which does not exist + String walNameNotExists = + "remoteWAL-12345-" + slaveId + ".12345" + ReplicationUtils.SYNC_WAL_SUFFIX; + Path wal = new Path(logDir, walNameNotExists); + manager.preLogRoll(wal); + manager.postLogRoll(wal); + + Path remoteLogDirForPeer = new Path(remoteLogDir, slaveId); + fs.mkdirs(remoteLogDirForPeer); + String walName = + "remoteWAL-12345-" + slaveId + ".23456" + ReplicationUtils.SYNC_WAL_SUFFIX; + Path remoteWAL = + new Path(remoteLogDirForPeer, walName).makeQualified(fs.getUri(), fs.getWorkingDirectory()); + fs.create(remoteWAL).close(); + wal = new Path(logDir, walName); + manager.preLogRoll(wal); + manager.postLogRoll(wal); + + ReplicationSourceInterface source = mockReplicationSource(peerId2); + manager.cleanOldLogs(walName, true, source); + // still there if peer id does not match + assertTrue(fs.exists(remoteWAL)); + + source = mockReplicationSource(slaveId); + manager.cleanOldLogs(walName, true, source); + assertFalse(fs.exists(remoteWAL)); + } finally { + removePeerAndWait(peerId2); + } + } + /** * Add a peer and wait for it to initialize - * @param peerId - * @param peerConfig * @param waitForSource Whether to wait for replication source to initialize - * @throws Exception */ private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig, final boolean waitForSource) throws Exception { final ReplicationPeers rp = manager.getReplicationPeers(); - rp.getPeerStorage().addPeer(peerId, peerConfig, true); + rp.getPeerStorage().addPeer(peerId, peerConfig, true, SyncReplicationState.NONE); try { manager.addPeer(peerId); } catch (Exception e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSyncReplicationShipperQuit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSyncReplicationShipperQuit.java new file mode 100644 index 0000000..f6dc3d7 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSyncReplicationShipperQuit.java @@ -0,0 +1,81 @@ +/** + * 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.replication.regionserver; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.hadoop.hbase.replication.SyncReplicationTestBase; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Testcase for HBASE-20456. + */ +@Category({ ReplicationTests.class, LargeTests.class }) +public class TestSyncReplicationShipperQuit extends SyncReplicationTestBase { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationShipperQuit.class); + + @Test + public void testShipperQuitWhenDA() throws Exception { + // set to serial replication + UTIL1.getAdmin().updateReplicationPeerConfig(PEER_ID, ReplicationPeerConfig + .newBuilder(UTIL1.getAdmin().getReplicationPeerConfig(PEER_ID)).setSerial(true).build()); + UTIL2.getAdmin().updateReplicationPeerConfig(PEER_ID, ReplicationPeerConfig + .newBuilder(UTIL2.getAdmin().getReplicationPeerConfig(PEER_ID)).setSerial(true).build()); + UTIL2.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.STANDBY); + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.ACTIVE); + + writeAndVerifyReplication(UTIL1, UTIL2, 0, 100); + HRegionServer rs = UTIL1.getRSForFirstRegionInTable(TABLE_NAME); + DualAsyncFSWAL wal = + (DualAsyncFSWAL) rs.getWAL(RegionInfoBuilder.newBuilder(TABLE_NAME).build()); + String walGroupId = + AbstractFSWALProvider.getWALPrefixFromWALName(wal.getCurrentFileName().getName()); + ReplicationSourceShipper shipper = + ((ReplicationSource) ((Replication) rs.getReplicationSourceService()).getReplicationManager() + .getSource(PEER_ID)).workerThreads.get(walGroupId); + assertFalse(shipper.isFinished()); + + UTIL1.getAdmin().transitReplicationPeerSyncReplicationState(PEER_ID, + SyncReplicationState.DOWNGRADE_ACTIVE); + writeAndVerifyReplication(UTIL1, UTIL2, 100, 200); + + ReplicationSource source = (ReplicationSource) ((Replication) rs.getReplicationSourceService()) + .getReplicationManager().getSource(PEER_ID); + // the peer is serial so here we can make sure that the previous wals have already been + // replicated, and finally the shipper should be removed from the worker pool + UTIL1.waitFor(10000, () -> !source.workerThreads.containsKey(walGroupId)); + assertTrue(shipper.isFinished()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java index 6299065..fd9ff29 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java @@ -28,6 +28,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellBuilder; @@ -127,7 +128,7 @@ public class TestWALEntrySinkFilter { conf.setClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY, IfTimeIsGreaterThanBOUNDARYWALEntrySinkFilterImpl.class, WALEntrySinkFilter.class); conf.setClass("hbase.client.connection.impl", DevNullConnection.class, - Connection.class); + Connection.class); ReplicationSink sink = new ReplicationSink(conf, STOPPABLE); // Create some dumb walentries. List< org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry > entries = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java index 35e4f82..fac6f74 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java @@ -413,9 +413,7 @@ public class TestWALEntryStream { batch = reader.take(); assertEquals(walPath, batch.getLastWalPath()); assertEquals(5, batch.getNbEntries()); - // Actually this should be true but we haven't handled this yet since for a normal queue the - // last one is always open... Not a big deal for now. - assertFalse(batch.isEndOfFile()); + assertTrue(batch.isEndOfFile()); assertSame(WALEntryBatch.NO_MORE_DATA, reader.take()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 870fa19..d2aa682 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -117,6 +117,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.Permission.Action; @@ -2941,6 +2942,21 @@ public class TestAccessController extends SecureTestUtil { } @Test + public void testTransitSyncReplicationPeerState() throws Exception { + AccessTestAction action = new AccessTestAction() { + @Override + public Object run() throws Exception { + ACCESS_CONTROLLER.preTransitReplicationPeerSyncReplicationState( + ObserverContextImpl.createAndPrepare(CP_ENV), "test", SyncReplicationState.NONE); + return null; + } + }; + + verifyAllowed(action, SUPERUSER, USER_ADMIN); + verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER); + } + + @Test public void testListReplicationPeers() throws Exception { AccessTestAction action = new AccessTestAction() { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java index 8911982..f5eca39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; @@ -67,9 +68,9 @@ public class TestHBaseFsckReplication { String peerId1 = "1"; String peerId2 = "2"; peerStorage.addPeer(peerId1, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(), - true); + true, SyncReplicationState.NONE); peerStorage.addPeer(peerId2, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(), - true); + true, SyncReplicationState.NONE); for (int i = 0; i < 10; i++) { queueStorage.addWAL(ServerName.valueOf("localhost", 10000 + i, 100000 + i), peerId1, "file-" + i); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java index 01f0dc6..453b742 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java @@ -103,8 +103,6 @@ public class IOTestProvider implements WALProvider { this.factory = factory; this.conf = conf; this.providerId = providerId != null ? providerId : DEFAULT_PROVIDER_ID; - - } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java new file mode 100644 index 0000000..8189cef --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java @@ -0,0 +1,172 @@ +/** + * 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.wal; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import java.io.IOException; +import java.util.Optional; +import java.util.function.BiPredicate; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; +import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL; +import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader; +import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper; +import org.apache.hadoop.hbase.replication.SyncReplicationState; +import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerInfoProvider; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestSyncReplicationWALProvider { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestSyncReplicationWALProvider.class); + + private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); + + private static String PEER_ID = "1"; + + private static String REMOTE_WAL_DIR = "/RemoteWAL"; + + private static TableName TABLE = TableName.valueOf("table"); + + private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep"); + + private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build(); + + private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build(); + + private static WALFactory FACTORY; + + public static final class InfoProvider implements SyncReplicationPeerInfoProvider { + + @Override + public Optional> getPeerIdAndRemoteWALDir(TableName table) { + if (table != null && table.equals(TABLE)) { + return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR)); + } else { + return Optional.empty(); + } + } + + @Override + public boolean checkState(TableName table, + BiPredicate checker) { + return false; + } + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + UTIL.startMiniDFSCluster(3); + FACTORY = new WALFactory(UTIL.getConfiguration(), "test"); + ((SyncReplicationWALProvider) FACTORY.getWALProvider()).setPeerInfoProvider(new InfoProvider()); + UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID)); + } + + @AfterClass + public static void tearDownAfterClass() throws IOException { + FACTORY.close(); + UTIL.shutdownMiniDFSCluster(); + } + + private void testReadWrite(DualAsyncFSWAL wal) throws Exception { + int recordCount = 100; + int columnCount = 10; + byte[] row = Bytes.toBytes("testRow"); + long timestamp = System.currentTimeMillis(); + MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl(); + ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp, + mvcc); + Path localFile = wal.getCurrentFileName(); + Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName()); + try (ProtobufLogReader reader = + (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) { + ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row, + timestamp); + } + try (ProtobufLogReader reader = + (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) { + ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row, + timestamp); + } + wal.rollWriter(); + DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem(); + UTIL.waitFor(5000, new ExplainingPredicate() { + + @Override + public boolean evaluate() throws Exception { + return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile); + } + + @Override + public String explainFailure() throws Exception { + StringBuilder sb = new StringBuilder(); + if (!dfs.isFileClosed(localFile)) { + sb.append(localFile + " has not been closed yet."); + } + if (!dfs.isFileClosed(remoteFile)) { + sb.append(remoteFile + " has not been closed yet."); + } + return sb.toString(); + } + }); + try (ProtobufLogReader reader = + (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) { + ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row, + timestamp); + } + try (ProtobufLogReader reader = + (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) { + ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row, + timestamp); + } + } + + @Test + public void test() throws Exception { + WAL walNoRep = FACTORY.getWAL(REGION_NO_REP); + assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class))); + DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION); + assertEquals(2, FACTORY.getWALs().size()); + testReadWrite(wal); + SyncReplicationWALProvider walProvider = (SyncReplicationWALProvider) FACTORY.getWALProvider(); + walProvider.peerSyncReplicationStateChange(PEER_ID, SyncReplicationState.ACTIVE, + SyncReplicationState.DOWNGRADE_ACTIVE, 1); + assertEquals(1, FACTORY.getWALs().size()); + } +} diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb index 5b87595..5f86365 100644 --- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb +++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb @@ -20,6 +20,7 @@ include Java java_import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil +java_import org.apache.hadoop.hbase.replication.SyncReplicationState java_import org.apache.hadoop.hbase.replication.ReplicationPeerConfig java_import org.apache.hadoop.hbase.util.Bytes java_import org.apache.hadoop.hbase.zookeeper.ZKConfig @@ -64,16 +65,20 @@ module Hbase table_cfs = args.fetch(TABLE_CFS, nil) namespaces = args.fetch(NAMESPACES, nil) peer_state = args.fetch(STATE, nil) + remote_wal_dir = args.fetch(REMOTE_WAL_DIR, nil) # Create and populate a ReplicationPeerConfig - builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig - .newBuilder() + builder = ReplicationPeerConfig.newBuilder() builder.set_cluster_key(cluster_key) unless endpoint_classname.nil? builder.set_replication_endpoint_impl(endpoint_classname) end + unless remote_wal_dir.nil? + builder.setRemoteWALDir(remote_wal_dir) + end + unless config.nil? builder.putAllConfiguration(config) end @@ -228,8 +233,7 @@ module Hbase namespaces.each do |n| ns_set.add(n) end - builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig - .newBuilder(rpc) + builder = ReplicationPeerConfig.newBuilder(rpc) builder.setNamespaces(ns_set) @admin.updateReplicationPeerConfig(id, builder.build) end @@ -248,8 +252,7 @@ module Hbase ns_set.remove(n) end end - builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig - .newBuilder(rpc) + builder = ReplicationPeerConfig.newBuilder(rpc) builder.setNamespaces(ns_set) @admin.updateReplicationPeerConfig(id, builder.build) end @@ -336,6 +339,20 @@ module Hbase '!' + ReplicationPeerConfigUtil.convertToString(tableCFs) end + # Transit current cluster to a new state in the specified synchronous + # replication peer + def transit_peer_sync_replication_state(id, state) + if 'ACTIVE'.eql?(state) + @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::ACTIVE) + elsif 'DOWNGRADE_ACTIVE'.eql?(state) + @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::DOWNGRADE_ACTIVE) + elsif 'STANDBY'.eql?(state) + @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::STANDBY) + else + raise(ArgumentError, 'synchronous replication state must be ACTIVE, DOWNGRADE_ACTIVE or STANDBY') + end + end + #---------------------------------------------------------------------------------------------- # Enables a table's replication switch def enable_tablerep(table_name) diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb index 28484cb..2870dfb 100644 --- a/hbase-shell/src/main/ruby/hbase_constants.rb +++ b/hbase-shell/src/main/ruby/hbase_constants.rb @@ -77,6 +77,7 @@ module HBaseConstants VALUE = 'VALUE'.freeze ENDPOINT_CLASSNAME = 'ENDPOINT_CLASSNAME'.freeze CLUSTER_KEY = 'CLUSTER_KEY'.freeze + REMOTE_WAL_DIR = 'REMOTE_WAL_DIR'.freeze TABLE_CFS = 'TABLE_CFS'.freeze NAMESPACES = 'NAMESPACES'.freeze STATE = 'STATE'.freeze diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb index 9a79658..934fa11 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -393,6 +393,7 @@ Shell.load_command_group( get_peer_config list_peer_configs update_peer_config + transit_peer_sync_replication_state ] ) diff --git a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb index eb2da83..4b6f294 100644 --- a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb +++ b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb @@ -35,7 +35,7 @@ to the peer cluster. An optional parameter for table column families identifies which tables and/or column families will be replicated to the peer cluster. -Notice: Set a namespace in the peer config means that all tables in this namespace +Note: Set a namespace in the peer config means that all tables in this namespace will be replicated to the peer cluster. So if you already have set a namespace in peer config, then you can't set this namespace's tables in the peer config again. @@ -74,6 +74,25 @@ the key TABLE_CFS. Note: Either CLUSTER_KEY or ENDPOINT_CLASSNAME must be specified. If ENDPOINT_CLASSNAME is specified, CLUSTER_KEY is optional and should only be specified if a particular custom endpoint requires it. +The default replication peer is asynchronous. You can also add a synchronous replication peer +with REMOTE_WAL_DIR parameter. Meanwhile, synchronous replication peer also support other optional +config for asynchronous replication peer. + +Examples: + + hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase", + REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase" + hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase", + STATE => "ENABLED", REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase" + hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase", + STATE => "DISABLED", REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase" + hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase", + REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase", NAMESPACES => ["ns1", "ns2"] + hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase", + REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase", TABLE_CFS => { "table1" => [] } + +Note: The REMOTE_WAL_DIR is not allowed to change. + EOF end diff --git a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb index eefcc42..f2ec014 100644 --- a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb +++ b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb @@ -39,7 +39,8 @@ EOF peers = replication_admin.list_peers formatter.header(%w[PEER_ID CLUSTER_KEY ENDPOINT_CLASSNAME - STATE REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH + REMOTE_ROOT_DIR SYNC_REPLICATION_STATE STATE + REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH SERIAL]) peers.each do |peer| @@ -53,8 +54,20 @@ EOF namespaces = replication_admin.show_peer_namespaces(config) tableCFs = replication_admin.show_peer_tableCFs_by_config(config) end - formatter.row([id, config.getClusterKey, - config.getReplicationEndpointImpl, state, + cluster_key = 'nil' + unless config.getClusterKey.nil? + cluster_key = config.getClusterKey + end + endpoint_classname = 'nil' + unless config.getReplicationEndpointImpl.nil? + endpoint_classname = config.getReplicationEndpointImpl + end + remote_root_dir = 'nil' + unless config.getRemoteWALDir.nil? + remote_root_dir = config.getRemoteWALDir + end + formatter.row([id, cluster_key, endpoint_classname, + remote_root_dir, peer.getSyncReplicationState, state, config.replicateAllUserTables, namespaces, tableCFs, config.getBandwidth, config.isSerial]) end diff --git a/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb b/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb new file mode 100644 index 0000000..6d4a963 --- /dev/null +++ b/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb @@ -0,0 +1,44 @@ +# +# Copyright The Apache Software Foundation +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +module Shell + module Commands + class TransitPeerSyncReplicationState < Command + def help + <<-EOF +Transit current cluster to new state in the specified synchronous replication peer. +Examples: + + # Transit cluster state to DOWNGRADE_ACTIVE in a synchronous replication peer + hbase> transit_peer_sync_replication_state '1', 'DOWNGRADE_ACTIVE' + # Transit cluster state to ACTIVE in a synchronous replication peer + hbase> transit_peer_sync_replication_state '1', 'ACTIVE' + # Transit cluster state to STANDBY in a synchronous replication peer + hbase> transit_peer_sync_replication_state '1', 'STANDBY' + +EOF + end + + def command(id, state) + replication_admin.transit_peer_sync_replication_state(id, state) + end + end + end +end diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb index 29de710..9d364ce 100644 --- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb +++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb @@ -23,6 +23,9 @@ require 'hbase/hbase' require 'hbase/table' include HBaseConstants +include Java + +java_import org.apache.hadoop.hbase.replication.SyncReplicationState module Hbase class ReplicationAdminTest < Test::Unit::TestCase @@ -97,6 +100,22 @@ module Hbase command(:remove_peer, @peer_id) end + define_test "add_peer: remote wal dir" do + cluster_key = "server1.cie.com:2181:/hbase" + remote_wal_dir = "hdfs://srv1:9999/hbase" + args = { CLUSTER_KEY => cluster_key, REMOTE_WAL_DIR => remote_wal_dir } + command(:add_peer, @peer_id, args) + + assert_equal(1, command(:list_peers).length) + peer = command(:list_peers).get(0) + assert_equal(@peer_id, peer.getPeerId) + assert_equal(cluster_key, peer.getPeerConfig.getClusterKey) + assert_equal(remote_wal_dir, peer.getPeerConfig.getRemoteWALDir) + + # cleanup for future tests + command(:remove_peer, @peer_id) + end + define_test "add_peer: single zk cluster key with enabled/disabled state" do cluster_key = "server1.cie.com:2181:/hbase" @@ -497,6 +516,27 @@ module Hbase command(:remove_peer, @peer_id) end + define_test "transit_peer_sync_replication_state: test" do + cluster_key = "server1.cie.com:2181:/hbase" + remote_wal_dir = "hdfs://srv1:9999/hbase" + args = { CLUSTER_KEY => cluster_key, REMOTE_WAL_DIR => remote_wal_dir } + command(:add_peer, @peer_id, args) + + assert_equal(1, command(:list_peers).length) + peer = command(:list_peers).get(0) + assert_equal(@peer_id, peer.getPeerId) + assert_equal(SyncReplicationState::DOWNGRADE_ACTIVE, peer.getSyncReplicationState) + + command(:transit_peer_sync_replication_state, @peer_id, 'ACTIVE') + assert_equal(1, command(:list_peers).length) + peer = command(:list_peers).get(0) + assert_equal(@peer_id, peer.getPeerId) + assert_equal(SyncReplicationState::ACTIVE, peer.getSyncReplicationState) + + # cleanup for future tests + command(:remove_peer, @peer_id) + end + define_test "get_peer_config: works with simple clusterKey peer" do cluster_key = "localhost:2181:/hbase-test" args = { CLUSTER_KEY => cluster_key } diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc index 60b749f..d2166e8 100644 --- a/src/main/asciidoc/_chapters/ops_mgt.adoc +++ b/src/main/asciidoc/_chapters/ops_mgt.adoc @@ -1473,6 +1473,9 @@ Some use cases for cluster replication include: NOTE: Replication is enabled at the granularity of the column family. Before enabling replication for a column family, create the table and all column families to be replicated, on the destination cluster. +NOTE: Replication is asynchronous as we send WAL to another cluster in background, which means that when you want to do recovery through replication, you could loss some data. To address this problem, we have introduced a new feature called synchronous replication. As the mechanism is a bit different so we use a separated section to describe it. Please see +<>. + === Replication Overview Cluster replication uses a source-push methodology. diff --git a/src/main/asciidoc/_chapters/sync_replication.adoc b/src/main/asciidoc/_chapters/sync_replication.adoc new file mode 100644 index 0000000..d28b9a9 --- /dev/null +++ b/src/main/asciidoc/_chapters/sync_replication.adoc @@ -0,0 +1,125 @@ +//// +/** + * + * 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. + */ +//// + +[[syncreplication]] += Synchronous Replication +:doctype: book +:numbered: +:toc: left +:icons: font +:experimental: +:source-language: java + +== Background + +The current <> in HBase in asynchronous. So if the master cluster crashes, the slave cluster may not have the +newest data. If users want strong consistency then they can not switch to the slave cluster. + +== Design + +Please see the design doc on link:https://issues.apache.org/jira/browse/HBASE-19064[HBASE-19064] + +== Operation and maintenance + +Case.1 Setup two synchronous replication clusters:: + +* Add a synchronous peer in both source cluster and peer cluster. + +For source cluster: +[source,ruby] +---- +hbase> add_peer '1', CLUSTER_KEY => 'lg-hadoop-tst-st01.bj:10010,lg-hadoop-tst-st02.bj:10010,lg-hadoop-tst-st03.bj:10010:/hbase/test-hbase-slave', REMOTE_WAL_DIR=>'hdfs://lg-hadoop-tst-st01.bj:20100/hbase/test-hbase-slave/remoteWALs', TABLE_CFS => {"ycsb-test"=>[]} +---- + +For peer cluster: +[source,ruby] +---- +hbase> add_peer '1', CLUSTER_KEY => 'lg-hadoop-tst-st01.bj:10010,lg-hadoop-tst-st02.bj:10010,lg-hadoop-tst-st03.bj:10010:/hbase/test-hbase', REMOTE_WAL_DIR=>'hdfs://lg-hadoop-tst-st01.bj:20100/hbase/test-hbase/remoteWALs', TABLE_CFS => {"ycsb-test"=>[]} +---- + +NOTE: For synchronous replication, the current implementation require that we have the same peer id for both source +and peer cluster. Another thing that need attention is: the peer does not support cluster-level, namespace-level, or +cf-level replication, only support table-level replication now. + +* Transit the peer cluster to be STANDBY state + +[source,ruby] +---- +hbase> transit_peer_sync_replication_state '1', 'STANDBY' +---- + +* Transit the source cluster to be ACTIVE state + +[source,ruby] +---- +hbase> transit_peer_sync_replication_state '1', 'ACTIVE' +---- + +Now, the synchronous replication has been set up successfully. the HBase client can only request to source cluster, if +request to peer cluster, the peer cluster which is STANDBY state now will reject the read/write requests. + +Case.2 How to operate when standby cluster crashed:: + +If the standby cluster has been crashed, it will fail to write remote WAL for the active cluster. So we need to transit +the source cluster to DOWNGRANDE_ACTIVE state, which means source cluster won't write any remote WAL any more, but +the normal replication (asynchronous Replication) can still work fine, it queue the newly written WALs, but the +replication block until the peer cluster come back. + +[source,ruby] +---- +hbase> transit_peer_sync_replication_state '1', 'DOWNGRADE_ACTIVE' +---- + +Once the peer cluster come back, we can just transit the source cluster to ACTIVE, to ensure that the replication will be +synchronous. + +[source,ruby] +---- +hbase> transit_peer_sync_replication_state '1', 'ACTIVE' +---- + +Case.3 How to operate when active cluster crashed:: + +If the active cluster has been crashed (it may be not reachable now), so let's just transit the standby cluster to +DOWNGRANDE_ACTIVE state, and after that, we should redirect all the requests from client to the DOWNGRADE_ACTIVE cluster. + +[source,ruby] +---- +hbase> transit_peer_sync_replication_state '1', 'DOWNGRADE_ACTIVE' +---- + +If the crashed cluster come back again, we just need to transit it to STANDBY directly. Otherwise if you transit the +cluster to DOWNGRADE_ACTIVE, the original ACTIVE cluster may have redundant data compared to the current ACTIVE +cluster. Because we designed to write source cluster WALs and remote cluster WALs concurrently, so it's possible that +the source cluster WALs has more data than the remote cluster, which result in data inconsistency. The procedure of +transiting ACTIVE to STANDBY has no problem, because we'll skip to replay the original WALs. + +[source,ruby] +---- +hbase> transit_peer_sync_replication_state '1', 'STANDBY' +---- + +After that, we can promote the DOWNGRADE_ACTIVE cluster to ACTIVE now, to ensure that the replication will be synchronous. + +[source,ruby] +---- +hbase> transit_peer_sync_replication_state '1', 'ACTIVE' +---- diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc index 02e488d..db95b21 100644 --- a/src/main/asciidoc/book.adoc +++ b/src/main/asciidoc/book.adoc @@ -64,6 +64,7 @@ include::_chapters/architecture.adoc[] include::_chapters/hbase_mob.adoc[] include::_chapters/inmemory_compaction.adoc[] include::_chapters/backup_restore.adoc[] +include::_chapters/sync_replication.adoc[] include::_chapters/hbase_apis.adoc[] include::_chapters/external_apis.adoc[] include::_chapters/thrift_filter_language.adoc[]