diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java index d7a415d..896bce0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/server/dns/RegistryDNS.java @@ -659,44 +659,44 @@ protected synchronized void stopExecutor() { * @throws IOException if the tcp processing generates an issue. */ public void nioTCPClient(SocketChannel ch) throws IOException { - try { - // query sizes are small, so the following two lines should work - // in all instances - ByteBuffer buf = ByteBuffer.allocate(1024); - ch.read(buf); - buf.flip(); - int messageLength = getMessgeLength(buf); + while(true) { + try { + // query sizes are small, so the following two lines should work + // in all instances + ByteBuffer buf = ByteBuffer.allocate(1024); + ch.read(buf); + buf.flip(); + int messageLength = getMessgeLength(buf); - byte[] in = new byte[messageLength]; + byte[] in = new byte[messageLength]; - buf.get(in, 0, messageLength); + buf.get(in, 0, messageLength); - Message query; - byte[] response = null; - try { - query = new Message(in); - response = generateReply(query, ch.socket()); - if (response == null) { - return; + Message query; + byte[] response = null; + try { + query = new Message(in); + response = generateReply(query, ch.socket()); + if (response == null) { + return; + } + } catch (IOException e) { + response = formErrorMessage(in); } - } catch (IOException e) { - response = formErrorMessage(in); - } - ByteBuffer out = ByteBuffer.allocate(response.length + 2); - out.putShort(0, (short) (response.length & 0xffff)); - out.put(response); + ByteBuffer out = ByteBuffer.allocate(response.length + 2); + out.putShort(0, (short) (response.length & 0xffff)); + out.put(response); - ch.write(out); - } catch (IOException e) { - throw NetUtils.wrapException(ch.socket().getInetAddress().getHostName(), - ch.socket().getPort(), - ch.socket().getLocalAddress().getHostName(), - ch.socket().getLocalPort(), e); - } finally { - IOUtils.closeStream(ch); + ch.write(out); + } catch (IOException e) { + LOG.error("Failed in TCPClient", NetUtils + .wrapException(ch.socket().getInetAddress().getHostName(), + ch.socket().getPort(), + ch.socket().getLocalAddress().getHostName(), + ch.socket().getLocalPort(), e)); + } } - } /** @@ -727,23 +727,19 @@ private int getMessgeLength(ByteBuffer buf) throws EOFException { public void serveNIOTCP(ServerSocketChannel serverSocketChannel, InetAddress addr, int port) throws Exception { try { - - while (true) { - final SocketChannel socketChannel = serverSocketChannel.accept(); - if (socketChannel != null) { - executor.submit(new Callable() { - @Override - public Boolean call() throws Exception { - nioTCPClient(socketChannel); - return true; - } - }); - - } + final SocketChannel socketChannel = serverSocketChannel.accept(); + if (socketChannel != null) { + executor.submit(new Callable() { + @Override + public Boolean call() throws Exception { + nioTCPClient(socketChannel); + return true; + } + }); } } catch (IOException e) { - throw NetUtils.wrapException(addr.getHostName(), port, - addr.getHostName(), port, e); + throw NetUtils + .wrapException(addr.getHostName(), port, addr.getHostName(), port, e); } }