|
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.AbstractFileSystem;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.fs.permission.FsAction;
|
|
|
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
import org.apache.hadoop.hdfs.*;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
@@ -47,6 +48,7 @@ import org.apache.hadoop.test.Whitebox;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
+import org.slf4j.event.Level;
|
|
|
|
|
|
import java.io.ByteArrayInputStream;
|
|
|
import java.io.DataInputStream;
|
|
@@ -112,6 +114,50 @@ public class TestDelegationTokensWithHA {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test that, when using ObserverReadProxyProvider with DT authentication,
|
|
|
+ * the ORPP gracefully handles when the Standby NN throws a StandbyException.
|
|
|
+ */
|
|
|
+ @Test(timeout = 300000)
|
|
|
+ public void testObserverReadProxyProviderWithDT() throws Exception {
|
|
|
+ // Make the first node standby, so that the ORPP will try it first
|
|
|
+ // instead of just using and succeeding on the active
|
|
|
+ cluster.transitionToStandby(0);
|
|
|
+ cluster.transitionToActive(1);
|
|
|
+
|
|
|
+ HATestUtil.setFailoverConfigurations(cluster, conf,
|
|
|
+ HATestUtil.getLogicalHostname(cluster), 0,
|
|
|
+ ObserverReadProxyProvider.class);
|
|
|
+ conf.setBoolean("fs.hdfs.impl.disable.cache", true);
|
|
|
+
|
|
|
+ dfs = (DistributedFileSystem) FileSystem.get(conf);
|
|
|
+ final UserGroupInformation ugi = UserGroupInformation
|
|
|
+ .createRemoteUser("JobTracker");
|
|
|
+ final Token<DelegationTokenIdentifier> token =
|
|
|
+ getDelegationToken(dfs, ugi.getShortUserName());
|
|
|
+ ugi.addToken(token);
|
|
|
+ // Recreate the DFS, this time authenticating using a DT
|
|
|
+ dfs = ugi.doAs((PrivilegedExceptionAction<DistributedFileSystem>)
|
|
|
+ () -> (DistributedFileSystem) FileSystem.get(conf));
|
|
|
+
|
|
|
+ GenericTestUtils.setLogLevel(ObserverReadProxyProvider.LOG, Level.DEBUG);
|
|
|
+ GenericTestUtils.LogCapturer logCapture = GenericTestUtils.LogCapturer
|
|
|
+ .captureLogs(ObserverReadProxyProvider.LOG);
|
|
|
+ try {
|
|
|
+ dfs.access(new Path("/"), FsAction.READ);
|
|
|
+ assertTrue(logCapture.getOutput()
|
|
|
+ .contains("threw StandbyException when fetching HAState"));
|
|
|
+ HATestUtil.isSentToAnyOfNameNodes(dfs, cluster, 1);
|
|
|
+
|
|
|
+ cluster.shutdownNameNode(0);
|
|
|
+ logCapture.clearOutput();
|
|
|
+ dfs.access(new Path("/"), FsAction.READ);
|
|
|
+ assertTrue(logCapture.getOutput().contains("Assuming Standby state"));
|
|
|
+ } finally {
|
|
|
+ logCapture.stopCapturing();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout = 300000)
|
|
|
public void testDelegationTokenDFSApi() throws Exception {
|
|
|
final Token<DelegationTokenIdentifier> token =
|