|
@@ -21,13 +21,17 @@ import java.net.URI;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystemTestHelper;
|
|
import org.apache.hadoop.fs.FileSystemTestHelper;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferTestCase;
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferTestCase;
|
|
|
|
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
|
+import org.apache.hadoop.io.DataInputBuffer;
|
|
|
|
+import org.apache.hadoop.security.token.Token;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
@@ -77,6 +81,85 @@ public class TestMultipleNNPortQOP extends SaslDataTransferTestCase {
|
|
clusterConf.setBoolean(DFS_NAMENODE_SEND_QOP_ENABLED, true);
|
|
clusterConf.setBoolean(DFS_NAMENODE_SEND_QOP_ENABLED, true);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Test that when NameNode returns back its established QOP,
|
|
|
|
+ * it only does this for auxiliary port(s), not the primary port.
|
|
|
|
+ *
|
|
|
|
+ * @throws Exception
|
|
|
|
+ */
|
|
|
|
+ @Test
|
|
|
|
+ public void testAuxiliaryPortSendingQOP() throws Exception {
|
|
|
|
+ MiniDFSCluster cluster = null;
|
|
|
|
+
|
|
|
|
+ final String pathPrefix = "/filetestAuxiliaryPortSendingQOP";
|
|
|
|
+ try {
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(clusterConf)
|
|
|
|
+ .numDataNodes(3).build();
|
|
|
|
+
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+ HdfsConfiguration clientConf = new HdfsConfiguration(clusterConf);
|
|
|
|
+ clientConf.unset(
|
|
|
|
+ CommonConfigurationKeys.HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS);
|
|
|
|
+
|
|
|
|
+ URI currentURI = cluster.getURI();
|
|
|
|
+ URI uriAuthPort = new URI(currentURI.getScheme() + "://" +
|
|
|
|
+ currentURI.getHost() + ":12000");
|
|
|
|
+ URI uriIntegrityPort = new URI(currentURI.getScheme() + "://" +
|
|
|
|
+ currentURI.getHost() + ":12100");
|
|
|
|
+ URI uriPrivacyPort = new URI(currentURI.getScheme() +
|
|
|
|
+ "://" + currentURI.getHost() + ":12200");
|
|
|
|
+
|
|
|
|
+ // If connecting to primary port, block token should not include
|
|
|
|
+ // handshake secret
|
|
|
|
+ byte[] secretOnPrimary = getHandshakeSecret(currentURI, clientConf,
|
|
|
|
+ new Path(pathPrefix + "Primary"));
|
|
|
|
+ assertTrue(secretOnPrimary == null || secretOnPrimary.length == 0);
|
|
|
|
+
|
|
|
|
+ // If connecting to auxiliary port, block token should include
|
|
|
|
+ // handshake secret
|
|
|
|
+ clientConf.set(HADOOP_RPC_PROTECTION, "privacy");
|
|
|
|
+ byte[] secretPrivacy = getHandshakeSecret(uriPrivacyPort, clientConf,
|
|
|
|
+ new Path(pathPrefix + "Privacy"));
|
|
|
|
+ assertTrue(secretPrivacy.length > 0);
|
|
|
|
+
|
|
|
|
+ clientConf.set(HADOOP_RPC_PROTECTION, "integrity");
|
|
|
|
+ byte[] secretIntegrity = getHandshakeSecret(uriIntegrityPort, clientConf,
|
|
|
|
+ new Path(pathPrefix + "Integrity"));
|
|
|
|
+ assertTrue(secretIntegrity.length > 0);
|
|
|
|
+
|
|
|
|
+ clientConf.set(HADOOP_RPC_PROTECTION, "authentication");
|
|
|
|
+ byte[] secretAuthentication = getHandshakeSecret(uriAuthPort,
|
|
|
|
+ clientConf, new Path(pathPrefix + "Authentication"));
|
|
|
|
+ assertTrue(secretAuthentication.length > 0);
|
|
|
|
+
|
|
|
|
+ } finally {
|
|
|
|
+ if (cluster != null) {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private byte[] getHandshakeSecret(URI uri, HdfsConfiguration conf,
|
|
|
|
+ Path path) throws Exception {
|
|
|
|
+ FileSystem fs = FileSystem.get(uri, conf);
|
|
|
|
+ FSDataOutputStream out = fs.create(
|
|
|
|
+ path, false, 4096, (short)1, BLOCK_SIZE);
|
|
|
|
+ try {
|
|
|
|
+ out.write(0);
|
|
|
|
+ out.hflush();
|
|
|
|
+ Token<BlockTokenIdentifier> token = DFSTestUtil.getBlockToken(out);
|
|
|
|
+ final byte[] tokenBytes = token.getIdentifier();
|
|
|
|
+ DataInputBuffer dib = new DataInputBuffer();
|
|
|
|
+
|
|
|
|
+ dib.reset(tokenBytes, tokenBytes.length);
|
|
|
|
+ BlockTokenIdentifier blockToken = new BlockTokenIdentifier();
|
|
|
|
+ blockToken.readFields(dib);
|
|
|
|
+ return blockToken.getHandshakeMsg();
|
|
|
|
+ } finally {
|
|
|
|
+ out.close();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Test accessing NameNode from three different ports.
|
|
* Test accessing NameNode from three different ports.
|
|
*
|
|
*
|
|
@@ -168,33 +251,54 @@ public class TestMultipleNNPortQOP extends SaslDataTransferTestCase {
|
|
clientConf.set(HADOOP_RPC_PROTECTION, "privacy");
|
|
clientConf.set(HADOOP_RPC_PROTECTION, "privacy");
|
|
FileSystem fsPrivacy = FileSystem.get(uriPrivacyPort, clientConf);
|
|
FileSystem fsPrivacy = FileSystem.get(uriPrivacyPort, clientConf);
|
|
doTest(fsPrivacy, PATH1);
|
|
doTest(fsPrivacy, PATH1);
|
|
- // add a wait so that data has reached not only first DN,
|
|
|
|
- // but also the rest
|
|
|
|
- Thread.sleep(100);
|
|
|
|
for (int i = 0; i < 2; i++) {
|
|
for (int i = 0; i < 2; i++) {
|
|
DataNode dn = dataNodes.get(i);
|
|
DataNode dn = dataNodes.get(i);
|
|
SaslDataTransferClient saslClient = dn.getSaslClient();
|
|
SaslDataTransferClient saslClient = dn.getSaslClient();
|
|
- assertEquals("auth", saslClient.getTargetQOP());
|
|
|
|
|
|
+ String qop = null;
|
|
|
|
+ // It may take some time for the qop to populate
|
|
|
|
+ // to all DNs, check in a loop.
|
|
|
|
+ for (int trial = 0; trial < 10; trial++) {
|
|
|
|
+ qop = saslClient.getTargetQOP();
|
|
|
|
+ if (qop != null) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ Thread.sleep(100);
|
|
|
|
+ }
|
|
|
|
+ assertEquals("auth", qop);
|
|
}
|
|
}
|
|
|
|
|
|
clientConf.set(HADOOP_RPC_PROTECTION, "integrity");
|
|
clientConf.set(HADOOP_RPC_PROTECTION, "integrity");
|
|
FileSystem fsIntegrity = FileSystem.get(uriIntegrityPort, clientConf);
|
|
FileSystem fsIntegrity = FileSystem.get(uriIntegrityPort, clientConf);
|
|
doTest(fsIntegrity, PATH2);
|
|
doTest(fsIntegrity, PATH2);
|
|
- Thread.sleep(100);
|
|
|
|
for (int i = 0; i < 2; i++) {
|
|
for (int i = 0; i < 2; i++) {
|
|
DataNode dn = dataNodes.get(i);
|
|
DataNode dn = dataNodes.get(i);
|
|
SaslDataTransferClient saslClient = dn.getSaslClient();
|
|
SaslDataTransferClient saslClient = dn.getSaslClient();
|
|
- assertEquals("auth", saslClient.getTargetQOP());
|
|
|
|
|
|
+ String qop = null;
|
|
|
|
+ for (int trial = 0; trial < 10; trial++) {
|
|
|
|
+ qop = saslClient.getTargetQOP();
|
|
|
|
+ if (qop != null) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ Thread.sleep(100);
|
|
|
|
+ }
|
|
|
|
+ assertEquals("auth", qop);
|
|
}
|
|
}
|
|
|
|
|
|
clientConf.set(HADOOP_RPC_PROTECTION, "authentication");
|
|
clientConf.set(HADOOP_RPC_PROTECTION, "authentication");
|
|
FileSystem fsAuth = FileSystem.get(uriAuthPort, clientConf);
|
|
FileSystem fsAuth = FileSystem.get(uriAuthPort, clientConf);
|
|
doTest(fsAuth, PATH3);
|
|
doTest(fsAuth, PATH3);
|
|
- Thread.sleep(100);
|
|
|
|
for (int i = 0; i < 3; i++) {
|
|
for (int i = 0; i < 3; i++) {
|
|
DataNode dn = dataNodes.get(i);
|
|
DataNode dn = dataNodes.get(i);
|
|
SaslDataTransferServer saslServer = dn.getSaslServer();
|
|
SaslDataTransferServer saslServer = dn.getSaslServer();
|
|
- assertEquals("auth", saslServer.getNegotiatedQOP());
|
|
|
|
|
|
+ String qop = null;
|
|
|
|
+ for (int trial = 0; trial < 10; trial++) {
|
|
|
|
+ qop = saslServer.getNegotiatedQOP();
|
|
|
|
+ if (qop != null) {
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ Thread.sleep(100);
|
|
|
|
+ }
|
|
|
|
+ assertEquals("auth", qop);
|
|
}
|
|
}
|
|
} finally {
|
|
} finally {
|
|
if (cluster != null) {
|
|
if (cluster != null) {
|