You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/04/09 17:44:04 UTC

svn commit: r1311287 - in /hbase/trunk: security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Author: stack
Date: Mon Apr  9 15:44:04 2012
New Revision: 1311287

URL: http://svn.apache.org/viewvc?rev=1311287&view=rev
Log:
HBASE-5727 secure hbase build broke because of 'HBASE-5451 Switch RPC call envelope/headers to PBs'

Modified:
    hbase/trunk/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java

Modified: hbase/trunk/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java?rev=1311287&r1=1311286&r2=1311287&view=diff
==============================================================================
--- hbase/trunk/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java (original)
+++ hbase/trunk/security/src/main/java/org/apache/hadoop/hbase/ipc/SecureClient.java Mon Apr  9 15:44:04 2012
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.security.
 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,6 +155,39 @@ public class SecureClient extends HBaseC
         }
       }
     }
+    
+    @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();

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1311287&r1=1311286&r2=1311287&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Mon Apr  9 15:44:04 2012
@@ -605,7 +605,7 @@ public class ZKUtil {
       ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
     List<String> nodes =
       ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
-    List<NodeAndData> newNodes = Collections.emptyList();
+    List<NodeAndData> newNodes = new ArrayList<NodeAndData>();
     if (nodes != null) {
       for (String node : nodes) {
         String nodePath = ZKUtil.joinZNode(baseNode, node);