Merge branch 'HDFS-12943' into trunk

This commit is contained in:
Konstantin V Shvachko 2018-12-24 10:00:02 -08:00
commit 47dd00a82e
99 changed files with 5540 additions and 158 deletions

View File

@ -129,7 +129,7 @@ public class FailoverController {
if (!toSvcStatus.getState().equals(HAServiceState.STANDBY)) {
throw new FailoverFailedException(
"Can't failover to an active service");
"Can't failover to an " + toSvcStatus.getState() + " service");
}
if (!toSvcStatus.isReadyToBecomeActive()) {

View File

@ -72,6 +72,9 @@ public abstract class HAAdmin extends Configured implements Tool {
new UsageInfo("[--"+FORCEACTIVE+"] <serviceId>", "Transitions the service into Active state"))
.put("-transitionToStandby",
new UsageInfo("<serviceId>", "Transitions the service into Standby state"))
.put("-transitionToObserver",
new UsageInfo("<serviceId>",
"Transitions the service into Observer state"))
.put("-failover",
new UsageInfo("[--"+FORCEFENCE+"] [--"+FORCEACTIVE+"] <serviceId> <serviceId>",
"Failover from the first service to the second.\n" +
@ -221,6 +224,28 @@ public abstract class HAAdmin extends Configured implements Tool {
HAServiceProtocolHelper.transitionToStandby(proto, createReqInfo());
return 0;
}
private int transitionToObserver(final CommandLine cmd)
throws IOException, ServiceFailedException {
String[] argv = cmd.getArgs();
if (argv.length != 1) {
errOut.println("transitionToObserver: incorrect number of arguments");
printUsage(errOut, "-transitionToObserver");
return -1;
}
HAServiceTarget target = resolveTarget(argv[0]);
if (!checkSupportObserver(target)) {
return -1;
}
if (!checkManualStateManagementOK(target)) {
return -1;
}
HAServiceProtocol proto = target.getProxy(getConf(), 0);
HAServiceProtocolHelper.transitionToObserver(proto, createReqInfo());
return 0;
}
/**
* Ensure that we are allowed to manually manage the HA state of the target
* service. If automatic failover is configured, then the automatic
@ -249,6 +274,21 @@ public abstract class HAAdmin extends Configured implements Tool {
return true;
}
/**
* Check if the target supports the Observer state.
* @param target the target to check
* @return true if the target support Observer state, false otherwise.
*/
private boolean checkSupportObserver(HAServiceTarget target) {
if (target.supportObserver()) {
return true;
} else {
errOut.println(
"The target " + target + " doesn't support Observer state.");
return false;
}
}
private StateChangeRequestInfo createReqInfo() {
return new StateChangeRequestInfo(requestSource);
}
@ -436,6 +476,7 @@ public abstract class HAAdmin extends Configured implements Tool {
// Mutative commands take FORCEMANUAL option
if ("-transitionToActive".equals(cmd) ||
"-transitionToStandby".equals(cmd) ||
"-transitionToObserver".equals(cmd) ||
"-failover".equals(cmd)) {
opts.addOption(FORCEMANUAL, false,
"force manual control even if auto-failover is enabled");
@ -461,6 +502,8 @@ public abstract class HAAdmin extends Configured implements Tool {
return transitionToActive(cmdLine);
} else if ("-transitionToStandby".equals(cmd)) {
return transitionToStandby(cmdLine);
} else if ("-transitionToObserver".equals(cmd)) {
return transitionToObserver(cmdLine);
} else if ("-failover".equals(cmd)) {
return failover(cmdLine);
} else if ("-getServiceState".equals(cmd)) {

View File

@ -51,6 +51,7 @@ public interface HAServiceProtocol {
INITIALIZING("initializing"),
ACTIVE("active"),
STANDBY("standby"),
OBSERVER("observer"),
STOPPING("stopping");
private String name;
@ -148,6 +149,23 @@ public interface HAServiceProtocol {
AccessControlException,
IOException;
/**
* Request service to transition to observer state. No operation, if the
* service is already in observer state.
*
* @throws ServiceFailedException
* if transition from standby to observer fails.
* @throws AccessControlException
* if access is denied.
* @throws IOException
* if other errors happen
*/
@Idempotent
void transitionToObserver(StateChangeRequestInfo reqInfo)
throws ServiceFailedException,
AccessControlException,
IOException;
/**
* Return the current status of the service. The status indicates
* the current <em>state</em> (e.g ACTIVE/STANDBY) as well as

View File

@ -60,4 +60,13 @@ public class HAServiceProtocolHelper {
throw e.unwrapRemoteException(ServiceFailedException.class);
}
}
public static void transitionToObserver(HAServiceProtocol svc,
StateChangeRequestInfo reqInfo) throws IOException {
try {
svc.transitionToObserver(reqInfo);
} catch (RemoteException e) {
throw e.unwrapRemoteException(ServiceFailedException.class);
}
}
}

View File

@ -170,4 +170,11 @@ public abstract class HAServiceTarget {
public boolean isAutoFailoverEnabled() {
return false;
}
/**
* @return true if this target supports the Observer state, false otherwise.
*/
public boolean supportObserver() {
return false;
}
}

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.MonitorHealthRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToObserverRequestProto;
import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
import org.apache.hadoop.ipc.ProtocolTranslator;
@ -115,6 +116,19 @@ public class HAServiceProtocolClientSideTranslatorPB implements
}
}
@Override
public void transitionToObserver(StateChangeRequestInfo reqInfo)
throws IOException {
try {
TransitionToObserverRequestProto req =
TransitionToObserverRequestProto.newBuilder()
.setReqInfo(convert(reqInfo)).build();
rpcProxy.transitionToObserver(NULL_CONTROLLER, req);
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public HAServiceStatus getServiceStatus() throws IOException {
GetServiceStatusResponseProto status;
@ -141,6 +155,8 @@ public class HAServiceProtocolClientSideTranslatorPB implements
return HAServiceState.ACTIVE;
case STANDBY:
return HAServiceState.STANDBY;
case OBSERVER:
return HAServiceState.OBSERVER;
case INITIALIZING:
default:
return HAServiceState.INITIALIZING;

View File

@ -35,6 +35,8 @@ import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveRequ
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToActiveResponseProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToStandbyResponseProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToObserverRequestProto;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.TransitionToObserverResponseProto;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
@ -61,6 +63,9 @@ public class HAServiceProtocolServerSideTranslatorPB implements
TransitionToActiveResponseProto.newBuilder().build();
private static final TransitionToStandbyResponseProto TRANSITION_TO_STANDBY_RESP =
TransitionToStandbyResponseProto.newBuilder().build();
private static final TransitionToObserverResponseProto
TRANSITION_TO_OBSERVER_RESP =
TransitionToObserverResponseProto.newBuilder().build();
private static final Logger LOG = LoggerFactory.getLogger(
HAServiceProtocolServerSideTranslatorPB.class);
@ -123,6 +128,18 @@ public class HAServiceProtocolServerSideTranslatorPB implements
}
}
@Override
public TransitionToObserverResponseProto transitionToObserver(
RpcController controller, TransitionToObserverRequestProto request)
throws ServiceException {
try {
server.transitionToObserver(convert(request.getReqInfo()));
return TRANSITION_TO_OBSERVER_RESP;
} catch (IOException e) {
throw new ServiceException(e);
}
}
@Override
public GetServiceStatusResponseProto getServiceStatus(RpcController controller,
GetServiceStatusRequestProto request) throws ServiceException {
@ -141,6 +158,9 @@ public class HAServiceProtocolServerSideTranslatorPB implements
case STANDBY:
retState = HAServiceStateProto.STANDBY;
break;
case OBSERVER:
retState = HAServiceStateProto.OBSERVER;
break;
case INITIALIZING:
default:
retState = HAServiceStateProto.INITIALIZING;

View File

@ -448,4 +448,9 @@ public class RetryInvocationHandler<T> implements RpcInvocationHandler {
public ConnectionId getConnectionId() {
return RPC.getConnectionIdForProxy(proxyDescriptor.getProxy());
}
@VisibleForTesting
public FailoverProxyProvider<T> getProxyProvider() {
return proxyDescriptor.fpp;
}
}

View File

@ -0,0 +1,96 @@
/**
* 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.ipc;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
/**
* This interface intends to align the state between client and server
* via RPC communication.
*
* This should be implemented separately on the client side and server side
* and can be used to pass state information on RPC responses from server
* to client.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface AlignmentContext {
/**
* This is the intended server method call to implement to pass state info
* during RPC response header construction.
*
* @param header The RPC response header builder.
*/
void updateResponseState(RpcResponseHeaderProto.Builder header);
/**
* This is the intended client method call to implement to recieve state info
* during RPC response processing.
*
* @param header The RPC response header.
*/
void receiveResponseState(RpcResponseHeaderProto header);
/**
* This is the intended client method call to pull last seen state info
* into RPC request processing.
*
* @param header The RPC request header builder.
*/
void updateRequestState(RpcRequestHeaderProto.Builder header);
/**
* This is the intended server method call to implement to receive
* client state info during RPC response header processing.
*
* @param header The RPC request header.
* @param threshold a parameter to verify a condition when server
* should reject client request due to its state being too far
* misaligned with the client state.
* See implementation for more details.
* @return state id required for the server to execute the call.
* @throws IOException
*/
long receiveRequestState(RpcRequestHeaderProto header, long threshold)
throws IOException;
/**
* Returns the last seen state id of the alignment context instance.
*
* @return the value of the last seen state id.
*/
long getLastSeenStateId();
/**
* Return true if this method call does need to be synced, false
* otherwise. sync meaning server state needs to have caught up with
* client state.
*
* @param protocolName the name of the protocol
* @param method the method call to check
* @return true if this method is async, false otherwise.
*/
boolean isCoordinatedCall(String protocolName, String method);
}

View File

@ -339,6 +339,7 @@ public class Client implements AutoCloseable {
final RPC.RpcKind rpcKind; // Rpc EngineKind
boolean done; // true when call is done
private final Object externalHandler;
private AlignmentContext alignmentContext;
private Call(RPC.RpcKind rpcKind, Writable param) {
this.rpcKind = rpcKind;
@ -380,6 +381,15 @@ public class Client implements AutoCloseable {
}
}
/**
* Set an AlignmentContext for the call to update when call is done.
*
* @param ac alignment context to update.
*/
public synchronized void setAlignmentContext(AlignmentContext ac) {
this.alignmentContext = ac;
}
/** Set the exception when there is an error.
* Notify the caller the call is done.
*
@ -1108,7 +1118,7 @@ public class Client implements AutoCloseable {
// Items '1' and '2' are prepared here.
RpcRequestHeaderProto header = ProtoUtil.makeRpcRequestHeader(
call.rpcKind, OperationProto.RPC_FINAL_PACKET, call.id, call.retry,
clientId);
clientId, call.alignmentContext);
final ResponseBuffer buf = new ResponseBuffer();
header.writeDelimitedTo(buf);
@ -1185,6 +1195,9 @@ public class Client implements AutoCloseable {
Writable value = packet.newInstance(valueClass, conf);
final Call call = calls.remove(callId);
call.setRpcResponse(value);
if (call.alignmentContext != null) {
call.alignmentContext.receiveResponseState(header);
}
}
// verify that packet length was correct
if (packet.remaining() > 0) {
@ -1365,7 +1378,15 @@ public class Client implements AutoCloseable {
ConnectionId remoteId, AtomicBoolean fallbackToSimpleAuth)
throws IOException {
return call(rpcKind, rpcRequest, remoteId, RPC.RPC_SERVICE_CLASS_DEFAULT,
fallbackToSimpleAuth);
fallbackToSimpleAuth, null);
}
public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId, AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
return call(rpcKind, rpcRequest, remoteId, RPC.RPC_SERVICE_CLASS_DEFAULT,
fallbackToSimpleAuth, alignmentContext);
}
private void checkAsyncCall() throws IOException {
@ -1382,6 +1403,14 @@ public class Client implements AutoCloseable {
}
}
Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId, int serviceClass,
AtomicBoolean fallbackToSimpleAuth)
throws IOException {
return call(rpcKind, rpcRequest, remoteId, serviceClass,
fallbackToSimpleAuth, null);
}
/**
* Make a call, passing <code>rpcRequest</code>, to the IPC server defined by
* <code>remoteId</code>, returning the rpc response.
@ -1392,14 +1421,17 @@ public class Client implements AutoCloseable {
* @param serviceClass - service class for RPC
* @param fallbackToSimpleAuth - set to true or false during this method to
* indicate if a secure client falls back to simple auth
* @param alignmentContext - state alignment context
* @return the rpc response
* Throws exceptions if there are network problems or if the remote code
* threw an exception.
*/
Writable call(RPC.RpcKind rpcKind, Writable rpcRequest,
ConnectionId remoteId, int serviceClass,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
final Call call = createCall(rpcKind, rpcRequest);
call.setAlignmentContext(alignmentContext);
final Connection connection = getConnection(remoteId, call, serviceClass,
fallbackToSimpleAuth);

View File

@ -24,6 +24,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.ipc.Server.Call;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcStatusProto;
import org.apache.hadoop.security.UserGroupInformation;
public abstract class ExternalCall<T> extends Call {
@ -78,7 +79,7 @@ public abstract class ExternalCall<T> extends Call {
}
@Override
final void doResponse(Throwable t) {
final void doResponse(Throwable t, RpcStatusProto status) {
synchronized(done) {
error = t;
done.set(true);

View File

@ -0,0 +1,37 @@
/**
* 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.ipc;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import java.io.IOException;
/**
* Thrown by a remote ObserverNode indicating the operation has failed and the
* client should retry active namenode directly (instead of retry other
* ObserverNodes).
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ObserverRetryOnActiveException extends IOException {
static final long serialVersionUID = 1L;
public ObserverRetryOnActiveException(String msg) {
super(msg);
}
}

View File

@ -86,7 +86,7 @@ public class ProtobufRpcEngine implements RpcEngine {
SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy
) throws IOException {
return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, null);
rpcTimeout, connectionRetryPolicy, null, null);
}
@Override
@ -94,10 +94,12 @@ public class ProtobufRpcEngine implements RpcEngine {
public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
final Invoker invoker = new Invoker(protocol, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, fallbackToSimpleAuth);
rpcTimeout, connectionRetryPolicy, fallbackToSimpleAuth,
alignmentContext);
return new ProtocolProxy<T>(protocol, (T) Proxy.newProxyInstance(
protocol.getClassLoader(), new Class[]{protocol}, invoker), false);
}
@ -122,15 +124,18 @@ public class ProtobufRpcEngine implements RpcEngine {
private final long clientProtocolVersion;
private final String protocolName;
private AtomicBoolean fallbackToSimpleAuth;
private AlignmentContext alignmentContext;
private Invoker(Class<?> protocol, InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf, SocketFactory factory,
int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
this(protocol, Client.ConnectionId.getConnectionId(
addr, protocol, ticket, rpcTimeout, connectionRetryPolicy, conf),
conf, factory);
this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.alignmentContext = alignmentContext;
}
/**
@ -227,7 +232,7 @@ public class ProtobufRpcEngine implements RpcEngine {
try {
val = (RpcWritable.Buffer) client.call(RPC.RpcKind.RPC_PROTOCOL_BUFFER,
new RpcProtobufRequest(rpcRequestHeader, theRequest), remoteId,
fallbackToSimpleAuth);
fallbackToSimpleAuth, alignmentContext);
} catch (Throwable e) {
if (LOG.isTraceEnabled()) {
@ -337,11 +342,11 @@ public class ProtobufRpcEngine implements RpcEngine {
String bindAddress, int port, int numHandlers, int numReaders,
int queueSizePerHandler, boolean verbose, Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager,
String portRangeConfig)
String portRangeConfig, AlignmentContext alignmentContext)
throws IOException {
return new Server(protocol, protocolImpl, conf, bindAddress, port,
numHandlers, numReaders, queueSizePerHandler, verbose, secretManager,
portRangeConfig);
portRangeConfig, alignmentContext);
}
public static class Server extends RPC.Server {
@ -410,18 +415,19 @@ public class ProtobufRpcEngine implements RpcEngine {
* @param numHandlers the number of method handler threads to run
* @param verbose whether each call should be logged
* @param portRangeConfig A config parameter that can be used to restrict
* the range of ports used when port is 0 (an ephemeral port)
* @param alignmentContext provides server state info on client responses
*/
public Server(Class<?> protocolClass, Object protocolImpl,
Configuration conf, String bindAddress, int port, int numHandlers,
int numReaders, int queueSizePerHandler, boolean verbose,
SecretManager<? extends TokenIdentifier> secretManager,
String portRangeConfig)
String portRangeConfig, AlignmentContext alignmentContext)
throws IOException {
super(bindAddress, port, null, numHandlers,
numReaders, queueSizePerHandler, conf,
serverNameFromClass(protocolImpl.getClass()), secretManager,
portRangeConfig);
setAlignmentContext(alignmentContext);
this.verbose = verbose;
registerProtocolAndImpl(RPC.RpcKind.RPC_PROTOCOL_BUFFER, protocolClass,
protocolImpl);

View File

@ -586,7 +586,44 @@ public class RPC {
}
return getProtocolEngine(protocol, conf).getProxy(protocol, clientVersion,
addr, ticket, conf, factory, rpcTimeout, connectionRetryPolicy,
fallbackToSimpleAuth);
fallbackToSimpleAuth, null);
}
/**
* Get a protocol proxy that contains a proxy connection to a remote server
* and a set of methods that are supported by the server.
*
* @param protocol protocol
* @param clientVersion client's version
* @param addr server address
* @param ticket security ticket
* @param conf configuration
* @param factory socket factory
* @param rpcTimeout max time for each rpc; 0 means no timeout
* @param connectionRetryPolicy retry policy
* @param fallbackToSimpleAuth set to true or false during calls to indicate
* if a secure client falls back to simple auth
* @param alignmentContext state alignment context
* @return the proxy
* @throws IOException if any error occurs
*/
public static <T> ProtocolProxy<T> getProtocolProxy(Class<T> protocol,
long clientVersion,
InetSocketAddress addr,
UserGroupInformation ticket,
Configuration conf,
SocketFactory factory,
int rpcTimeout,
RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
if (UserGroupInformation.isSecurityEnabled()) {
SaslRpcServer.init(conf);
}
return getProtocolEngine(protocol, conf).getProxy(protocol, clientVersion,
addr, ticket, conf, factory, rpcTimeout, connectionRetryPolicy,
fallbackToSimpleAuth, alignmentContext);
}
/**
@ -719,6 +756,7 @@ public class RPC {
private final Configuration conf;
private SecretManager<? extends TokenIdentifier> secretManager = null;
private String portRangeConfig = null;
private AlignmentContext alignmentContext = null;
public Builder(Configuration conf) {
this.conf = conf;
@ -785,6 +823,12 @@ public class RPC {
return this;
}
/** Default: null */
public Builder setAlignmentContext(AlignmentContext alignmentContext) {
this.alignmentContext = alignmentContext;
return this;
}
/**
* Build the RPC Server.
* @throws IOException on error
@ -804,7 +848,8 @@ public class RPC {
return getProtocolEngine(this.protocol, this.conf).getServer(
this.protocol, this.instance, this.bindAddress, this.port,
this.numHandlers, this.numReaders, this.queueSizePerHandler,
this.verbose, this.conf, this.secretManager, this.portRangeConfig);
this.verbose, this.conf, this.secretManager, this.portRangeConfig,
this.alignmentContext);
}
}

View File

@ -50,7 +50,8 @@ public interface RpcEngine {
UserGroupInformation ticket, Configuration conf,
SocketFactory factory, int rpcTimeout,
RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException;
AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext) throws IOException;
/**
* Construct a server for a protocol implementation instance.
@ -67,6 +68,7 @@ public interface RpcEngine {
* @param secretManager The secret manager to use to validate incoming requests.
* @param portRangeConfig A config parameter that can be used to restrict
* the range of ports used when port is 0 (an ephemeral port)
* @param alignmentContext provides server state info on client responses
* @return The Server instance
* @throws IOException on any error
*/
@ -75,8 +77,8 @@ public interface RpcEngine {
int queueSizePerHandler, boolean verbose,
Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager,
String portRangeConfig
) throws IOException;
String portRangeConfig,
AlignmentContext alignmentContext) throws IOException;
/**
* Returns a proxy for ProtocolMetaInfoPB, which uses the given connection

View File

@ -140,11 +140,12 @@ public abstract class Server {
private RpcSaslProto negotiateResponse;
private ExceptionsHandler exceptionsHandler = new ExceptionsHandler();
private Tracer tracer;
private AlignmentContext alignmentContext;
/**
* Logical name of the server used in metrics and monitor.
*/
private final String serverName;
/**
* Add exception classes for which server won't log stack traces.
*
@ -163,6 +164,15 @@ public abstract class Server {
exceptionsHandler.addSuppressedLoggingExceptions(exceptionClass);
}
/**
* Set alignment context to pass state info thru RPC.
*
* @param alignmentContext alignment state context
*/
public void setAlignmentContext(AlignmentContext alignmentContext) {
this.alignmentContext = alignmentContext;
}
/**
* ExceptionsHandler manages Exception groups for special handling
* e.g., terse exception group for concise logging messages
@ -716,6 +726,8 @@ public abstract class Server {
private boolean deferredResponse = false;
private int priorityLevel;
// the priority level assigned by scheduler, 0 by default
private long clientStateId;
private boolean isCallCoordinated;
Call() {
this(RpcConstants.INVALID_CALL_ID, RpcConstants.INVALID_RETRY_COUNT,
@ -746,6 +758,8 @@ public abstract class Server {
this.clientId = clientId;
this.traceScope = traceScope;
this.callerContext = callerContext;
this.clientStateId = Long.MIN_VALUE;
this.isCallCoordinated = false;
}
@Override
@ -806,7 +820,11 @@ public abstract class Server {
}
}
void doResponse(Throwable t) throws IOException {}
void doResponse(Throwable t) throws IOException {
doResponse(t, RpcStatusProto.FATAL);
}
void doResponse(Throwable t, RpcStatusProto proto) throws IOException {}
// For Schedulable
@Override
@ -823,6 +841,22 @@ public abstract class Server {
this.priorityLevel = priorityLevel;
}
public long getClientStateId() {
return this.clientStateId;
}
public void setClientStateId(long stateId) {
this.clientStateId = stateId;
}
public void markCallCoordinated(boolean flag) {
this.isCallCoordinated = flag;
}
public boolean isCallCoordinated() {
return this.isCallCoordinated;
}
@InterfaceStability.Unstable
public void deferResponse() {
this.deferredResponse = true;
@ -846,10 +880,15 @@ public abstract class Server {
final Writable rpcRequest; // Serialized Rpc request from client
ByteBuffer rpcResponse; // the response for this call
private ResponseParams responseParams; // the response params
private Writable rv; // the byte response
RpcCall(RpcCall call) {
super(call);
this.connection = call.connection;
this.rpcRequest = call.rpcRequest;
this.rv = call.rv;
this.responseParams = call.responseParams;
}
RpcCall(Connection connection, int id) {
@ -870,6 +909,12 @@ public abstract class Server {
this.rpcRequest = param;
}
void setResponseFields(Writable returnValue,
ResponseParams responseParams) {
this.rv = returnValue;
this.responseParams = responseParams;
}
@Override
public String getProtocol() {
return "rpc";
@ -901,9 +946,7 @@ public abstract class Server {
populateResponseParamsOnError(e, responseParams);
}
if (!isResponseDeferred()) {
setupResponse(this, responseParams.returnStatus,
responseParams.detailedErr,
value, responseParams.errorClass, responseParams.error);
setResponseFields(value, responseParams);
sendResponse();
} else {
if (LOG.isDebugEnabled()) {
@ -948,16 +991,23 @@ public abstract class Server {
}
@Override
void doResponse(Throwable t) throws IOException {
void doResponse(Throwable t, RpcStatusProto status) throws IOException {
RpcCall call = this;
if (t != null) {
if (status == null) {
status = RpcStatusProto.FATAL;
}
// clone the call to prevent a race with another thread stomping
// on the response while being sent. the original call is
// effectively discarded since the wait count won't hit zero
call = new RpcCall(this);
setupResponse(call,
RpcStatusProto.FATAL, RpcErrorCodeProto.ERROR_RPC_SERVER,
setupResponse(call, status, RpcErrorCodeProto.ERROR_RPC_SERVER,
null, t.getClass().getName(), StringUtils.stringifyException(t));
} else {
setupResponse(call, call.responseParams.returnStatus,
call.responseParams.detailedErr, call.rv,
call.responseParams.errorClass,
call.responseParams.error);
}
connection.sendResponse(call);
}
@ -2529,6 +2579,31 @@ public abstract class Server {
// Save the priority level assignment by the scheduler
call.setPriorityLevel(callQueue.getPriorityLevel(call));
call.markCallCoordinated(false);
if(alignmentContext != null && call.rpcRequest != null &&
(call.rpcRequest instanceof ProtobufRpcEngine.RpcProtobufRequest)) {
// if call.rpcRequest is not RpcProtobufRequest, will skip the following
// step and treat the call as uncoordinated. As currently only certain
// ClientProtocol methods request made through RPC protobuf needs to be
// coordinated.
String methodName;
String protoName;
ProtobufRpcEngine.RpcProtobufRequest req =
(ProtobufRpcEngine.RpcProtobufRequest) call.rpcRequest;
try {
methodName = req.getRequestHeader().getMethodName();
protoName = req.getRequestHeader().getDeclaringClassProtocolName();
if (alignmentContext.isCoordinatedCall(protoName, methodName)) {
call.markCallCoordinated(true);
long stateId;
stateId = alignmentContext.receiveRequestState(
header, getMaxIdleTime());
call.setClientStateId(stateId);
}
} catch (IOException ioe) {
throw new RpcServerException("Processing RPC request caught ", ioe);
}
}
try {
internalQueueCall(call);
@ -2680,8 +2755,18 @@ public abstract class Server {
private void internalQueueCall(Call call)
throws IOException, InterruptedException {
internalQueueCall(call, true);
}
private void internalQueueCall(Call call, boolean blocking)
throws IOException, InterruptedException {
try {
callQueue.put(call); // queue the call; maybe blocked here
// queue the call, may be blocked if blocking is true.
if (blocking) {
callQueue.put(call);
} else {
callQueue.add(call);
}
} catch (CallQueueOverflowException cqe) {
// If rpc scheduler indicates back off based on performance degradation
// such as response time or rpc queue is full, we will ask the client
@ -2711,6 +2796,24 @@ public abstract class Server {
TraceScope traceScope = null;
try {
final Call call = callQueue.take(); // pop the queue; maybe blocked here
if (alignmentContext != null && call.isCallCoordinated() &&
call.getClientStateId() > alignmentContext.getLastSeenStateId()) {
/*
* The call processing should be postponed until the client call's
* state id is aligned (<=) with the server state id.
* NOTE:
* Inserting the call back to the queue can change the order of call
* execution comparing to their original placement into the queue.
* This is not a problem, because Hadoop RPC does not have any
* constraints on ordering the incoming rpc requests.
* In case of Observer, it handles only reads, which are
* commutative.
*/
// Re-queue the call and continue
requeueCall(call);
continue;
}
if (LOG.isDebugEnabled()) {
LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
}
@ -2750,6 +2853,15 @@ public abstract class Server {
LOG.debug(Thread.currentThread().getName() + ": exiting");
}
private void requeueCall(Call call)
throws IOException, InterruptedException {
try {
internalQueueCall(call, false);
} catch (RpcServerException rse) {
call.doResponse(rse.getCause(), rse.getRpcStatusProto());
}
}
}
@VisibleForTesting
@ -2977,6 +3089,9 @@ public abstract class Server {
headerBuilder.setRetryCount(call.retryCount);
headerBuilder.setStatus(status);
headerBuilder.setServerIpcVersionNum(CURRENT_VERSION);
if (alignmentContext != null) {
alignmentContext.updateResponseState(headerBuilder);
}
if (status == RpcStatusProto.SUCCESS) {
RpcResponseHeaderProto header = headerBuilder.build();
@ -3607,6 +3722,10 @@ public abstract class Server {
}
}
protected int getMaxIdleTime() {
return connectionManager.maxIdleTime;
}
public String getServerName() {
return serverName;
}

View File

@ -214,16 +214,19 @@ public class WritableRpcEngine implements RpcEngine {
private Client client;
private boolean isClosed = false;
private final AtomicBoolean fallbackToSimpleAuth;
private final AlignmentContext alignmentContext;
public Invoker(Class<?> protocol,
InetSocketAddress address, UserGroupInformation ticket,
Configuration conf, SocketFactory factory,
int rpcTimeout, AtomicBoolean fallbackToSimpleAuth)
int rpcTimeout, AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
this.remoteId = Client.ConnectionId.getConnectionId(address, protocol,
ticket, rpcTimeout, null, conf);
this.client = CLIENTS.getClient(conf, factory);
this.fallbackToSimpleAuth = fallbackToSimpleAuth;
this.alignmentContext = alignmentContext;
}
@Override
@ -246,7 +249,7 @@ public class WritableRpcEngine implements RpcEngine {
try {
value = (ObjectWritable)
client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args),
remoteId, fallbackToSimpleAuth);
remoteId, fallbackToSimpleAuth, alignmentContext);
} finally {
if (traceScope != null) traceScope.close();
}
@ -289,7 +292,7 @@ public class WritableRpcEngine implements RpcEngine {
int rpcTimeout, RetryPolicy connectionRetryPolicy)
throws IOException {
return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, null);
rpcTimeout, connectionRetryPolicy, null, null);
}
/** Construct a client-side proxy object that implements the named protocol,
@ -301,7 +304,8 @@ public class WritableRpcEngine implements RpcEngine {
InetSocketAddress addr, UserGroupInformation ticket,
Configuration conf, SocketFactory factory,
int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth)
AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
if (connectionRetryPolicy != null) {
@ -311,7 +315,7 @@ public class WritableRpcEngine implements RpcEngine {
T proxy = (T) Proxy.newProxyInstance(protocol.getClassLoader(),
new Class[] { protocol }, new Invoker(protocol, addr, ticket, conf,
factory, rpcTimeout, fallbackToSimpleAuth));
factory, rpcTimeout, fallbackToSimpleAuth, alignmentContext));
return new ProtocolProxy<T>(protocol, proxy, true);
}
@ -323,11 +327,11 @@ public class WritableRpcEngine implements RpcEngine {
int numHandlers, int numReaders, int queueSizePerHandler,
boolean verbose, Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager,
String portRangeConfig)
String portRangeConfig, AlignmentContext alignmentContext)
throws IOException {
return new Server(protocolClass, protocolImpl, conf, bindAddress, port,
numHandlers, numReaders, queueSizePerHandler, verbose, secretManager,
portRangeConfig);
portRangeConfig, alignmentContext);
}
@ -397,18 +401,45 @@ public class WritableRpcEngine implements RpcEngine {
* @param port the port to listen for connections on
* @param numHandlers the number of method handler threads to run
* @param verbose whether each call should be logged
*
* @deprecated use Server#Server(Class, Object,
* Configuration, String, int, int, int, int, boolean, SecretManager)
*/
@Deprecated
public Server(Class<?> protocolClass, Object protocolImpl,
Configuration conf, String bindAddress, int port,
int numHandlers, int numReaders, int queueSizePerHandler,
boolean verbose, SecretManager<? extends TokenIdentifier> secretManager,
String portRangeConfig)
throws IOException {
this(null, protocolImpl, conf, bindAddress, port,
numHandlers, numReaders, queueSizePerHandler, verbose,
secretManager, null, null);
}
/**
* Construct an RPC server.
* @param protocolClass - the protocol being registered
* can be null for compatibility with old usage (see below for details)
* @param protocolImpl the protocol impl that will be called
* @param conf the configuration to use
* @param bindAddress the address to bind on to listen for connection
* @param port the port to listen for connections on
* @param numHandlers the number of method handler threads to run
* @param verbose whether each call should be logged
* @param alignmentContext provides server state info on client responses
*/
public Server(Class<?> protocolClass, Object protocolImpl,
Configuration conf, String bindAddress, int port,
int numHandlers, int numReaders, int queueSizePerHandler,
boolean verbose, SecretManager<? extends TokenIdentifier> secretManager,
String portRangeConfig, AlignmentContext alignmentContext)
throws IOException {
super(bindAddress, port, null, numHandlers, numReaders,
queueSizePerHandler, conf,
serverNameFromClass(protocolImpl.getClass()), secretManager,
portRangeConfig);
setAlignmentContext(alignmentContext);
this.verbose = verbose;

View File

@ -288,6 +288,24 @@ public class LogThrottlingHelper {
return null;
}
/**
* Helper function to create a message about how many log statements were
* suppressed in the provided log action. If no statements were suppressed,
* this returns an empty string. The message has the format (without quotes):
*
* <p/>' (suppressed logging <i>{suppression_count}</i> times)'
*
* @param action The log action to produce a message about.
* @return A message about suppression within this action.
*/
public static String getLogSupressionMessage(LogAction action) {
if (action.getCount() > 1) {
return " (suppressed logging " + (action.getCount() - 1) + " times)";
} else {
return "";
}
}
/**
* A standard log action which keeps track of all of the values which have
* been logged. This is also used for internal bookkeeping via its private

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.util;
import java.io.DataInput;
import java.io.IOException;
import org.apache.hadoop.ipc.AlignmentContext;
import org.apache.hadoop.ipc.CallerContext;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.IpcConnectionContextProto;
@ -165,6 +166,13 @@ public abstract class ProtoUtil {
public static RpcRequestHeaderProto makeRpcRequestHeader(RPC.RpcKind rpcKind,
RpcRequestHeaderProto.OperationProto operation, int callId,
int retryCount, byte[] uuid) {
return makeRpcRequestHeader(rpcKind, operation, callId, retryCount, uuid,
null);
}
public static RpcRequestHeaderProto makeRpcRequestHeader(RPC.RpcKind rpcKind,
RpcRequestHeaderProto.OperationProto operation, int callId,
int retryCount, byte[] uuid, AlignmentContext alignmentContext) {
RpcRequestHeaderProto.Builder result = RpcRequestHeaderProto.newBuilder();
result.setRpcKind(convert(rpcKind)).setRpcOp(operation).setCallId(callId)
.setRetryCount(retryCount).setClientId(ByteString.copyFrom(uuid));
@ -190,6 +198,11 @@ public abstract class ProtoUtil {
result.setCallerContext(contextBuilder);
}
// Add alignment context if it is not null
if (alignmentContext != null) {
alignmentContext.updateRequestState(result);
}
return result.build();
}
}

View File

@ -32,6 +32,7 @@ enum HAServiceStateProto {
INITIALIZING = 0;
ACTIVE = 1;
STANDBY = 2;
OBSERVER = 3;
}
enum HARequestSource {
@ -82,6 +83,19 @@ message TransitionToStandbyRequestProto {
message TransitionToStandbyResponseProto {
}
/**
* void request
*/
message TransitionToObserverRequestProto {
required HAStateChangeRequestInfoProto reqInfo = 1;
}
/**
* void response
*/
message TransitionToObserverResponseProto {
}
/**
* void request
*/
@ -126,6 +140,12 @@ service HAServiceProtocolService {
rpc transitionToStandby(TransitionToStandbyRequestProto)
returns(TransitionToStandbyResponseProto);
/**
* Request service to transition to observer state.
*/
rpc transitionToObserver(TransitionToObserverRequestProto)
returns(TransitionToObserverResponseProto);
/**
* Get the current status of the service.
*/

View File

@ -90,6 +90,7 @@ message RpcRequestHeaderProto { // the header for the RpcRequest
optional sint32 retryCount = 5 [default = -1];
optional RPCTraceInfoProto traceInfo = 6; // tracing info
optional RPCCallerContextProto callerContext = 7; // call context
optional int64 stateId = 8; // The last seen Global State ID
}
@ -155,6 +156,7 @@ message RpcResponseHeaderProto {
optional RpcErrorCodeProto errorDetail = 6; // in case of error
optional bytes clientId = 7; // Globally unique client ID
optional sint32 retryCount = 8 [default = -1];
optional int64 stateId = 9; // The last written Global State ID
}
message RpcSaslProto {

View File

@ -315,6 +315,11 @@ The server-side metrics for a journal from the JournalNode's perspective. Each m
| `LastWrittenTxId` | The highest transaction id stored on this JournalNode |
| `LastPromisedEpoch` | The last epoch number which this node has promised not to accept any lower epoch, or 0 if no promises have been made |
| `LastJournalTimestamp` | The timestamp of last successfully written transaction |
| `TxnsServedViaRpc` | Number of transactions served via the RPC mechanism |
| `BytesServedViaRpc` | Number of bytes served via the RPC mechanism |
| `RpcRequestCacheMissAmountNumMisses` | Number of RPC requests which could not be served due to lack of data in the cache |
| `RpcRequestCacheMissAmountAvgTxns` | The average number of transactions by which a request missed the cache; for example if transaction ID 10 is requested and the cache's oldest transaction is ID 15, value 5 will be added to this average |
| `RpcEmptyResponses` | Number of RPC requests with zero edits returned |
datanode
--------

View File

@ -56,7 +56,8 @@ class DummyHAService extends HAServiceTarget {
InetSocketAddress address, healthMonitorAddress;
boolean isHealthy = true;
boolean actUnreachable = false;
boolean failToBecomeActive, failToBecomeStandby, failToFence;
boolean failToBecomeActive, failToBecomeStandby, failToBecomeObserver,
failToFence;
DummySharedResource sharedResource;
public int fenceCount = 0;
@ -216,6 +217,11 @@ class DummyHAService extends HAServiceTarget {
return true;
}
@Override
public boolean supportObserver() {
return true;
}
@Override
public String toString() {
return "DummyHAService #" + index;
@ -263,6 +269,16 @@ class DummyHAService extends HAServiceTarget {
state = HAServiceState.STANDBY;
}
@Override
public void transitionToObserver(StateChangeRequestInfo req)
throws ServiceFailedException, AccessControlException, IOException {
checkUnreachable();
if (failToBecomeObserver) {
throw new ServiceFailedException("injected failure");
}
state = HAServiceState.OBSERVER;
}
@Override
public HAServiceStatus getServiceStatus() throws IOException {
checkUnreachable();

View File

@ -187,6 +187,10 @@ public class MiniZKFCCluster {
svcs.get(idx).actUnreachable = unreachable;
}
public void setFailToBecomeObserver(int idx, boolean doFail) {
svcs.get(idx).failToBecomeObserver = doFail;
}
/**
* Wait for the given HA service to enter the given HA state.
* This is based on the state of ZKFC, not the state of HA service.

View File

@ -278,7 +278,7 @@ public class TestRPC extends TestRpcBase {
SocketFactory factory, int rpcTimeout,
RetryPolicy connectionRetryPolicy) throws IOException {
return getProxy(protocol, clientVersion, addr, ticket, conf, factory,
rpcTimeout, connectionRetryPolicy, null);
rpcTimeout, connectionRetryPolicy, null, null);
}
@SuppressWarnings("unchecked")
@ -287,7 +287,8 @@ public class TestRPC extends TestRpcBase {
Class<T> protocol, long clientVersion, InetSocketAddress addr,
UserGroupInformation ticket, Configuration conf, SocketFactory factory,
int rpcTimeout, RetryPolicy connectionRetryPolicy,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
AtomicBoolean fallbackToSimpleAuth, AlignmentContext alignmentContext)
throws IOException {
T proxy = (T) Proxy.newProxyInstance(protocol.getClassLoader(),
new Class[] { protocol }, new StoppedInvocationHandler());
return new ProtocolProxy<T>(protocol, proxy, false);
@ -299,7 +300,8 @@ public class TestRPC extends TestRpcBase {
int numHandlers, int numReaders, int queueSizePerHandler,
boolean verbose, Configuration conf,
SecretManager<? extends TokenIdentifier> secretManager,
String portRangeConfig) throws IOException {
String portRangeConfig, AlignmentContext alignmentContext)
throws IOException {
return null;
}

View File

@ -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.hdfs;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ipc.AlignmentContext;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
import java.io.IOException;
import java.util.concurrent.atomic.LongAccumulator;
/**
* Global State Id context for the client.
* <p/>
* This is the client side implementation responsible for receiving
* state alignment info from server(s).
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ClientGSIContext implements AlignmentContext {
private final LongAccumulator lastSeenStateId =
new LongAccumulator(Math::max, Long.MIN_VALUE);
@Override
public long getLastSeenStateId() {
return lastSeenStateId.get();
}
@Override
public boolean isCoordinatedCall(String protocolName, String method) {
throw new UnsupportedOperationException(
"Client should not be checking uncoordinated call");
}
/**
* Client side implementation only receives state alignment info.
* It does not provide state alignment info therefore this does nothing.
*/
@Override
public void updateResponseState(RpcResponseHeaderProto.Builder header) {
// Do nothing.
}
/**
* Client side implementation for receiving state alignment info
* in responses.
*/
@Override
public void receiveResponseState(RpcResponseHeaderProto header) {
lastSeenStateId.accumulate(header.getStateId());
}
/**
* Client side implementation for providing state alignment info in requests.
*/
@Override
public void updateRequestState(RpcRequestHeaderProto.Builder header) {
header.setStateId(lastSeenStateId.longValue());
}
/**
* Client side implementation only provides state alignment info in requests.
* Client does not receive RPC requests therefore this does nothing.
*/
@Override
public long receiveRequestState(RpcRequestHeaderProto header, long threshold)
throws IOException {
// Do nothing.
return 0;
}
}

View File

@ -93,6 +93,7 @@ import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsCreateModes;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
@ -3181,4 +3182,30 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
checkOpen();
return new OpenFilesIterator(namenode, tracer, openFilesTypes, path);
}
/**
* A blocking call to wait for Observer NameNode state ID to reach to the
* current client state ID. Current client state ID is given by the client
* alignment context.
* An assumption is that client alignment context has the state ID set at this
* point. This is become ObserverReadProxyProvider sets up the initial state
* ID when it is being created.
*
* @throws IOException
*/
public void msync() throws IOException {
namenode.msync();
}
/**
* An unblocking call to get the HA service state of NameNode.
*
* @return HA state of NameNode
* @throws IOException
*/
@VisibleForTesting
public HAServiceProtocol.HAServiceState getHAServiceState()
throws IOException {
return namenode.getHAServiceState();
}
}

View File

@ -28,8 +28,10 @@ import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.hdfs.server.namenode.ha.ClientHAProxyFactory;
import org.apache.hadoop.hdfs.server.namenode.ha.HAProxyFactory;
import org.apache.hadoop.ipc.AlignmentContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -61,8 +63,9 @@ import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
/**
* Create proxy objects with {@link ClientProtocol} to communicate with a remote
* NN. Generally use {@link NameNodeProxiesClient#createProxyWithClientProtocol(
* Create proxy objects with {@link ClientProtocol} and
* {@link HAServiceProtocol} to communicate with a remote NN. For the former,
* generally use {@link NameNodeProxiesClient#createProxyWithClientProtocol(
* Configuration, URI, AtomicBoolean)}, which will create either an HA- or
* non-HA-enabled client proxy as appropriate.
*
@ -75,6 +78,11 @@ public class NameNodeProxiesClient {
private static final Logger LOG = LoggerFactory.getLogger(
NameNodeProxiesClient.class);
/** Maximum # of retries for HAProxy with HAServiceProtocol. */
private static final int MAX_RETRIES = 3;
/** Initial retry delay for HAProxy with HAServiceProtocol. */
private static final int DELAY_MILLISECONDS = 200;
/**
* Wrapper for a client proxy as well as its associated service ID.
* This is simply used as a tuple-like return type for created NN proxy.
@ -337,6 +345,15 @@ public class NameNodeProxiesClient {
InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
boolean withRetries, AtomicBoolean fallbackToSimpleAuth)
throws IOException {
return createProxyWithAlignmentContext(address, conf, ugi, withRetries,
fallbackToSimpleAuth, null);
}
public static ClientProtocol createProxyWithAlignmentContext(
InetSocketAddress address, Configuration conf, UserGroupInformation ugi,
boolean withRetries, AtomicBoolean fallbackToSimpleAuth,
AlignmentContext alignmentContext)
throws IOException {
RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
ProtobufRpcEngine.class);
@ -354,7 +371,7 @@ public class NameNodeProxiesClient {
ClientNamenodeProtocolPB.class, version, address, ugi, conf,
NetUtils.getDefaultSocketFactory(conf),
org.apache.hadoop.ipc.Client.getTimeout(conf), defaultPolicy,
fallbackToSimpleAuth).getProxy();
fallbackToSimpleAuth, alignmentContext).getProxy();
if (withRetries) { // create the proxy with retries
Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<>();

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.crypto.CryptoProtocolVersion;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.hdfs.AddBlockFlag;
import org.apache.hadoop.fs.CacheFlag;
import org.apache.hadoop.fs.ContentSummary;
@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
import org.apache.hadoop.hdfs.server.namenode.ha.ReadOnly;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
import org.apache.hadoop.io.EnumSetWritable;
import org.apache.hadoop.io.Text;
@ -128,6 +130,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(atimeAffected = true, isCoordinated = true)
LocatedBlocks getBlockLocations(String src, long offset, long length)
throws IOException;
@ -137,6 +140,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
FsServerDefaults getServerDefaults() throws IOException;
/**
@ -277,6 +281,7 @@ public interface ClientProtocol {
* @return All the in-use block storage policies currently.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
BlockStoragePolicy[] getStoragePolicies() throws IOException;
/**
@ -319,6 +324,7 @@ public interface ClientProtocol {
* If file/dir <code>src</code> is not found
*/
@Idempotent
@ReadOnly(isCoordinated = true)
BlockStoragePolicy getStoragePolicy(String path) throws IOException;
/**
@ -685,6 +691,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
DirectoryListing getListing(String src, byte[] startAfter,
boolean needLocation) throws IOException;
@ -695,6 +702,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
SnapshottableDirectoryStatus[] getSnapshottableDirListing()
throws IOException;
@ -775,6 +783,7 @@ public interface ClientProtocol {
* </ul>
*/
@Idempotent
@ReadOnly
long[] getStats() throws IOException;
/**
@ -782,6 +791,7 @@ public interface ClientProtocol {
* in the filesystem.
*/
@Idempotent
@ReadOnly
ReplicatedBlockStats getReplicatedBlockStats() throws IOException;
/**
@ -789,6 +799,7 @@ public interface ClientProtocol {
* in the filesystem.
*/
@Idempotent
@ReadOnly
ECBlockGroupStats getECBlockGroupStats() throws IOException;
/**
@ -798,6 +809,7 @@ public interface ClientProtocol {
* otherwise all datanodes if type is ALL.
*/
@Idempotent
@ReadOnly
DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
throws IOException;
@ -805,6 +817,7 @@ public interface ClientProtocol {
* Get a report on the current datanode storages.
*/
@Idempotent
@ReadOnly
DatanodeStorageReport[] getDatanodeStorageReport(
HdfsConstants.DatanodeReportType type) throws IOException;
@ -817,6 +830,7 @@ public interface ClientProtocol {
* a symlink.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
long getPreferredBlockSize(String filename)
throws IOException;
@ -971,6 +985,7 @@ public interface ClientProtocol {
* cookie returned from the previous call.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
throws IOException;
@ -1006,6 +1021,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
HdfsFileStatus getFileInfo(String src) throws IOException;
/**
@ -1020,6 +1036,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
boolean isFileClosed(String src) throws IOException;
/**
@ -1036,6 +1053,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
HdfsFileStatus getFileLinkInfo(String src) throws IOException;
/**
@ -1050,6 +1068,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
HdfsLocatedFileStatus getLocatedFileInfo(String src, boolean needBlockToken)
throws IOException;
@ -1064,6 +1083,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
ContentSummary getContentSummary(String path) throws IOException;
/**
@ -1176,6 +1196,7 @@ public interface ClientProtocol {
* or an I/O error occurred
*/
@Idempotent
@ReadOnly(isCoordinated = true)
String getLinkTarget(String path) throws IOException;
/**
@ -1245,6 +1266,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
DataEncryptionKey getDataEncryptionKey() throws IOException;
/**
@ -1313,6 +1335,7 @@ public interface ClientProtocol {
* @throws IOException on error
*/
@Idempotent
@ReadOnly(isCoordinated = true)
SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
String fromSnapshot, String toSnapshot) throws IOException;
@ -1340,6 +1363,7 @@ public interface ClientProtocol {
* @throws IOException on error
*/
@Idempotent
@ReadOnly(isCoordinated = true)
SnapshotDiffReportListing getSnapshotDiffReportListing(String snapshotRoot,
String fromSnapshot, String toSnapshot, byte[] startPath, int index)
throws IOException;
@ -1386,6 +1410,7 @@ public interface ClientProtocol {
* @return A batch of CacheDirectiveEntry objects.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
long prevId, CacheDirectiveInfo filter) throws IOException;
@ -1427,6 +1452,7 @@ public interface ClientProtocol {
* @return A batch of CachePoolEntry objects.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
throws IOException;
@ -1473,6 +1499,7 @@ public interface ClientProtocol {
* Gets the ACLs of files and directories.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
AclStatus getAclStatus(String src) throws IOException;
/**
@ -1486,6 +1513,7 @@ public interface ClientProtocol {
* Get the encryption zone for a path.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
EncryptionZone getEZForPath(String src)
throws IOException;
@ -1497,6 +1525,7 @@ public interface ClientProtocol {
* @return Batch of encryption zones.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
BatchedEntries<EncryptionZone> listEncryptionZones(
long prevId) throws IOException;
@ -1521,6 +1550,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
BatchedEntries<ZoneReencryptionStatus> listReencryptionStatus(long prevId)
throws IOException;
@ -1554,6 +1584,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
throws IOException;
@ -1569,6 +1600,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
List<XAttr> listXAttrs(String src)
throws IOException;
@ -1603,6 +1635,7 @@ public interface ClientProtocol {
* @throws IOException see specific implementation
*/
@Idempotent
@ReadOnly(isCoordinated = true)
void checkAccess(String path, FsAction mode) throws IOException;
/**
@ -1611,6 +1644,7 @@ public interface ClientProtocol {
* the starting point for the inotify event stream.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
long getCurrentEditLogTxid() throws IOException;
/**
@ -1618,6 +1652,7 @@ public interface ClientProtocol {
* transactions for txids equal to or greater than txid.
*/
@Idempotent
@ReadOnly(isCoordinated = true)
EventBatchList getEditsFromTxid(long txid) throws IOException;
/**
@ -1675,6 +1710,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
ErasureCodingPolicyInfo[] getErasureCodingPolicies() throws IOException;
/**
@ -1683,6 +1719,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
Map<String, String> getErasureCodingCodecs() throws IOException;
/**
@ -1693,6 +1730,7 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException;
/**
@ -1704,6 +1742,11 @@ public interface ClientProtocol {
/**
* Get {@link QuotaUsage} rooted at the specified directory.
*
* Note: due to HDFS-6763, standby/observer doesn't keep up-to-date info
* about quota usage, and thus even though this is ReadOnly, it can only be
* directed to the active namenode.
*
* @param path The string representation of the path
*
* @throws AccessControlException permission denied
@ -1713,6 +1756,7 @@ public interface ClientProtocol {
* @throws IOException If an I/O error occurred
*/
@Idempotent
@ReadOnly(activeOnly = true)
QuotaUsage getQuotaUsage(String path) throws IOException;
/**
@ -1726,6 +1770,7 @@ public interface ClientProtocol {
*/
@Idempotent
@Deprecated
@ReadOnly(isCoordinated = true)
BatchedEntries<OpenFileEntry> listOpenFiles(long prevId) throws IOException;
/**
@ -1740,9 +1785,31 @@ public interface ClientProtocol {
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException;
/**
* Get HA service state of the server.
*
* @return server HA state
* @throws IOException
*/
@Idempotent
@ReadOnly
HAServiceProtocol.HAServiceState getHAServiceState() throws IOException;
/**
* Called by client to wait until the server has reached the state id of the
* client. The client and server state id are given by client side and server
* side alignment context respectively. This can be a blocking call.
*
* @throws IOException
*/
@Idempotent
@ReadOnly(isCoordinated = true)
void msync() throws IOException;
/**
* Satisfy the storage policy for a file/directory.
* @param path Path of an existing file/directory.

View File

@ -48,6 +48,8 @@ import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
import org.apache.hadoop.hdfs.AddBlockFlag;
import org.apache.hadoop.hdfs.inotify.EventBatchList;
import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
@ -146,6 +148,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSna
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ListCacheDirectivesResponseProto;
@ -158,6 +161,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MetaSa
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MkdirsRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.OpenFilesBatchResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
@ -1947,6 +1951,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
}
}
@Override
public void msync() throws IOException {
MsyncRequestProto.Builder req = MsyncRequestProto.newBuilder();
try {
rpcProxy.msync(null, req.build());
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public void satisfyStoragePolicy(String src) throws IOException {
SatisfyStoragePolicyRequestProto req =
@ -1957,4 +1971,29 @@ public class ClientNamenodeProtocolTranslatorPB implements
throw ProtobufHelper.getRemoteException(e);
}
}
@Override
public HAServiceProtocol.HAServiceState getHAServiceState()
throws IOException {
HAServiceStateRequestProto req =
HAServiceStateRequestProto.newBuilder().build();
try {
HAServiceStateProto res =
rpcProxy.getHAServiceState(null, req).getState();
switch(res) {
case ACTIVE:
return HAServiceProtocol.HAServiceState.ACTIVE;
case STANDBY:
return HAServiceProtocol.HAServiceState.STANDBY;
case OBSERVER:
return HAServiceProtocol.HAServiceState.OBSERVER;
case INITIALIZING:
default:
return HAServiceProtocol.HAServiceState.INITIALIZING;
}
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
}

View File

@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.HAUtilClient;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@ -111,6 +112,12 @@ public abstract class AbstractNNFailoverProxyProvider<T> implements
*/
public static class NNProxyInfo<T> extends ProxyInfo<T> {
private InetSocketAddress address;
/**
* The currently known state of the NameNode represented by this ProxyInfo.
* This may be out of date if the NameNode has changed state since the last
* time the state was checked.
*/
private HAServiceState cachedState;
public NNProxyInfo(InetSocketAddress address) {
super(null, address.toString());
@ -120,6 +127,14 @@ public abstract class AbstractNNFailoverProxyProvider<T> implements
public InetSocketAddress getAddress() {
return address;
}
public void setCachedState(HAServiceState state) {
cachedState = state;
}
public HAServiceState getCachedState() {
return cachedState;
}
}
@Override
@ -137,8 +152,8 @@ public abstract class AbstractNNFailoverProxyProvider<T> implements
pi.proxy = factory.createProxy(conf,
pi.getAddress(), xface, ugi, false, getFallbackToSimpleAuth());
} catch (IOException ioe) {
LOG.error("{} Failed to create RPC proxy to NameNode",
this.getClass().getSimpleName(), ioe);
LOG.error("{} Failed to create RPC proxy to NameNode at {}",
this.getClass().getSimpleName(), pi.address, ioe);
throw new RuntimeException(ioe);
}
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.NameNodeProxiesClient;
import org.apache.hadoop.ipc.AlignmentContext;
import org.apache.hadoop.security.UserGroupInformation;
import java.io.IOException;
@ -26,11 +27,22 @@ import java.net.InetSocketAddress;
import java.util.concurrent.atomic.AtomicBoolean;
public class ClientHAProxyFactory<T> implements HAProxyFactory<T> {
private AlignmentContext alignmentContext;
public void setAlignmentContext(AlignmentContext alignmentContext) {
this.alignmentContext = alignmentContext;
}
@Override
@SuppressWarnings("unchecked")
public T createProxy(Configuration conf, InetSocketAddress nnAddr,
Class<T> xface, UserGroupInformation ugi, boolean withRetries,
AtomicBoolean fallbackToSimpleAuth) throws IOException {
if (alignmentContext != null) {
return (T) NameNodeProxiesClient.createProxyWithAlignmentContext(
nnAddr, conf, ugi, false, fallbackToSimpleAuth, alignmentContext);
}
return (T) NameNodeProxiesClient.createNonHAProxyWithClientProtocol(
nnAddr, conf, ugi, false, fallbackToSimpleAuth);
}

View File

@ -48,7 +48,7 @@ public class IPFailoverProxyProvider<T> extends
public IPFailoverProxyProvider(Configuration conf, URI uri,
Class<T> xface, HAProxyFactory<T> factory) {
super(conf, uri, xface, factory);
this.nnProxyInfo = new NNProxyInfo<T>(DFSUtilClient.getNNAddress(uri));
this.nnProxyInfo = new NNProxyInfo<>(DFSUtilClient.getNNAddress(uri));
}
@Override

View File

@ -0,0 +1,357 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdfs.server.namenode.ha;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.lang.reflect.Proxy;
import java.net.URI;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.ClientGSIContext;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.io.retry.AtMostOnce;
import org.apache.hadoop.io.retry.Idempotent;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
import org.apache.hadoop.ipc.AlignmentContext;
import org.apache.hadoop.ipc.Client.ConnectionId;
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.RpcInvocationHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
/**
* A {@link org.apache.hadoop.io.retry.FailoverProxyProvider} implementation
* that supports reading from observer namenode(s).
*
* This constructs a wrapper proxy that sends the request to observer
* namenode(s), if observer read is enabled. In case there are multiple
* observer namenodes, it will try them one by one in case the RPC failed. It
* will fail back to the active namenode after it has exhausted all the
* observer namenodes.
*
* Read and write requests will still be sent to active NN if reading from
* observer is turned off.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ObserverReadProxyProvider<T extends ClientProtocol>
extends AbstractNNFailoverProxyProvider<T> {
private static final Logger LOG = LoggerFactory.getLogger(
ObserverReadProxyProvider.class);
/** Client-side context for syncing with the NameNode server side. */
private final AlignmentContext alignmentContext;
/** The inner proxy provider used for active/standby failover. */
private final AbstractNNFailoverProxyProvider<T> failoverProxy;
/** List of all NameNode proxies. */
private final List<NNProxyInfo<T>> nameNodeProxies;
/** The policy used to determine if an exception is fatal or retriable. */
private final RetryPolicy observerRetryPolicy;
/** The combined proxy which redirects to other proxies as necessary. */
private final ProxyInfo<T> combinedProxy;
/**
* Whether reading from observer is enabled. If this is false, all read
* requests will still go to active NN.
*/
private boolean observerReadEnabled;
/**
* The index into the nameNodeProxies list currently being used. Should only
* be accessed in synchronized methods.
*/
private int currentIndex = -1;
/**
* The proxy being used currently; this will match with currentIndex above.
* This field is volatile to allow reads without synchronization; updates
* should still be performed synchronously to maintain consistency between
* currentIndex and this field.
*/
private volatile NNProxyInfo<T> currentProxy;
/** The last proxy that has been used. Only used for testing. */
private volatile ProxyInfo<T> lastProxy = null;
/**
* By default ObserverReadProxyProvider uses
* {@link ConfiguredFailoverProxyProvider} for failover.
*/
public ObserverReadProxyProvider(
Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory) {
this(conf, uri, xface, factory,
new ConfiguredFailoverProxyProvider<>(conf, uri, xface, factory));
}
@SuppressWarnings("unchecked")
public ObserverReadProxyProvider(
Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory,
AbstractNNFailoverProxyProvider<T> failoverProxy) {
super(conf, uri, xface, factory);
this.failoverProxy = failoverProxy;
this.alignmentContext = new ClientGSIContext();
((ClientHAProxyFactory<T>) factory).setAlignmentContext(alignmentContext);
// Don't bother configuring the number of retries and such on the retry
// policy since it is mainly only used for determining whether or not an
// exception is retriable or fatal
observerRetryPolicy = RetryPolicies.failoverOnNetworkException(
RetryPolicies.TRY_ONCE_THEN_FAIL, 1);
// Get all NameNode proxies
nameNodeProxies = getProxyAddresses(uri,
HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY);
// Create a wrapped proxy containing all the proxies. Since this combined
// proxy is just redirecting to other proxies, all invocations can share it.
StringBuilder combinedInfo = new StringBuilder("[");
for (int i = 0; i < nameNodeProxies.size(); i++) {
if (i > 0) {
combinedInfo.append(",");
}
combinedInfo.append(nameNodeProxies.get(i).proxyInfo);
}
combinedInfo.append(']');
T wrappedProxy = (T) Proxy.newProxyInstance(
ObserverReadInvocationHandler.class.getClassLoader(),
new Class<?>[] {xface}, new ObserverReadInvocationHandler());
combinedProxy = new ProxyInfo<>(wrappedProxy, combinedInfo.toString());
// TODO : make this configurable or remove this variable
this.observerReadEnabled = true;
}
public AlignmentContext getAlignmentContext() {
return alignmentContext;
}
@Override
public ProxyInfo<T> getProxy() {
return combinedProxy;
}
@Override
public void performFailover(T currentProxy) {
failoverProxy.performFailover(currentProxy);
}
/**
* Check if a method is read-only.
*
* @return whether the 'method' is a read-only operation.
*/
private static boolean isRead(Method method) {
if (!method.isAnnotationPresent(ReadOnly.class)) {
return false;
}
return !method.getAnnotationsByType(ReadOnly.class)[0].activeOnly();
}
@VisibleForTesting
void setObserverReadEnabled(boolean flag) {
this.observerReadEnabled = flag;
}
@VisibleForTesting
ProxyInfo<T> getLastProxy() {
return lastProxy;
}
/**
* Return the currently used proxy. If there is none, first calls
* {@link #changeProxy(NNProxyInfo)} to initialize one.
*/
private NNProxyInfo<T> getCurrentProxy() {
if (currentProxy == null) {
changeProxy(null);
}
return currentProxy;
}
/**
* Move to the next proxy in the proxy list. If the NNProxyInfo supplied by
* the caller does not match the current proxy, the call is ignored; this is
* to handle concurrent calls (to avoid changing the proxy multiple times).
* The service state of the newly selected proxy will be updated before
* returning.
*
* @param initial The expected current proxy
*/
private synchronized void changeProxy(NNProxyInfo<T> initial) {
if (currentProxy != initial) {
// Must have been a concurrent modification; ignore the move request
return;
}
// Attempt to force concurrent callers of getCurrentProxy to wait for the
// new proxy; best-effort by setting currentProxy to null
currentProxy = null;
currentIndex = (currentIndex + 1) % nameNodeProxies.size();
currentProxy = createProxyIfNeeded(nameNodeProxies.get(currentIndex));
try {
HAServiceState state = currentProxy.proxy.getHAServiceState();
currentProxy.setCachedState(state);
} catch (IOException e) {
LOG.info("Failed to connect to {}. Setting cached state to Standby",
currentProxy.getAddress(), e);
currentProxy.setCachedState(HAServiceState.STANDBY);
}
LOG.debug("Changed current proxy from {} to {}",
initial == null ? "none" : initial.proxyInfo,
currentProxy.proxyInfo);
}
/**
* An InvocationHandler to handle incoming requests. This class's invoke
* method contains the primary logic for redirecting to observers.
*
* If observer reads are enabled, attempt to send read operations to the
* current proxy. If it is not an observer, or the observer fails, adjust
* the current proxy and retry on the next one. If all proxies are tried
* without success, the request is forwarded to the active.
*
* Write requests are always forwarded to the active.
*/
private class ObserverReadInvocationHandler implements RpcInvocationHandler {
@Override
public Object invoke(Object proxy, final Method method, final Object[] args)
throws Throwable {
lastProxy = null;
Object retVal;
if (observerReadEnabled && isRead(method)) {
int failedObserverCount = 0;
int activeCount = 0;
int standbyCount = 0;
for (int i = 0; i < nameNodeProxies.size(); i++) {
NNProxyInfo<T> current = getCurrentProxy();
HAServiceState currState = current.getCachedState();
if (currState != HAServiceState.OBSERVER) {
if (currState == HAServiceState.ACTIVE) {
activeCount++;
} else if (currState == HAServiceState.STANDBY) {
standbyCount++;
}
LOG.debug("Skipping proxy {} for {} because it is in state {}",
current.proxyInfo, method.getName(), currState);
changeProxy(current);
continue;
}
LOG.debug("Attempting to service {} using proxy {}",
method.getName(), current.proxyInfo);
try {
retVal = method.invoke(current.proxy, args);
lastProxy = current;
LOG.debug("Invocation of {} using {} was successful",
method.getName(), current.proxyInfo);
return retVal;
} catch (InvocationTargetException ite) {
if (!(ite.getCause() instanceof Exception)) {
throw ite.getCause();
}
Exception e = (Exception) ite.getCause();
if (e instanceof RemoteException) {
RemoteException re = (RemoteException) e;
Exception unwrapped = re.unwrapRemoteException(
ObserverRetryOnActiveException.class);
if (unwrapped instanceof ObserverRetryOnActiveException) {
LOG.info("Encountered ObserverRetryOnActiveException from {}." +
" Retry active namenode directly.", current.proxyInfo);
break;
}
}
RetryAction retryInfo = observerRetryPolicy.shouldRetry(e, 0, 0,
method.isAnnotationPresent(Idempotent.class)
|| method.isAnnotationPresent(AtMostOnce.class));
if (retryInfo.action == RetryAction.RetryDecision.FAIL) {
throw e;
} else {
failedObserverCount++;
LOG.warn(
"Invocation returned exception on [{}]; {} failure(s) so far",
current.proxyInfo, failedObserverCount, e);
changeProxy(current);
}
}
}
// If we get here, it means all observers have failed.
LOG.warn("{} observers have failed for read request {}; also found " +
"{} standby and {} active. Falling back to active.",
failedObserverCount, method.getName(), standbyCount, activeCount);
}
// Either all observers have failed, or that it is a write request.
// In either case, we'll forward the request to active NameNode.
LOG.debug("Using failoverProxy to service {}", method.getName());
ProxyInfo<T> activeProxy = failoverProxy.getProxy();
try {
retVal = method.invoke(activeProxy.proxy, args);
} catch (InvocationTargetException e) {
// This exception will be handled by higher layers
throw e.getCause();
}
lastProxy = activeProxy;
return retVal;
}
@Override
public void close() throws IOException {}
@Override
public ConnectionId getConnectionId() {
return RPC.getConnectionIdForProxy(getCurrentProxy().proxy);
}
}
@Override
public synchronized void close() throws IOException {
for (ProxyInfo<T> pi : nameNodeProxies) {
if (pi.proxy != null) {
if (pi.proxy instanceof Closeable) {
((Closeable)pi.proxy).close();
} else {
RPC.stopProxy(pi.proxy);
}
// Set to null to avoid the failoverProxy having to re-do the close
// if it is sharing a proxy instance
pi.proxy = null;
}
}
failoverProxy.close();
}
@Override
public boolean useLogicalURI() {
return failoverProxy.useLogicalURI();
}
}

View File

@ -0,0 +1,142 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdfs.server.namenode.ha;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.Collections;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.HAUtilClient;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Extends {@link ObserverReadProxyProvider} to support NameNode IP failover.
*
* For Observer reads a client needs to know physical addresses of all
* NameNodes, so that it could switch between active and observer nodes
* for write and read requests.
*
* Traditional {@link IPFailoverProxyProvider} works with a virtual
* address of the NameNode. If active NameNode fails the virtual address
* is assigned to the standby NameNode, and IPFailoverProxyProvider, which
* keeps talking to the same virtual address is in fact now connects to
* the new physical server.
*
* To combine these behaviors ObserverReadProxyProviderWithIPFailover
* should both
* <ol>
* <li> Maintain all physical addresses of NameNodes in order to allow
* observer reads, and
* <li> Should rely on the virtual address of the NameNode in order to
* perform failover by assuming that the virtual address always points
* to the active NameNode.
* </ol>
*
* An example of a configuration to leverage
* ObserverReadProxyProviderWithIPFailover
* should include the following values:
* <pre>{@code
* fs.defaultFS = hdfs://mycluster
* dfs.nameservices = mycluster
* dfs.ha.namenodes.mycluster = ha1,ha2
* dfs.namenode.rpc-address.mycluster.ha1 = nn01-ha1.com:8020
* dfs.namenode.rpc-address.mycluster.ha2 = nn01-ha2.com:8020
* dfs.client.failover.ipfailover.virtual-address.mycluster =
* hdfs://nn01.com:8020
* dfs.client.failover.proxy.provider.mycluster =
* org.apache...ObserverReadProxyProviderWithIPFailover
* }</pre>
* Here {@code nn01.com:8020} is the virtual address of the active NameNode,
* while {@code nn01-ha1.com:8020} and {@code nn01-ha2.com:8020}
* are the physically addresses the two NameNodes.
*
* With this configuration, client will use
* ObserverReadProxyProviderWithIPFailover, which creates proxies for both
* nn01-ha1 and nn01-ha2, used for read/write RPC calls, but for the failover,
* it relies on the virtual address nn01.com
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class ObserverReadProxyProviderWithIPFailover<T extends ClientProtocol>
extends ObserverReadProxyProvider<T> {
private static final Logger LOG = LoggerFactory.getLogger(
ObserverReadProxyProviderWithIPFailover.class);
private static final String IPFAILOVER_CONFIG_PREFIX =
HdfsClientConfigKeys.Failover.PREFIX + "ipfailover.virtual-address";
/**
* By default ObserverReadProxyProviderWithIPFailover
* uses {@link IPFailoverProxyProvider} for failover.
*/
public ObserverReadProxyProviderWithIPFailover(
Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory) {
this(conf, uri, xface, factory,
new IPFailoverProxyProvider<>(conf,
getFailoverVirtualIP(conf, uri.getHost()), xface, factory));
}
@Override
public boolean useLogicalURI() {
return true;
}
public ObserverReadProxyProviderWithIPFailover(
Configuration conf, URI uri, Class<T> xface, HAProxyFactory<T> factory,
AbstractNNFailoverProxyProvider<T> failoverProxy) {
super(conf, uri, xface, factory, failoverProxy);
cloneDelegationTokenForVirtualIP(conf, uri);
}
/**
* Clone delegation token for the virtual IP. Specifically
* clone the dt that corresponds to the name service uri,
* to the configured corresponding virtual IP.
*
* @param conf configuration
* @param haURI the ha uri, a name service id in this case.
*/
private void cloneDelegationTokenForVirtualIP(
Configuration conf, URI haURI) {
URI virtualIPURI = getFailoverVirtualIP(conf, haURI.getHost());
InetSocketAddress vipAddress = new InetSocketAddress(
virtualIPURI.getHost(), virtualIPURI.getPort());
HAUtilClient.cloneDelegationTokenForLogicalUri(
ugi, haURI, Collections.singleton(vipAddress));
}
private static URI getFailoverVirtualIP(
Configuration conf, String nameServiceID) {
String configKey = IPFAILOVER_CONFIG_PREFIX + "." + nameServiceID;
String virtualIP = conf.get(configKey);
LOG.info("Name service ID {} will use virtual IP {} for failover",
nameServiceID, virtualIP);
if (virtualIP == null) {
throw new IllegalArgumentException("Virtual IP for failover not found,"
+ "misconfigured " + configKey + "?");
}
return URI.create(virtualIP);
}
}

View File

@ -0,0 +1,57 @@
/*
* 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.hdfs.server.namenode.ha;
import java.lang.annotation.ElementType;
import java.lang.annotation.Inherited;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Marker interface used to annotate methods that are readonly.
*/
@Inherited
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
@InterfaceAudience.Private
@InterfaceStability.Evolving
public @interface ReadOnly {
/**
* @return if true, the annotated method may update the last accessed time
* while performing its read, if access time is enabled.
*/
boolean atimeAffected() default false;
/**
* @return if true, the target method should only be invoked on the active
* namenode. This applies to operations that need to access information that
* is only available on the active namenode.
*/
boolean activeOnly() default false;
/**
* @return if true, when processing the rpc call of the target method, the
* server side will wait if server state id is behind client (msync). If
* false, the method will be processed regardless of server side state.
*/
boolean isCoordinated() default false;
}

View File

@ -35,6 +35,7 @@ import "xattr.proto";
import "encryption.proto";
import "inotify.proto";
import "erasurecoding.proto";
import "HAServiceProtocol.proto";
/**
* The ClientNamenodeProtocol Service defines the interface between a client
@ -830,6 +831,12 @@ message ListOpenFilesResponseProto {
repeated OpenFilesTypeProto types = 3;
}
message MsyncRequestProto {
}
message MsyncResponseProto {
}
message SatisfyStoragePolicyRequestProto {
required string src = 1;
}
@ -838,6 +845,13 @@ message SatisfyStoragePolicyResponseProto {
}
message HAServiceStateRequestProto {
}
message HAServiceStateResponseProto {
required hadoop.common.HAServiceStateProto state = 1;
}
service ClientNamenodeProtocol {
rpc getBlockLocations(GetBlockLocationsRequestProto)
returns(GetBlockLocationsResponseProto);
@ -1024,6 +1038,10 @@ service ClientNamenodeProtocol {
returns(GetQuotaUsageResponseProto);
rpc listOpenFiles(ListOpenFilesRequestProto)
returns(ListOpenFilesResponseProto);
rpc msync(MsyncRequestProto)
returns(MsyncResponseProto);
rpc satisfyStoragePolicy(SatisfyStoragePolicyRequestProto)
returns(SatisfyStoragePolicyResponseProto);
rpc getHAServiceState(HAServiceStateRequestProto)
returns(HAServiceStateResponseProto);
}

View File

@ -0,0 +1,103 @@
/**
* 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.hdfs.protocol;
import org.apache.hadoop.hdfs.server.namenode.ha.ReadOnly;
import org.junit.Test;
import java.lang.reflect.Method;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import static org.junit.Assert.assertEquals;
/**
* Testing class for {@link ReadOnly} annotation on {@link ClientProtocol}.
*/
public class TestReadOnly {
private static final Method[] ALL_METHODS = ClientProtocol.class.getMethods();
private static final Set<String> READONLY_METHOD_NAMES = new HashSet<>(
Arrays.asList(
"getBlockLocations",
"getServerDefaults",
"getStoragePolicies",
"getStoragePolicy",
"getListing",
"getSnapshottableDirListing",
"getPreferredBlockSize",
"listCorruptFileBlocks",
"getFileInfo",
"isFileClosed",
"getFileLinkInfo",
"getLocatedFileInfo",
"getContentSummary",
"getLinkTarget",
"getSnapshotDiffReport",
"getSnapshotDiffReportListing",
"listCacheDirectives",
"listCachePools",
"getAclStatus",
"getEZForPath",
"listEncryptionZones",
"listReencryptionStatus",
"getXAttrs",
"listXAttrs",
"checkAccess",
"getErasureCodingPolicies",
"getErasureCodingCodecs",
"getErasureCodingPolicy",
"listOpenFiles",
"getStats",
"getReplicatedBlockStats",
"getECBlockGroupStats",
"getDatanodeReport",
"getDatanodeStorageReport",
"getDataEncryptionKey",
"getCurrentEditLogTxid",
"getEditsFromTxid",
"getQuotaUsage",
"msync",
"getHAServiceState"
)
);
@Test
public void testReadOnly() {
for (Method m : ALL_METHODS) {
boolean expected = READONLY_METHOD_NAMES.contains(m.getName());
checkIsReadOnly(m.getName(), expected);
}
}
private void checkIsReadOnly(String methodName, boolean expected) {
for (Method m : ALL_METHODS) {
// Note here we only check the FIRST result of overloaded methods
// with the same name. The assumption is that all these methods should
// share the same annotation.
if (m.getName().equals(methodName)) {
assertEquals("Expected ReadOnly for method '" + methodName +
"' to be " + expected,
m.isAnnotationPresent(ReadOnly.class), expected);
return;
}
}
throw new IllegalArgumentException("Unknown method name: " + methodName);
}
}

View File

@ -34,6 +34,7 @@ protobuf_generate_cpp(PROTO_SRCS PROTO_HDRS
${PROTO_HADOOP_DIR}/ProtobufRpcEngine.proto
${PROTO_HADOOP_DIR}/RpcHeader.proto
${PROTO_HADOOP_DIR}/Security.proto
${PROTO_HADOOP_DIR}/HAServiceProtocol.proto
)
add_executable(protoc-gen-hrpc protoc_gen_hrpc.cc)

View File

@ -35,6 +35,9 @@ public enum FederationNamenodeServiceState {
case ACTIVE:
return FederationNamenodeServiceState.ACTIVE;
case STANDBY:
// TODO: we should probably have a separate state OBSERVER for RBF and
// treat it differently.
case OBSERVER:
return FederationNamenodeServiceState.STANDBY;
case INITIALIZING:
return FederationNamenodeServiceState.UNAVAILABLE;

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.hdfs.AddBlockFlag;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
@ -1532,11 +1533,23 @@ public class RouterClientProtocol implements ClientProtocol {
return null;
}
@Override
public void msync() throws IOException {
rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
}
@Override
public void satisfyStoragePolicy(String path) throws IOException {
rpcServer.checkOperation(NameNode.OperationCategory.WRITE, false);
}
@Override
public HAServiceProtocol.HAServiceState getHAServiceState()
throws IOException {
rpcServer.checkOperation(NameNode.OperationCategory.READ, false);
return null;
}
/**
* Determines combinations of eligible src/dst locations for a rename. A
* rename cannot change the namespace. Renames are only allowed if there is an

View File

@ -58,6 +58,7 @@ import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.hdfs.AddBlockFlag;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.inotify.EventBatchList;
@ -1187,12 +1188,23 @@ public class RouterRpcServer extends AbstractService
return clientProto.listOpenFiles(prevId);
}
@Override // ClientProtocol
public HAServiceProtocol.HAServiceState getHAServiceState()
throws IOException {
return clientProto.getHAServiceState();
}
@Override // ClientProtocol
public BatchedEntries<OpenFileEntry> listOpenFiles(long prevId,
EnumSet<OpenFilesType> openFilesTypes, String path) throws IOException {
return clientProto.listOpenFiles(prevId, openFilesTypes, path);
}
@Override // ClientProtocol
public void msync() throws IOException {
clientProto.msync();
}
@Override // ClientProtocol
public void satisfyStoragePolicy(String path) throws IOException {
clientProto.satisfyStoragePolicy(path);

View File

@ -1086,6 +1086,9 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
public static final String DFS_JOURNALNODE_SYNC_INTERVAL_KEY =
"dfs.journalnode.sync.interval";
public static final long DFS_JOURNALNODE_SYNC_INTERVAL_DEFAULT = 2*60*1000L;
public static final String DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY =
"dfs.journalnode.edit-cache-size.bytes";
public static final int DFS_JOURNALNODE_EDIT_CACHE_SIZE_DEFAULT = 1024 * 1024;
// Journal-node related configs for the client side.
public static final String DFS_QJOURNAL_QUEUE_SIZE_LIMIT_KEY = "dfs.qjournal.queued-edits.limit.mb";

View File

@ -30,6 +30,9 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.permission.FsCreateModes;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos.HAServiceStateProto;
import org.apache.hadoop.hdfs.AddBlockFlag;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
@ -155,6 +158,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetSto
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePoliciesResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetStoragePolicyResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.HAServiceStateResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.IsFileClosedResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpgradeStatusRequestProto;
@ -175,6 +180,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Modify
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCacheDirectiveResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.ModifyCachePoolResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.MsyncResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseRequestProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RecoverLeaseResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RefreshNodesRequestProto;
@ -1893,6 +1900,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
}
}
@Override
public MsyncResponseProto msync(RpcController controller,
MsyncRequestProto req) throws ServiceException {
try {
server.msync();
return MsyncResponseProto.newBuilder().build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
@Override
public SatisfyStoragePolicyResponseProto satisfyStoragePolicy(
RpcController controller,
@ -1904,4 +1922,35 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
}
return VOID_SATISFYSTORAGEPOLICY_RESPONSE;
}
@Override
public HAServiceStateResponseProto getHAServiceState(
RpcController controller,
HAServiceStateRequestProto request) throws ServiceException {
try {
HAServiceProtocol.HAServiceState state = server.getHAServiceState();
HAServiceStateProto retState;
switch (state) {
case ACTIVE:
retState = HAServiceProtocolProtos.HAServiceStateProto.ACTIVE;
break;
case STANDBY:
retState = HAServiceProtocolProtos.HAServiceStateProto.STANDBY;
break;
case OBSERVER:
retState = HAServiceProtocolProtos.HAServiceStateProto.OBSERVER;
break;
case INITIALIZING:
default:
retState = HAServiceProtocolProtos.HAServiceStateProto.INITIALIZING;
break;
}
HAServiceStateResponseProto.Builder builder =
HAServiceStateResponseProto.newBuilder();
builder.setState(retState);
return builder.build();
} catch (IOException e) {
throw new ServiceException(e);
}
}
}

View File

@ -779,6 +779,8 @@ public class PBHelper {
return HAServiceState.ACTIVE;
case STANDBY:
return HAServiceState.STANDBY;
case OBSERVER:
return HAServiceState.OBSERVER;
default:
throw new IllegalArgumentException("Unexpected HAServiceStateProto:"
+ s);
@ -794,6 +796,8 @@ public class PBHelper {
return NNHAStatusHeartbeatProto.State.ACTIVE;
case STANDBY:
return NNHAStatusHeartbeatProto.State.STANDBY;
case OBSERVER:
return NNHAStatusHeartbeatProto.State.OBSERVER;
default:
throw new IllegalArgumentException("Unexpected HAServiceState:"
+ s);

View File

@ -22,6 +22,7 @@ import java.net.URL;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -107,6 +108,12 @@ interface AsyncLogger {
* Begin a new epoch on the target node.
*/
public ListenableFuture<NewEpochResponseProto> newEpoch(long epoch);
/**
* Fetch journaled edits from the cache.
*/
public ListenableFuture<GetJournaledEditsResponseProto> getJournaledEdits(
long fromTxnId, int maxTransactions);
/**
* Fetch the list of edit logs available on the remote node.

View File

@ -26,6 +26,7 @@ import java.util.concurrent.TimeoutException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
@ -261,6 +262,19 @@ class AsyncLoggerSet {
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, GetJournaledEditsResponseProto>
getJournaledEdits(long fromTxnId, int maxTransactions) {
Map<AsyncLogger,
ListenableFuture<GetJournaledEditsResponseProto>> calls
= Maps.newHashMap();
for (AsyncLogger logger : loggers) {
ListenableFuture<GetJournaledEditsResponseProto> future =
logger.getJournaledEdits(fromTxnId, maxTransactions);
calls.put(logger, future);
}
return QuorumCall.create(calls);
}
public QuorumCall<AsyncLogger, RemoteEditLogManifest> getEditLogManifest(
long fromTxnId, boolean inProgressOk) {
Map<AsyncLogger,

View File

@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -559,6 +560,19 @@ public class IPCLoggerChannel implements AsyncLogger {
});
}
@Override
public ListenableFuture<GetJournaledEditsResponseProto> getJournaledEdits(
long fromTxnId, int maxTransactions) {
return parallelExecutor.submit(
new Callable<GetJournaledEditsResponseProto>() {
@Override
public GetJournaledEditsResponseProto call() throws IOException {
return getProxy().getJournaledEdits(journalId, nameServiceId,
fromTxnId, maxTransactions);
}
});
}
@Override
public ListenableFuture<RemoteEditLogManifest> getEditLogManifest(
final long fromTxnId, final boolean inProgressOk) {

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
@ -36,6 +37,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -52,6 +54,8 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
import org.apache.hadoop.log.LogThrottlingHelper;
import org.apache.hadoop.log.LogThrottlingHelper.LogAction;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
@ -67,6 +71,16 @@ import com.google.protobuf.TextFormat;
public class QuorumJournalManager implements JournalManager {
static final Logger LOG = LoggerFactory.getLogger(QuorumJournalManager.class);
// This config is not publicly exposed
static final String QJM_RPC_MAX_TXNS_KEY =
"dfs.ha.tail-edits.qjm.rpc.max-txns";
static final int QJM_RPC_MAX_TXNS_DEFAULT = 5000;
// Maximum number of transactions to fetch at a time when using the
// RPC edit fetch mechanism
private final int maxTxnsPerRpc;
// Whether or not in-progress tailing is enabled in the configuration
private final boolean inProgressTailingEnabled;
// Timeouts for which the QJM will wait for each of the following actions.
private final int startSegmentTimeoutMs;
private final int prepareRecoveryTimeoutMs;
@ -93,6 +107,11 @@ public class QuorumJournalManager implements JournalManager {
private int outputBufferCapacity = 512 * 1024;
private final URLConnectionFactory connectionFactory;
/** Limit logging about input stream selection to every 5 seconds max. */
private static final long SELECT_INPUT_STREAM_LOG_INTERVAL_MS = 5000;
private final LogThrottlingHelper selectInputStreamLogHelper =
new LogThrottlingHelper(SELECT_INPUT_STREAM_LOG_INTERVAL_MS);
@VisibleForTesting
public QuorumJournalManager(Configuration conf,
URI uri,
@ -125,6 +144,13 @@ public class QuorumJournalManager implements JournalManager {
this.nameServiceId = nameServiceId;
this.loggers = new AsyncLoggerSet(createLoggers(loggerFactory));
this.maxTxnsPerRpc =
conf.getInt(QJM_RPC_MAX_TXNS_KEY, QJM_RPC_MAX_TXNS_DEFAULT);
Preconditions.checkArgument(maxTxnsPerRpc > 0,
"Must specify %s greater than 0!", QJM_RPC_MAX_TXNS_KEY);
this.inProgressTailingEnabled = conf.getBoolean(
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_DEFAULT);
// Configure timeouts.
this.startSegmentTimeoutMs = conf.getInt(
DFSConfigKeys.DFS_QJOURNAL_START_SEGMENT_TIMEOUT_KEY,
@ -406,11 +432,8 @@ public class QuorumJournalManager implements JournalManager {
layoutVersion);
loggers.waitForWriteQuorum(q, startSegmentTimeoutMs,
"startLogSegment(" + txId + ")");
boolean updateCommittedTxId = conf.getBoolean(
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_DEFAULT);
return new QuorumOutputStream(loggers, txId, outputBufferCapacity,
writeTxnsTimeoutMs, updateCommittedTxId);
writeTxnsTimeoutMs);
}
@Override
@ -478,17 +501,113 @@ public class QuorumJournalManager implements JournalManager {
public void selectInputStreams(Collection<EditLogInputStream> streams,
long fromTxnId, boolean inProgressOk,
boolean onlyDurableTxns) throws IOException {
// Some calls will use inProgressOK to get in-progress edits even if
// the cache used for RPC calls is not enabled; fall back to using the
// streaming mechanism to serve such requests
if (inProgressOk && inProgressTailingEnabled) {
if (LOG.isDebugEnabled()) {
LOG.debug("Tailing edits starting from txn ID " + fromTxnId +
" via RPC mechanism");
}
try {
Collection<EditLogInputStream> rpcStreams = new ArrayList<>();
selectRpcInputStreams(rpcStreams, fromTxnId, onlyDurableTxns);
streams.addAll(rpcStreams);
return;
} catch (IOException ioe) {
LOG.warn("Encountered exception while tailing edits >= " + fromTxnId +
" via RPC; falling back to streaming.", ioe);
}
}
selectStreamingInputStreams(streams, fromTxnId, inProgressOk,
onlyDurableTxns);
}
/**
* Select input streams from the journals, specifically using the RPC
* mechanism optimized for low latency.
*
* @param streams The collection to store the return streams into.
* @param fromTxnId Select edits starting from this transaction ID
* @param onlyDurableTxns Iff true, only include transactions which have been
* committed to a quorum of the journals.
* @throws IOException Upon issues, including cache misses on the journals.
*/
private void selectRpcInputStreams(Collection<EditLogInputStream> streams,
long fromTxnId, boolean onlyDurableTxns) throws IOException {
QuorumCall<AsyncLogger, GetJournaledEditsResponseProto> q =
loggers.getJournaledEdits(fromTxnId, maxTxnsPerRpc);
Map<AsyncLogger, GetJournaledEditsResponseProto> responseMap =
loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs,
"selectRpcInputStreams");
assert responseMap.size() >= loggers.getMajoritySize() :
"Quorum call returned without a majority";
List<Integer> responseCounts = new ArrayList<>();
for (GetJournaledEditsResponseProto resp : responseMap.values()) {
responseCounts.add(resp.getTxnCount());
}
Collections.sort(responseCounts);
int highestTxnCount = responseCounts.get(responseCounts.size() - 1);
if (LOG.isDebugEnabled() || highestTxnCount < 0) {
StringBuilder msg = new StringBuilder("Requested edits starting from ");
msg.append(fromTxnId).append("; got ").append(responseMap.size())
.append(" responses: <");
for (Map.Entry<AsyncLogger, GetJournaledEditsResponseProto> ent :
responseMap.entrySet()) {
msg.append("[").append(ent.getKey()).append(", ")
.append(ent.getValue().getTxnCount()).append("],");
}
msg.append(">");
if (highestTxnCount < 0) {
throw new IOException("Did not get any valid JournaledEdits " +
"responses: " + msg);
} else {
LOG.debug(msg.toString());
}
}
int maxAllowedTxns = !onlyDurableTxns ? highestTxnCount :
responseCounts.get(responseCounts.size() - loggers.getMajoritySize());
if (maxAllowedTxns == 0) {
LOG.debug("No new edits available in logs; requested starting from " +
"ID " + fromTxnId);
return;
}
LogAction logAction = selectInputStreamLogHelper.record(fromTxnId);
if (logAction.shouldLog()) {
LOG.info("Selected loggers with >= " + maxAllowedTxns + " transactions " +
"starting from lowest txn ID " + logAction.getStats(0).getMin() +
LogThrottlingHelper.getLogSupressionMessage(logAction));
}
PriorityQueue<EditLogInputStream> allStreams = new PriorityQueue<>(
JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR);
for (GetJournaledEditsResponseProto resp : responseMap.values()) {
long endTxnId = fromTxnId - 1 +
Math.min(maxAllowedTxns, resp.getTxnCount());
allStreams.add(EditLogFileInputStream.fromByteString(
resp.getEditLog(), fromTxnId, endTxnId, true));
}
JournalSet.chainAndMakeRedundantStreams(streams, allStreams, fromTxnId);
}
/**
* Select input streams from the journals, specifically using the streaming
* mechanism optimized for resiliency / bulk load.
*/
private void selectStreamingInputStreams(
Collection<EditLogInputStream> streams, long fromTxnId,
boolean inProgressOk, boolean onlyDurableTxns) throws IOException {
QuorumCall<AsyncLogger, RemoteEditLogManifest> q =
loggers.getEditLogManifest(fromTxnId, inProgressOk);
Map<AsyncLogger, RemoteEditLogManifest> resps =
loggers.waitForWriteQuorum(q, selectInputStreamsTimeoutMs,
"selectInputStreams");
LOG.debug("selectInputStream manifests:\n" +
"selectStreamingInputStreams");
LOG.debug("selectStreamingInputStream manifests:\n" +
Joiner.on("\n").withKeyValueSeparator(": ").join(resps));
final PriorityQueue<EditLogInputStream> allStreams =
final PriorityQueue<EditLogInputStream> allStreams =
new PriorityQueue<EditLogInputStream>(64,
JournalSet.EDIT_LOG_INPUT_STREAM_COMPARATOR);
for (Map.Entry<AsyncLogger, RemoteEditLogManifest> e : resps.entrySet()) {

View File

@ -33,17 +33,15 @@ class QuorumOutputStream extends EditLogOutputStream {
private EditsDoubleBuffer buf;
private final long segmentTxId;
private final int writeTimeoutMs;
private final boolean updateCommittedTxId;
public QuorumOutputStream(AsyncLoggerSet loggers,
long txId, int outputBufferCapacity,
int writeTimeoutMs, boolean updateCommittedTxId) throws IOException {
int writeTimeoutMs) throws IOException {
super();
this.buf = new EditsDoubleBuffer(outputBufferCapacity);
this.loggers = loggers;
this.segmentTxId = txId;
this.writeTimeoutMs = writeTimeoutMs;
this.updateCommittedTxId = updateCommittedTxId;
}
@Override
@ -112,15 +110,6 @@ class QuorumOutputStream extends EditLogOutputStream {
// RPCs will thus let the loggers know of the most recent transaction, even
// if a logger has fallen behind.
loggers.setCommittedTxId(firstTxToFlush + numReadyTxns - 1);
// If we don't have this dummy send, committed TxId might be one-batch
// stale on the Journal Nodes
if (updateCommittedTxId) {
QuorumCall<AsyncLogger, Void> fakeCall = loggers.sendEdits(
segmentTxId, firstTxToFlush,
0, new byte[0]);
loggers.waitForWriteQuorum(fakeCall, writeTimeoutMs, "sendEdits");
}
}
}

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -139,7 +140,28 @@ public interface QJournalProtocol {
long sinceTxId,
boolean inProgressOk)
throws IOException;
/**
* Fetch edit logs present in the Journal's in-memory cache of edits
* ({@link org.apache.hadoop.hdfs.qjournal.server.JournaledEditsCache}).
* To enable this cache, in-progress edit log tailing must be enabled via the
* {@value DFSConfigKeys#DFS_HA_TAILEDITS_INPROGRESS_KEY} configuration key.
*
* @param jid The ID of the journal from which to fetch edits.
* @param nameServiceId The ID of the namespace for which to fetch edits.
* @param sinceTxId Fetch edits starting at this transaction ID
* @param maxTxns Request at most this many transactions to be returned
* @throws IOException If there was an issue encountered while fetching edits
* from the cache, including a cache miss (cache does not contain the
* requested edits). The caller should then attempt to fetch the edits via
* the streaming mechanism (starting with
* {@link #getEditLogManifest(String, String, long, boolean)}).
* @return Response containing serialized edits to be loaded
* @see org.apache.hadoop.hdfs.qjournal.server.JournaledEditsCache
*/
GetJournaledEditsResponseProto getJournaledEdits(String jid,
String nameServiceId, long sinceTxId, int maxTxns) throws IOException;
/**
* Begin the recovery process for a given segment. See the HDFS-3077
* design document for details.

View File

@ -45,6 +45,8 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatReq
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateRequestProto;
@ -235,6 +237,18 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
}
}
@Override
public GetJournaledEditsResponseProto getJournaledEdits(
RpcController controller, GetJournaledEditsRequestProto request)
throws ServiceException {
try {
return impl.getJournaledEdits(request.getJid().getIdentifier(),
request.hasNameServiceId() ? request.getNameServiceId() : null,
request.getSinceTxId(), request.getMaxTxns());
} catch (IOException ioe) {
throw new ServiceException(ioe);
}
}
@Override
public PrepareRecoveryResponseProto prepareRecovery(RpcController controller,

View File

@ -40,6 +40,8 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FinalizeL
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.FormatRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeRequestProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalCTimeResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateRequestProto;
@ -283,6 +285,24 @@ public class QJournalProtocolTranslatorPB implements ProtocolMetaInterface,
}
}
@Override
public GetJournaledEditsResponseProto getJournaledEdits(String jid,
String nameServiceId, long sinceTxId, int maxTxns) throws IOException {
try {
GetJournaledEditsRequestProto.Builder req =
GetJournaledEditsRequestProto.newBuilder()
.setJid(convertJournalId(jid))
.setSinceTxId(sinceTxId)
.setMaxTxns(maxTxns);
if (nameServiceId != null) {
req.setNameServiceId(nameServiceId);
}
return rpcProxy.getJournaledEdits(NULL_CONTROLLER, req.build());
} catch (ServiceException se) {
throw ProtobufHelper.getRemoteException(se);
}
}
@Override
public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo,
long segmentTxId) throws IOException {

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdfs.qjournal.server;
import com.google.protobuf.ByteString;
import java.io.Closeable;
import java.io.File;
import java.io.FileInputStream;
@ -24,9 +25,11 @@ import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStreamWriter;
import java.net.URL;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.StandardCopyOption;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.TimeUnit;
@ -36,10 +39,12 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.qjournal.protocol.JournalNotFormattedException;
import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PersistedRecoveryPaxosData;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -85,6 +90,7 @@ public class Journal implements Closeable {
// Current writing state
private EditLogOutputStream curSegment;
private long curSegmentTxId = HdfsServerConstants.INVALID_TXID;
private int curSegmentLayoutVersion = 0;
private long nextTxId = HdfsServerConstants.INVALID_TXID;
private long highestWrittenTxId = 0;
@ -133,6 +139,8 @@ public class Journal implements Closeable {
private final FileJournalManager fjm;
private final JournaledEditsCache cache;
private final JournalMetrics metrics;
private long lastJournalTimestamp = 0;
@ -156,6 +164,13 @@ public class Journal implements Closeable {
refreshCachedData();
this.fjm = storage.getJournalManager();
if (conf.getBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_DEFAULT)) {
this.cache = new JournaledEditsCache(conf);
} else {
this.cache = null;
}
this.metrics = JournalMetrics.create(this);
@ -361,6 +376,7 @@ public class Journal implements Closeable {
curSegment.abort();
curSegment = null;
curSegmentTxId = HdfsServerConstants.INVALID_TXID;
curSegmentLayoutVersion = 0;
}
/**
@ -406,6 +422,9 @@ public class Journal implements Closeable {
LOG.trace("Writing txid " + firstTxnId + "-" + lastTxnId +
" ; journal id: " + journalId);
}
if (cache != null) {
cache.storeEdits(records, firstTxnId, lastTxnId, curSegmentLayoutVersion);
}
// If the edit has already been marked as committed, we know
// it has been fsynced on a quorum of other nodes, and we are
@ -593,6 +612,7 @@ public class Journal implements Closeable {
curSegment = fjm.startLogSegment(txid, layoutVersion);
curSegmentTxId = txid;
curSegmentLayoutVersion = layoutVersion;
nextTxId = txid;
}
@ -612,6 +632,7 @@ public class Journal implements Closeable {
curSegment.close();
curSegment = null;
curSegmentTxId = HdfsServerConstants.INVALID_TXID;
curSegmentLayoutVersion = 0;
}
checkSync(nextTxId == endTxId + 1,
@ -712,6 +733,44 @@ public class Journal implements Closeable {
return new RemoteEditLogManifest(logs, getCommittedTxnId());
}
/**
* @see QJournalProtocol#getJournaledEdits(String, String, long, int)
*/
public GetJournaledEditsResponseProto getJournaledEdits(long sinceTxId,
int maxTxns) throws IOException {
if (cache == null) {
throw new IOException("The journal edits cache is not enabled, which " +
"is a requirement to fetch journaled edits via RPC. Please enable " +
"it via " + DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY);
}
if (sinceTxId > getHighestWrittenTxId()) {
// Requested edits that don't exist yet; short-circuit the cache here
metrics.rpcEmptyResponses.incr();
return GetJournaledEditsResponseProto.newBuilder().setTxnCount(0).build();
}
try {
List<ByteBuffer> buffers = new ArrayList<>();
int txnCount = cache.retrieveEdits(sinceTxId, maxTxns, buffers);
int totalSize = 0;
for (ByteBuffer buf : buffers) {
totalSize += buf.remaining();
}
metrics.txnsServedViaRpc.incr(txnCount);
metrics.bytesServedViaRpc.incr(totalSize);
ByteString.Output output = ByteString.newOutput(totalSize);
for (ByteBuffer buf : buffers) {
output.write(buf.array(), buf.position(), buf.remaining());
}
return GetJournaledEditsResponseProto.newBuilder()
.setTxnCount(txnCount)
.setEditLog(output.toByteString())
.build();
} catch (JournaledEditsCache.CacheMissException cme) {
metrics.rpcRequestCacheMissAmount.add(cme.getCacheMissAmount());
throw cme;
}
}
/**
* @return the current state of the given segment, or null if the
* segment does not exist.
@ -1150,4 +1209,10 @@ public class Journal implements Closeable {
public Long getJournalCTime() throws IOException {
return storage.getJournalManager().getJournalCTime();
}
@VisibleForTesting
JournaledEditsCache getJournaledEditsCache() {
return cache;
}
}

View File

@ -25,6 +25,8 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.MetricsRegistry;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableQuantiles;
import org.apache.hadoop.metrics2.lib.MutableStat;
/**
* The server-side metrics for a journal from the JournalNode's
@ -42,7 +44,23 @@ class JournalMetrics {
@Metric("Number of bytes written since startup")
MutableCounterLong bytesWritten;
@Metric("Number of txns served via RPC")
MutableCounterLong txnsServedViaRpc;
@Metric("Number of bytes served via RPC")
MutableCounterLong bytesServedViaRpc;
@Metric
MutableStat rpcRequestCacheMissAmount = new MutableStat(
"RpcRequestCacheMissAmount", "Number of RPC requests unable to be " +
"served due to lack of availability in cache, and how many " +
"transactions away the request was from being in the cache.",
"Misses", "Txns");
@Metric("Number of RPC requests with zero edits returned")
MutableCounterLong rpcEmptyResponses;
@Metric("Number of batches written where this node was lagging")
MutableCounterLong batchesWrittenWhileLagging;

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.InterQJournalProtocolProtos.InterQJournalProtocolService;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocol;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetEditLogManifestResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
@ -234,6 +235,13 @@ public class JournalNodeRpcServer implements QJournalProtocol,
.build();
}
@Override
public GetJournaledEditsResponseProto getJournaledEdits(String jid,
String nameServiceId, long sinceTxId, int maxTxns) throws IOException {
return jn.getOrCreateJournal(jid, nameServiceId)
.getJournaledEdits(sinceTxId, maxTxns);
}
@Override
public PrepareRecoveryResponseProto prepareRecovery(RequestInfo reqInfo,
long segmentTxId) throws IOException {

View File

@ -0,0 +1,417 @@
/**
* 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.hdfs.qjournal.server;
import com.google.common.annotations.VisibleForTesting;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
import org.apache.hadoop.util.AutoCloseableLock;
/**
* An in-memory cache of edits in their serialized form. This is used to serve
* the {@link Journal#getJournaledEdits(long, int)} call, used by the
* QJM when {@value DFSConfigKeys#DFS_HA_TAILEDITS_INPROGRESS_KEY} is
* enabled.
*
* <p>When a batch of edits is received by the JournalNode, it is put into this
* cache via {@link #storeEdits(byte[], long, long, int)}. Edits must be
* stored contiguously; if a batch of edits is stored that does not align with
* the previously stored edits, the cache will be cleared before storing new
* edits to avoid gaps. This decision is made because gaps are only handled
* when in recovery mode, which the cache is not intended to be used for.
*
* <p>Batches of edits are stored in a {@link TreeMap} mapping the starting
* transaction ID of the batch to the data buffer. Upon retrieval, the
* relevant data buffers are concatenated together and a header is added
* to construct a fully-formed edit data stream.
*
* <p>The cache is of a limited size capacity determined by
* {@value DFSConfigKeys#DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY}. If the capacity
* is exceeded after adding a new batch of edits, batches of edits are removed
* until the total size is less than the capacity, starting from the ones
* containing the oldest transactions. Transactions range in size, but a
* decent rule of thumb is that 200 bytes are needed per transaction. Monitoring
* the {@link JournalMetrics#rpcRequestCacheMissAmount} metric is recommended
* to determine if the cache is too small; it will indicate both how many
* cache misses occurred, and how many more transactions would have been
* needed in the cache to serve the request.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
class JournaledEditsCache {
private static final int INVALID_LAYOUT_VERSION = 0;
private static final long INVALID_TXN_ID = -1;
/** The capacity, in bytes, of this cache. */
private final int capacity;
/**
* Read/write lock pair wrapped in AutoCloseable; these refer to the same
* underlying lock.
*/
private final AutoCloseableLock readLock;
private final AutoCloseableLock writeLock;
// ** Start lock-protected fields **
/**
* Stores the actual data as a mapping of the StartTxnId of a batch of edits
* to the serialized batch of edits. Stores only contiguous ranges; that is,
* the last transaction ID in one batch is always one less than the first
* transaction ID in the next batch. Though the map is protected by the lock,
* individual data buffers are immutable and can be accessed without locking.
*/
private final NavigableMap<Long, byte[]> dataMap = new TreeMap<>();
/** Stores the layout version currently present in the cache. */
private int layoutVersion = INVALID_LAYOUT_VERSION;
/** Stores the serialized version of the header for the current version. */
private ByteBuffer layoutHeader;
/**
* The lowest/highest transaction IDs present in the cache.
* {@value INVALID_TXN_ID} if there are no transactions in the cache.
*/
private long lowestTxnId;
private long highestTxnId;
/**
* The lowest transaction ID that was ever present in the cache since last
* being reset (i.e. since initialization or since reset due to being out of
* sync with the Journal). Until the cache size goes above capacity, this is
* equal to lowestTxnId.
*/
private long initialTxnId;
/** The current total size of all buffers in this cache. */
private int totalSize;
// ** End lock-protected fields **
JournaledEditsCache(Configuration conf) {
capacity = conf.getInt(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY,
DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_DEFAULT);
if (capacity > 0.9 * Runtime.getRuntime().maxMemory()) {
Journal.LOG.warn(String.format("Cache capacity is set at %d bytes but " +
"maximum JVM memory is only %d bytes. It is recommended that you " +
"decrease the cache size or increase the heap size.",
capacity, Runtime.getRuntime().maxMemory()));
}
Journal.LOG.info("Enabling the journaled edits cache with a capacity " +
"of bytes: " + capacity);
ReadWriteLock lock = new ReentrantReadWriteLock(true);
readLock = new AutoCloseableLock(lock.readLock());
writeLock = new AutoCloseableLock(lock.writeLock());
initialize(INVALID_TXN_ID);
}
/**
* Fetch the data for edits starting at the specific transaction ID, fetching
* up to {@code maxTxns} transactions. Populates a list of output buffers
* which contains a serialized version of the edits, and returns the count of
* edits contained within the serialized buffers. The serialized edits are
* prefixed with a standard edit log header containing information about the
* layout version. The transactions returned are guaranteed to have contiguous
* transaction IDs.
*
* If {@code requestedStartTxn} is higher than the highest transaction which
* has been added to this cache, a response with an empty buffer and a
* transaction count of 0 will be returned. If {@code requestedStartTxn} is
* lower than the lowest transaction currently contained in this cache, or no
* transactions have yet been added to the cache, an exception will be thrown.
*
* @param requestedStartTxn The ID of the first transaction to return. If any
* transactions are returned, it is guaranteed that
* the first one will have this ID.
* @param maxTxns The maximum number of transactions to return.
* @param outputBuffers A list to populate with output buffers. When
* concatenated, these form a full response.
* @return The number of transactions contained within the set of output
* buffers.
* @throws IOException If transactions are requested which cannot be served
* by this cache.
*/
int retrieveEdits(long requestedStartTxn, int maxTxns,
List<ByteBuffer> outputBuffers) throws IOException {
int txnCount = 0;
try (AutoCloseableLock l = readLock.acquire()) {
if (lowestTxnId == INVALID_TXN_ID || requestedStartTxn < lowestTxnId) {
throw getCacheMissException(requestedStartTxn);
} else if (requestedStartTxn > highestTxnId) {
return 0;
}
outputBuffers.add(layoutHeader);
Iterator<Map.Entry<Long, byte[]>> incrBuffIter =
dataMap.tailMap(dataMap.floorKey(requestedStartTxn), true)
.entrySet().iterator();
long prevTxn = requestedStartTxn;
byte[] prevBuf = null;
// Stop when maximum transactions reached...
while ((txnCount < maxTxns) &&
// ... or there are no more entries ...
(incrBuffIter.hasNext() || prevBuf != null)) {
long currTxn;
byte[] currBuf;
if (incrBuffIter.hasNext()) {
Map.Entry<Long, byte[]> ent = incrBuffIter.next();
currTxn = ent.getKey();
currBuf = ent.getValue();
} else {
// This accounts for the trailing entry
currTxn = highestTxnId + 1;
currBuf = null;
}
if (prevBuf != null) { // True except for the first loop iteration
outputBuffers.add(ByteBuffer.wrap(prevBuf));
// if prevTxn < requestedStartTxn, the extra transactions will get
// removed after the loop, so don't include them in the txn count
txnCount += currTxn - Math.max(requestedStartTxn, prevTxn);
}
prevTxn = currTxn;
prevBuf = currBuf;
}
// Release the lock before doing operations on the buffers (deserializing
// to find transaction boundaries, and copying into an output buffer)
}
// Remove extra leading transactions in the first buffer
ByteBuffer firstBuf = outputBuffers.get(1); // 0th is the header
firstBuf.position(
findTransactionPosition(firstBuf.array(), requestedStartTxn));
// Remove trailing transactions in the last buffer if necessary
if (txnCount > maxTxns) {
ByteBuffer lastBuf = outputBuffers.get(outputBuffers.size() - 1);
int limit =
findTransactionPosition(lastBuf.array(), requestedStartTxn + maxTxns);
lastBuf.limit(limit);
txnCount = maxTxns;
}
return txnCount;
}
/**
* Store a batch of serialized edits into this cache. Removes old batches
* as necessary to keep the total size of the cache below the capacity.
* See the class Javadoc for more info.
*
* This attempts to always handle malformed inputs gracefully rather than
* throwing an exception, to allow the rest of the Journal's operations
* to proceed normally.
*
* @param inputData A buffer containing edits in serialized form
* @param newStartTxn The txn ID of the first edit in {@code inputData}
* @param newEndTxn The txn ID of the last edit in {@code inputData}
* @param newLayoutVersion The version of the layout used to serialize
* the edits
*/
void storeEdits(byte[] inputData, long newStartTxn, long newEndTxn,
int newLayoutVersion) {
if (newStartTxn < 0 || newEndTxn < newStartTxn) {
Journal.LOG.error(String.format("Attempted to cache data of length %d " +
"with newStartTxn %d and newEndTxn %d",
inputData.length, newStartTxn, newEndTxn));
return;
}
try (AutoCloseableLock l = writeLock.acquire()) {
if (newLayoutVersion != layoutVersion) {
try {
updateLayoutVersion(newLayoutVersion, newStartTxn);
} catch (IOException ioe) {
Journal.LOG.error(String.format("Unable to save new edits [%d, %d] " +
"due to exception when updating to new layout version %d",
newStartTxn, newEndTxn, newLayoutVersion), ioe);
return;
}
} else if (lowestTxnId == INVALID_TXN_ID) {
Journal.LOG.info("Initializing edits cache starting from txn ID " +
newStartTxn);
initialize(newStartTxn);
} else if (highestTxnId + 1 != newStartTxn) {
// Cache is out of sync; clear to avoid storing noncontiguous regions
Journal.LOG.error(String.format("Edits cache is out of sync; " +
"looked for next txn id at %d but got start txn id for " +
"cache put request at %d. Reinitializing at new request.",
highestTxnId + 1, newStartTxn));
initialize(newStartTxn);
}
while ((totalSize + inputData.length) > capacity && !dataMap.isEmpty()) {
Map.Entry<Long, byte[]> lowest = dataMap.firstEntry();
dataMap.remove(lowest.getKey());
totalSize -= lowest.getValue().length;
}
if (inputData.length > capacity) {
initialize(INVALID_TXN_ID);
Journal.LOG.warn(String.format("A single batch of edits was too " +
"large to fit into the cache: startTxn = %d, endTxn = %d, " +
"input length = %d. The capacity of the cache (%s) must be " +
"increased for it to work properly (current capacity %d)." +
"Cache is now empty.",
newStartTxn, newEndTxn, inputData.length,
DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, capacity));
return;
}
if (dataMap.isEmpty()) {
lowestTxnId = newStartTxn;
} else {
lowestTxnId = dataMap.firstKey();
}
dataMap.put(newStartTxn, inputData);
highestTxnId = newEndTxn;
totalSize += inputData.length;
}
}
/**
* Skip through a given stream of edits until the given transaction ID is
* found. Return the number of bytes that appear prior to the given
* transaction.
*
* @param buf A buffer containing a stream of serialized edits
* @param txnId The transaction ID to search for
* @return The number of bytes appearing in {@code buf} <i>before</i>
* the start of the transaction with ID {@code txnId}.
*/
private int findTransactionPosition(byte[] buf, long txnId)
throws IOException {
ByteArrayInputStream bais = new ByteArrayInputStream(buf);
FSEditLogLoader.PositionTrackingInputStream tracker =
new FSEditLogLoader.PositionTrackingInputStream(bais);
FSEditLogOp.Reader reader = FSEditLogOp.Reader.create(
new DataInputStream(tracker), tracker, layoutVersion);
long previousPos = 0;
while (reader.scanOp() < txnId) {
previousPos = tracker.getPos();
}
// tracker is backed by a byte[]; position cannot go above an integer
return (int) previousPos;
}
/**
* Update the layout version of the cache. This clears out all existing
* entries, and populates the new layout version and header for that version.
*
* @param newLayoutVersion The new layout version to be stored in the cache
* @param newStartTxn The new lowest transaction in the cache
*/
private void updateLayoutVersion(int newLayoutVersion, long newStartTxn)
throws IOException {
StringBuilder logMsg = new StringBuilder()
.append("Updating edits cache to use layout version ")
.append(newLayoutVersion)
.append(" starting from txn ID ")
.append(newStartTxn);
if (layoutVersion != INVALID_LAYOUT_VERSION) {
logMsg.append("; previous version was ").append(layoutVersion)
.append("; old entries will be cleared.");
}
Journal.LOG.info(logMsg.toString());
initialize(newStartTxn);
ByteArrayOutputStream baos = new ByteArrayOutputStream();
EditLogFileOutputStream.writeHeader(newLayoutVersion,
new DataOutputStream(baos));
layoutVersion = newLayoutVersion;
layoutHeader = ByteBuffer.wrap(baos.toByteArray());
}
/**
* Initialize the cache back to a clear state.
*
* @param newInitialTxnId The new lowest transaction ID stored in the cache.
* This should be {@value INVALID_TXN_ID} if the cache
* is to remain empty at this time.
*/
private void initialize(long newInitialTxnId) {
dataMap.clear();
totalSize = 0;
initialTxnId = newInitialTxnId;
lowestTxnId = initialTxnId;
highestTxnId = INVALID_TXN_ID; // this will be set later
}
/**
* Return the underlying data buffer used to store information about the
* given transaction ID.
*
* @param txnId Transaction ID whose containing buffer should be fetched.
* @return The data buffer for the transaction
*/
@VisibleForTesting
byte[] getRawDataForTests(long txnId) {
try (AutoCloseableLock l = readLock.acquire()) {
return dataMap.floorEntry(txnId).getValue();
}
}
private CacheMissException getCacheMissException(long requestedTxnId) {
if (lowestTxnId == INVALID_TXN_ID) {
return new CacheMissException(0, "Cache is empty; either it was never " +
"written to or the last write overflowed the cache capacity.");
} else if (requestedTxnId < initialTxnId) {
return new CacheMissException(initialTxnId - requestedTxnId,
"Cache started at txn ID %d but requested txns starting at %d.",
initialTxnId, requestedTxnId);
} else {
return new CacheMissException(lowestTxnId - requestedTxnId,
"Oldest txn ID available in the cache is %d, but requested txns " +
"starting at %d. The cache size (%s) may need to be increased " +
"to hold more transactions (currently %d bytes containing %d " +
"transactions)", lowestTxnId, requestedTxnId,
DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, capacity,
highestTxnId - lowestTxnId + 1);
}
}
static class CacheMissException extends IOException {
private static final long serialVersionUID = 0L;
private final long cacheMissAmount;
CacheMissException(long cacheMissAmount, String msgFormat,
Object... msgArgs) {
super(String.format(msgFormat, msgArgs));
this.cacheMissAmount = cacheMissAmount;
}
long getCacheMissAmount() {
return cacheMissAmount;
}
}
}

View File

@ -157,7 +157,9 @@ public interface HdfsServerConstants {
// only used for StorageDirectory.analyzeStorage() in hot swap drive scenario.
// TODO refactor StorageDirectory.analyzeStorage() so that we can do away with
// this in StartupOption.
HOTSWAP("-hotswap");
HOTSWAP("-hotswap"),
// Startup the namenode in observer mode.
OBSERVER("-observer");
private static final Pattern ENUM_WITH_ROLLING_UPGRADE_OPTION = Pattern.compile(
"(\\w+)\\((\\w+)\\)");

View File

@ -928,7 +928,7 @@ class BPServiceActor implements Runnable {
scheduler.scheduleHeartbeat();
// HDFS-9917,Standby NN IBR can be very huge if standby namenode is down
// for sometime.
if (state == HAServiceState.STANDBY) {
if (state == HAServiceState.STANDBY || state == HAServiceState.OBSERVER) {
ibrManager.clearIBRs();
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.namenode;
import com.google.protobuf.ByteString;
import java.io.BufferedInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
@ -119,6 +120,23 @@ public class EditLogFileInputStream extends EditLogInputStream {
return new EditLogFileInputStream(new URLLog(connectionFactory, url),
startTxId, endTxId, inProgress);
}
/**
* Create an EditLogInputStream from a {@link ByteString}, i.e. an in-memory
* collection of bytes.
*
* @param bytes The byte string to read from
* @param startTxId the expected starting transaction ID
* @param endTxId the expected ending transaction ID
* @param inProgress whether the log is in-progress
* @return An edit stream to read from
*/
public static EditLogInputStream fromByteString(ByteString bytes,
long startTxId, long endTxId, boolean inProgress) {
return new EditLogFileInputStream(new ByteStringLog(bytes,
String.format("ByteStringEditLog[%d, %d]", startTxId, endTxId)),
startTxId, endTxId, inProgress);
}
private EditLogFileInputStream(LogSource log,
long firstTxId, long lastTxId,
@ -376,6 +394,32 @@ public class EditLogFileInputStream extends EditLogInputStream {
public long length();
public String getName();
}
private static class ByteStringLog implements LogSource {
private final ByteString bytes;
private final String name;
public ByteStringLog(ByteString bytes, String name) {
this.bytes = bytes;
this.name = name;
}
@Override
public InputStream getInputStream() {
return bytes.newInput();
}
@Override
public long length() {
return bytes.size();
}
@Override
public String getName() {
return name;
}
}
private static class FileLog implements LogSource {
private final File file;

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.namenode.FSImageFormat.renameReservedPathsOnUpgrade;
import static org.apache.hadoop.util.Time.monotonicNow;
import java.io.FilterInputStream;
import java.io.IOException;
@ -113,11 +112,16 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
import org.apache.hadoop.hdfs.util.Holder;
import org.apache.hadoop.log.LogThrottlingHelper;
import org.apache.hadoop.util.ChunkedArrayList;
import org.apache.hadoop.util.Timer;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import static org.apache.hadoop.log.LogThrottlingHelper.LogAction;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class FSEditLogLoader {
@ -125,16 +129,29 @@ public class FSEditLogLoader {
LoggerFactory.getLogger(FSEditLogLoader.class.getName());
static final long REPLAY_TRANSACTION_LOG_INTERVAL = 1000; // 1sec
/** Limit logging about edit loading to every 5 seconds max. */
@VisibleForTesting
static final long LOAD_EDIT_LOG_INTERVAL_MS = 5000;
private final LogThrottlingHelper loadEditsLogHelper =
new LogThrottlingHelper(LOAD_EDIT_LOG_INTERVAL_MS);
private final FSNamesystem fsNamesys;
private final BlockManager blockManager;
private final Timer timer;
private long lastAppliedTxId;
/** Total number of end transactions loaded. */
private int totalEdits = 0;
public FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId) {
this(fsNamesys, lastAppliedTxId, new Timer());
}
@VisibleForTesting
FSEditLogLoader(FSNamesystem fsNamesys, long lastAppliedTxId, Timer timer) {
this.fsNamesys = fsNamesys;
this.blockManager = fsNamesys.getBlockManager();
this.lastAppliedTxId = lastAppliedTxId;
this.timer = timer;
}
long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
@ -155,14 +172,26 @@ public class FSEditLogLoader {
prog.beginStep(Phase.LOADING_EDITS, step);
fsNamesys.writeLock();
try {
long startTime = monotonicNow();
FSImage.LOG.info("Start loading edits file " + edits.getName()
+ " maxTxnsToRead = " + maxTxnsToRead);
long startTime = timer.monotonicNow();
LogAction preLogAction = loadEditsLogHelper.record("pre", startTime);
if (preLogAction.shouldLog()) {
FSImage.LOG.info("Start loading edits file " + edits.getName()
+ " maxTxnsToRead = " + maxTxnsToRead +
LogThrottlingHelper.getLogSupressionMessage(preLogAction));
}
long numEdits = loadEditRecords(edits, false, expectedStartingTxId,
maxTxnsToRead, startOpt, recovery);
FSImage.LOG.info("Edits file " + edits.getName()
+ " of size " + edits.length() + " edits # " + numEdits
+ " loaded in " + (monotonicNow()-startTime)/1000 + " seconds");
long endTime = timer.monotonicNow();
LogAction postLogAction = loadEditsLogHelper.record("post", endTime,
numEdits, edits.length(), endTime - startTime);
if (postLogAction.shouldLog()) {
FSImage.LOG.info("Loaded {} edits file(s) (the last named {}) of " +
"total size {}, total edits {}, total load time {} ms",
postLogAction.getCount(), edits.getName(),
postLogAction.getStats(1).getSum(),
postLogAction.getStats(0).getSum(),
postLogAction.getStats(2).getSum());
}
return numEdits;
} finally {
edits.close();
@ -203,7 +232,7 @@ public class FSEditLogLoader {
Step step = createStartupProgressStep(in);
prog.setTotal(Phase.LOADING_EDITS, step, numTxns);
Counter counter = prog.getCounter(Phase.LOADING_EDITS, step);
long lastLogTime = monotonicNow();
long lastLogTime = timer.monotonicNow();
long lastInodeId = fsNamesys.dir.getLastInodeId();
try {
@ -283,7 +312,7 @@ public class FSEditLogLoader {
}
// log progress
if (op.hasTransactionId()) {
long now = monotonicNow();
long now = timer.monotonicNow();
if (now - lastLogTime > REPLAY_TRANSACTION_LOG_INTERVAL) {
long deltaTxId = lastAppliedTxId - expectedStartingTxId + 1;
int percent = Math.round((float) deltaTxId / numTxns * 100);

View File

@ -69,6 +69,8 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.util.Canceler;
import org.apache.hadoop.hdfs.util.MD5FileUtils;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.log.LogThrottlingHelper;
import org.apache.hadoop.log.LogThrottlingHelper.LogAction;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.Time;
@ -124,6 +126,11 @@ public class FSImage implements Closeable {
private final Set<Long> currentlyCheckpointing =
Collections.<Long>synchronizedSet(new HashSet<Long>());
/** Limit logging about edit loading to every 5 seconds max. */
private static final long LOAD_EDIT_LOG_INTERVAL_MS = 5000;
private final LogThrottlingHelper loadEditLogHelper =
new LogThrottlingHelper(LOAD_EDIT_LOG_INTERVAL_MS);
/**
* Construct an FSImage
* @param conf Configuration
@ -886,8 +893,16 @@ public class FSImage implements Closeable {
// Load latest edits
for (EditLogInputStream editIn : editStreams) {
LOG.info("Reading " + editIn + " expecting start txid #" +
(lastAppliedTxId + 1));
LogAction logAction = loadEditLogHelper.record();
if (logAction.shouldLog()) {
String logSuppressed = "";
if (logAction.getCount() > 1) {
logSuppressed = "; suppressed logging for " +
(logAction.getCount() - 1) + " edit reads";
}
LOG.info("Reading " + editIn + " expecting start txid #" +
(lastAppliedTxId + 1) + logSuppressed);
}
try {
loader.loadFSEdits(editIn, lastAppliedTxId + 1, maxTxnsToRead,
startOpt, recovery);

View File

@ -93,6 +93,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LI
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.ACTIVE;
import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.OBSERVER;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
import org.apache.hadoop.hdfs.protocol.OpenFilesIterator.OpenFilesType;
@ -286,6 +288,7 @@ import org.apache.hadoop.hdfs.web.JsonUtil;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.CallerContext;
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
import org.apache.hadoop.ipc.RetriableException;
import org.apache.hadoop.ipc.RetryCache;
import org.apache.hadoop.ipc.Server;
@ -463,7 +466,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
/** The namespace tree. */
FSDirectory dir;
private final BlockManager blockManager;
private BlockManager blockManager;
private final SnapshotManager snapshotManager;
private final CacheManager cacheManager;
private final DatanodeStatistics datanodeStatistics;
@ -542,7 +545,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
private final ReentrantLock cpLock;
/**
* Used when this NN is in standby state to read from the shared edit log.
* Used when this NN is in standby or observer state to read from the
* shared edit log.
*/
private EditLogTailer editLogTailer = null;
@ -1380,24 +1384,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
}
/**
* Start services required in standby state
* Start services required in standby or observer state
*
* @throws IOException
*/
void startStandbyServices(final Configuration conf) throws IOException {
LOG.info("Starting services required for standby state");
void startStandbyServices(final Configuration conf, boolean isObserver)
throws IOException {
LOG.info("Starting services required for " +
(isObserver ? "observer" : "standby") + " state");
if (!getFSImage().editLog.isOpenForRead()) {
// During startup, we're already open for read.
getFSImage().editLog.initSharedJournalsForRead();
}
blockManager.setPostponeBlocksFromFuture(true);
// Disable quota checks while in standby.
dir.disableQuotaChecks();
editLogTailer = new EditLogTailer(this, conf);
editLogTailer.start();
if (standbyShouldCheckpoint) {
if (!isObserver && standbyShouldCheckpoint) {
standbyCheckpointer = new StandbyCheckpointer(conf, this);
standbyCheckpointer.start();
}
@ -1731,7 +1736,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return haEnabled;
}
return HAServiceState.STANDBY == haContext.getState().getServiceState();
return HAServiceState.STANDBY == haContext.getState().getServiceState() ||
HAServiceState.OBSERVER == haContext.getState().getServiceState();
}
/**
@ -1963,13 +1969,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
SafeModeException se = newSafemodeException(
"Zero blocklocations for " + srcArg);
if (haEnabled && haContext != null &&
haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
(haContext.getState().getServiceState() == ACTIVE ||
haContext.getState().getServiceState() == OBSERVER)) {
throw new RetriableException(se);
} else {
throw se;
}
}
}
} else if (haEnabled && haContext != null &&
haContext.getState().getServiceState() == OBSERVER) {
for (LocatedBlock b : res.blocks.getLocatedBlocks()) {
if (b.getLocations() == null || b.getLocations().length == 0) {
throw new ObserverRetryOnActiveException("Zero blocklocations for "
+ srcArg);
}
}
}
} catch (AccessControlException e) {
logAuditEvent(false, operationName, srcArg);
@ -6298,6 +6313,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
return blockManager;
}
@VisibleForTesting
public void setBlockManagerForTesting(BlockManager bm) {
this.blockManager = bm;
}
/** @return the FSDirectory. */
@Override
public FSDirectory getFSDirectory() {

View File

@ -0,0 +1,158 @@
/**
* 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.hdfs.server.namenode;
import java.lang.reflect.Method;
import java.util.HashSet;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.server.namenode.ha.ReadOnly;
import org.apache.hadoop.ipc.AlignmentContext;
import org.apache.hadoop.ipc.RetriableException;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
/**
* This is the server side implementation responsible for passing
* state alignment info to clients.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
class GlobalStateIdContext implements AlignmentContext {
/**
* Estimated number of journal transactions a typical NameNode can execute
* per second. The number is used to estimate how long a client's
* RPC request will wait in the call queue before the Observer catches up
* with its state id.
*/
private static final long ESTIMATED_TRANSACTIONS_PER_SECOND = 10000L;
/**
* The client wait time on an RPC request is composed of
* the server execution time plus the communication time.
* This is an expected fraction of the total wait time spent on
* server execution.
*/
private static final float ESTIMATED_SERVER_TIME_MULTIPLIER = 0.8f;
private final FSNamesystem namesystem;
private final HashSet<String> coordinatedMethods;
/**
* Server side constructor.
* @param namesystem server side state provider
*/
GlobalStateIdContext(FSNamesystem namesystem) {
this.namesystem = namesystem;
this.coordinatedMethods = new HashSet<>();
// For now, only ClientProtocol methods can be coordinated, so only checking
// against ClientProtocol.
for (Method method : ClientProtocol.class.getDeclaredMethods()) {
if (method.isAnnotationPresent(ReadOnly.class) &&
method.getAnnotationsByType(ReadOnly.class)[0].isCoordinated()) {
coordinatedMethods.add(method.getName());
}
}
}
/**
* Server side implementation for providing state alignment info in responses.
*/
@Override
public void updateResponseState(RpcResponseHeaderProto.Builder header) {
// Using getCorrectLastAppliedOrWrittenTxId will acquire the lock on
// FSEditLog. This is needed so that ANN will return the correct state id
// it currently has. But this may not be necessary for Observer, may want
// revisit for optimization. Same goes to receiveRequestState.
header.setStateId(getLastSeenStateId());
}
/**
* Server side implementation only provides state alignment info.
* It does not receive state alignment info therefore this does nothing.
*/
@Override
public void receiveResponseState(RpcResponseHeaderProto header) {
// Do nothing.
}
/**
* Server side implementation only receives state alignment info.
* It does not build RPC requests therefore this does nothing.
*/
@Override
public void updateRequestState(RpcRequestHeaderProto.Builder header) {
// Do nothing.
}
/**
* Server-side implementation for processing state alignment info in
* requests.
* For Observer it compares the client and the server states and determines
* if it makes sense to wait until the server catches up with the client
* state. If not the server throws RetriableException so that the client
* could retry the call according to the retry policy with another Observer
* or the Active NameNode.
*
* @param header The RPC request header.
* @param clientWaitTime time in milliseconds indicating how long client
* waits for the server response. It is used to verify if the client's
* state is too far ahead of the server's
* @return the minimum of the state ids of the client or the server.
* @throws RetriableException if Observer is too far behind.
*/
@Override
public long receiveRequestState(RpcRequestHeaderProto header,
long clientWaitTime) throws RetriableException {
long serverStateId =
namesystem.getFSImage().getCorrectLastAppliedOrWrittenTxId();
long clientStateId = header.getStateId();
if (clientStateId > serverStateId &&
HAServiceState.ACTIVE.equals(namesystem.getState())) {
FSNamesystem.LOG.warn("A client sent stateId: " + clientStateId +
", but server state is: " + serverStateId);
return serverStateId;
}
if (HAServiceState.OBSERVER.equals(namesystem.getState()) &&
clientStateId - serverStateId >
ESTIMATED_TRANSACTIONS_PER_SECOND
* TimeUnit.MILLISECONDS.toSeconds(clientWaitTime)
* ESTIMATED_SERVER_TIME_MULTIPLIER) {
throw new RetriableException(
"Observer Node is too far behind: serverStateId = "
+ serverStateId + " clientStateId = " + clientStateId);
}
return clientStateId;
}
@Override
public long getLastSeenStateId() {
return namesystem.getFSImage().getCorrectLastAppliedOrWrittenTxId();
}
@Override
public boolean isCoordinatedCall(String protocolName, String methodName) {
return protocolName.equals(ClientProtocol.class.getCanonicalName())
&& coordinatedMethods.contains(methodName);
}
}

View File

@ -365,6 +365,7 @@ public class NameNode extends ReconfigurableBase implements
LoggerFactory.getLogger("BlockStateChange");
public static final HAState ACTIVE_STATE = new ActiveState();
public static final HAState STANDBY_STATE = new StandbyState();
public static final HAState OBSERVER_STATE = new StandbyState(true);
private static final String NAMENODE_HTRACE_PREFIX = "namenode.htrace.";
@ -984,9 +985,11 @@ public class NameNode extends ReconfigurableBase implements
}
protected HAState createHAState(StartupOption startOpt) {
if (!haEnabled || startOpt == StartupOption.UPGRADE
if (!haEnabled || startOpt == StartupOption.UPGRADE
|| startOpt == StartupOption.UPGRADEONLY) {
return ACTIVE_STATE;
} else if (startOpt == StartupOption.OBSERVER) {
return OBSERVER_STATE;
} else {
return STANDBY_STATE;
}
@ -1481,6 +1484,8 @@ public class NameNode extends ReconfigurableBase implements
startOpt = StartupOption.BACKUP;
} else if (StartupOption.CHECKPOINT.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.CHECKPOINT;
} else if (StartupOption.OBSERVER.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.OBSERVER;
} else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)
|| StartupOption.UPGRADEONLY.getName().equalsIgnoreCase(cmd)) {
startOpt = StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd) ?
@ -1794,10 +1799,15 @@ public class NameNode extends ReconfigurableBase implements
if (!haEnabled) {
throw new ServiceFailedException("HA for namenode is not enabled");
}
if (state == OBSERVER_STATE) {
throw new ServiceFailedException(
"Cannot transition from '" + OBSERVER_STATE + "' to '" +
ACTIVE_STATE + "'");
}
state.setState(haContext, ACTIVE_STATE);
}
synchronized void transitionToStandby()
synchronized void transitionToStandby()
throws ServiceFailedException, AccessControlException {
namesystem.checkSuperuserPrivilege();
if (!haEnabled) {
@ -1806,6 +1816,21 @@ public class NameNode extends ReconfigurableBase implements
state.setState(haContext, STANDBY_STATE);
}
synchronized void transitionToObserver()
throws ServiceFailedException, AccessControlException {
namesystem.checkSuperuserPrivilege();
if (!haEnabled) {
throw new ServiceFailedException("HA for namenode is not enabled");
}
// Transition from ACTIVE to OBSERVER is forbidden.
if (state == ACTIVE_STATE) {
throw new ServiceFailedException(
"Cannot transition from '" + ACTIVE_STATE + "' to '" +
OBSERVER_STATE + "'");
}
state.setState(haContext, OBSERVER_STATE);
}
synchronized HAServiceStatus getServiceStatus()
throws ServiceFailedException, AccessControlException {
if (!haEnabled) {
@ -1957,7 +1982,8 @@ public class NameNode extends ReconfigurableBase implements
@Override
public void startStandbyServices() throws IOException {
try {
namesystem.startStandbyServices(getConf());
namesystem.startStandbyServices(getConf(),
state == NameNode.OBSERVER_STATE);
} catch (Throwable t) {
doImmediateShutdown(t);
}
@ -2004,6 +2030,9 @@ public class NameNode extends ReconfigurableBase implements
@Override
public boolean allowStaleReads() {
if (state == OBSERVER_STATE) {
return true;
}
return allowStaleStandbyReads;
}
@ -2017,6 +2046,10 @@ public class NameNode extends ReconfigurableBase implements
return (state.equals(ACTIVE_STATE));
}
public boolean isObserverState() {
return state.equals(OBSERVER_STATE);
}
/**
* Returns whether the NameNode is completely started
*/

View File

@ -456,6 +456,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
.setNumHandlers(handlerCount)
.setVerbose(false)
.setSecretManager(namesystem.getDelegationTokenSecretManager())
.setAlignmentContext(new GlobalStateIdContext(namesystem))
.build();
// Add all the RPC protocols that the namenode implements
@ -1379,6 +1380,17 @@ public class NameNodeRpcServer implements NamenodeProtocols {
return namesystem.listOpenFiles(prevId, openFilesTypes, path);
}
@Override // ClientProtocol
public void msync() throws IOException {
// TODO : need to be filled up if needed. May be a no-op here.
}
@Override // ClientProtocol
public HAServiceState getHAServiceState() throws IOException {
checkNNStartup();
return nn.getServiceStatus().getState();
}
@Override // ClientProtocol
public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
throws IOException {
@ -1575,7 +1587,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
if (nn.getFSImage().isUpgradeFinalized() &&
!namesystem.isRollingUpgrade() &&
!nn.isStandbyState() &&
nn.isActiveState() &&
noStaleStorages) {
return new FinalizeCommand(poolId);
}
@ -1764,6 +1776,14 @@ public class NameNodeRpcServer implements NamenodeProtocols {
nn.transitionToStandby();
}
@Override // HAServiceProtocol
public synchronized void transitionToObserver(StateChangeRequestInfo req)
throws ServiceFailedException, AccessControlException, IOException {
checkNNStartup();
nn.checkHaStateChange(req);
nn.transitionToObserver();
}
@Override // HAServiceProtocol
public synchronized HAServiceStatus getServiceStatus()
throws AccessControlException, ServiceFailedException, IOException {

View File

@ -28,6 +28,8 @@ import org.apache.hadoop.io.IOUtils;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import org.apache.hadoop.log.LogThrottlingHelper;
import org.apache.hadoop.log.LogThrottlingHelper.LogAction;
/**
* A merged input stream that handles failover between different edit logs.
@ -43,6 +45,11 @@ class RedundantEditLogInputStream extends EditLogInputStream {
private long prevTxId;
private final EditLogInputStream[] streams;
/** Limit logging about fast forwarding the stream to every 5 seconds max. */
private static final long FAST_FORWARD_LOGGING_INTERVAL_MS = 5000;
private final LogThrottlingHelper fastForwardLoggingHelper =
new LogThrottlingHelper(FAST_FORWARD_LOGGING_INTERVAL_MS);
/**
* States that the RedundantEditLogInputStream can be in.
*
@ -174,8 +181,12 @@ class RedundantEditLogInputStream extends EditLogInputStream {
case SKIP_UNTIL:
try {
if (prevTxId != HdfsServerConstants.INVALID_TXID) {
LOG.info("Fast-forwarding stream '" + streams[curIdx].getName() +
"' to transaction ID " + (prevTxId + 1));
LogAction logAction = fastForwardLoggingHelper.record();
if (logAction.shouldLog()) {
LOG.info("Fast-forwarding stream '" + streams[curIdx].getName() +
"' to transaction ID " + (prevTxId + 1) +
LogThrottlingHelper.getLogSupressionMessage(logAction));
}
streams[curIdx].skipUntil(prevTxId + 1);
}
} catch (IOException e) {

View File

@ -145,7 +145,11 @@ public class EditLogTailer {
private int maxRetries;
/**
* Whether the tailer should tail the in-progress edit log segments.
* Whether the tailer should tail the in-progress edit log segments. If true,
* this will also attempt to optimize for latency when tailing the edit logs
* (if using the
* {@link org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager}, this
* implies using the RPC-based mechanism to tail edits).
*/
private final boolean inProgressOk;
@ -277,7 +281,7 @@ public class EditLogTailer {
}
@VisibleForTesting
void doTailEdits() throws IOException, InterruptedException {
public void doTailEdits() throws IOException, InterruptedException {
// Write lock needs to be interruptible here because the
// transitionToActive RPC takes the write lock before calling
// tailer.stop() -- so if we're not interruptible, it will

View File

@ -39,13 +39,23 @@ import org.apache.hadoop.ipc.StandbyException;
*/
@InterfaceAudience.Private
public class StandbyState extends HAState {
// TODO: consider implementing a ObserverState instead of using the flag.
private final boolean isObserver;
public StandbyState() {
super(HAServiceState.STANDBY);
this(false);
}
public StandbyState(boolean isObserver) {
super(isObserver ? HAServiceState.OBSERVER : HAServiceState.STANDBY);
this.isObserver = isObserver;
}
@Override
public void setState(HAContext context, HAState s) throws ServiceFailedException {
if (s == NameNode.ACTIVE_STATE) {
if (s == NameNode.ACTIVE_STATE ||
(!isObserver && s == NameNode.OBSERVER_STATE) ||
(isObserver && s == NameNode.STANDBY_STATE)) {
setStateInternal(context, s);
return;
}
@ -92,5 +102,10 @@ public class StandbyState extends HAState {
public boolean shouldPopulateReplQueues() {
return false;
}
@Override
public String toString() {
return isObserver ? "observer" : "standby";
}
}

View File

@ -186,4 +186,9 @@ public class NNHAServiceTarget extends HAServiceTarget {
public boolean isAutoFailoverEnabled() {
return autoFailoverEnabled;
}
@Override
public boolean supportObserver() {
return true;
}
}

View File

@ -212,6 +212,7 @@ message NNHAStatusHeartbeatProto {
enum State {
ACTIVE = 0;
STANDBY = 1;
OBSERVER = 2;
}
required State state = 1;
required uint64 txid = 2;

View File

@ -285,6 +285,21 @@ message GetEditLogManifestResponseProto {
// required NamespaceInfoProto nsInfo = 2;
}
/**
* getJournaledEdits()
*/
message GetJournaledEditsRequestProto {
required JournalIdProto jid = 1;
required uint64 sinceTxId = 2;
required uint32 maxTxns = 3;
optional string nameServiceId = 4;
}
message GetJournaledEditsResponseProto {
required uint32 txnCount = 1;
optional bytes editLog = 2;
}
/**
* prepareRecovery()
*/
@ -364,6 +379,9 @@ service QJournalProtocolService {
rpc getEditLogManifest(GetEditLogManifestRequestProto)
returns (GetEditLogManifestResponseProto);
rpc getJournaledEdits(GetJournaledEditsRequestProto)
returns (GetJournaledEditsResponseProto);
rpc prepareRecovery(PrepareRecoveryRequestProto)
returns (PrepareRecoveryResponseProto);

View File

@ -3193,7 +3193,9 @@
<description>
Whether enable standby namenode to tail in-progress edit logs.
Clients might want to turn it on when they want Standby NN to have
more up-to-date data.
more up-to-date data. When using the QuorumJournalManager, this enables
tailing of edit logs via the RPC-based mechanism, rather than streaming,
which allows for much fresher data.
</description>
</property>
@ -4167,6 +4169,18 @@
</description>
</property>
<property>
<name>dfs.journalnode.edit-cache-size.bytes</name>
<value>1048576</value>
<description>
The size, in bytes, of the in-memory cache of edits to keep on the
JournalNode. This cache is used to serve edits for tailing via the RPC-based
mechanism, and is only enabled when dfs.ha.tail-edits.in-progress is true.
Transactions range in size but are around 200 bytes on average, so the
default of 1MB can store around 5000 transactions.
</description>
</property>
<property>
<name>dfs.journalnode.kerberos.internal.spnego.principal</name>
<value></value>

View File

@ -509,6 +509,7 @@ Usage:
hdfs haadmin -transitionToActive <serviceId> [--forceactive]
hdfs haadmin -transitionToStandby <serviceId>
hdfs haadmin -transitionToObserver <serviceId>
hdfs haadmin -failover [--forcefence] [--forceactive] <serviceId> <serviceId>
hdfs haadmin -getServiceState <serviceId>
hdfs haadmin -getAllServiceState
@ -524,6 +525,7 @@ Usage:
| `-getAllServiceState` | returns the state of all the NameNodes | |
| `-transitionToActive` | transition the state of the given NameNode to Active (Warning: No fencing is done) |
| `-transitionToStandby` | transition the state of the given NameNode to Standby (Warning: No fencing is done) |
| `-transitionToObserver` | transition the state of the given NameNode to Observer (Warning: No fencing is done) |
| `-help` [cmd] | Displays help for the given command or all commands if none is specified. |
See [HDFS HA with NFS](./HDFSHighAvailabilityWithNFS.html#Administrative_commands) or [HDFS HA with QJM](./HDFSHighAvailabilityWithQJM.html#Administrative_commands) for more information on this command.

View File

@ -27,7 +27,12 @@ This document assumes that the reader has a general understanding of general com
Note: Using the Quorum Journal Manager or Conventional Shared Storage
---------------------------------------------------------------------
This guide discusses how to configure and use HDFS HA using the Quorum Journal Manager (QJM) to share edit logs between the Active and Standby NameNodes. For information on how to configure HDFS HA using NFS for shared storage instead of the QJM, please see [this alternative guide.](./HDFSHighAvailabilityWithNFS.html)
This guide discusses how to configure and use HDFS HA using the Quorum
Journal Manager (QJM) to share edit logs between the Active and Standby
NameNodes. For information on how to configure HDFS HA using NFS for
shared storage instead of the QJM, please see [this alternative
guide.](./HDFSHighAvailabilityWithNFS.html). For information on how to
configure HDFS HA with Observer NameNode, please see [this guide](./ObserverNameNode.html)
Background
----------
@ -431,6 +436,34 @@ http://NN_HOSTNAME/isActive will return a 200 status code response if the NN is
### In-Progress Edit Log Tailing
Under the default settings, the Standby NameNode will only apply edits that are present in an edit
log segments which has been finalized. If it is desirable to have a Standby NameNode which has more
up-to-date namespace information, it is possible to enable tailing of in-progress edit segments.
This setting will attempt to fetch edits from an in-memory cache on the JournalNodes and can reduce
the lag time before a transaction is applied on the Standby NameNode to the order of milliseconds.
If an edit cannot be served from the cache, the Standby will still be able to retrieve it, but the
lag time will be much longer. The relevant configurations are:
* **dfs.ha.tail-edits.in-progress** - Whether or not to enable tailing on in-progress edits logs.
This will also enable the in-memory edit cache on the JournalNodes. Disabled by default.
* **dfs.journalnode.edit-cache-size.bytes** - The size of the in-memory cache of edits on the
JournalNode. Edits take around 200 bytes each in a typical environment, so, for example, the
default of 1048576 (1MB) can hold around 5000 transactions. It is recommended to monitor the
JournalNode metrics RpcRequestCacheMissAmountNumMisses and RpcRequestCacheMissAmountAvgTxns,
which respectively count the number of requests unable to be served by the cache, and the extra
number of transactions which would have needed to have been in the cache for the request to
succeed. For example, if a request attempted to fetch edits starting at transaction ID 10, but
the oldest data in the cache was at transaction ID 20, a value of 10 would be added to the
average.
This feature is primarily useful in conjunction with the Standby/Observer Read feature. Using this
feature, read requests can be serviced from non-active NameNodes; thus tailing in-progress edits
provides these nodes with the ability to serve requests with data which is much more fresh. See the
Apache JIRA ticket HDFS-12943 for more information on this feature.
Automatic Failover
------------------

View File

@ -0,0 +1,187 @@
<!---
Licensed 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. See accompanying LICENSE file.
-->
Consistent Reads from HDFS Observer NameNode
=============================================================
<!-- MACRO{toc|fromDepth=0|toDepth=3} -->
Purpose
--------
This guide provides an overview of the HDFS Observer NameNode feature
and how to configure/install it in a typical HA-enabled cluster. For a
detailed technical design overview, please check the doc attached to
[HDFS-12943](https://issues.apache.org/jira/browse/HDFS-12943).
Background
-----------
In a HA-enabled HDFS cluster (for more information, check
[HDFSHighAvailabilityWithQJM](./HDFSHighAvailabilityWithQJM.html)), there
is a single Active NameNode and one or more Standby NameNode(s). The
Active NameNode is responsible for serving all client requests, while
Standby NameNode just keep the up-to-date information regarding the
namespace, by tailing edit logs from JournalNodes, as well as block
location information, by receiving block reports from all the DataNodes.
One drawback of this architecture is that the Active NameNode could be a
single bottle-neck and be overloaded with client requests, especially in
a busy cluster.
The Consistent Reads from HDFS Observer NameNode feature addresses the
above by introducing a new type of NameNode called **Observer
NameNode**. Similar to Standby NameNode, Observer NameNode keeps itself
up-to-date regarding the namespace and block location information.
In addition, it also has the ability to serve consistent reads, like
Active NameNode. Since read requests are the majority in a typical
environment, this can help to load balancing the NameNode traffic and
improve overall throughput.
Architecture
--------------
In the new architecture, a HA cluster could consists of namenodes in 3
different states: active, standby and observer. State transition can
happen between active and standby, standby and observer, but not
directly between active and observer.
To ensure read-after-write consistency within a single client, a state
ID, which is implemented using transaction ID within NameNode, is
introduced in RPC headers. When a client performs write through Active
NameNode, it updates its state ID using the latest transaction ID from
the NameNode. When performing a subsequent read, the client passes this
state ID to Observe NameNode, which will then check against its own
transaction ID, and will ensure its own transaction ID has caught up
with the request's state ID, before serving the read request.
Edit log tailing is critical for Observer NameNode as it directly affects
the latency between when a transaction is applied in Active NameNode and
when it is applied in the Observer NameNode. A new edit log tailing
mechanism, named "Edit Tailing Fast-Path", is introduced to
significantly reduce this latency. This is built on top of the existing
in-progress edit log tailing feature, with further improvements such as
RPC-based tailing instead of HTTP, a in-memory cache on the JournalNode,
etc. For more details, please see the design doc attached to HDFS-13150.
New client-side proxy providers are also introduced.
ObserverReadProxyProvider, which inherits the existing
ConfiguredFailoverProxyProvider, should be used to replace the latter to
enable reads from Observer NameNode. When submitting a client read
request, the proxy provider will first try each Observer NameNode
available in the cluster, and only fall back to Active NameNode if all
of the former failed. Similarly, ObserverReadProxyProviderWithIPFailover
is introduced to replace IPFailoverProxyProvider in a IP failover setup.
Deployment
-----------
### Configurations
To enable consistent reads from Observer NameNode, you'll need to add a
few configurations to your **hdfs-site.xml**:
* **dfs.ha.tail-edits.in-progress** - to enable fast tailing on
in-progress edit logs.
This enables fast edit log tailing through in-progress edit logs and
also other mechanisms such as RPC-based edit log fetching, in-memory
cache in JournalNodes, and so on. It is disabled by default, but is
**required to be turned on** for the Observer NameNode feature.
<property>
<name>dfs.ha.tail-edits.in-progress</name>
<value>true</value>
</property>
* **dfs.ha.tail-edits.period** - how often Standby/Observer NameNodes
should fetch edits from JournalNodes.
This determines the staleness of Observer NameNode w.r.t the Active.
If too large, RPC time will increase as client requests will wait
longer in the RPC queue before Observer tails edit logs and catches
up the latest state of Active. The default value is 1min. It is
**highly recommend** to configure this to a much lower value.
<property>
<name>dfs.ha.tail-edits.period</name>
<value>0ms</value>
</property>
* **dfs.journalnode.edit-cache-size.bytes** - the in-memory cache size,
in bytes, on the JournalNodes.
This is the size, in bytes, of the in-memory cache for storing edits
on the JournalNode side. The cache is used for serving edits via
RPC-based tailing. This is only effective when
dfs.ha.tail-edits.in-progress is turned on.
<property>
<name>dfs.journalnode.edit-cache-size.bytes</name>
<value>1048576</value>
</property>
### New administrative command
A new HA admin command is introduced to transition a Standby NameNode
into observer state:
haadmin -transitionToObserver
Note this can only be executed on Standby NameNode. Exception will be
thrown when invoking this on Active NameNode.
Similarly, existing **transitionToStandby** can also be run on an
Observer NameNode, which transition it to the standby state.
**NOTE**: the feature for Observer NameNode to participate in failover
is not implemented yet. Therefore, as described in the next section, you
should only use **transitionToObserver** to bring up an observer and put
it outside the ZooKeeper controlled failover group. You should not use
**transitionToStandby** since the host for the Observer NameNode cannot
have ZKFC running.
### Deployment details
To enable observer support, first you'll need a HA-enabled HDFS cluster
with more than 2 namenodes. Then, you need to transition Standby
NameNode(s) into the observer state. An minimum setup would be running 3
namenodes in the cluster, one active, one standby and one observer. For
large HDFS clusters we recommend running two or more Observers depending
on the intensity of read requests and HA requirements.
Note that currently Observer NameNode doesn't integrate fully when
automatic failover is enabled. If the
**dfs.ha.automatic-failover.enabled** is turned on, you'll also need to
disable ZKFC on the namenode for observer. In addition to that, you'll
also need to add **forcemanual** flag to the **transitionToObserver**
command:
haadmin -transitionToObserver -forcemanual
In future, this restriction will be lifted.
### Client configuration
Clients who wish to use Observer NameNode for read accesses can
specify the ObserverReadProxyProvider class for proxy provider
implementation, in the client-side **hdfs-site.xml** configuration file:
<property>
<name>dfs.client.failover.proxy.provider.<nameservice></name>
<value>org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider</value>
</property>
Clients who do not wish to use Observer NameNode can still use the
existing ConfiguredFailoverProxyProvider and should not see any behavior
change.

View File

@ -71,7 +71,6 @@ import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import com.google.common.base.Charsets;
import com.google.common.base.Joiner;
@ -2022,18 +2021,17 @@ public class DFSTestUtil {
* Get the RefreshUserMappingsProtocol RPC proxy for the NN associated with
* this DFSClient object
*
* @param nameNodeUri the URI of the NN to get a proxy for.
* @param nnAddr the address of the NN to get a proxy for.
*
* @return the RefreshUserMappingsProtocol RPC proxy associated with this
* DFSClient object
*/
@VisibleForTesting
public static RefreshUserMappingsProtocol getRefreshUserMappingsProtocolProxy(
Configuration conf, URI nameNodeUri) throws IOException {
final AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
return NameNodeProxies.createProxy(conf,
nameNodeUri, RefreshUserMappingsProtocol.class,
nnFallbackToSimpleAuth).getProxy();
Configuration conf, InetSocketAddress nnAddr) throws IOException {
return NameNodeProxies.createNonHAProxy(
conf, nnAddr, RefreshUserMappingsProtocol.class,
UserGroupInformation.getCurrentUser(), false).getProxy();
}
/**

View File

@ -2646,8 +2646,20 @@ public class MiniDFSCluster implements AutoCloseable {
getNameNode(nnIndex).getRpcServer().transitionToStandby(
new StateChangeRequestInfo(RequestSource.REQUEST_BY_USER_FORCED));
}
public void transitionToObserver(int nnIndex) throws IOException,
ServiceFailedException {
getNameNode(nnIndex).getRpcServer().transitionToObserver(
new StateChangeRequestInfo(RequestSource.REQUEST_BY_USER_FORCED));
}
public void rollEditLogAndTail(int nnIndex) throws Exception {
getNameNode(nnIndex).getRpcServer().rollEditLog();
for (int i = 2; i < getNumNameNodes(); i++) {
getNameNode(i).getNamesystem().getEditLogTailer().doTailEdits();
}
}
public void triggerBlockReports()
throws IOException {
for (DataNode dn : getDataNodes()) {

View File

@ -0,0 +1,347 @@
/**
* 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.hdfs;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
import org.apache.hadoop.hdfs.server.namenode.ha.HAProxyFactory;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
/**
* Class is used to test server sending state alignment information to clients
* via RPC and likewise clients receiving and updating their last known
* state alignment info.
* These tests check that after a single RPC call a client will have caught up
* to the most recent alignment state of the server.
*/
public class TestStateAlignmentContextWithHA {
public static final Logger LOG =
LoggerFactory.getLogger(TestStateAlignmentContextWithHA.class.getName());
private static final int NUMDATANODES = 1;
private static final int NUMCLIENTS = 10;
private static final int NUMFILES = 120;
private static final Configuration CONF = new HdfsConfiguration();
private static final List<ClientGSIContext> AC_LIST = new ArrayList<>();
private static MiniQJMHACluster qjmhaCluster;
private static MiniDFSCluster cluster;
private static List<Worker> clients;
private DistributedFileSystem dfs;
private int active = 0;
private int standby = 1;
static class ORPPwithAlignmentContexts<T extends ClientProtocol>
extends ObserverReadProxyProvider<T> {
public ORPPwithAlignmentContexts(
Configuration conf, URI uri, Class<T> xface,
HAProxyFactory<T> factory) throws IOException {
super(conf, uri, xface, factory);
AC_LIST.add((ClientGSIContext) getAlignmentContext());
}
}
@BeforeClass
public static void startUpCluster() throws IOException {
// Set short retry timeouts so this test runs faster
CONF.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
CONF.setBoolean(String.format(
"fs.%s.impl.disable.cache", HdfsConstants.HDFS_URI_SCHEME), true);
CONF.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, NUMDATANODES);
qjmhaCluster = HATestUtil.setUpObserverCluster(CONF, 1, NUMDATANODES, true);
cluster = qjmhaCluster.getDfsCluster();
}
@Before
public void before() throws IOException, URISyntaxException {
dfs = HATestUtil.configureObserverReadFs(
cluster, CONF, ORPPwithAlignmentContexts.class, true);
}
@AfterClass
public static void shutDownCluster() throws IOException {
if (qjmhaCluster != null) {
qjmhaCluster.shutdown();
}
}
@After
public void after() throws IOException {
killWorkers();
cluster.transitionToStandby(1);
cluster.transitionToActive(0);
active = 0;
standby = 1;
if (dfs != null) {
dfs.close();
dfs = null;
}
AC_LIST.clear();
}
/**
* This test checks if after a client writes we can see the state id in
* updated via the response.
*/
@Test
public void testStateTransferOnWrite() throws Exception {
long preWriteState =
cluster.getNamesystem(active).getLastWrittenTransactionId();
DFSTestUtil.writeFile(dfs, new Path("/testFile1"), "abc");
long clientState = getContext(0).getLastSeenStateId();
long postWriteState =
cluster.getNamesystem(active).getLastWrittenTransactionId();
// Write(s) should have increased state. Check for greater than.
assertTrue(clientState > preWriteState);
// Client and server state should be equal.
assertEquals(clientState, postWriteState);
}
/**
* This test checks if after a client reads we can see the state id in
* updated via the response.
*/
@Test
public void testStateTransferOnRead() throws Exception {
DFSTestUtil.writeFile(dfs, new Path("/testFile2"), "123");
long lastWrittenId =
cluster.getNamesystem(active).getLastWrittenTransactionId();
DFSTestUtil.readFile(dfs, new Path("/testFile2"));
// Read should catch client up to last written state.
long clientState = getContext(0).getLastSeenStateId();
assertEquals(clientState, lastWrittenId);
}
/**
* This test checks that a fresh client starts with no state and becomes
* updated of state from RPC call.
*/
@Test
public void testStateTransferOnFreshClient() throws Exception {
DFSTestUtil.writeFile(dfs, new Path("/testFile3"), "ezpz");
long lastWrittenId =
cluster.getNamesystem(active).getLastWrittenTransactionId();
try (DistributedFileSystem clearDfs = HATestUtil.configureObserverReadFs(
cluster, CONF, ORPPwithAlignmentContexts.class, true);) {
ClientGSIContext clientState = getContext(1);
assertEquals(clientState.getLastSeenStateId(), Long.MIN_VALUE);
DFSTestUtil.readFile(clearDfs, new Path("/testFile3"));
assertEquals(clientState.getLastSeenStateId(), lastWrittenId);
}
}
/**
* This test checks if after a client writes we can see the state id in
* updated via the response.
*/
@Test
public void testStateTransferOnWriteWithFailover() throws Exception {
long preWriteState =
cluster.getNamesystem(active).getLastWrittenTransactionId();
// Write using HA client.
DFSTestUtil.writeFile(dfs, new Path("/testFile1FO"), "123");
long clientState = getContext(0).getLastSeenStateId();
long postWriteState =
cluster.getNamesystem(active).getLastWrittenTransactionId();
// Write(s) should have increased state. Check for greater than.
assertTrue(clientState > preWriteState);
// Client and server state should be equal.
assertEquals(clientState, postWriteState);
// Failover NameNode.
failOver();
// Write using HA client.
DFSTestUtil.writeFile(dfs, new Path("/testFile2FO"), "456");
long clientStateFO = getContext(0).getLastSeenStateId();
long writeStateFO =
cluster.getNamesystem(active).getLastWrittenTransactionId();
// Write(s) should have increased state. Check for greater than.
assertTrue(clientStateFO > postWriteState);
// Client and server state should be equal.
assertEquals(clientStateFO, writeStateFO);
}
@Test(timeout=300000)
public void testMultiClientStatesWithRandomFailovers() throws Exception {
// First run, half the load, with one failover.
runClientsWithFailover(1, NUMCLIENTS/2, NUMFILES/2);
// Second half, with fail back.
runClientsWithFailover(NUMCLIENTS/2 + 1, NUMCLIENTS, NUMFILES/2);
}
private void runClientsWithFailover(int clientStartId,
int numClients,
int numFiles)
throws Exception {
ExecutorService execService = Executors.newFixedThreadPool(2);
clients = new ArrayList<>(numClients);
for (int i = clientStartId; i <= numClients; i++) {
DistributedFileSystem haClient = HATestUtil.configureObserverReadFs(
cluster, CONF, ORPPwithAlignmentContexts.class, true);
clients.add(new Worker(haClient, numFiles, "/testFile3FO_", i));
}
// Execute workers in threadpool with random failovers.
List<Future<STATE>> futures = submitAll(execService, clients);
execService.shutdown();
boolean finished = false;
failOver();
while (!finished) {
finished = execService.awaitTermination(20L, TimeUnit.SECONDS);
}
// Validation.
for (Future<STATE> future : futures) {
assertEquals(future.get(), STATE.SUCCESS);
}
clients.clear();
}
private ClientGSIContext getContext(int clientCreationIndex) {
return AC_LIST.get(clientCreationIndex);
}
private void failOver() throws IOException {
LOG.info("Transitioning Active to Standby");
cluster.transitionToStandby(active);
LOG.info("Transitioning Standby to Active");
cluster.transitionToActive(standby);
int tempActive = active;
active = standby;
standby = tempActive;
}
/* Executor.invokeAll() is blocking so utilizing submit instead. */
private static List<Future<STATE>> submitAll(ExecutorService executor,
Collection<Worker> calls) {
List<Future<STATE>> futures = new ArrayList<>(calls.size());
for (Worker call : calls) {
Future<STATE> future = executor.submit(call);
futures.add(future);
}
return futures;
}
private void killWorkers() throws IOException {
if (clients != null) {
for(Worker worker : clients) {
worker.kill();
}
clients = null;
}
}
private enum STATE { SUCCESS, FAIL, ERROR }
private class Worker implements Callable<STATE> {
private final DistributedFileSystem client;
private final int filesToMake;
private String filePath;
private final int nonce;
Worker(DistributedFileSystem client,
int filesToMake,
String filePath,
int nonce) {
this.client = client;
this.filesToMake = filesToMake;
this.filePath = filePath;
this.nonce = nonce;
}
@Override
public STATE call() {
int i = -1;
try {
for (i = 0; i < filesToMake; i++) {
ClientGSIContext gsiContext = getContext(nonce);
long preClientStateFO = gsiContext.getLastSeenStateId();
// Write using HA client.
Path path = new Path(filePath + nonce + "_" + i);
DFSTestUtil.writeFile(client, path, "erk");
long postClientStateFO = gsiContext.getLastSeenStateId();
// Write(s) should have increased state. Check for greater than.
if (postClientStateFO < 0 || postClientStateFO <= preClientStateFO) {
LOG.error("FAIL: Worker started with: {} , but finished with: {}",
preClientStateFO, postClientStateFO);
return STATE.FAIL;
}
if(i % (NUMFILES/10) == 0) {
LOG.info("Worker {} created {} files", nonce, i);
LOG.info("LastSeenStateId = {}", postClientStateFO);
}
}
return STATE.SUCCESS;
} catch (Exception e) {
LOG.error("ERROR: Worker failed with: ", e);
return STATE.ERROR;
} finally {
LOG.info("Worker {} created {} files", nonce, i);
}
}
public void kill() throws IOException {
client.dfs.closeAllFilesBeingWritten(true);
client.dfs.closeOutputStreams(true);
client.dfs.closeConnectionToNamenode();
client.dfs.close();
client.close();
}
}
}

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import java.io.IOException;
@ -171,7 +172,8 @@ public class MiniQJMHACluster {
}
// use standard failover configurations
HATestUtil.setFailoverConfigurations(conf, NAMESERVICE, nns);
HATestUtil.setFailoverConfigurations(conf, NAMESERVICE, nns,
ConfiguredFailoverProxyProvider.class);
return conf;
}

View File

@ -44,6 +44,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
@ -91,6 +92,10 @@ public class TestQuorumJournalManager {
conf = new Configuration();
// Don't retry connections - it just slows down the tests.
conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
// Turn off IPC client caching to handle daemon restarts.
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY, 0);
conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
cluster = new MiniJournalCluster.Builder(conf)
.baseDir(GenericTestUtils.getRandomizedTestDir().getAbsolutePath())
@ -959,6 +964,135 @@ public class TestQuorumJournalManager {
qjm2.selectInputStreams(streams, 1, true, true);
verifyEdits(streams, 1, 8);
}
@Test
public void testSelectViaRpcWithDurableTransactions() throws Exception {
// Two loggers will have up to ID 5, one will have up to ID 6
failLoggerAtTxn(spies.get(0), 6);
failLoggerAtTxn(spies.get(1), 6);
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 5);
try {
writeTxns(stm, 6, 1);
fail("Did not fail to write when only a minority succeeded");
} catch (QuorumException qe) {
GenericTestUtils.assertExceptionContains(
"too many exceptions to achieve quorum size 2/3", qe);
}
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, true);
verifyEdits(streams, 1, 5);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
}
@Test
public void testSelectViaRpcWithoutDurableTransactions() throws Exception {
setupLoggers345();
futureThrows(new IOException()).when(spies.get(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, false);
verifyEdits(streams, 1, 5);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
}
@Test
public void testSelectViaRpcOneDeadJN() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
cluster.getJournalNode(0).stopAndJoin(0);
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, false);
verifyEdits(streams, 1, 10);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
}
@Test
public void testSelectViaRpcTwoDeadJNs() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
cluster.getJournalNode(0).stopAndJoin(0);
cluster.getJournalNode(1).stopAndJoin(0);
try {
qjm.selectInputStreams(new ArrayList<>(), 1, true, false);
fail("");
} catch (QuorumException qe) {
GenericTestUtils.assertExceptionContains(
"too many exceptions to achieve quorum size 2/3", qe);
}
}
@Test
public void testSelectViaRpcTwoJNsError() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
writeTxns(stm, 11, 1);
// One last sync whose transactions are not expected to be seen in the
// input streams because the JournalNodes have not updated their concept
// of the committed transaction ID yet
writeTxns(stm, 12, 1);
futureThrows(new IOException()).when(spies.get(0)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
futureThrows(new IOException()).when(spies.get(1)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, true);
// This should still succeed as the QJM should fall back to the streaming
// mechanism for fetching edits
verifyEdits(streams, 1, 11);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getEditLogManifest(1, true);
}
}
@Test
public void testSelectViaRpcAfterJNRestart() throws Exception {
EditLogOutputStream stm =
qjm.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
writeTxns(stm, 1, 10);
qjm.finalizeLogSegment(1, 10);
// Close to avoid connections hanging around after the JNs are restarted
for (int i = 0; i < cluster.getNumNodes(); i++) {
cluster.restartJournalNode(i);
}
cluster.waitActive();
qjm = createSpyingQJM();
spies = qjm.getLoggerSetForTests().getLoggersForTests();
List<EditLogInputStream> streams = new ArrayList<>();
qjm.selectInputStreams(streams, 1, true, true);
// This should still succeed as the QJM should fall back to the streaming
// mechanism for fetching edits
verifyEdits(streams, 1, 10);
IOUtils.closeStreams(streams.toArray(new Closeable[0]));
for (AsyncLogger logger : spies) {
Mockito.verify(logger, Mockito.times(1)).getEditLogManifest(1, true);
}
}
private QuorumJournalManager createSpyingQJM()
throws IOException, URISyntaxException {

View File

@ -17,11 +17,14 @@
*/
package org.apache.hadoop.hdfs.qjournal.client;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.eq;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.net.URI;
import java.util.List;
@ -29,11 +32,12 @@ import java.util.List;
import org.junit.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.qjournal.client.AsyncLogger;
import org.apache.hadoop.hdfs.qjournal.client.QuorumException;
import org.apache.hadoop.hdfs.qjournal.client.QuorumJournalManager;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournalStateResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
import org.apache.hadoop.hdfs.server.namenode.EditLogInputStream;
import org.apache.hadoop.hdfs.server.namenode.EditLogOutputStream;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@ -45,11 +49,15 @@ import org.mockito.Mockito;
import org.mockito.stubbing.Stubber;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import com.google.protobuf.ByteString;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.writeOp;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.createTxnData;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.verifyEdits;
/**
* True unit tests for QuorumJournalManager
@ -72,6 +80,7 @@ public class TestQuorumJournalManagerUnit {
mockLogger(),
mockLogger());
conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
qjm = new QuorumJournalManager(conf, new URI("qjournal://host/jid"), FAKE_NSINFO) {
@Override
protected List<AsyncLogger> createLoggers(AsyncLogger.Factory factory) {
@ -217,6 +226,94 @@ public class TestQuorumJournalManagerUnit {
Mockito.verify(spyLoggers.get(0)).setCommittedTxId(1L);
}
@Test
public void testReadRpcInputStreams() throws Exception {
for (int jn = 0; jn < 3; jn++) {
futureReturns(getJournaledEditsReponse(1, 3))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 3);
}
@Test
public void testReadRpcMismatchedInputStreams() throws Exception {
for (int jn = 0; jn < 3; jn++) {
futureReturns(getJournaledEditsReponse(1, jn + 1))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 2);
}
@Test
public void testReadRpcInputStreamsOneSlow() throws Exception {
for (int jn = 0; jn < 2; jn++) {
futureReturns(getJournaledEditsReponse(1, jn + 1))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
Mockito.doReturn(SettableFuture.create())
.when(spyLoggers.get(2)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 1);
}
@Test
public void testReadRpcInputStreamsOneException() throws Exception {
for (int jn = 0; jn < 2; jn++) {
futureReturns(getJournaledEditsReponse(1, jn + 1))
.when(spyLoggers.get(jn)).getJournaledEdits(1,
QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
futureThrows(new IOException()).when(spyLoggers.get(2))
.getJournaledEdits(1, QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(1, streams.size());
verifyEdits(streams, 1, 1);
}
@Test
public void testReadRpcInputStreamsNoNewEdits() throws Exception {
for (int jn = 0; jn < 3; jn++) {
futureReturns(GetJournaledEditsResponseProto.newBuilder()
.setTxnCount(0).setEditLog(ByteString.EMPTY).build())
.when(spyLoggers.get(jn))
.getJournaledEdits(1, QuorumJournalManager.QJM_RPC_MAX_TXNS_DEFAULT);
}
List<EditLogInputStream> streams = Lists.newArrayList();
qjm.selectInputStreams(streams, 1, true, true);
assertEquals(0, streams.size());
}
private GetJournaledEditsResponseProto getJournaledEditsReponse(
int startTxn, int numTxns) throws Exception {
ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
EditLogFileOutputStream.writeHeader(
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION,
new DataOutputStream(byteStream));
byteStream.write(createTxnData(startTxn, numTxns));
return GetJournaledEditsResponseProto.newBuilder()
.setTxnCount(numTxns)
.setEditLog(ByteString.copyFrom(byteStream.toByteArray()))
.build();
}
private EditLogOutputStream createLogSegment() throws IOException {
futureReturns(null).when(spyLoggers.get(0)).startLogSegment(Mockito.anyLong(),
Mockito.eq(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION));

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdfs.qjournal.server;
/**
* Utilities for testing {@link Journal} instances.
*/
public class JournalTestUtil {
/**
* Corrupt the cache of a {@link Journal} to simulate some corrupt entries
* being present.
*
* @param txid The transaction ID whose containing buffer in the cache
* should be corrupted.
* @param journal The journal whose cache should be corrupted.
*/
public static void corruptJournaledEditsCache(long txid, Journal journal) {
JournaledEditsCache cache = journal.getJournaledEditsCache();
byte[] buf = cache.getRawDataForTests(txid);
// Change a few arbitrary bytes in the buffer
for (int i = 0; i < buf.length; i += 9) {
buf[i] = 0;
}
for (int i = 3; i < buf.length; i += 9) {
buf[i] += 10;
}
for (int i = 6; i < buf.length; i += 9) {
buf[i] -= 10;
}
}
}

View File

@ -17,19 +17,25 @@
*/
package org.apache.hadoop.hdfs.qjournal.server;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import com.google.common.primitives.Bytes;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
import org.apache.hadoop.hdfs.qjournal.protocol.JournalOutOfSyncException;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJournaledEditsResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProtoOrBuilder;
import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
@ -38,6 +44,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils;
@ -71,6 +78,8 @@ public class TestJournal {
public void setup() throws Exception {
FileUtil.fullyDelete(TEST_LOG_DIR);
conf = new Configuration();
// Enable fetching edits via RPC
conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
journal = new Journal(conf, TEST_LOG_DIR, JID, StartupOption.REGULAR,
mockErrorReporter);
journal.format(FAKE_NSINFO, false);
@ -434,6 +443,44 @@ public class TestJournal {
}
}
@Test
public void testReadFromCache() throws Exception {
journal.newEpoch(FAKE_NSINFO, 1);
journal.startLogSegment(makeRI(1), 1,
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
journal.journal(makeRI(2), 1, 1, 5, QJMTestUtil.createTxnData(1, 5));
journal.journal(makeRI(3), 1, 6, 5, QJMTestUtil.createTxnData(6, 5));
journal.journal(makeRI(4), 1, 11, 5, QJMTestUtil.createTxnData(11, 5));
assertJournaledEditsTxnCountAndContents(1, 7, 7,
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
assertJournaledEditsTxnCountAndContents(1, 30, 15,
NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
journal.finalizeLogSegment(makeRI(5), 1, 15);
int newLayoutVersion = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1;
journal.startLogSegment(makeRI(6), 16, newLayoutVersion);
journal.journal(makeRI(7), 16, 16, 5, QJMTestUtil.createTxnData(16, 5));
assertJournaledEditsTxnCountAndContents(16, 10, 20, newLayoutVersion);
}
private void assertJournaledEditsTxnCountAndContents(int startTxn,
int requestedMaxTxns, int expectedEndTxn, int layoutVersion)
throws Exception {
GetJournaledEditsResponseProto result =
journal.getJournaledEdits(startTxn, requestedMaxTxns);
int expectedTxnCount = expectedEndTxn - startTxn + 1;
ByteArrayOutputStream headerBytes = new ByteArrayOutputStream();
EditLogFileOutputStream.writeHeader(layoutVersion,
new DataOutputStream(headerBytes));
assertEquals(expectedTxnCount, result.getTxnCount());
assertArrayEquals(
Bytes.concat(
headerBytes.toByteArray(),
QJMTestUtil.createTxnData(startTxn, expectedTxnCount)),
result.getEditLog().toByteArray());
}
@Test
public void testFormatNonEmptyStorageDirectoriesWhenforceOptionIsTrue()
throws Exception {

View File

@ -0,0 +1,257 @@
/**
* 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.hdfs.qjournal.server;
import com.google.common.primitives.Bytes;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.apache.hadoop.test.PathUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.createGabageTxns;
import static org.apache.hadoop.hdfs.qjournal.QJMTestUtil.createTxnData;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/**
* Test the {@link JournaledEditsCache} used for caching edits in-memory on the
* {@link Journal}.
*/
public class TestJournaledEditsCache {
private static final int EDITS_CAPACITY = 100;
private static final File TEST_DIR =
PathUtils.getTestDir(TestJournaledEditsCache.class, false);
private JournaledEditsCache cache;
@Before
public void setup() throws Exception {
Configuration conf = new Configuration();
conf.setInt(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY,
createTxnData(1, 1).length * EDITS_CAPACITY);
cache = new JournaledEditsCache(conf);
TEST_DIR.mkdirs();
}
@After
public void cleanup() throws Exception {
FileUtils.deleteQuietly(TEST_DIR);
}
@Test
public void testCacheSingleSegment() throws Exception {
storeEdits(1, 20);
// Leading part of the segment
assertTxnCountAndContents(1, 5, 5);
// All of the segment
assertTxnCountAndContents(1, 20, 20);
// Past the segment
assertTxnCountAndContents(1, 40, 20);
// Trailing part of the segment
assertTxnCountAndContents(10, 11, 20);
// Trailing part of the segment, past the end
assertTxnCountAndContents(10, 20, 20);
}
@Test
public void testCacheBelowCapacityRequestOnBoundary() throws Exception {
storeEdits(1, 5);
storeEdits(6, 20);
storeEdits(21, 30);
// First segment only
assertTxnCountAndContents(1, 3, 3);
// Second segment only
assertTxnCountAndContents(6, 10, 15);
// First and second segment
assertTxnCountAndContents(1, 7, 7);
// All three segments
assertTxnCountAndContents(1, 25, 25);
// Second and third segment
assertTxnCountAndContents(6, 20, 25);
// Second and third segment; request past the end
assertTxnCountAndContents(6, 50, 30);
// Third segment only; request past the end
assertTxnCountAndContents(21, 20, 30);
}
@Test
public void testCacheBelowCapacityRequestOffBoundary() throws Exception {
storeEdits(1, 5);
storeEdits(6, 20);
storeEdits(21, 30);
// First segment only
assertTxnCountAndContents(3, 1, 3);
// First and second segment
assertTxnCountAndContents(3, 6, 8);
// Second and third segment
assertTxnCountAndContents(15, 10, 24);
// Second and third segment; request past the end
assertTxnCountAndContents(15, 50, 30);
// Start read past the end
List<ByteBuffer> buffers = new ArrayList<>();
assertEquals(0, cache.retrieveEdits(31, 10, buffers));
assertTrue(buffers.isEmpty());
}
@Test
public void testCacheAboveCapacity() throws Exception {
int thirdCapacity = EDITS_CAPACITY / 3;
storeEdits(1, thirdCapacity);
storeEdits(thirdCapacity + 1, thirdCapacity * 2);
storeEdits(thirdCapacity * 2 + 1, EDITS_CAPACITY);
storeEdits(EDITS_CAPACITY + 1, thirdCapacity * 4);
storeEdits(thirdCapacity * 4 + 1, thirdCapacity * 5);
try {
cache.retrieveEdits(1, 10, new ArrayList<>());
fail();
} catch (IOException ioe) {
// expected
}
assertTxnCountAndContents(EDITS_CAPACITY + 1, EDITS_CAPACITY,
thirdCapacity * 5);
}
@Test
public void testCacheSingleAdditionAboveCapacity() throws Exception {
LogCapturer logs = LogCapturer.captureLogs(Journal.LOG);
storeEdits(1, EDITS_CAPACITY * 2);
logs.stopCapturing();
assertTrue(logs.getOutput().contains("batch of edits was too large"));
try {
cache.retrieveEdits(1, 1, new ArrayList<>());
fail();
} catch (IOException ioe) {
// expected
}
storeEdits(EDITS_CAPACITY * 2 + 1, EDITS_CAPACITY * 2 + 5);
assertTxnCountAndContents(EDITS_CAPACITY * 2 + 1, 5,
EDITS_CAPACITY * 2 + 5);
}
@Test
public void testCacheWithFutureLayoutVersion() throws Exception {
byte[] firstHalf = createGabageTxns(1, 5);
byte[] secondHalf = createGabageTxns(6, 5);
int futureVersion = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1;
cache.storeEdits(Bytes.concat(firstHalf, secondHalf), 1, 10,
futureVersion);
List<ByteBuffer> buffers = new ArrayList<>();
assertEquals(5, cache.retrieveEdits(6, 5, buffers));
assertArrayEquals(getHeaderForLayoutVersion(futureVersion),
buffers.get(0).array());
byte[] retBytes = new byte[buffers.get(1).remaining()];
System.arraycopy(buffers.get(1).array(), buffers.get(1).position(),
retBytes, 0, buffers.get(1).remaining());
assertArrayEquals(secondHalf, retBytes);
}
@Test
public void testCacheWithMultipleLayoutVersions() throws Exception {
int oldLayout = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION + 1;
cache.storeEdits(createTxnData(1, 5), 1, 5, oldLayout);
storeEdits(6, 10);
// Ensure the cache will only return edits from a single
// layout version at a time
try {
cache.retrieveEdits(1, 50, new ArrayList<>());
fail("Expected a cache miss");
} catch (JournaledEditsCache.CacheMissException cme) {
// expected
}
assertTxnCountAndContents(6, 50, 10);
}
@Test
public void testCacheEditsWithGaps() throws Exception {
storeEdits(1, 5);
storeEdits(10, 15);
try {
cache.retrieveEdits(1, 20, new ArrayList<>());
fail();
} catch (JournaledEditsCache.CacheMissException cme) {
assertEquals(9, cme.getCacheMissAmount());
}
assertTxnCountAndContents(10, 10, 15);
}
@Test(expected = JournaledEditsCache.CacheMissException.class)
public void testReadUninitializedCache() throws Exception {
cache.retrieveEdits(1, 10, new ArrayList<>());
}
@Test(expected = JournaledEditsCache.CacheMissException.class)
public void testCacheMalformedInput() throws Exception {
storeEdits(1, 1);
cache.retrieveEdits(-1, 10, new ArrayList<>());
}
private void storeEdits(int startTxn, int endTxn) throws Exception {
cache.storeEdits(createTxnData(startTxn, endTxn - startTxn + 1), startTxn,
endTxn, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
}
private void assertTxnCountAndContents(int startTxn, int requestedMaxTxns,
int expectedEndTxn) throws Exception {
List<ByteBuffer> buffers = new ArrayList<>();
int expectedTxnCount = expectedEndTxn - startTxn + 1;
assertEquals(expectedTxnCount,
cache.retrieveEdits(startTxn, requestedMaxTxns, buffers));
byte[] expectedBytes = Bytes.concat(
getHeaderForLayoutVersion(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION),
createTxnData(startTxn, expectedTxnCount));
byte[] actualBytes =
new byte[buffers.stream().mapToInt(ByteBuffer::remaining).sum()];
int pos = 0;
for (ByteBuffer buf : buffers) {
System.arraycopy(buf.array(), buf.position(), actualBytes, pos,
buf.remaining());
pos += buf.remaining();
}
assertArrayEquals(expectedBytes, actualBytes);
}
private static byte[] getHeaderForLayoutVersion(int version)
throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
EditLogFileOutputStream.writeHeader(version, new DataOutputStream(baos));
return baos.toByteArray();
}
}

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
@ -1519,10 +1520,11 @@ public class NNThroughputBenchmark implements Tool {
nameNodeProto = DFSTestUtil.getNamenodeProtocolProxy(config, nnUri,
UserGroupInformation.getCurrentUser());
clientProto = dfs.getClient().getNamenode();
InetSocketAddress nnAddr = DFSUtilClient.getNNAddress(nnUri);
dataNodeProto = new DatanodeProtocolClientSideTranslatorPB(
DFSUtilClient.getNNAddress(nnUri), config);
nnAddr, config);
refreshUserMappingsProto =
DFSTestUtil.getRefreshUserMappingsProtocolProxy(config, nnUri);
DFSTestUtil.getRefreshUserMappingsProtocolProxy(config, nnAddr);
getBlockPoolId(dfs);
}
// run each benchmark

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
import static org.mockito.Mockito.spy;
@ -175,6 +177,11 @@ public class NameNodeAdapter {
return l == null ? -1 : l.getLastUpdate();
}
public static HAServiceState getServiceState(NameNode nn) {
return nn.getServiceState();
}
/**
* Return the datanode descriptor for the given datanode.
*/
@ -223,6 +230,12 @@ public class NameNodeAdapter {
return fsnSpy;
}
public static BlockManager spyOnBlockManager(NameNode nn) {
BlockManager bmSpy = Mockito.spy(nn.getNamesystem().getBlockManager());
nn.getNamesystem().setBlockManagerForTesting(bmSpy);
return bmSpy;
}
public static ReentrantReadWriteLock spyOnFsLock(FSNamesystem fsn) {
ReentrantReadWriteLock spy = Mockito.spy(fsn.getFsLockForTests());
fsn.setFsLockForTests(spy);

View File

@ -32,6 +32,7 @@ import java.net.HttpURLConnection;
import java.net.URL;
import java.util.EnumMap;
import com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
@ -80,6 +81,23 @@ public class TestEditLogFileInputStream {
elis.close();
}
@Test
public void testByteStringLog() throws Exception {
ByteString bytes = ByteString.copyFrom(FAKE_LOG_DATA);
EditLogInputStream elis = EditLogFileInputStream.fromByteString(bytes,
HdfsServerConstants.INVALID_TXID, HdfsServerConstants.INVALID_TXID,
true);
// Read the edit log and verify that all of the data is present
EnumMap<FSEditLogOpCodes, Holder<Integer>> counts = FSImageTestUtil
.countEditLogOpTypes(elis);
assertThat(counts.get(FSEditLogOpCodes.OP_ADD).held, is(1));
assertThat(counts.get(FSEditLogOpCodes.OP_SET_GENSTAMP_V1).held, is(1));
assertThat(counts.get(FSEditLogOpCodes.OP_CLOSE).held, is(1));
assertEquals(FAKE_LOG_DATA.length, elis.length());
elis.close();
}
/**
* Regression test for HDFS-8965 which verifies that
* FSEditLogFileInputStream#scanOp verifies Op checksums.

View File

@ -19,10 +19,13 @@ package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.io.BufferedInputStream;
import java.io.File;
@ -61,7 +64,9 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.util.FakeTimer;
import org.slf4j.event.Level;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -101,6 +106,7 @@ public class TestFSEditLogLoader {
private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
private static final int NUM_DATA_NODES = 0;
private static final String FAKE_EDIT_STREAM_NAME = "FAKE_STREAM";
private final ErasureCodingPolicy testECPolicy
= StripedFileTestUtil.getDefaultECPolicy();
@ -799,4 +805,46 @@ public class TestFSEditLogLoader {
}
}
}
@Test
public void setLoadFSEditLogThrottling() throws Exception {
FSNamesystem namesystem = mock(FSNamesystem.class);
namesystem.dir = mock(FSDirectory.class);
FakeTimer timer = new FakeTimer();
FSEditLogLoader loader = new FSEditLogLoader(namesystem, 0, timer);
LogCapturer capture = LogCapturer.captureLogs(FSImage.LOG);
loader.loadFSEdits(getFakeEditLogInputStream(1, 10), 1);
assertTrue(capture.getOutput().contains("Start loading edits file " +
FAKE_EDIT_STREAM_NAME));
assertTrue(capture.getOutput().contains("Loaded 1 edits file(s)"));
assertFalse(capture.getOutput().contains("suppressed"));
timer.advance(FSEditLogLoader.LOAD_EDIT_LOG_INTERVAL_MS / 2);
capture.clearOutput();
loader.loadFSEdits(getFakeEditLogInputStream(11, 20), 11);
assertFalse(capture.getOutput().contains("Start loading edits file"));
assertFalse(capture.getOutput().contains("edits file(s)"));
timer.advance(FSEditLogLoader.LOAD_EDIT_LOG_INTERVAL_MS);
capture.clearOutput();
loader.loadFSEdits(getFakeEditLogInputStream(21, 30), 21);
assertTrue(capture.getOutput().contains("Start loading edits file " +
FAKE_EDIT_STREAM_NAME));
assertTrue(capture.getOutput().contains("suppressed logging 1 times"));
assertTrue(capture.getOutput().contains("Loaded 2 edits file(s)"));
assertTrue(capture.getOutput().contains("total size 2.0"));
}
private EditLogInputStream getFakeEditLogInputStream(long startTx, long endTx)
throws IOException {
EditLogInputStream fakeStream = mock(EditLogInputStream.class);
when(fakeStream.getName()).thenReturn(FAKE_EDIT_STREAM_NAME);
when(fakeStream.getFirstTxId()).thenReturn(startTx);
when(fakeStream.getLastTxId()).thenReturn(endTx);
when(fakeStream.length()).thenReturn(1L);
return fakeStream;
}
}

View File

@ -165,7 +165,7 @@ public class TestFsck {
private static final String LINE_SEPARATOR =
System.getProperty("line.separator");
static String runFsck(Configuration conf, int expectedErrCode,
public static String runFsck(Configuration conf, int expectedErrCode,
boolean checkErrorCode, String... path)
throws Exception {
ByteArrayOutputStream bStream = new ByteArrayOutputStream();

View File

@ -17,35 +17,47 @@
*/
package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
import static org.apache.hadoop.hdfs.DFSUtil.createUri;
import java.io.IOException;
import java.lang.reflect.Field;
import java.lang.reflect.Proxy;
import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.LongAccumulator;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.ClientGSIContext;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.io.retry.FailoverProxyProvider;
import org.apache.hadoop.io.retry.RetryInvocationHandler;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
@ -158,7 +170,88 @@ public abstract class HATestUtil {
FileSystem fs = FileSystem.get(new URI("hdfs://" + logicalName), conf);
return (DistributedFileSystem)fs;
}
public static <P extends ObserverReadProxyProvider<?>>
DistributedFileSystem configureObserverReadFs(
MiniDFSCluster cluster, Configuration conf,
Class<P> classFPP, boolean isObserverReadEnabled)
throws IOException, URISyntaxException {
String logicalName = conf.get(DFSConfigKeys.DFS_NAMESERVICES);
URI nnUri = new URI(HdfsConstants.HDFS_URI_SCHEME + "://" + logicalName);
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+ "." + logicalName, classFPP.getName());
conf.set("fs.defaultFS", nnUri.toString());
DistributedFileSystem dfs = (DistributedFileSystem)
FileSystem.get(nnUri, conf);
@SuppressWarnings("unchecked")
P provider = (P) ((RetryInvocationHandler<?>) Proxy.getInvocationHandler(
dfs.getClient().getNamenode())).getProxyProvider();
provider.setObserverReadEnabled(isObserverReadEnabled);
return dfs;
}
public static boolean isSentToAnyOfNameNodes(
DistributedFileSystem dfs,
MiniDFSCluster cluster, int... nnIndices) throws IOException {
ObserverReadProxyProvider<?> provider = (ObserverReadProxyProvider<?>)
((RetryInvocationHandler<?>) Proxy.getInvocationHandler(
dfs.getClient().getNamenode())).getProxyProvider();
FailoverProxyProvider.ProxyInfo<?> pi = provider.getLastProxy();
for (int nnIdx : nnIndices) {
if (pi.proxyInfo.equals(
cluster.getNameNode(nnIdx).getNameNodeAddress().toString())) {
return true;
}
}
return false;
}
public static MiniQJMHACluster setUpObserverCluster(
Configuration conf, int numObservers, int numDataNodes,
boolean fastTailing) throws IOException {
// disable block scanner
conf.setInt(DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, -1);
conf.setBoolean(DFS_HA_TAILEDITS_INPROGRESS_KEY, fastTailing);
if(fastTailing) {
conf.setTimeDuration(
DFS_HA_TAILEDITS_PERIOD_KEY, 100, TimeUnit.MILLISECONDS);
} else {
// disable fast tailing so that coordination takes time.
conf.setTimeDuration(DFS_HA_LOGROLL_PERIOD_KEY, 300, TimeUnit.SECONDS);
conf.setTimeDuration(DFS_HA_TAILEDITS_PERIOD_KEY, 200, TimeUnit.SECONDS);
}
MiniQJMHACluster.Builder qjmBuilder = new MiniQJMHACluster.Builder(conf)
.setNumNameNodes(2 + numObservers);
qjmBuilder.getDfsBuilder().numDataNodes(numDataNodes);
MiniQJMHACluster qjmhaCluster = qjmBuilder.build();
MiniDFSCluster dfsCluster = qjmhaCluster.getDfsCluster();
dfsCluster.transitionToActive(0);
dfsCluster.waitActive(0);
for (int i = 0; i < numObservers; i++) {
dfsCluster.transitionToObserver(2 + i);
}
return qjmhaCluster;
}
public static <P extends FailoverProxyProvider<?>>
void setupHAConfiguration(MiniDFSCluster cluster,
Configuration conf, int nsIndex, Class<P> classFPP) {
MiniDFSCluster.NameNodeInfo[] nns = cluster.getNameNodeInfos(nsIndex);
List<String> nnAddresses = new ArrayList<String>();
for (MiniDFSCluster.NameNodeInfo nn : nns) {
InetSocketAddress addr = nn.nameNode.getNameNodeAddress();
nnAddresses.add(
createUri(HdfsConstants.HDFS_URI_SCHEME, addr).toString());
}
setFailoverConfigurations(
conf, getLogicalHostname(cluster), nnAddresses, classFPP);
}
public static void setFailoverConfigurations(MiniDFSCluster cluster,
Configuration conf) {
setFailoverConfigurations(cluster, conf, getLogicalHostname(cluster));
@ -199,11 +292,13 @@ public abstract class HATestUtil {
public String apply(InetSocketAddress addr) {
return "hdfs://" + addr.getHostName() + ":" + addr.getPort();
}
}));
}), ConfiguredFailoverProxyProvider.class);
}
public static void setFailoverConfigurations(Configuration conf, String logicalName,
Iterable<String> nnAddresses) {
public static <P extends FailoverProxyProvider<?>>
void setFailoverConfigurations(
Configuration conf, String logicalName,
Iterable<String> nnAddresses, Class<P> classFPP) {
List<String> nnids = new ArrayList<String>();
int i = 0;
for (String address : nnAddresses) {
@ -215,8 +310,8 @@ public abstract class HATestUtil {
conf.set(DFSConfigKeys.DFS_NAMESERVICES, logicalName);
conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, logicalName),
Joiner.on(',').join(nnids));
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
ConfiguredFailoverProxyProvider.class.getName());
conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX
+ "." + logicalName, classFPP.getName());
conf.set("fs.defaultFS", "hdfs://" + logicalName);
}
@ -246,4 +341,21 @@ public abstract class HATestUtil {
}
}
}
/**
* Customize stateId of the client AlignmentContext for testing.
*/
public static long setACStateId(DistributedFileSystem dfs,
long stateId) throws Exception {
ObserverReadProxyProvider<?> provider = (ObserverReadProxyProvider<?>)
((RetryInvocationHandler<?>) Proxy.getInvocationHandler(
dfs.getClient().getNamenode())).getProxyProvider();
ClientGSIContext ac = (ClientGSIContext)(provider.getAlignmentContext());
Field f = ac.getClass().getDeclaredField("lastSeenStateId");
f.setAccessible(true);
LongAccumulator lastSeenStateId = (LongAccumulator)f.get(ac);
long currentStateId = lastSeenStateId.getThenReset();
lastSeenStateId.accumulate(stateId);
return currentStateId;
}
}

View File

@ -0,0 +1,235 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdfs.server.namenode.ha;
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.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.RpcScheduler;
import org.apache.hadoop.ipc.Schedulable;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test consistency of reads while accessing an ObserverNode.
* The tests are based on traditional (non fast path) edits tailing.
*/
public class TestConsistentReadsObserver {
public static final Logger LOG =
LoggerFactory.getLogger(TestConsistentReadsObserver.class.getName());
private static Configuration conf;
private static MiniQJMHACluster qjmhaCluster;
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
private final Path testPath= new Path("/TestConsistentReadsObserver");
@BeforeClass
public static void startUpCluster() throws Exception {
conf = new Configuration();
// disable fast tailing here because this test's assertions are based on the
// timing of explicitly called rollEditLogAndTail. Although this means this
// test takes some time to run
// TODO: revisit if there is a better way.
qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 1, 0, false);
dfsCluster = qjmhaCluster.getDfsCluster();
}
@Before
public void setUp() throws Exception {
setObserverRead(true);
}
@After
public void cleanUp() throws IOException {
dfs.delete(testPath, true);
}
@AfterClass
public static void shutDownCluster() throws IOException {
if (qjmhaCluster != null) {
qjmhaCluster.shutdown();
}
}
@Test
public void testRequeueCall() throws Exception {
setObserverRead(true);
// Update the configuration just for the observer, by enabling
// IPC backoff and using the test scheduler class, which starts to backoff
// after certain number of calls.
final int observerIdx = 2;
NameNode nn = dfsCluster.getNameNode(observerIdx);
int port = nn.getNameNodeAddress().getPort();
Configuration configuration = dfsCluster.getConfiguration(observerIdx);
String prefix = CommonConfigurationKeys.IPC_NAMESPACE + "." + port + ".";
configuration.set(prefix + CommonConfigurationKeys.IPC_SCHEDULER_IMPL_KEY,
TestRpcScheduler.class.getName());
configuration.setBoolean(prefix
+ CommonConfigurationKeys.IPC_BACKOFF_ENABLE, true);
dfsCluster.restartNameNode(observerIdx);
dfsCluster.transitionToObserver(observerIdx);
dfs.create(testPath, (short)1).close();
assertSentTo(0);
// Since we haven't tailed edit logs on the observer, it will fall behind
// and keep re-queueing the incoming request. Eventually, RPC backoff will
// be triggered and client should retry active NN.
dfs.getFileStatus(testPath);
assertSentTo(0);
}
@Test
public void testMsyncSimple() throws Exception {
// 0 == not completed, 1 == succeeded, -1 == failed
AtomicInteger readStatus = new AtomicInteger(0);
// Making an uncoordinated call, which initialize the proxy
// to Observer node.
dfs.getClient().getHAServiceState();
dfs.mkdir(testPath, FsPermission.getDefault());
assertSentTo(0);
Thread reader = new Thread(() -> {
try {
// this read will block until roll and tail edits happen.
dfs.getFileStatus(testPath);
readStatus.set(1);
} catch (IOException e) {
e.printStackTrace();
readStatus.set(-1);
}
});
reader.start();
// the reader is still blocking, not succeeded yet.
assertEquals(0, readStatus.get());
dfsCluster.rollEditLogAndTail(0);
// wait a while for all the change to be done
GenericTestUtils.waitFor(() -> readStatus.get() != 0, 100, 10000);
// the reader should have succeed.
assertEquals(1, readStatus.get());
}
// @Ignore("Move to another test file")
@Test
public void testUncoordinatedCall() throws Exception {
// make a write call so that client will be ahead of
// observer for now.
dfs.mkdir(testPath, FsPermission.getDefault());
// a status flag, initialized to 0, after reader finished, this will be
// updated to 1, -1 on error
AtomicInteger readStatus = new AtomicInteger(0);
// create a separate thread to make a blocking read.
Thread reader = new Thread(() -> {
try {
// this read call will block until server state catches up. But due to
// configuration, this will take a very long time.
dfs.getClient().getFileInfo("/");
readStatus.set(1);
fail("Should have been interrupted before getting here.");
} catch (IOException e) {
e.printStackTrace();
readStatus.set(-1);
}
});
reader.start();
long before = Time.now();
dfs.getClient().datanodeReport(HdfsConstants.DatanodeReportType.ALL);
long after = Time.now();
// should succeed immediately, because datanodeReport is marked an
// uncoordinated call, and will not be waiting for server to catch up.
assertTrue(after - before < 200);
// by this time, reader thread should still be blocking, so the status not
// updated
assertEquals(0, readStatus.get());
Thread.sleep(5000);
// reader thread status should still be unchanged after 5 sec...
assertEquals(0, readStatus.get());
// and the reader thread is not dead, so it must be still waiting
assertEquals(Thread.State.WAITING, reader.getState());
reader.interrupt();
}
private void assertSentTo(int nnIdx) throws IOException {
assertTrue("Request was not sent to the expected namenode " + nnIdx,
HATestUtil.isSentToAnyOfNameNodes(dfs, dfsCluster, nnIdx));
}
private static void setObserverRead(boolean flag) throws Exception {
dfs = HATestUtil.configureObserverReadFs(
dfsCluster, conf, ObserverReadProxyProvider.class, flag);
}
/**
* A dummy test scheduler that starts backoff after a fixed number
* of requests.
*/
public static class TestRpcScheduler implements RpcScheduler {
// Allow a number of RPCs to pass in order for the NN restart to succeed.
private int allowed = 10;
public TestRpcScheduler() {}
@Override
public int getPriorityLevel(Schedulable obj) {
return 0;
}
@Override
public boolean shouldBackOff(Schedulable obj) {
return --allowed < 0;
}
@Override
public void addResponseTime(String name, int priorityLevel, int queueTime,
int processingTime) {
}
@Override
public void stop() {
}
}
}

View File

@ -0,0 +1,159 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdfs.server.namenode.ha;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Tests multiple ObserverNodes.
*/
public class TestMultiObserverNode {
private static Configuration conf;
private static MiniQJMHACluster qjmhaCluster;
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
private final Path testPath= new Path("/TestMultiObserverNode");
@BeforeClass
public static void startUpCluster() throws Exception {
conf = new Configuration();
qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 2, 0, true);
dfsCluster = qjmhaCluster.getDfsCluster();
dfs = HATestUtil.configureObserverReadFs(
dfsCluster, conf, ObserverReadProxyProvider.class, true);
}
@After
public void cleanUp() throws IOException {
dfs.delete(testPath, true);
}
@AfterClass
public static void shutDownCluster() throws IOException {
if (qjmhaCluster != null) {
qjmhaCluster.shutdown();
}
}
@Test
public void testObserverFailover() throws Exception {
dfs.mkdir(testPath, FsPermission.getDefault());
dfsCluster.rollEditLogAndTail(0);
dfs.getFileStatus(testPath);
assertSentTo(2, 3);
// Transition observer #2 to standby, request should go to the #3.
dfsCluster.transitionToStandby(2);
dfs.getFileStatus(testPath);
assertSentTo(3);
// Transition observer #3 to standby, request should go to active
dfsCluster.transitionToStandby(3);
dfs.getFileStatus(testPath);
assertSentTo(0);
// Transition #2 back to observer, request should go to #2
dfsCluster.transitionToObserver(2);
dfs.getFileStatus(testPath);
assertSentTo(2);
// Transition #3 back to observer, request should go to either #2 or #3
dfsCluster.transitionToObserver(3);
dfs.getFileStatus(testPath);
assertSentTo(2, 3);
}
@Test
public void testMultiObserver() throws Exception {
Path testPath2 = new Path(testPath, "test2");
Path testPath3 = new Path(testPath, "test3");
dfs.mkdir(testPath, FsPermission.getDefault());
assertSentTo(0);
dfsCluster.rollEditLogAndTail(0);
dfs.getFileStatus(testPath);
assertSentTo(2, 3);
dfs.mkdir(testPath2, FsPermission.getDefault());
dfsCluster.rollEditLogAndTail(0);
// Shutdown first observer, request should go to the second one
dfsCluster.shutdownNameNode(2);
dfs.listStatus(testPath2);
assertSentTo(3);
// Restart the first observer
dfsCluster.restartNameNode(2);
dfs.listStatus(testPath);
assertSentTo(3);
dfsCluster.transitionToObserver(2);
dfs.listStatus(testPath);
assertSentTo(2, 3);
dfs.mkdir(testPath3, FsPermission.getDefault());
dfsCluster.rollEditLogAndTail(0);
// Now shutdown the second observer, request should go to the first one
dfsCluster.shutdownNameNode(3);
dfs.listStatus(testPath3);
assertSentTo(2);
// Shutdown both, request should go to active
dfsCluster.shutdownNameNode(2);
dfs.listStatus(testPath3);
assertSentTo(0);
dfsCluster.restartNameNode(2);
dfsCluster.transitionToObserver(2);
dfsCluster.restartNameNode(3);
dfsCluster.transitionToObserver(3);
}
@Test
public void testObserverFallBehind() throws Exception {
dfs.mkdir(testPath, FsPermission.getDefault());
assertSentTo(0);
// Set large state Id on the client
long realStateId = HATestUtil.setACStateId(dfs, 500000);
dfs.getFileStatus(testPath);
// Should end up on ANN
assertSentTo(0);
HATestUtil.setACStateId(dfs, realStateId);
}
private void assertSentTo(int... nnIndices) throws IOException {
assertTrue("Request was not sent to any of the expected namenodes.",
HATestUtil.isSentToAnyOfNameNodes(dfs, dfsCluster, nnIndices));
}
}

View File

@ -0,0 +1,364 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdfs.server.namenode.ha;
import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getServiceState;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyBoolean;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Mockito.doAnswer;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.ServiceFailedException;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.namenode.TestFsck;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.Mockito;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test main functionality of ObserverNode.
*/
public class TestObserverNode {
public static final Logger LOG =
LoggerFactory.getLogger(TestObserverNode.class.getName());
private static Configuration conf;
private static MiniQJMHACluster qjmhaCluster;
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
private final Path testPath= new Path("/TestObserverNode");
@BeforeClass
public static void startUpCluster() throws Exception {
conf = new Configuration();
qjmhaCluster = HATestUtil.setUpObserverCluster(conf, 1, 0, true);
dfsCluster = qjmhaCluster.getDfsCluster();
}
@Before
public void setUp() throws Exception {
setObserverRead(true);
}
@After
public void cleanUp() throws IOException {
dfs.delete(testPath, true);
assertEquals("NN[0] should be active", HAServiceState.ACTIVE,
getServiceState(dfsCluster.getNameNode(0)));
assertEquals("NN[1] should be standby", HAServiceState.STANDBY,
getServiceState(dfsCluster.getNameNode(1)));
assertEquals("NN[2] should be observer", HAServiceState.OBSERVER,
getServiceState(dfsCluster.getNameNode(2)));
}
@AfterClass
public static void shutDownCluster() throws IOException {
if (qjmhaCluster != null) {
qjmhaCluster.shutdown();
}
}
@Test
public void testNoActiveToObserver() throws Exception {
try {
dfsCluster.transitionToObserver(0);
} catch (ServiceFailedException e) {
return;
}
fail("active cannot be transitioned to observer");
}
@Test
public void testNoObserverToActive() throws Exception {
try {
dfsCluster.transitionToActive(2);
} catch (ServiceFailedException e) {
return;
}
fail("observer cannot be transitioned to active");
}
@Test
public void testSimpleRead() throws Exception {
Path testPath2 = new Path(testPath, "test2");
dfs.mkdir(testPath, FsPermission.getDefault());
assertSentTo(0);
dfsCluster.rollEditLogAndTail(0);
dfs.getFileStatus(testPath);
assertSentTo(2);
dfs.mkdir(testPath2, FsPermission.getDefault());
assertSentTo(0);
}
@Test
public void testFailover() throws Exception {
Path testPath2 = new Path(testPath, "test2");
setObserverRead(false);
dfs.mkdir(testPath, FsPermission.getDefault());
assertSentTo(0);
dfs.getFileStatus(testPath);
assertSentTo(0);
dfsCluster.transitionToStandby(0);
dfsCluster.transitionToActive(1);
dfsCluster.waitActive(1);
dfs.mkdir(testPath2, FsPermission.getDefault());
assertSentTo(1);
dfs.getFileStatus(testPath);
assertSentTo(1);
dfsCluster.transitionToStandby(1);
dfsCluster.transitionToActive(0);
dfsCluster.waitActive(0);
}
@Test
public void testDoubleFailover() throws Exception {
Path testPath2 = new Path(testPath, "test2");
Path testPath3 = new Path(testPath, "test3");
dfs.mkdir(testPath, FsPermission.getDefault());
assertSentTo(0);
dfsCluster.rollEditLogAndTail(0);
dfs.getFileStatus(testPath);
assertSentTo(2);
dfs.mkdir(testPath2, FsPermission.getDefault());
assertSentTo(0);
dfsCluster.transitionToStandby(0);
dfsCluster.transitionToActive(1);
dfsCluster.waitActive(1);
dfsCluster.rollEditLogAndTail(1);
dfs.getFileStatus(testPath2);
assertSentTo(2);
dfs.mkdir(testPath3, FsPermission.getDefault());
assertSentTo(1);
dfsCluster.transitionToStandby(1);
dfsCluster.transitionToActive(0);
dfsCluster.waitActive(0);
dfsCluster.rollEditLogAndTail(0);
dfs.getFileStatus(testPath3);
assertSentTo(2);
dfs.delete(testPath3, false);
assertSentTo(0);
}
@Test
public void testObserverShutdown() throws Exception {
dfs.mkdir(testPath, FsPermission.getDefault());
dfsCluster.rollEditLogAndTail(0);
dfs.getFileStatus(testPath);
assertSentTo(2);
// Shutdown the observer - requests should go to active
dfsCluster.shutdownNameNode(2);
dfs.getFileStatus(testPath);
assertSentTo(0);
// Start the observer again - requests should go to observer
dfsCluster.restartNameNode(2);
dfsCluster.transitionToObserver(2);
// The first request goes to the active because it has not refreshed yet;
// the second will properly go to the observer
dfs.getFileStatus(testPath);
dfs.getFileStatus(testPath);
assertSentTo(2);
}
@Test
public void testObserverFailOverAndShutdown() throws Exception {
dfs.mkdir(testPath, FsPermission.getDefault());
dfsCluster.rollEditLogAndTail(0);
dfs.getFileStatus(testPath);
assertSentTo(2);
dfsCluster.transitionToStandby(0);
dfsCluster.transitionToActive(1);
dfsCluster.waitActive(1);
// Shutdown the observer - requests should go to active
dfsCluster.shutdownNameNode(2);
dfs.getFileStatus(testPath);
assertSentTo(1);
// Start the observer again - requests should go to observer
dfsCluster.restartNameNode(2);
dfs.getFileStatus(testPath);
assertSentTo(1);
dfsCluster.transitionToObserver(2);
dfs.getFileStatus(testPath);
// The first request goes to the active because it has not refreshed yet;
// the second will properly go to the observer
dfs.getFileStatus(testPath);
assertSentTo(2);
dfsCluster.transitionToStandby(1);
dfsCluster.transitionToActive(0);
dfsCluster.waitActive(0);
}
@Test
public void testBootstrap() throws Exception {
for (URI u : dfsCluster.getNameDirs(2)) {
File dir = new File(u.getPath());
assertTrue(FileUtil.fullyDelete(dir));
}
int rc = BootstrapStandby.run(
new String[]{"-nonInteractive"},
dfsCluster.getConfiguration(2)
);
assertEquals(0, rc);
}
/**
* Test the case where Observer should throw RetriableException, just like
* active NN, for certain open() calls where block locations are not
* available. See HDFS-13898 for details.
*/
@Test
public void testObserverNodeSafeModeWithBlockLocations() throws Exception {
// Create a new file - the request should go to active.
dfs.create(testPath, (short)1).close();
assertSentTo(0);
dfsCluster.rollEditLogAndTail(0);
dfs.open(testPath).close();
assertSentTo(2);
// Set observer to safe mode.
dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.SAFEMODE_ENTER);
// Mock block manager for observer to generate some fake blocks which
// will trigger the (retriable) safe mode exception.
BlockManager bmSpy =
NameNodeAdapter.spyOnBlockManager(dfsCluster.getNameNode(2));
doAnswer((invocation) -> {
ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
LocatedBlock fakeBlock = new LocatedBlock(b, DatanodeInfo.EMPTY_ARRAY);
List<LocatedBlock> fakeBlocks = new ArrayList<>();
fakeBlocks.add(fakeBlock);
return new LocatedBlocks(0, false, fakeBlocks, null, true, null, null);
}).when(bmSpy).createLocatedBlocks(any(), anyLong(), anyBoolean(),
anyLong(), anyLong(), anyBoolean(), anyBoolean(), any(), any());
// Open the file again - it should throw retriable exception and then
// failover to active.
dfs.open(testPath).close();
assertSentTo(0);
Mockito.reset(bmSpy);
// Remove safe mode on observer, request should still go to it.
dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
dfs.open(testPath).close();
assertSentTo(2);
}
@Test
public void testObserverNodeBlockMissingRetry() throws Exception {
setObserverRead(true);
dfs.create(testPath, (short)1).close();
assertSentTo(0);
dfsCluster.rollEditLogAndTail(0);
// Mock block manager for observer to generate some fake blocks which
// will trigger the block missing exception.
BlockManager bmSpy = NameNodeAdapter
.spyOnBlockManager(dfsCluster.getNameNode(2));
doAnswer((invocation) -> {
List<LocatedBlock> fakeBlocks = new ArrayList<>();
// Remove the datanode info for the only block so it will throw
// BlockMissingException and retry.
ExtendedBlock b = new ExtendedBlock("fake-pool", new Block(12345L));
LocatedBlock fakeBlock = new LocatedBlock(b, DatanodeInfo.EMPTY_ARRAY);
fakeBlocks.add(fakeBlock);
return new LocatedBlocks(0, false, fakeBlocks, null, true, null, null);
}).when(bmSpy).createLocatedBlocks(Mockito.any(), anyLong(),
anyBoolean(), anyLong(), anyLong(), anyBoolean(), anyBoolean(),
Mockito.any(), Mockito.any());
dfs.open(testPath);
assertSentTo(0);
Mockito.reset(bmSpy);
}
@Test
public void testFsckWithObserver() throws Exception {
setObserverRead(true);
dfs.create(testPath, (short)1).close();
assertSentTo(0);
final String result = TestFsck.runFsck(conf, 0, true, "/");
LOG.info("result=" + result);
assertTrue(result.contains("Status: HEALTHY"));
}
private void assertSentTo(int nnIdx) throws IOException {
assertTrue("Request was not sent to the expected namenode " + nnIdx,
HATestUtil.isSentToAnyOfNameNodes(dfs, dfsCluster, nnIdx));
}
private static void setObserverRead(boolean flag) throws Exception {
dfs = HATestUtil.configureObserverReadFs(
dfsCluster, conf, ObserverReadProxyProvider.class, flag);
}
}

View File

@ -0,0 +1,403 @@
/**
* 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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.hdfs.server.namenode.ha;
import com.google.common.base.Joiner;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.ipc.ObserverRetryOnActiveException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.ipc.StandbyException;
import org.apache.hadoop.security.UserGroupInformation;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import static org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
/**
* Tests for {@link ObserverReadProxyProvider} under various configurations of
* NameNode states. Mainly testing that the proxy provider picks the correct
* NameNode to communicate with.
*/
public class TestObserverReadProxyProvider {
private static final LocatedBlock[] EMPTY_BLOCKS = new LocatedBlock[0];
private String ns;
private URI nnURI;
private Configuration conf;
private ObserverReadProxyProvider<ClientProtocol> proxyProvider;
private NameNodeAnswer[] namenodeAnswers;
private String[] namenodeAddrs;
@Before
public void setup() throws Exception {
ns = "testcluster";
nnURI = URI.create("hdfs://" + ns);
conf = new Configuration();
conf.set(HdfsClientConfigKeys.DFS_NAMESERVICES, ns);
}
private void setupProxyProvider(int namenodeCount) throws Exception {
String[] namenodeIDs = new String[namenodeCount];
namenodeAddrs = new String[namenodeCount];
namenodeAnswers = new NameNodeAnswer[namenodeCount];
ClientProtocol[] proxies = new ClientProtocol[namenodeCount];
Map<String, ClientProtocol> proxyMap = new HashMap<>();
for (int i = 0; i < namenodeCount; i++) {
namenodeIDs[i] = "nn" + i;
namenodeAddrs[i] = "namenode" + i + ".test:8020";
conf.set(HdfsClientConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY + "." + ns +
"." + namenodeIDs[i], namenodeAddrs[i]);
namenodeAnswers[i] = new NameNodeAnswer();
proxies[i] = mock(ClientProtocol.class);
doWrite(Mockito.doAnswer(namenodeAnswers[i].clientAnswer)
.when(proxies[i]));
doRead(Mockito.doAnswer(namenodeAnswers[i].clientAnswer)
.when(proxies[i]));
Mockito.doAnswer(namenodeAnswers[i].clientAnswer)
.when(proxies[i]).getHAServiceState();
proxyMap.put(namenodeAddrs[i], proxies[i]);
}
conf.set(HdfsClientConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX + "." + ns,
Joiner.on(",").join(namenodeIDs));
proxyProvider = new ObserverReadProxyProvider<ClientProtocol>(conf, nnURI,
ClientProtocol.class,
new ClientHAProxyFactory<ClientProtocol>() {
@Override
public ClientProtocol createProxy(Configuration config,
InetSocketAddress nnAddr, Class<ClientProtocol> xface,
UserGroupInformation ugi, boolean withRetries,
AtomicBoolean fallbackToSimpleAuth) {
return proxyMap.get(nnAddr.toString());
}
}) {
@Override
protected List<NNProxyInfo<ClientProtocol>> getProxyAddresses(
URI uri, String addressKey) {
List<NNProxyInfo<ClientProtocol>> nnProxies =
super.getProxyAddresses(uri, addressKey);
return nnProxies;
}
};
proxyProvider.setObserverReadEnabled(true);
}
@Test
public void testReadOperationOnObserver() throws Exception {
setupProxyProvider(3);
namenodeAnswers[0].setActiveState();
namenodeAnswers[2].setObserverState();
doRead();
assertHandledBy(2);
}
@Test
public void testWriteOperationOnActive() throws Exception {
setupProxyProvider(3);
namenodeAnswers[0].setActiveState();
namenodeAnswers[2].setObserverState();
doWrite();
assertHandledBy(0);
}
@Test
public void testUnreachableObserverWithNoBackup() throws Exception {
setupProxyProvider(2);
namenodeAnswers[0].setActiveState();
namenodeAnswers[1].setObserverState();
namenodeAnswers[1].setUnreachable(true);
// Confirm that read still succeeds even though observer is not available
doRead();
assertHandledBy(0);
}
@Test
public void testUnreachableObserverWithMultiple() throws Exception {
setupProxyProvider(4);
namenodeAnswers[0].setActiveState();
namenodeAnswers[2].setObserverState();
namenodeAnswers[3].setObserverState();
doRead();
assertHandledBy(2);
namenodeAnswers[2].setUnreachable(true);
doRead();
// Fall back to the second observer node
assertHandledBy(3);
namenodeAnswers[2].setUnreachable(false);
doRead();
// Current index has changed, so although the first observer is back,
// it should continue requesting from the second observer
assertHandledBy(3);
namenodeAnswers[3].setUnreachable(true);
doRead();
// Now that second is unavailable, go back to using the first observer
assertHandledBy(2);
namenodeAnswers[2].setUnreachable(true);
doRead();
// Both observers are now unavailable, so it should fall back to active
assertHandledBy(0);
}
@Test
public void testObserverToActive() throws Exception {
setupProxyProvider(3);
namenodeAnswers[0].setActiveState();
namenodeAnswers[1].setObserverState();
namenodeAnswers[2].setObserverState();
doWrite();
assertHandledBy(0);
// Transition an observer to active
namenodeAnswers[0].setStandbyState();
namenodeAnswers[1].setActiveState();
try {
doWrite();
fail("Write should fail; failover required");
} catch (RemoteException re) {
assertEquals(re.getClassName(),
StandbyException.class.getCanonicalName());
}
proxyProvider.performFailover(proxyProvider.getProxy().proxy);
doWrite();
// After failover, previous observer is now active
assertHandledBy(1);
doRead();
assertHandledBy(2);
// Transition back to original state but second observer not available
namenodeAnswers[0].setActiveState();
namenodeAnswers[1].setObserverState();
namenodeAnswers[2].setUnreachable(true);
for (int i = 0; i < 2; i++) {
try {
doWrite();
fail("Should have failed");
} catch (IOException ioe) {
proxyProvider.performFailover(proxyProvider.getProxy().proxy);
}
}
doWrite();
assertHandledBy(0);
doRead();
assertHandledBy(1);
}
@Test
public void testObserverToStandby() throws Exception {
setupProxyProvider(3);
namenodeAnswers[0].setActiveState();
namenodeAnswers[1].setObserverState();
namenodeAnswers[2].setObserverState();
doRead();
assertHandledBy(1);
namenodeAnswers[1].setStandbyState();
doRead();
assertHandledBy(2);
namenodeAnswers[2].setStandbyState();
doRead();
assertHandledBy(0);
namenodeAnswers[1].setObserverState();
doRead();
assertHandledBy(1);
}
@Test
public void testSingleObserverToStandby() throws Exception {
setupProxyProvider(2);
namenodeAnswers[0].setActiveState();
namenodeAnswers[1].setObserverState();
doRead();
assertHandledBy(1);
namenodeAnswers[1].setStandbyState();
doRead();
assertHandledBy(0);
namenodeAnswers[1].setObserverState();
// The proxy provider still thinks the second NN is in observer state,
// so it will take a second call for it to notice the new observer
doRead();
doRead();
assertHandledBy(1);
}
@Test
public void testObserverRetriableException() throws Exception {
setupProxyProvider(3);
namenodeAnswers[0].setActiveState();
namenodeAnswers[1].setObserverState();
namenodeAnswers[2].setObserverState();
// Set the first observer to throw "ObserverRetryOnActiveException" so that
// the request should skip the second observer and be served by the active.
namenodeAnswers[1].setRetryActive(true);
doRead();
assertHandledBy(0);
namenodeAnswers[1].setRetryActive(false);
doRead();
assertHandledBy(1);
}
private void doRead() throws Exception {
doRead(proxyProvider.getProxy().proxy);
}
private void doWrite() throws Exception {
doWrite(proxyProvider.getProxy().proxy);
}
private void assertHandledBy(int namenodeIdx) {
assertEquals(namenodeAddrs[namenodeIdx],
proxyProvider.getLastProxy().proxyInfo);
}
private static void doWrite(ClientProtocol client) throws Exception {
client.reportBadBlocks(EMPTY_BLOCKS);
}
private static void doRead(ClientProtocol client) throws Exception {
client.checkAccess("/", FsAction.READ);
}
/**
* An {@link Answer} used for mocking of {@link ClientProtocol}.
* Setting the state or unreachability of this
* Answer will make the linked ClientProtocol respond as if it was
* communicating with a NameNode of the corresponding state. It is in Standby
* state by default.
*/
private static class NameNodeAnswer {
private volatile boolean unreachable = false;
private volatile boolean retryActive = false;
// Standby state by default
private volatile boolean allowWrites = false;
private volatile boolean allowReads = false;
private ClientProtocolAnswer clientAnswer = new ClientProtocolAnswer();
private class ClientProtocolAnswer implements Answer<Object> {
@Override
public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
if (unreachable) {
throw new IOException("Unavailable");
}
// retryActive should be checked before getHAServiceState.
// Check getHAServiceState first here only because in test,
// it relies read call, which relies on getHAServiceState
// to have passed already. May revisit future.
if (invocationOnMock.getMethod()
.getName().equals("getHAServiceState")) {
HAServiceState status;
if (allowReads && allowWrites) {
status = HAServiceState.ACTIVE;
} else if (allowReads) {
status = HAServiceState.OBSERVER;
} else {
status = HAServiceState.STANDBY;
}
return status;
}
if (retryActive) {
throw new RemoteException(
ObserverRetryOnActiveException.class.getCanonicalName(),
"Try active!"
);
}
switch (invocationOnMock.getMethod().getName()) {
case "reportBadBlocks":
if (!allowWrites) {
throw new RemoteException(
StandbyException.class.getCanonicalName(), "No writes!");
}
return null;
case "checkAccess":
if (!allowReads) {
throw new RemoteException(
StandbyException.class.getCanonicalName(), "No reads!");
}
return null;
default:
throw new IllegalArgumentException(
"Only reportBadBlocks and checkAccess supported!");
}
}
}
void setUnreachable(boolean unreachable) {
this.unreachable = unreachable;
}
void setActiveState() {
allowReads = true;
allowWrites = true;
}
void setStandbyState() {
allowReads = false;
allowWrites = false;
}
void setObserverState() {
allowReads = true;
allowWrites = false;
}
void setRetryActive(boolean shouldRetryActive) {
retryActive = shouldRetryActive;
}
}
}

View File

@ -17,12 +17,15 @@
*/
package org.apache.hadoop.hdfs.server.namenode.ha;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException;
import java.net.URI;
import java.util.Iterator;
import java.util.List;
import org.slf4j.Logger;
@ -30,9 +33,11 @@ import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
import org.apache.hadoop.hdfs.qjournal.server.JournalTestUtil;
import org.apache.hadoop.hdfs.server.namenode.NNStorage;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.test.GenericTestUtils;
@ -43,6 +48,7 @@ import org.junit.Before;
import org.junit.Test;
import com.google.common.base.Joiner;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
/**
@ -64,6 +70,8 @@ public class TestStandbyInProgressTail {
// Set period of tail edits to a large value (20 mins) for test purposes
conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 20 * 60);
conf.setBoolean(DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY, true);
conf.setInt(DFSConfigKeys.DFS_QJOURNAL_SELECT_INPUT_STREAMS_TIMEOUT_KEY,
500);
HAUtil.setAllowStandbyReads(conf, true);
qjmhaCluster = new MiniQJMHACluster.Builder(conf).build();
cluster = qjmhaCluster.getDfsCluster();
@ -179,12 +187,7 @@ public class TestStandbyInProgressTail {
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// After waiting for 5 seconds, StandbyNameNode should finish tailing
// in-progress logs
assertNotNull(getFileInfo(cluster.getNameNode(1),
"/test", true, false, false));
waitForFileInfo(nn1, "/test");
// Restarting the standby should not finalize any edits files
// in the shared directory when it starts up!
@ -227,10 +230,9 @@ public class TestStandbyInProgressTail {
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode should tail the in-progress edit
assertNotNull(getFileInfo(nn1, "/test", true, false, false));
waitForFileInfo(nn1, "/test");
// Create a new edit and finalized it
cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
@ -238,17 +240,14 @@ public class TestStandbyInProgressTail {
nn0.getRpcServer().rollEditLog();
// StandbyNameNode shouldn't tail the edit since we do not call the method
assertNull(getFileInfo(nn1, "/test2", true, false, false));
waitForFileInfo(nn1, "/test2");
// Create a new in-progress edit and let SBNN do the tail
cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode should tail the finalized edit and the new in-progress
assertNotNull(getFileInfo(nn1, "/test", true, false, false));
assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
assertNotNull(getFileInfo(nn1, "/test3", true, false, false));
waitForFileInfo(nn1, "/test", "/test2", "/test3");
}
@Test
@ -275,12 +274,8 @@ public class TestStandbyInProgressTail {
assertNull(getFileInfo(nn1, "/test2", true, false, false));
assertNull(getFileInfo(nn1, "/test3", true, false, false));
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode shoudl tail the finalized edit and the new in-progress
assertNotNull(getFileInfo(nn1, "/test", true, false, false));
assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
assertNotNull(getFileInfo(nn1, "/test3", true, false, false));
// StandbyNameNode should tail the finalized edit and the new in-progress
waitForFileInfo(nn1, "/test", "/test2", "/test3");
}
@Test
@ -295,19 +290,14 @@ public class TestStandbyInProgressTail {
FsPermission.createImmutable((short) 0755), true);
cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
waitForFileInfo(nn1, "/test", "/test2");
nn0.getRpcServer().rollEditLog();
assertNotNull(getFileInfo(nn1, "/test", true, false, false));
assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
FsPermission.createImmutable((short) 0755), true);
nn1.getNamesystem().getEditLogTailer().doTailEdits();
// StandbyNameNode shoudl tail the finalized edit and the new in-progress
assertNotNull(getFileInfo(nn1, "/test", true, false, false));
assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
assertNotNull(getFileInfo(nn1, "/test3", true, false, false));
// StandbyNameNode should tail the finalized edit and the new in-progress
waitForFileInfo(nn1, "/test", "/test2", "/test3");
}
@Test
@ -325,8 +315,85 @@ public class TestStandbyInProgressTail {
FsPermission.createImmutable((short) 0755), true);
cluster.getNameNode(0).getRpcServer().rollEdits();
cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits();
assertNotNull(getFileInfo(nn1, "/test", true, false, false));
waitForFileInfo(nn1, "/test");
}
@Test
public void testEditsServedViaCache() throws Exception {
cluster.transitionToActive(0);
cluster.waitActive(0);
mkdirs(nn0, "/test", "/test2");
nn0.getRpcServer().rollEditLog();
for (int idx = 0; idx < qjmhaCluster.getJournalCluster().getNumNodes();
idx++) {
File[] startingEditFile = qjmhaCluster.getJournalCluster()
.getCurrentDir(idx, DFSUtil.getNamenodeNameServiceId(conf))
.listFiles(new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
return name.matches("edits_0+1-[0-9]+");
}
});
assertNotNull(startingEditFile);
assertEquals(1, startingEditFile.length);
// Delete this edit file to ensure that edits can't be served via the
// streaming mechanism - RPC/cache-based only
startingEditFile[0].delete();
}
// Ensure edits were not tailed before the edit files were deleted;
// quick spot check of a single dir
assertNull(getFileInfo(nn1, "/tmp0", false, false, false));
waitForFileInfo(nn1, "/test", "/test2");
}
@Test
public void testCorruptJournalCache() throws Exception {
cluster.transitionToActive(0);
cluster.waitActive(0);
// Shut down one JN so there is only a quorum remaining to make it easier
// to manage the remaining two
qjmhaCluster.getJournalCluster().getJournalNode(0).stopAndJoin(0);
mkdirs(nn0, "/test", "/test2");
JournalTestUtil.corruptJournaledEditsCache(1,
qjmhaCluster.getJournalCluster().getJournalNode(1)
.getJournal(DFSUtil.getNamenodeNameServiceId(conf)));
nn0.getRpcServer().rollEditLog();
waitForFileInfo(nn1, "/test", "/test2");
mkdirs(nn0, "/test3", "/test4");
JournalTestUtil.corruptJournaledEditsCache(3,
qjmhaCluster.getJournalCluster().getJournalNode(2)
.getJournal(DFSUtil.getNamenodeNameServiceId(conf)));
waitForFileInfo(nn1, "/test3", "/test4");
}
@Test
public void testTailWithoutCache() throws Exception {
qjmhaCluster.shutdown();
// Effectively disable the cache by setting its size too small to be used
conf.setInt(DFSConfigKeys.DFS_JOURNALNODE_EDIT_CACHE_SIZE_KEY, 1);
qjmhaCluster = new MiniQJMHACluster.Builder(conf).build();
cluster = qjmhaCluster.getDfsCluster();
cluster.transitionToActive(0);
cluster.waitActive(0);
nn0 = cluster.getNameNode(0);
nn1 = cluster.getNameNode(1);
mkdirs(nn0, "/test", "/test2");
nn0.getRpcServer().rollEditLog();
mkdirs(nn0, "/test3", "/test4");
// Skip the last directory; the JournalNodes' idea of the committed
// txn ID may not have been updated to include it yet
waitForFileInfo(nn1, "/test", "/test2", "/test3");
}
/**
@ -356,4 +423,43 @@ public class TestStandbyInProgressTail {
GenericTestUtils.assertGlobEquals(editDir, "edits_.*", files);
}
}
/**
* Create the given directories on the provided NameNode.
*/
private static void mkdirs(NameNode nameNode, String... dirNames)
throws Exception {
for (String dirName : dirNames) {
nameNode.getRpcServer().mkdirs(dirName,
FsPermission.createImmutable((short) 0755), true);
}
}
/**
* Wait up to 1 second until the given NameNode is aware of the existing of
* all of the provided fileNames.
*/
private static void waitForFileInfo(NameNode standbyNN, String... fileNames)
throws Exception {
List<String> remainingFiles = Lists.newArrayList(fileNames);
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
try {
standbyNN.getNamesystem().getEditLogTailer().doTailEdits();
for (Iterator<String> it = remainingFiles.iterator(); it.hasNext();) {
if (getFileInfo(standbyNN, it.next(), true, false, false) == null) {
return false;
} else {
it.remove();
}
}
return true;
} catch (IOException|InterruptedException e) {
throw new AssertionError("Exception while waiting: " + e);
}
}
}, 10, 1000);
}
}

View File

@ -216,11 +216,16 @@ public class TestDFSHAAdmin {
assertTrue(errOutput.contains("Refusing to manually manage"));
assertEquals(-1, runTool("-transitionToStandby", "nn1"));
assertTrue(errOutput.contains("Refusing to manually manage"));
assertEquals(-1, runTool("-transitionToObserver", "nn1"));
assertTrue(errOutput.contains("Refusing to manually manage"));
Mockito.verify(mockProtocol, Mockito.never())
.transitionToActive(anyReqInfo());
Mockito.verify(mockProtocol, Mockito.never())
.transitionToStandby(anyReqInfo());
.transitionToStandby(anyReqInfo());
Mockito.verify(mockProtocol, Mockito.never())
.transitionToObserver(anyReqInfo());
// Force flag should bypass the check and change the request source
// for the RPC
@ -228,12 +233,16 @@ public class TestDFSHAAdmin {
assertEquals(0, runTool("-transitionToActive", "-forcemanual", "nn1"));
setupConfirmationOnSystemIn();
assertEquals(0, runTool("-transitionToStandby", "-forcemanual", "nn1"));
setupConfirmationOnSystemIn();
assertEquals(0, runTool("-transitionToObserver", "-forcemanual", "nn1"));
Mockito.verify(mockProtocol, Mockito.times(1)).transitionToActive(
reqInfoCaptor.capture());
Mockito.verify(mockProtocol, Mockito.times(1)).transitionToStandby(
reqInfoCaptor.capture());
Mockito.verify(mockProtocol, Mockito.times(1)).transitionToObserver(
reqInfoCaptor.capture());
// All of the RPCs should have had the "force" source
for (StateChangeRequestInfo ri : reqInfoCaptor.getAllValues()) {
assertEquals(RequestSource.REQUEST_BY_USER_FORCED, ri.getSource());
@ -275,6 +284,12 @@ public class TestDFSHAAdmin {
Mockito.verify(mockProtocol).transitionToStandby(anyReqInfo());
}
@Test
public void testTransitionToObserver() throws Exception {
assertEquals(0, runTool("-transitionToObserver", "nn1"));
Mockito.verify(mockProtocol).transitionToObserver(anyReqInfo());
}
@Test
public void testFailoverWithNoFencerConfigured() throws Exception {
Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();

View File

@ -116,6 +116,50 @@ public class TestDFSHAAdminMiniCluster {
assertFalse(nnode2.isStandbyState());
assertEquals(0, runTool("-transitionToStandby", "nn2"));
assertTrue(nnode2.isStandbyState());
assertEquals(0, runTool("-transitionToObserver", "nn2"));
assertFalse(nnode2.isStandbyState());
assertTrue(nnode2.isObserverState());
}
@Test
public void testObserverTransition() throws Exception {
NameNode nnode1 = cluster.getNameNode(0);
assertTrue(nnode1.isStandbyState());
// Should be able to transition from STANDBY to OBSERVER
assertEquals(0, runTool("-transitionToObserver", "nn1"));
assertFalse(nnode1.isStandbyState());
assertTrue(nnode1.isObserverState());
// Transition from Observer to Observer should be no-op
assertEquals(0, runTool("-transitionToObserver", "nn1"));
assertTrue(nnode1.isObserverState());
// Should also be able to transition back from OBSERVER to STANDBY
assertEquals(0, runTool("-transitionToStandby", "nn1"));
assertTrue(nnode1.isStandbyState());
assertFalse(nnode1.isObserverState());
}
@Test
public void testObserverIllegalTransition() throws Exception {
NameNode nnode1 = cluster.getNameNode(0);
assertTrue(nnode1.isStandbyState());
assertEquals(0, runTool("-transitionToActive", "nn1"));
assertFalse(nnode1.isStandbyState());
assertTrue(nnode1.isActiveState());
// Should NOT be able to transition from ACTIVE to OBSERVER
assertEquals(-1, runTool("-transitionToObserver", "nn1"));
assertTrue(nnode1.isActiveState());
// Should NOT be able to transition from OBSERVER to ACTIVE
assertEquals(0, runTool("-transitionToStandby", "nn1"));
assertTrue(nnode1.isStandbyState());
assertEquals(0, runTool("-transitionToObserver", "nn1"));
assertTrue(nnode1.isObserverState());
assertEquals(-1, runTool("-transitionToActive", "nn1"));
assertFalse(nnode1.isActiveState());
}
@Test

View File

@ -362,6 +362,13 @@ public class AdminService extends CompositeService implements
}
}
@Override
public synchronized void transitionToObserver(
StateChangeRequestInfo reqInfo) throws IOException {
// Should NOT get here, as RMHAServiceTarget doesn't support observer.
throw new ServiceFailedException("Does not support transition to Observer");
}
/**
* Return the HA status of this RM. This includes the current state and
* whether the RM is ready to become active.