-
Notifications
You must be signed in to change notification settings - Fork 7.3k
ZOOKEEPER-2549 Add exception handling to sendResponse #99
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -667,21 +667,17 @@ public static void closeSock(SocketChannel sock) { | |
| * org.apache.jute.Record, java.lang.String) | ||
| */ | ||
| @Override | ||
| public void sendResponse(ReplyHeader h, Record r, String tag) { | ||
| public void sendResponse(ReplyHeader h, Record r, String tag) throws IOException { | ||
| try { | ||
| ByteArrayOutputStream baos = new ByteArrayOutputStream(); | ||
| // Make space for length | ||
| BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos); | ||
| try { | ||
| baos.write(fourBytes); | ||
| bos.writeRecord(h, "header"); | ||
| if (r != null) { | ||
| bos.writeRecord(r, tag); | ||
| } | ||
| baos.close(); | ||
| } catch (IOException e) { | ||
| LOG.error("Error serializing response"); | ||
| baos.write(fourBytes); | ||
| bos.writeRecord(h, "header"); | ||
| if (r != null) { | ||
| bos.writeRecord(r, tag); | ||
| } | ||
| baos.close(); | ||
| byte b[] = baos.toByteArray(); | ||
| ByteBuffer bb = ByteBuffer.wrap(b); | ||
| bb.putInt(b.length - 4).rewind(); | ||
|
|
@@ -694,7 +690,7 @@ public void sendResponse(ReplyHeader h, Record r, String tag) { | |
| } | ||
| } | ||
| } catch(Exception e) { | ||
| LOG.warn("Unexpected exception. Destruction averted.", e); | ||
| throw new IOException(e); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -716,7 +712,11 @@ public void process(WatchedEvent event) { | |
| // Convert WatchedEvent to a type that can be sent over the wire | ||
| WatcherEvent e = event.getWrapper(); | ||
|
|
||
| sendResponse(h, e, "notification"); | ||
| try { | ||
| sendResponse(h, e, "notification"); | ||
| } catch (IOException ex) { | ||
| LOG.debug("Problem sending to " + getRemoteSocketAddress(), ex); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i have mixed feelings with concatenating strings in a hot path (IOException happening here is a hot path when, for instance, a network blip happens).
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Any suggestion here? Not to getRemoteSocketAddress() at all?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We're using LOG.debug, so it shouldn't be an issue on prod.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: I would use a modern debug format:
|
||
| } | ||
| } | ||
|
|
||
| /* | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -71,7 +71,7 @@ public class NettyServerCnxn extends ServerCnxn { | |
| NettyServerCnxnFactory factory; | ||
| boolean initialized; | ||
|
|
||
| NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) { | ||
| public NettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is not obvious to me why the access specifier of
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think I did this to match NIOServerCnxn constructor. It can be kept package level, as my test class is in the same package namespace. I can change it back, but it will be inconsistent with NIO
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Take my words back. I need it to be public as I use reflection to create it in NettyServerCnxnFactory and if it is not I would have to do couple of more steps during init to set access to public which is unnecessary
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @yufeldman Thanks for explaining, makes sense to me. It is required to run the tests, however I do find another problem with tests: it looks like the Netty tests (testNetty*) never run with current configuration. Proof: remove the public access specifier appertain to It's likely caused by the intervening of the java system properties that controls the Netty vs NIO server selection. One solution is to split the
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @hanm - there is nothing wrong with the tests, they do run fine. They do not use ctor from NettyServerCnxn, but from MockNettyServerCnxn - that has public ctor.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @yufeldman Yeah the tests are fine. What I mentioned that leads to
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @hanm - >>> so all tests fall back to create NIOServerCnxnx instead It is not the case, they don't fall back to NIO - they fail.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
The fallback I was referring to is https://github.com/apache/zookeeper/blob/master/src/java/main/org/apache/zookeeper/server/ServerCnxnFactory.java#L130. It gets hit when public was removed from NIOServerCnxn for Netty* tests. An example call stack (note that a Netty test complaining about NIOServerCnxnFactory):
Thanks for persisting on this, but I don't think this erroneous case need to be investigated further as it would not happen when real test cases were running.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Interesting. For me it actually is: 2016-12-05 11:11:02,523 [myid:] - INFO [Time-limited test:JUnit4ZKTestRunner$LoggedInvokeMethod@98] - TEST METHOD FAILED testNettyRunTimeException
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @hanm - thank you for all the reviews and feedback. Really appreciate it. |
||
| this.channel = channel; | ||
| this.zkServer = zks; | ||
| this.factory = factory; | ||
|
|
@@ -139,9 +139,7 @@ public void process(WatchedEvent event) { | |
| try { | ||
| sendResponse(h, e, "notification"); | ||
| } catch (IOException e1) { | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Problem sending to " + getRemoteSocketAddress(), e1); | ||
| } | ||
| LOG.debug("Problem sending to " + getRemoteSocketAddress(), e1); | ||
| close(); | ||
| } | ||
| } | ||
|
|
@@ -165,31 +163,31 @@ static class ResumeMessageEvent implements MessageEvent { | |
| @Override | ||
| public void sendResponse(ReplyHeader h, Record r, String tag) | ||
| throws IOException { | ||
| if (!channel.isOpen()) { | ||
| return; | ||
| } | ||
| ByteArrayOutputStream baos = new ByteArrayOutputStream(); | ||
| // Make space for length | ||
| BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos); | ||
| try { | ||
| if (!channel.isOpen()) { | ||
| return; | ||
| } | ||
| ByteArrayOutputStream baos = new ByteArrayOutputStream(); | ||
| // Make space for length | ||
| BinaryOutputArchive bos = BinaryOutputArchive.getArchive(baos); | ||
| baos.write(fourBytes); | ||
| bos.writeRecord(h, "header"); | ||
| if (r != null) { | ||
| bos.writeRecord(r, tag); | ||
| } | ||
| baos.close(); | ||
| } catch (IOException e) { | ||
| LOG.error("Error serializing response"); | ||
| } | ||
| byte b[] = baos.toByteArray(); | ||
| ByteBuffer bb = ByteBuffer.wrap(b); | ||
| bb.putInt(b.length - 4).rewind(); | ||
| sendBuffer(bb); | ||
| if (h.getXid() > 0) { | ||
| // zks cannot be null otherwise we would not have gotten here! | ||
| if (!zkServer.shouldThrottle(outstandingCount.decrementAndGet())) { | ||
| enableRecv(); | ||
| byte b[] = baos.toByteArray(); | ||
| ByteBuffer bb = ByteBuffer.wrap(b); | ||
| bb.putInt(b.length - 4).rewind(); | ||
| sendBuffer(bb); | ||
| if (h.getXid() > 0) { | ||
| // zks cannot be null otherwise we would not have gotten here! | ||
| if (!zkServer.shouldThrottle(outstandingCount.decrementAndGet())) { | ||
| enableRecv(); | ||
| } | ||
| } | ||
| } catch (Exception e) { | ||
| throw new IOException(e); | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,65 @@ | ||
| /** | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.zookeeper.server; | ||
|
|
||
| import org.apache.jute.Record; | ||
| import org.apache.zookeeper.proto.ReplyHeader; | ||
| import org.jboss.netty.channel.Channel; | ||
|
|
||
| import java.io.IOException; | ||
| import java.lang.reflect.Field; | ||
| import java.lang.reflect.Modifier; | ||
|
|
||
| /** | ||
| * Helper class to test different scenarios in NettyServerCnxn | ||
| */ | ||
| public class MockNettyServerCnxn extends NettyServerCnxn { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In this file, tab is indented with 2 spaces while the rest of the ZooKeeper files use 4 spaces (I only discovered this 'cause my IDE complained about it). |
||
|
|
||
| public MockNettyServerCnxn(Channel channel, ZooKeeperServer zks, NettyServerCnxnFactory factory) { | ||
| super(channel, zks, factory); | ||
| } | ||
|
|
||
| @Override | ||
| public void sendResponse(ReplyHeader h, Record r, String tag) throws IOException { | ||
| String exceptionType = System.getProperty("exception.type", "IOException"); | ||
| switch(exceptionType) { | ||
| case "IOException": | ||
| throw new IOException("test IOException"); | ||
| case "NoException": | ||
| super.sendResponse(h,r,tag); | ||
| break; | ||
| case "RunTimeException": | ||
| try { | ||
| Field zkServerField = NettyServerCnxn.class.getDeclaredField("zkServer"); | ||
| zkServerField.setAccessible(true); | ||
| Field modifiersField = Field.class.getDeclaredField("modifiers"); | ||
| modifiersField.setAccessible(true); | ||
| modifiersField.setInt(zkServerField, zkServerField.getModifiers() & ~Modifier.FINAL); | ||
| zkServerField.set((NettyServerCnxn)this, null); | ||
| super.sendResponse(h,r,tag); | ||
| } catch (NoSuchFieldException e) { | ||
| e.printStackTrace(); | ||
| } catch (IllegalAccessException e) { | ||
| e.printStackTrace(); | ||
| } | ||
| break; | ||
| default: | ||
| break; | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just a suggestion: wdyt about bubbling up a more custom message with the IOException instead of just encapsulate the Exception? I mean, something like:
throw new IOException("sendMessage exception: blah blah", e);