HDFS-7431. log message for InvalidMagicNumberException may be incorrect. Contributed by Yi Liu.

This commit is contained in:
cnauroth 2014-12-18 15:03:52 -08:00
parent b0b9084433
commit 5df7ecb33a
5 changed files with 54 additions and 9 deletions

View File

@ -618,6 +618,9 @@ Release 2.7.0 - UNRELEASED
HDFS-7494. Checking of closed in DFSInputStream#pread() should be protected
by synchronization (Ted Yu via Colin P. McCabe)
HDFS-7431. log message for InvalidMagicNumberException may be incorrect.
(Yi Liu via cnauroth)
Release 2.6.1 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -31,14 +31,26 @@
public class InvalidMagicNumberException extends IOException {
private static final long serialVersionUID = 1L;
private final boolean handshake4Encryption;
/**
* Creates a new InvalidMagicNumberException.
*
* @param magicNumber expected value
*/
public InvalidMagicNumberException(int magicNumber) {
public InvalidMagicNumberException(final int magicNumber,
final boolean handshake4Encryption) {
super(String.format("Received %x instead of %x from client.",
magicNumber, SASL_TRANSFER_MAGIC_NUMBER));
this.handshake4Encryption = handshake4Encryption;
}
/**
* Return true if it's handshake for encryption
*
* @return boolean true if it's handshake for encryption
*/
public boolean isHandshake4Encryption() {
return handshake4Encryption;
}
}

View File

@ -357,7 +357,8 @@ private IOStreamPair doSaslHandshake(OutputStream underlyingOut,
int magicNumber = in.readInt();
if (magicNumber != SASL_TRANSFER_MAGIC_NUMBER) {
throw new InvalidMagicNumberException(magicNumber);
throw new InvalidMagicNumberException(magicNumber,
dnConf.getEncryptDataTransfer());
}
try {
// step 1

View File

@ -192,10 +192,17 @@ public void run() {
HdfsConstants.SMALL_BUFFER_SIZE);
socketOut = saslStreams.out;
} catch (InvalidMagicNumberException imne) {
LOG.info("Failed to read expected encryption handshake from client " +
"at " + peer.getRemoteAddressString() + ". Perhaps the client " +
"is running an older version of Hadoop which does not support " +
"encryption");
if (imne.isHandshake4Encryption()) {
LOG.info("Failed to read expected encryption handshake from client " +
"at " + peer.getRemoteAddressString() + ". Perhaps the client " +
"is running an older version of Hadoop which does not support " +
"encryption");
} else {
LOG.info("Failed to read expected SASL data transfer protection " +
"handshake from client at " + peer.getRemoteAddressString() +
". Perhaps the client is running an older version of Hadoop " +
"which does not support SASL data transfer protection");
}
return;
}

View File

@ -25,16 +25,22 @@
import java.io.IOException;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
import org.junit.After;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
@ -110,12 +116,28 @@ public void testClientAndServerDoNotHaveCommonQop() throws Exception {
public void testServerSaslNoClientSasl() throws Exception {
HdfsConfiguration clusterConf = createSecureConfig(
"authentication,integrity,privacy");
// Set short retry timeouts so this test runs faster
clusterConf.setInt(DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE, 10);
startCluster(clusterConf);
HdfsConfiguration clientConf = new HdfsConfiguration(clusterConf);
clientConf.set(DFS_DATA_TRANSFER_PROTECTION_KEY, "");
exception.expect(IOException.class);
exception.expectMessage("could only be replicated to 0 nodes");
doTest(clientConf);
LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(
LogFactory.getLog(DataNode.class));
try {
doTest(clientConf);
Assert.fail("Should fail if SASL data transfer protection is not " +
"configured or not supported in client");
} catch (IOException e) {
GenericTestUtils.assertMatches(e.getMessage(),
"could only be replicated to 0 nodes");
} finally {
logs.stopCapturing();
}
GenericTestUtils.assertMatches(logs.getOutput(),
"Failed to read expected SASL data transfer protection " +
"handshake from client at");
}
@Test