Skip to content

Commit 02bd02b

Browse files
committed
HDFS-14660. [SBN Read] ObserverNameNode should throw StandbyException for requests not from ObserverProxyProvider. Contributed by Chao Sun.
1 parent 2fe450c commit 02bd02b

File tree

2 files changed

+52
-1
lines changed

2 files changed

+52
-1
lines changed

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GlobalStateIdContext.java

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
package org.apache.hadoop.hdfs.server.namenode;
2020

21+
import java.io.IOException;
2122
import java.lang.reflect.Method;
2223
import java.util.HashSet;
2324
import java.util.concurrent.TimeUnit;
@@ -26,9 +27,11 @@
2627
import org.apache.hadoop.classification.InterfaceStability;
2728
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
2829
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
30+
import org.apache.hadoop.hdfs.server.namenode.ha.ObserverReadProxyProvider;
2931
import org.apache.hadoop.hdfs.server.namenode.ha.ReadOnly;
3032
import org.apache.hadoop.ipc.AlignmentContext;
3133
import org.apache.hadoop.ipc.RetriableException;
34+
import org.apache.hadoop.ipc.StandbyException;
3235
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcRequestHeaderProto;
3336
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto;
3437

@@ -123,7 +126,18 @@ public void updateRequestState(RpcRequestHeaderProto.Builder header) {
123126
*/
124127
@Override
125128
public long receiveRequestState(RpcRequestHeaderProto header,
126-
long clientWaitTime) throws RetriableException {
129+
long clientWaitTime) throws IOException {
130+
if (!header.hasStateId() &&
131+
HAServiceState.OBSERVER.equals(namesystem.getState())) {
132+
// This could happen if client configured with non-observer proxy provider
133+
// (e.g., ConfiguredFailoverProxyProvider) is accessing a cluster with
134+
// observers. In this case, we should let the client failover to the
135+
// active node, rather than potentially serving stale result (client
136+
// stateId is 0 if not set).
137+
throw new StandbyException("Observer Node received request without "
138+
+ "stateId. This mostly likely is because client is not configured "
139+
+ "with " + ObserverReadProxyProvider.class.getSimpleName());
140+
}
127141
long serverStateId = getLastSeenStateId();
128142
long clientStateId = header.getStateId();
129143
FSNamesystem.LOG.trace("Client State ID= {} and Server State ID= {}",

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java

Lines changed: 37 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import static org.junit.Assert.fail;
2323

2424
import java.io.IOException;
25+
import java.util.Collections;
2526
import java.util.concurrent.TimeUnit;
2627
import java.util.concurrent.TimeoutException;
2728
import java.util.concurrent.atomic.AtomicInteger;
@@ -33,12 +34,15 @@
3334
import org.apache.hadoop.fs.permission.FsPermission;
3435
import org.apache.hadoop.hdfs.DistributedFileSystem;
3536
import org.apache.hadoop.hdfs.MiniDFSCluster;
37+
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
3638
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
3739
import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
3840
import org.apache.hadoop.hdfs.server.namenode.NameNode;
3941
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
42+
import org.apache.hadoop.ipc.RemoteException;
4043
import org.apache.hadoop.ipc.RpcScheduler;
4144
import org.apache.hadoop.ipc.Schedulable;
45+
import org.apache.hadoop.ipc.StandbyException;
4246
import org.apache.hadoop.test.GenericTestUtils;
4347
import org.apache.hadoop.util.Time;
4448
import org.junit.After;
@@ -344,6 +348,39 @@ public void testUncoordinatedCall() throws Exception {
344348
reader.interrupt();
345349
}
346350

351+
@Test
352+
public void testRequestFromNonObserverProxyProvider() throws Exception {
353+
// Create another HDFS client using ConfiguredFailoverProvider
354+
Configuration conf2 = new Configuration(conf);
355+
356+
// Populate the above configuration with only a single observer in the
357+
// namenode list. Also reduce retries to make test finish faster.
358+
HATestUtil.setFailoverConfigurations(
359+
conf2,
360+
HATestUtil.getLogicalHostname(dfsCluster),
361+
Collections.singletonList(
362+
dfsCluster.getNameNode(2).getNameNodeAddress()),
363+
ConfiguredFailoverProxyProvider.class);
364+
conf2.setBoolean("fs.hdfs.impl.disable.cache", true);
365+
conf2.setInt(HdfsClientConfigKeys.Retry.MAX_ATTEMPTS_KEY, 1);
366+
conf2.setInt(HdfsClientConfigKeys.Failover.MAX_ATTEMPTS_KEY, 1);
367+
FileSystem dfs2 = FileSystem.get(conf2);
368+
369+
dfs.mkdir(testPath, FsPermission.getDefault());
370+
dfsCluster.rollEditLogAndTail(0);
371+
372+
try {
373+
// Request should be rejected by observer and throw StandbyException
374+
dfs2.listStatus(testPath);
375+
fail("listStatus should have thrown exception");
376+
} catch (RemoteException re) {
377+
IOException e = re.unwrapRemoteException();
378+
assertTrue("should have thrown StandbyException but got "
379+
+ e.getClass().getSimpleName(),
380+
e instanceof StandbyException);
381+
}
382+
}
383+
347384
private void assertSentTo(int nnIdx) throws IOException {
348385
assertTrue("Request was not sent to the expected namenode " + nnIdx,
349386
HATestUtil.isSentToAnyOfNameNodes(dfs, dfsCluster, nnIdx));

0 commit comments

Comments
 (0)