virajjasani commented on a change in pull request #1945: URL: https://github.com/apache/hbase/pull/1945#discussion_r445695334
########## File path: hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java ########## @@ -479,29 +502,15 @@ public ZNodePaths getZNodePaths() { return znodePaths; } - /** - * Method called from ZooKeeper for events and connection status. - * <p> - * Valid events are passed along to listeners. Connection status changes - * are dealt with locally. - */ - @Override - public void process(WatchedEvent event) { - LOG.debug(prefix("Received ZooKeeper Event, " + - "type=" + event.getType() + ", " + - "state=" + event.getState() + ", " + - "path=" + event.getPath())); - + void processEvent(WatchedEvent event) { Review comment: keep it private? ########## File path: hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java ########## @@ -530,10 +539,26 @@ public void process(WatchedEvent event) { break; } default: - throw new IllegalStateException("Received event is not valid: " + event.getState()); + LOG.error("Invalid event of type {} received for path {}. Ignoring.", Review comment: Yeah, this looks better. ########## File path: hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java ########## @@ -79,8 +86,22 @@ // listeners to be notified private final List<ZKListener> listeners = new CopyOnWriteArrayList<>(); + // Single threaded executor pool that processes event notifications from Zookeeper. Events are + // processed in the order in which they arrive (pool backed by an unbounded fifo queue). We do + // this to decouple the event processing from Zookeeper's ClientCnxn's EventThread context. + // EventThread internally runs a single while loop to serially process all the events. When events + // are processed by the listeners in the same thread, that blocks the EventThread from processing + // subsequent events. Processing events in a separate thread frees up the event thread to continue + // and further prevents deadlocks if the process method itself makes other zookeeper calls. + // It is ok to do it in a single thread because the Zookeeper ClientCnxn already serializes the + // requests using a single while loop and hence there is no performance degradation. + private final ExecutorService zkEventProcessor = + Executors.newSingleThreadExecutor(Threads.getNamedThreadFactory("zk-event-processor")); Review comment: Why not use `new ThreadFactoryBuilder()` and set name format? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org