Index: security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java =================================================================== --- security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java (revision 1311068) +++ security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java (working copy) @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier; import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PoolMap; import org.apache.hadoop.io.*; import org.apache.hadoop.ipc.RemoteException; @@ -154,7 +155,40 @@ } } } + + @Override + protected void sendParam(Call call) { + if (shouldCloseConnection.get()) { + return; + } + // For serializing the data to be written. + final DataOutputBuffer d = new DataOutputBuffer(); + try { + if (LOG.isDebugEnabled()) + LOG.debug(getName() + " sending #" + call.id); + + d.writeInt(0xdeadbeef); // placeholder for data length + d.writeInt(call.id); + call.param.write(d); + byte[] data = d.getData(); + int dataLength = d.getLength(); + // fill in the placeholder + Bytes.putInt(data, 0, dataLength - 4); + //noinspection SynchronizeOnNonFinalField + synchronized (this.out) { // FindBugs IS2_INCONSISTENT_SYNC + out.write(data, 0, dataLength); + out.flush(); + } + } catch(IOException e) { + markClosed(e); + } finally { + //the buffer is just an in-memory buffer, but it is still polite to + // close early + IOUtils.closeStream(d); + } + } + private synchronized boolean shouldAuthenticateOverKrb() throws IOException { UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); UserGroupInformation currentUser = Index: src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (revision 1311068) +++ src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (working copy) @@ -605,7 +605,7 @@ ZooKeeperWatcher zkw, String baseNode) throws KeeperException { List nodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode); - List newNodes = Collections.emptyList(); + List newNodes = new ArrayList(); if (nodes != null) { for (String node : nodes) { String nodePath = ZKUtil.joinZNode(baseNode, node);