[
https://issues.apache.org/jira/browse/HADOOP-16006?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
liyunzhang updated HADOOP-16006:
--------------------------------
Description:
there are two ips, one is 10.8.180.90 other is 10.103.117.121
{code:java}
ifconfig
eth0 Link encap:Ethernet HWaddr 18:FB:7B:FB:2E:CE
inet addr:10.8.180.90 Bcast:10.8.180.255 Mask:255.255.255.0
UP BROADCAST RUNNING MULTICAST MTU:1500 Metric:1
RX packets:3964517028 errors:0 dropped:0 overruns:0 frame:0
TX packets:47963875962 errors:0 dropped:0 overruns:0 carrier:0
collisions:0 txqueuelen:1000
RX bytes:1620406656577 (1.4 TiB) TX bytes:72182813057382 (65.6 TiB)
eth1 Link encap:Ethernet HWaddr 18:FB:7B:FB:2E:CF
inet addr:10.103.117.121 Bcast:10.103.117.255 Mask:255.255.255.0
UP BROADCAST RUNNING MULTICAST MTU:1500 Metric:1
RX packets:25879294429 errors:0 dropped:0 overruns:0 frame:0
TX packets:44433206604 errors:0 dropped:0 overruns:0 carrier:0
collisions:0 txqueuelen:1000
RX bytes:30107497390425 (27.3 TiB)
{code}
hadoop version
{code:java}
#hadoop version
Hadoop 2.7.1.2.4.2.98-1
{code}
although there are two ips, only 1 ip(10.8.180.90) can ping the namenode
artemis-nn.vip.ebay.com(10.8.211.45)
{code:java}
My traceroute [v0.75]
hms031.stratus.lvs.ebay.com (0.0.0.0) Fri Dec 14 15:26:55 2018
Keys: Help Display mode Restart statistics Order of fields quit
Packets Pings
Host Loss% Snt Last Avg Best Wrst StDev
1. 10.8.180.1 0.0% 4 0.1 0.2 0.1 0.3 0.1
2. eth5-lvs2-ed20.net.ebay.com 0.0% 4 0.2 0.2 0.2 0.2 0.0
3. te50-4-lvs2-lc97.net.ebay.com 0.0% 4 0.2 0.2 0.1 0.2 0.0
4. artemis-nn.vip.ebay.com 0.0% 3 0.2 0.2 0.2 0.2 0.0
{code}
I use kerberos. the principal is not specified hostname
{code:java}
klist
Ticket cache: FILE:/tmp/1artemis
Default principal: [email protected]
Valid starting Expires Service principal
12/14/18 15:02:28 12/15/18 01:02:28 krbtgt/[email protected]
renew until 12/21/18 15:02:28
{code}
when i executed command like "hadoop fs -put /tmp/1236 /tmp/", there are some
possiblity that it will throw exception like
{code:java}
Info [Fri Dec 14 00:10:48 GMT-07:00 2018] Stderr:18/12/14 00:10:47 WARN
retry.RetryInvocationHandler: Exception while invoking class
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo
over artemis-nn-2.vip.ebay.com/10.8.211.47:8020. Not retrying because
failovers (15) exceeded maximum allowed (15)
org.apache.hadoop.net.ConnectTimeoutException: Call From
hms031.stratus.lvs.ebay.com/10.8.180.90 to artemis-nn-2.vip.ebay.com:8020
failed on socket timeout exception:
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
waiting for channel to be ready for connect. ch :
java.nio.channels.SocketChannel[connection-pending local=/10.103.117.121:45121
remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020]; For more details see:
http://wiki.apache.org/hadoop/SocketTimeout at
sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at
org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:801) at
org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:751) at
org.apache.hadoop.ipc.Client.call(Client.java:1430) at
org.apache.hadoop.ipc.Client.call(Client.java:1363) at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
at com.sun.proxy.$Proxy9.getFileInfo(Unknown Source) at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:771)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498) at
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:256)
at
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
at com.sun.proxy.$Proxy10.getFileInfo(Unknown Source) at
org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2116) at
org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1305)
at
org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
at
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
at org.apache.hadoop.fs.Globber.getFileStatus(Globber.java:57) at
org.apache.hadoop.fs.Globber.glob(Globber.java:252) at
org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:1655) at
org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:326) at
org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235) at
org.apache.hadoop.fs.shell.Delete$Rm.expandArgument(Delete.java:92) at
org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218) at
org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:201) at
org.apache.hadoop.fs.shell.Command.run(Command.java:165) at
org.apache.hadoop.fs.FsShell.run(FsShell.java:287) at
org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at
org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90) at
org.apache.hadoop.fs.FsShell.main(FsShell.java:340) Caused by:
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
waiting for channel to be ready for connect. ch :
java.nio.channels.SocketChannel[connection-pending local=/10.103.117.121:45121
remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020] at
org.apache.hadoop.net.NetUtils.connect(NetUtils.java:534) at
org.apache.hadoop.net.NetUtils.connect(NetUtils.java:495) at
org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:617) at
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:715) at
org.apache.hadoop.ipc.Client$Connection.access$2900(Client.java:378) at
org.apache.hadoop.ipc.Client.getConnection(Client.java:1492) at
org.apache.hadoop.ipc.Client.call(Client.java:1402) ... 29 more rm: Call From
hms031.stratus.lvs.ebay.com/10.8.180.90 to artemis-nn-2.vip.ebay.com:8020
failed on socket timeout exception:
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
waiting for channel to be ready for connect. ch :
java.nio.channels.SocketChannel[connection-pending local=/10.103.117.121:45121
remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020]; For more details see
{code}
the problem here is as i did not provide hostname in my kerberos principal,
client use one of ip randomly
Client#setupConnection
{code:java}
private synchronized void setupConnection() throws IOException {
short ioFailures = 0;
short timeoutFailures = 0;
while (true) {
try {
this.socket = socketFactory.createSocket();
this.socket.setTcpNoDelay(tcpNoDelay);
this.socket.setKeepAlive(true);
/*
* Bind the socket to the host specified in the principal name of the
* client, to ensure Server matching address of the client connection
* to host name in principal passed.
*/
UserGroupInformation ticket = remoteId.getTicket();
if (ticket != null && ticket.hasKerberosCredentials()) {
KerberosInfo krbInfo =
remoteId.getProtocol().getAnnotation(KerberosInfo.class);
if (krbInfo != null && krbInfo.clientPrincipal() != null) {
String host =
SecurityUtil.getHostFromPrincipal(remoteId.getTicket().getUserName());
// If host name is a valid local address then bind socket to it
InetAddress localAddr = NetUtils.getLocalInetAddress(host);
if (localAddr != null) {
this.socket.bind(new InetSocketAddress(localAddr, 0));
}
}
}
{code}
#Here as i do not provide hostname in keytab , here socket will bind one of my
two ip
randomly to this.socket
{code:java}
NetUtils.connect(this.socket, server, connectionTimeout);
if (rpcTimeout > 0) {
pingInterval = rpcTimeout; // rpcTimeout overwrites pingInterval
}
this.socket.setSoTimeout(pingInterval);
return;
} catch (ConnectTimeoutException toe) {
/* Check for an address change and update the local reference.
* Reset the failure counter if the address was changed
*/
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
}
handleConnectionTimeout(timeoutFailures++,
maxRetriesOnSocketTimeouts, toe);
} catch (IOException ie) {
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
}
handleConnectionFailure(ioFailures++, ie);
}
}
}
{code}
I have added dfs.client.local.interfaces to hdfs-site.xml to force client use
eth0, but seems that there is some possiblity that it will throw above
exception.
{code:java}
<property>
<name>dfs.client.local.interfaces</name>
<value>eth0</value>
</property>
{code}
was:
there are two ips, one is 10.8.180.90 other is 10.103.117.121
{code:java}
ifconfig
eth0 Link encap:Ethernet HWaddr 18:FB:7B:FB:2E:CE
inet addr:10.8.180.90 Bcast:10.8.180.255 Mask:255.255.255.0
UP BROADCAST RUNNING MULTICAST MTU:1500 Metric:1
RX packets:3964517028 errors:0 dropped:0 overruns:0 frame:0
TX packets:47963875962 errors:0 dropped:0 overruns:0 carrier:0
collisions:0 txqueuelen:1000
RX bytes:1620406656577 (1.4 TiB) TX bytes:72182813057382 (65.6 TiB)
eth1 Link encap:Ethernet HWaddr 18:FB:7B:FB:2E:CF
inet addr:10.103.117.121 Bcast:10.103.117.255 Mask:255.255.255.0
UP BROADCAST RUNNING MULTICAST MTU:1500 Metric:1
RX packets:25879294429 errors:0 dropped:0 overruns:0 frame:0
TX packets:44433206604 errors:0 dropped:0 overruns:0 carrier:0
collisions:0 txqueuelen:1000
RX bytes:30107497390425 (27.3 TiB)
{code}
hadoop version
{code}
#hadoop version
Hadoop 2.7.1.2.4.2.98-1
{code}
I use kerberos. the principal is not specified hostname
{code}
klist
Ticket cache: FILE:/tmp/1artemis
Default principal: [email protected]
Valid starting Expires Service principal
12/14/18 15:02:28 12/15/18 01:02:28 krbtgt/[email protected]
renew until 12/21/18 15:02:28
{code}
when i executed command like "hadoop fs -put /tmp/1236 /tmp/", there are some
possiblity that it will throw exception like
{code}
Info [Fri Dec 14 00:10:48 GMT-07:00 2018] Stderr:18/12/14 00:10:47 WARN
retry.RetryInvocationHandler: Exception while invoking class
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo
over artemis-nn-2.vip.ebay.com/10.8.211.47:8020. Not retrying because
failovers (15) exceeded maximum allowed (15)
org.apache.hadoop.net.ConnectTimeoutException: Call From
hms031.stratus.lvs.ebay.com/10.8.180.90 to artemis-nn-2.vip.ebay.com:8020
failed on socket timeout exception:
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
waiting for channel to be ready for connect. ch :
java.nio.channels.SocketChannel[connection-pending local=/10.103.117.121:45121
remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020]; For more details see:
http://wiki.apache.org/hadoop/SocketTimeout at
sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at
org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:801) at
org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:751) at
org.apache.hadoop.ipc.Client.call(Client.java:1430) at
org.apache.hadoop.ipc.Client.call(Client.java:1363) at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
at com.sun.proxy.$Proxy9.getFileInfo(Unknown Source) at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:771)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498) at
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:256)
at
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
at com.sun.proxy.$Proxy10.getFileInfo(Unknown Source) at
org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2116) at
org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1305)
at
org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
at
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
at org.apache.hadoop.fs.Globber.getFileStatus(Globber.java:57) at
org.apache.hadoop.fs.Globber.glob(Globber.java:252) at
org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:1655) at
org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:326) at
org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235) at
org.apache.hadoop.fs.shell.Delete$Rm.expandArgument(Delete.java:92) at
org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218) at
org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:201) at
org.apache.hadoop.fs.shell.Command.run(Command.java:165) at
org.apache.hadoop.fs.FsShell.run(FsShell.java:287) at
org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at
org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90) at
org.apache.hadoop.fs.FsShell.main(FsShell.java:340) Caused by:
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
waiting for channel to be ready for connect. ch :
java.nio.channels.SocketChannel[connection-pending local=/10.103.117.121:45121
remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020] at
org.apache.hadoop.net.NetUtils.connect(NetUtils.java:534) at
org.apache.hadoop.net.NetUtils.connect(NetUtils.java:495) at
org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:617) at
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:715) at
org.apache.hadoop.ipc.Client$Connection.access$2900(Client.java:378) at
org.apache.hadoop.ipc.Client.getConnection(Client.java:1492) at
org.apache.hadoop.ipc.Client.call(Client.java:1402) ... 29 more rm: Call From
hms031.stratus.lvs.ebay.com/10.8.180.90 to artemis-nn-2.vip.ebay.com:8020
failed on socket timeout exception:
org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
waiting for channel to be ready for connect. ch :
java.nio.channels.SocketChannel[connection-pending local=/10.103.117.121:45121
remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020]; For more details see
{code}
the problem here is as i did not provide hostname in my kerberos principal,
client use one of ip randomly
Client#setupConnection
{code}
private synchronized void setupConnection() throws IOException {
short ioFailures = 0;
short timeoutFailures = 0;
while (true) {
try {
this.socket = socketFactory.createSocket();
this.socket.setTcpNoDelay(tcpNoDelay);
this.socket.setKeepAlive(true);
/*
* Bind the socket to the host specified in the principal name of the
* client, to ensure Server matching address of the client connection
* to host name in principal passed.
*/
UserGroupInformation ticket = remoteId.getTicket();
if (ticket != null && ticket.hasKerberosCredentials()) {
KerberosInfo krbInfo =
remoteId.getProtocol().getAnnotation(KerberosInfo.class);
if (krbInfo != null && krbInfo.clientPrincipal() != null) {
String host =
SecurityUtil.getHostFromPrincipal(remoteId.getTicket().getUserName());
// If host name is a valid local address then bind socket to it
InetAddress localAddr = NetUtils.getLocalInetAddress(host);
if (localAddr != null) {
this.socket.bind(new InetSocketAddress(localAddr, 0));
}
}
}
{code}
#Here as i do not provide hostname in keytab , here socket will bind
one of my two ip
randomly to this.socket
{code}
NetUtils.connect(this.socket, server, connectionTimeout);
if (rpcTimeout > 0) {
pingInterval = rpcTimeout; // rpcTimeout overwrites pingInterval
}
this.socket.setSoTimeout(pingInterval);
return;
} catch (ConnectTimeoutException toe) {
/* Check for an address change and update the local reference.
* Reset the failure counter if the address was changed
*/
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
}
handleConnectionTimeout(timeoutFailures++,
maxRetriesOnSocketTimeouts, toe);
} catch (IOException ie) {
if (updateAddress()) {
timeoutFailures = ioFailures = 0;
}
handleConnectionFailure(ioFailures++, ie);
}
}
}
{code}
I have added dfs.client.local.interfaces to hdfs-site.xml to force client use
eth0, but seems that there is some possiblity that Client will bind eth1 to
socket.
{code}
<property>
<name>dfs.client.local.interfaces</name>
<value>eth0</value>
</property>
{code}
> Client use random ip when there are multiple ips even setting
> dfs.client.local.interfaces when hostname is not set in kerberos principle
> ----------------------------------------------------------------------------------------------------------------------------------------
>
> Key: HADOOP-16006
> URL: https://issues.apache.org/jira/browse/HADOOP-16006
> Project: Hadoop Common
> Issue Type: Bug
> Reporter: liyunzhang
> Priority: Major
>
> there are two ips, one is 10.8.180.90 other is 10.103.117.121
> {code:java}
> ifconfig
> eth0 Link encap:Ethernet HWaddr 18:FB:7B:FB:2E:CE
> inet addr:10.8.180.90 Bcast:10.8.180.255 Mask:255.255.255.0
> UP BROADCAST RUNNING MULTICAST MTU:1500 Metric:1
> RX packets:3964517028 errors:0 dropped:0 overruns:0 frame:0
> TX packets:47963875962 errors:0 dropped:0 overruns:0 carrier:0
> collisions:0 txqueuelen:1000
> RX bytes:1620406656577 (1.4 TiB) TX bytes:72182813057382 (65.6 TiB)
> eth1 Link encap:Ethernet HWaddr 18:FB:7B:FB:2E:CF
> inet addr:10.103.117.121 Bcast:10.103.117.255 Mask:255.255.255.0
> UP BROADCAST RUNNING MULTICAST MTU:1500 Metric:1
> RX packets:25879294429 errors:0 dropped:0 overruns:0 frame:0
> TX packets:44433206604 errors:0 dropped:0 overruns:0 carrier:0
> collisions:0 txqueuelen:1000
> RX bytes:30107497390425 (27.3 TiB)
> {code}
> hadoop version
> {code:java}
> #hadoop version
> Hadoop 2.7.1.2.4.2.98-1
> {code}
>
> although there are two ips, only 1 ip(10.8.180.90) can ping the namenode
> artemis-nn.vip.ebay.com(10.8.211.45)
> {code:java}
> My traceroute [v0.75]
> hms031.stratus.lvs.ebay.com (0.0.0.0) Fri Dec 14 15:26:55 2018
> Keys: Help Display mode Restart statistics Order of fields quit
> Packets Pings
> Host Loss% Snt Last Avg Best Wrst StDev
> 1. 10.8.180.1 0.0% 4 0.1 0.2 0.1 0.3 0.1
> 2. eth5-lvs2-ed20.net.ebay.com 0.0% 4 0.2 0.2 0.2 0.2 0.0
> 3. te50-4-lvs2-lc97.net.ebay.com 0.0% 4 0.2 0.2 0.1 0.2 0.0
> 4. artemis-nn.vip.ebay.com 0.0% 3 0.2 0.2 0.2 0.2 0.0
> {code}
> I use kerberos. the principal is not specified hostname
> {code:java}
> klist
> Ticket cache: FILE:/tmp/1artemis
> Default principal: [email protected]
> Valid starting Expires Service principal
> 12/14/18 15:02:28 12/15/18 01:02:28 krbtgt/[email protected]
> renew until 12/21/18 15:02:28
> {code}
> when i executed command like "hadoop fs -put /tmp/1236 /tmp/", there are some
> possiblity that it will throw exception like
> {code:java}
> Info [Fri Dec 14 00:10:48 GMT-07:00 2018] Stderr:18/12/14 00:10:47 WARN
> retry.RetryInvocationHandler: Exception while invoking class
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo
> over artemis-nn-2.vip.ebay.com/10.8.211.47:8020. Not retrying because
> failovers (15) exceeded maximum allowed (15)
> org.apache.hadoop.net.ConnectTimeoutException: Call From
> hms031.stratus.lvs.ebay.com/10.8.180.90 to artemis-nn-2.vip.ebay.com:8020
> failed on socket timeout exception:
> org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
> waiting for channel to be ready for connect. ch :
> java.nio.channels.SocketChannel[connection-pending
> local=/10.103.117.121:45121
> remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020]; For more details see:
> http://wiki.apache.org/hadoop/SocketTimeout at
> sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> at
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at
> org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:801) at
> org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:751) at
> org.apache.hadoop.ipc.Client.call(Client.java:1430) at
> org.apache.hadoop.ipc.Client.call(Client.java:1363) at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
> at com.sun.proxy.$Proxy9.getFileInfo(Unknown Source) at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:771)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498) at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:256)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
> at com.sun.proxy.$Proxy10.getFileInfo(Unknown Source) at
> org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2116) at
> org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1305)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
> at
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
> at org.apache.hadoop.fs.Globber.getFileStatus(Globber.java:57) at
> org.apache.hadoop.fs.Globber.glob(Globber.java:252) at
> org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:1655) at
> org.apache.hadoop.fs.shell.PathData.expandAsGlob(PathData.java:326) at
> org.apache.hadoop.fs.shell.Command.expandArgument(Command.java:235) at
> org.apache.hadoop.fs.shell.Delete$Rm.expandArgument(Delete.java:92) at
> org.apache.hadoop.fs.shell.Command.expandArguments(Command.java:218) at
> org.apache.hadoop.fs.shell.Command.processRawArguments(Command.java:201) at
> org.apache.hadoop.fs.shell.Command.run(Command.java:165) at
> org.apache.hadoop.fs.FsShell.run(FsShell.java:287) at
> org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at
> org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90) at
> org.apache.hadoop.fs.FsShell.main(FsShell.java:340) Caused by:
> org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
> waiting for channel to be ready for connect. ch :
> java.nio.channels.SocketChannel[connection-pending
> local=/10.103.117.121:45121
> remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020] at
> org.apache.hadoop.net.NetUtils.connect(NetUtils.java:534) at
> org.apache.hadoop.net.NetUtils.connect(NetUtils.java:495) at
> org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:617) at
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:715) at
> org.apache.hadoop.ipc.Client$Connection.access$2900(Client.java:378) at
> org.apache.hadoop.ipc.Client.getConnection(Client.java:1492) at
> org.apache.hadoop.ipc.Client.call(Client.java:1402) ... 29 more rm: Call From
> hms031.stratus.lvs.ebay.com/10.8.180.90 to artemis-nn-2.vip.ebay.com:8020
> failed on socket timeout exception:
> org.apache.hadoop.net.ConnectTimeoutException: 20000 millis timeout while
> waiting for channel to be ready for connect. ch :
> java.nio.channels.SocketChannel[connection-pending
> local=/10.103.117.121:45121
> remote=artemis-nn-2.vip.ebay.com/10.8.211.47:8020]; For more details see
> {code}
> the problem here is as i did not provide hostname in my kerberos principal,
> client use one of ip randomly
> Client#setupConnection
> {code:java}
>
> private synchronized void setupConnection() throws IOException {
> short ioFailures = 0;
> short timeoutFailures = 0;
> while (true) {
> try {
> this.socket = socketFactory.createSocket();
> this.socket.setTcpNoDelay(tcpNoDelay);
> this.socket.setKeepAlive(true);
>
> /*
> * Bind the socket to the host specified in the principal name of
> the
> * client, to ensure Server matching address of the client
> connection
> * to host name in principal passed.
> */
> UserGroupInformation ticket = remoteId.getTicket();
> if (ticket != null && ticket.hasKerberosCredentials()) {
> KerberosInfo krbInfo =
> remoteId.getProtocol().getAnnotation(KerberosInfo.class);
> if (krbInfo != null && krbInfo.clientPrincipal() != null) {
> String host =
>
> SecurityUtil.getHostFromPrincipal(remoteId.getTicket().getUserName());
>
> // If host name is a valid local address then bind socket to it
> InetAddress localAddr = NetUtils.getLocalInetAddress(host);
> if (localAddr != null) {
> this.socket.bind(new InetSocketAddress(localAddr, 0));
> }
> }
> }
> {code}
> #Here as i do not provide hostname in keytab , here socket will bind one of
> my two ip
> randomly to this.socket
> {code:java}
> NetUtils.connect(this.socket, server, connectionTimeout);
> if (rpcTimeout > 0) {
> pingInterval = rpcTimeout; // rpcTimeout overwrites pingInterval
> }
> this.socket.setSoTimeout(pingInterval);
> return;
> } catch (ConnectTimeoutException toe) {
> /* Check for an address change and update the local reference.
> * Reset the failure counter if the address was changed
> */
> if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
> }
> handleConnectionTimeout(timeoutFailures++,
> maxRetriesOnSocketTimeouts, toe);
> } catch (IOException ie) {
> if (updateAddress()) {
> timeoutFailures = ioFailures = 0;
> }
> handleConnectionFailure(ioFailures++, ie);
> }
> }
> }
> {code}
> I have added dfs.client.local.interfaces to hdfs-site.xml to force client use
> eth0, but seems that there is some possiblity that it will throw above
> exception.
> {code:java}
> <property>
> <name>dfs.client.local.interfaces</name>
> <value>eth0</value>
> </property>
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]