Skip to content

Commit e77d54d

Browse files
simbadzinaomalley
authored andcommitted
HDFS-13522: Add federated nameservices states to client protocol and propagate it between routers and clients.
Fixes #4311 Signed-off-by: Owen O'Malley <oomalley@linkedin.com>
1 parent e76ffbf commit e77d54d

File tree

14 files changed

+426
-53
lines changed

14 files changed

+426
-53
lines changed

hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -937,6 +937,9 @@ public static class Call implements Schedulable,
937937
// the priority level assigned by scheduler, 0 by default
938938
private long clientStateId;
939939
private boolean isCallCoordinated;
940+
// Serialized RouterFederatedStateProto message to
941+
// store last seen states for multiple namespaces.
942+
private ByteString federatedNamespaceState;
940943

941944
Call() {
942945
this(RpcConstants.INVALID_CALL_ID, RpcConstants.INVALID_RETRY_COUNT,
@@ -994,6 +997,14 @@ public ProcessingDetails getProcessingDetails() {
994997
return processingDetails;
995998
}
996999

1000+
public void setFederatedNamespaceState(ByteString federatedNamespaceState) {
1001+
this.federatedNamespaceState = federatedNamespaceState;
1002+
}
1003+
1004+
public ByteString getFederatedNamespaceState() {
1005+
return this.federatedNamespaceState;
1006+
}
1007+
9971008
@Override
9981009
public String toString() {
9991010
return "Call#" + callId + " Retry#" + retryCount;
@@ -2868,6 +2879,9 @@ private void processRpcRequest(RpcRequestHeaderProto header,
28682879
stateId = alignmentContext.receiveRequestState(
28692880
header, getMaxIdleTime());
28702881
call.setClientStateId(stateId);
2882+
if (header.hasRouterFederatedState()) {
2883+
call.setFederatedNamespaceState(header.getRouterFederatedState());
2884+
}
28712885
}
28722886
} catch (IOException ioe) {
28732887
throw new RpcServerException("Processing RPC request caught ", ioe);

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientGSIContext.java

Lines changed: 25 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626

2727
import java.io.IOException;
2828
import java.util.concurrent.atomic.LongAccumulator;
29+
import org.apache.hadoop.thirdparty.protobuf.ByteString;
2930

3031
/**
3132
* Global State Id context for the client.
@@ -37,8 +38,17 @@
3738
@InterfaceStability.Evolving
3839
public class ClientGSIContext implements AlignmentContext {
3940

40-
private final LongAccumulator lastSeenStateId =
41-
new LongAccumulator(Math::max, Long.MIN_VALUE);
41+
private final LongAccumulator lastSeenStateId;
42+
private ByteString routerFederatedState;
43+
44+
public ClientGSIContext() {
45+
this(new LongAccumulator(Math::max, Long.MIN_VALUE));
46+
}
47+
48+
public ClientGSIContext(LongAccumulator lastSeenStateId) {
49+
this.lastSeenStateId = lastSeenStateId;
50+
routerFederatedState = null;
51+
}
4252

4353
@Override
4454
public long getLastSeenStateId() {
@@ -65,16 +75,25 @@ public void updateResponseState(RpcResponseHeaderProto.Builder header) {
6575
* in responses.
6676
*/
6777
@Override
68-
public void receiveResponseState(RpcResponseHeaderProto header) {
69-
lastSeenStateId.accumulate(header.getStateId());
78+
public synchronized void receiveResponseState(RpcResponseHeaderProto header) {
79+
if (header.hasRouterFederatedState()) {
80+
routerFederatedState = header.getRouterFederatedState();
81+
} else {
82+
lastSeenStateId.accumulate(header.getStateId());
83+
}
7084
}
7185

7286
/**
7387
* Client side implementation for providing state alignment info in requests.
7488
*/
7589
@Override
76-
public void updateRequestState(RpcRequestHeaderProto.Builder header) {
77-
header.setStateId(lastSeenStateId.longValue());
90+
public synchronized void updateRequestState(RpcRequestHeaderProto.Builder header) {
91+
if (lastSeenStateId.get() != Long.MIN_VALUE) {
92+
header.setStateId(lastSeenStateId.get());
93+
}
94+
if (routerFederatedState != null) {
95+
header.setRouterFederatedState(routerFederatedState);
96+
}
7897
}
7998

8099
/**

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -349,6 +349,9 @@ public static ClientProtocol createProxyWithAlignmentContext(
349349
boolean withRetries, AtomicBoolean fallbackToSimpleAuth,
350350
AlignmentContext alignmentContext)
351351
throws IOException {
352+
if (alignmentContext == null) {
353+
alignmentContext = new ClientGSIContext();
354+
}
352355
RPC.setProtocolEngine(conf, ClientNamenodeProtocolPB.class,
353356
ProtobufRpcEngine2.class);
354357

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java

Lines changed: 30 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,14 @@ public class ConnectionManager {
7373

7474
/** Queue for creating new connections. */
7575
private final BlockingQueue<ConnectionPool> creatorQueue;
76+
/**
77+
* Global federated namespace context for router.
78+
*/
79+
private final RouterStateIdContext routerStateIdContext;
80+
/**
81+
* Map from connection pool ID to namespace.
82+
*/
83+
private final Map<ConnectionPoolId, String> connectionPoolToNamespaceMap;
7684
/** Max size of queue for creating new connections. */
7785
private final int creatorQueueMaxSize;
7886

@@ -85,15 +93,19 @@ public class ConnectionManager {
8593
/** If the connection manager is running. */
8694
private boolean running = false;
8795

96+
public ConnectionManager(Configuration config) {
97+
this(config, new RouterStateIdContext(config));
98+
}
8899

89100
/**
90101
* Creates a proxy client connection pool manager.
91102
*
92103
* @param config Configuration for the connections.
93104
*/
94-
public ConnectionManager(Configuration config) {
105+
public ConnectionManager(Configuration config, RouterStateIdContext routerStateIdContext) {
95106
this.conf = config;
96-
107+
this.routerStateIdContext = routerStateIdContext;
108+
this.connectionPoolToNamespaceMap = new HashMap<>();
97109
// Configure minimum, maximum and active connection pools
98110
this.maxSize = this.conf.getInt(
99111
RBFConfigKeys.DFS_ROUTER_NAMENODE_CONNECTION_POOL_SIZE,
@@ -160,6 +172,10 @@ public void close() {
160172
pool.close();
161173
}
162174
this.pools.clear();
175+
for (String nsID: connectionPoolToNamespaceMap.values()) {
176+
routerStateIdContext.removeNamespaceStateId(nsID);
177+
}
178+
connectionPoolToNamespaceMap.clear();
163179
} finally {
164180
writeLock.unlock();
165181
}
@@ -172,12 +188,12 @@ public void close() {
172188
* @param ugi User group information.
173189
* @param nnAddress Namenode address for the connection.
174190
* @param protocol Protocol for the connection.
191+
* @param nsId Nameservice identity.
175192
* @return Proxy client to connect to nnId as UGI.
176193
* @throws IOException If the connection cannot be obtained.
177194
*/
178195
public ConnectionContext getConnection(UserGroupInformation ugi,
179-
String nnAddress, Class<?> protocol) throws IOException {
180-
196+
String nnAddress, Class<?> protocol, String nsId) throws IOException {
181197
// Check if the manager is shutdown
182198
if (!this.running) {
183199
LOG.error(
@@ -205,9 +221,13 @@ public ConnectionContext getConnection(UserGroupInformation ugi,
205221
if (pool == null) {
206222
pool = new ConnectionPool(
207223
this.conf, nnAddress, ugi, this.minSize, this.maxSize,
208-
this.minActiveRatio, protocol);
224+
this.minActiveRatio, protocol,
225+
new PoolAlignmentContext(this.routerStateIdContext, nsId));
209226
this.pools.put(connectionId, pool);
227+
this.connectionPoolToNamespaceMap.put(connectionId, nsId);
210228
}
229+
long clientStateId = RouterStateIdContext.getClientStateIdFromCurrentCall(nsId);
230+
pool.getPoolAlignmentContext().advanceClientStateId(clientStateId);
211231
} finally {
212232
writeLock.unlock();
213233
}
@@ -430,6 +450,11 @@ public void run() {
430450
try {
431451
for (ConnectionPoolId poolId : toRemove) {
432452
pools.remove(poolId);
453+
String nsID = connectionPoolToNamespaceMap.get(poolId);
454+
connectionPoolToNamespaceMap.remove(poolId);
455+
if (!connectionPoolToNamespaceMap.values().contains(nsID)) {
456+
routerStateIdContext.removeNamespaceStateId(nsID);
457+
}
433458
}
434459
} finally {
435460
writeLock.unlock();

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java

Lines changed: 22 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import javax.net.SocketFactory;
3333

3434
import org.apache.hadoop.classification.VisibleForTesting;
35+
import org.apache.hadoop.ipc.AlignmentContext;
3536
import org.apache.hadoop.classification.InterfaceAudience;
3637
import org.apache.hadoop.classification.InterfaceStability;
3738
import org.apache.hadoop.conf.Configuration;
@@ -108,6 +109,8 @@ public class ConnectionPool {
108109

109110
/** Enable using multiple physical socket or not. **/
110111
private final boolean enableMultiSocket;
112+
/** StateID alignment context. */
113+
private final PoolAlignmentContext alignmentContext;
111114

112115
/** Map for the protocols and their protobuf implementations. */
113116
private final static Map<Class<?>, ProtoImpl> PROTO_MAP = new HashMap<>();
@@ -138,7 +141,8 @@ private static class ProtoImpl {
138141

139142
protected ConnectionPool(Configuration config, String address,
140143
UserGroupInformation user, int minPoolSize, int maxPoolSize,
141-
float minActiveRatio, Class<?> proto) throws IOException {
144+
float minActiveRatio, Class<?> proto, PoolAlignmentContext alignmentContext)
145+
throws IOException {
142146

143147
this.conf = config;
144148

@@ -157,6 +161,8 @@ protected ConnectionPool(Configuration config, String address,
157161
RBFConfigKeys.DFS_ROUTER_NAMENODE_ENABLE_MULTIPLE_SOCKET_KEY,
158162
RBFConfigKeys.DFS_ROUTER_NAMENODE_ENABLE_MULTIPLE_SOCKET_DEFAULT);
159163

164+
this.alignmentContext = alignmentContext;
165+
160166
// Add minimum connections to the pool
161167
for (int i = 0; i < this.minSize; i++) {
162168
ConnectionContext newConnection = newConnection();
@@ -211,6 +217,14 @@ public AtomicInteger getClientIndex() {
211217
return this.clientIndex;
212218
}
213219

220+
/**
221+
* Get the alignment context for this pool
222+
* @return Alignment context
223+
*/
224+
public PoolAlignmentContext getPoolAlignmentContext() {
225+
return this.alignmentContext;
226+
}
227+
214228
/**
215229
* Return the next connection round-robin.
216230
*
@@ -398,7 +412,7 @@ public String getJSON() {
398412
public ConnectionContext newConnection() throws IOException {
399413
return newConnection(this.conf, this.namenodeAddress,
400414
this.ugi, this.protocol, this.enableMultiSocket,
401-
this.socketIndex.incrementAndGet());
415+
this.socketIndex.incrementAndGet(), alignmentContext);
402416
}
403417

404418
/**
@@ -413,13 +427,15 @@ public ConnectionContext newConnection() throws IOException {
413427
* @param ugi User context.
414428
* @param proto Interface of the protocol.
415429
* @param enableMultiSocket Enable multiple socket or not.
430+
* @param alignmentContext Client alignment context.
416431
* @return proto for the target ClientProtocol that contains the user's
417432
* security context.
418433
* @throws IOException If it cannot be created.
419434
*/
420435
protected static <T> ConnectionContext newConnection(Configuration conf,
421436
String nnAddress, UserGroupInformation ugi, Class<T> proto,
422-
boolean enableMultiSocket, int socketIndex) throws IOException {
437+
boolean enableMultiSocket, int socketIndex,
438+
AlignmentContext alignmentContext) throws IOException {
423439
if (!PROTO_MAP.containsKey(proto)) {
424440
String msg = "Unsupported protocol for connection to NameNode: "
425441
+ ((proto != null) ? proto.getName() : "null");
@@ -448,10 +464,11 @@ protected static <T> ConnectionContext newConnection(Configuration conf,
448464
socket, classes.protoPb, ugi, RPC.getRpcTimeout(conf),
449465
defaultPolicy, conf, socketIndex);
450466
proxy = RPC.getProtocolProxy(classes.protoPb, version, connectionId,
451-
conf, factory).getProxy();
467+
conf, factory, alignmentContext).getProxy();
452468
} else {
453469
proxy = RPC.getProtocolProxy(classes.protoPb, version, socket, ugi,
454-
conf, factory, RPC.getRpcTimeout(conf), defaultPolicy, null).getProxy();
470+
conf, factory, RPC.getRpcTimeout(conf), defaultPolicy, null,
471+
alignmentContext).getProxy();
455472
}
456473

457474
T client = newProtoClient(proto, classes, proxy);
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,103 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.hdfs.server.federation.router;
20+
21+
import java.io.IOException;
22+
import java.util.concurrent.atomic.LongAccumulator;
23+
import org.apache.hadoop.ipc.AlignmentContext;
24+
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos;
25+
26+
27+
/**
28+
* An alignment context shared by all connections in a {@link ConnectionPool}.
29+
* There is a distinct connection pool for each [namespace,UGI] pairing.
30+
* <p>
31+
* {@link #sharedGlobalStateId} is a reference to a
32+
* shared {@link LongAccumulator} object in the {@link RouterStateIdContext}.
33+
* {@link #poolLocalStateId} is specific to each PoolAlignmentContext.
34+
* <p>
35+
* The shared {@link #sharedGlobalStateId} is updated only using
36+
* responses from NameNodes, so clients cannot poison it.
37+
* {@link #poolLocalStateId} is used to propagate client observed
38+
* state into NameNode requests. A misbehaving client can poison this but the effect is only
39+
* visible to other clients with the same UGI and accessing the same namespace.
40+
*/
41+
public class PoolAlignmentContext implements AlignmentContext {
42+
private LongAccumulator sharedGlobalStateId;
43+
private LongAccumulator poolLocalStateId;
44+
45+
PoolAlignmentContext(RouterStateIdContext routerStateIdContext, String namespaceId) {
46+
sharedGlobalStateId = routerStateIdContext.getNamespaceStateId(namespaceId);
47+
poolLocalStateId = new LongAccumulator(Math::max, Long.MIN_VALUE);
48+
}
49+
50+
/**
51+
* Client side implementation only receives state alignment info.
52+
* It does not provide state alignment info therefore this does nothing.
53+
*/
54+
@Override
55+
public void updateResponseState(RpcHeaderProtos.RpcResponseHeaderProto.Builder header) {
56+
// Do nothing.
57+
}
58+
59+
/**
60+
* Router updates a globally shared value using response from
61+
* namenodes.
62+
*/
63+
@Override
64+
public void receiveResponseState(RpcHeaderProtos.RpcResponseHeaderProto header) {
65+
sharedGlobalStateId.accumulate(header.getStateId());
66+
}
67+
68+
/**
69+
* Client side implementation for routers to provide state info in requests to
70+
* namenodes.
71+
*/
72+
@Override
73+
public void updateRequestState(RpcHeaderProtos.RpcRequestHeaderProto.Builder header) {
74+
long maxStateId = Long.max(poolLocalStateId.get(), sharedGlobalStateId.get());
75+
header.setStateId(maxStateId);
76+
}
77+
78+
/**
79+
* Client side implementation only provides state alignment info in requests.
80+
* Client does not receive RPC requests therefore this does nothing.
81+
*/
82+
@Override
83+
public long receiveRequestState(RpcHeaderProtos.RpcRequestHeaderProto header, long threshold)
84+
throws IOException {
85+
// Do nothing.
86+
return 0;
87+
}
88+
89+
@Override
90+
public long getLastSeenStateId() {
91+
return sharedGlobalStateId.get();
92+
}
93+
94+
@Override
95+
public boolean isCoordinatedCall(String protocolName, String method) {
96+
throw new UnsupportedOperationException(
97+
"Client should not be checking uncoordinated call");
98+
}
99+
100+
public void advanceClientStateId(Long clientStateId) {
101+
poolLocalStateId.accumulate(clientStateId);
102+
}
103+
}

hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -191,6 +191,10 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic {
191191
FEDERATION_STORE_PREFIX + "enable";
192192
public static final boolean DFS_ROUTER_STORE_ENABLE_DEFAULT = true;
193193

194+
public static final String DFS_ROUTER_OBSERVER_FEDERATED_STATE_PROPAGATION_MAXSIZE =
195+
FEDERATION_ROUTER_PREFIX + "observer.federated.state.propagation.maxsize";
196+
public static final int DFS_ROUTER_OBSERVER_FEDERATED_STATE_PROPAGATION_MAXSIZE_DEFAULT = 5;
197+
194198
public static final String FEDERATION_STORE_SERIALIZER_CLASS =
195199
FEDERATION_STORE_PREFIX + "serializer";
196200
public static final Class<StateStoreSerializerPBImpl>

0 commit comments

Comments
 (0)