[
https://issues.apache.org/jira/browse/HADOOP-17975?focusedWorklogId=684666&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-684666
]
ASF GitHub Bot logged work on HADOOP-17975:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 22/Nov/21 12:42
Start Date: 22/Nov/21 12:42
Worklog Time Spent: 10m
Work Description: fapifta opened a new pull request #3579:
URL: https://github.com/apache/hadoop/pull/3579
### Description of PR
When an ipc.Client.Connection is created for the first time, it is added to
the connections pool based on ConnectionId, and then it is there until the
connection is closed.
If someone instantiates two clients (eg. when fs.hdfs.impl.disable.cache is
true), then the same connection is shared between the two DfsClient due to this
pooling.
Connection.setupIOStreams is responsible to set the fallbackToSimpleAuth
AtomicBoolean, but when a connection is accessed the second time,
setupIOStreams returns without setting the AtomicBoolean properly.
This leads to read failures in the second DfsClient, when the client is
running with a secure configuration, but connects to a SIMPLE auth cluster with
ipc.client.fallback-to-simple-auth-allowed set to true unless it is created
after the first one is closed.
The fix is to properly set the fallbackToSimpleAuth AtomicBoolean even
though the socket is already created. In this case the authMethod is not null,
but there is a sanity check for that, so if somehow we get there with an
authMethod equals null, we just do not touch the AtomicBoolean, similarly if
the AtomicBoolean itself is null.
### How was this patch tested?
In a real environment with different iterations of the code that is shown in
the JIRA ticket. Event orders that were tested:
Get two DfsClient with cache enabled - pass (passed before patch)
Get one DfsClient with cache disabled, then get on DfsClient with cache
enabled - pass (failed before patch)
Get two DfsClient with cache disabled - pass (failed before patch)
Next tests were with cache disabled:
Get one DfsClient close it get the second DfsClient - pass (passed before
patch)
Get one DfsClient close it after the second DfsClient is created - pass
(failed before patch)
Get two DfsClient without deliberately closing any of them - pass (failed
before patch)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 684666)
Time Spent: 9h (was: 8h 50m)
> Fallback to simple auth does not work for a secondary DistributedFileSystem
> instance
> ------------------------------------------------------------------------------------
>
> Key: HADOOP-17975
> URL: https://issues.apache.org/jira/browse/HADOOP-17975
> Project: Hadoop Common
> Issue Type: Bug
> Components: ipc
> Reporter: István Fajth
> Assignee: István Fajth
> Priority: Major
> Labels: pull-request-available
> Time Spent: 9h
> Remaining Estimate: 0h
>
> The following code snippet demonstrates what is necessary to cause a failure
> in connection to a non secure cluster with fallback to SIMPLE auth allowed
> from a secure cluster.
> {code:java}
> Configuration conf = new Configuration();
> conf.setBoolean("ipc.client.fallback-to-simple-auth-allowed", true);
> URI fsUri = new URI("hdfs://<nn_uri>");
> conf.setBoolean("fs.hdfs.impl.disable.cache", true);
> FileSystem fs = FileSystem.get(fsUri, conf);
> FSDataInputStream src = fs.open(new Path("/path/to/a/file"));
> FileOutputStream dst = new FileOutputStream(File.createTempFile("foo",
> "bar"));
> IOUtils.copyBytes(src, dst, 1024);
> // The issue happens even if we re-enable cache at this point
> //conf.setBoolean("fs.hdfs.impl.disable.cache", false);
> // The issue does not happen when we close the first FileSystem object
> // before creating the second.
> //fs.close();
> FileSystem fs2 = FileSystem.get(fsUri, conf);
> FSDataInputStream src2 = fs2.open(new Path("/path/to/a/file"));
> FileOutputStream dst2 = new FileOutputStream(File.createTempFile("foo",
> "bar"));
> IOUtils.copyBytes(src2, dst2, 1024);
> {code}
> The problem is that when the DfsClient is created it creates an instance of
> AtomicBoolean, which is propagated down into the IPC layer, where the
> Client.Connection instance in setupIOStreams sets its value. This connection
> object is cached and re-used to multiplex requests against the same DataNode.
> In case of creating a second DfsClient, the AtomicBoolean reference in the
> client is a new AtomicBoolean, but the Client.Connection instance is the
> same, and as it has a socket already open to the DataNode, it returns
> immediatelly from setupIOStreams, leaving the fallbackToSimpleAuth
> AtomicBoolean false as it is created in the DfsClient.
> This AtomicBoolean on the other hand controls how the SaslDataTransferClient
> handles the connection in the above level, and with this value left on the
> default false, the SaslDataTransferClient of the second DfsClient will not
> fall back to SIMPLE authentication but will try to send a SASL handshake when
> connecting to the DataNode.
>
> The access to the FileSystem via the second DfsClient fails with exceptions
> like the following one, then fails the read with a BlockMissingException like
> below:
> {code}
> WARN hdfs.DFSClient: Failed to connect to /<dn_ip>:<dn_port> for file <file>
> for block BP-531773307-<nn_ip>-1634685133591:blk_1073741826_1002, add to
> deadNodes and continue.
> java.io.EOFException: Unexpected EOF while trying to read response from server
> at
> org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed(PBHelperClient.java:552)
> at
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.readSaslMessage(DataTransferSaslUtil.java:215)
> at
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:455)
> at
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getSaslStreams(SaslDataTransferClient.java:393)
> at
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:267)
> at
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:215)
> at
> org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160)
> at
> org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:648)
> at
> org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2980)
> at
> org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:822)
> at
> org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:747)
> at
> org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:380)
> at
> org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:658)
> at
> org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:589)
> at
> org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:771)
> at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:840)
> at java.io.DataInputStream.read(DataInputStream.java:100)
> at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:94)
> at DfsClientTest3.main(DfsClientTest3.java:30)
> {code}
> {code}
> org.apache.hadoop.hdfs.BlockMissingException: Could not obtain block:
> BP-813026743-<nn_ip>-1495248833293:blk_1139767762_66027405 file=/path/to/file
> {code}
>
> The DataNode in the meantime logs the following:
> {code}
> ERROR org.apache.hadoop.hdfs.server.datanode.DataNode:
> <dn_host>:<dn_port>:DataXceiver error processing unknown operation src:
> /<client_ip>:<client_port> dst: /<dn_ip>:<dn_port>
> java.io.IOException: Version Mismatch (Expected: 28, Received: -8531 )
> at
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.readOp(Receiver.java:70)
> at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:222)
> at java.lang.Thread.run(Thread.java:748)
> {code}
> This happens only if the second client is connecting to the same DataNode as
> the first one did, so might seem intermittent in case the clients are reading
> different files, but happens always if the two client reads the same file
> with replication factor 1.
> We ran into this issue during running HBase ExportSnapshot tool to move a
> snapshot from a non-secure to a secure cluster, the issue is loosely related
> to HBASE-12819 and HBASE-20433 and similar problems, I am linking these so
> that HBase team will see how this is relevant for them.
--
This message was sent by Atlassian Jira
(v8.20.1#820001)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]