HBASE-5727 secure hbase build broke because of 'HBASE-5451 Switch RPC call envelope/headers to PBs'

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1311287 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-04-09 15:44:04 +00:00
parent f66532d9dc
commit 66772ce043
2 changed files with 35 additions and 1 deletions

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.security.TokenInfo;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier; import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSelector; 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.hbase.util.PoolMap;
import org.apache.hadoop.io.*; import org.apache.hadoop.io.*;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
@ -155,6 +156,39 @@ public class SecureClient extends HBaseClient {
} }
} }
@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 { private synchronized boolean shouldAuthenticateOverKrb() throws IOException {
UserGroupInformation loginUser = UserGroupInformation.getLoginUser(); UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
UserGroupInformation currentUser = UserGroupInformation currentUser =

View File

@ -605,7 +605,7 @@ public class ZKUtil {
ZooKeeperWatcher zkw, String baseNode) throws KeeperException { ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
List<String> nodes = List<String> nodes =
ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode); ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
List<NodeAndData> newNodes = Collections.emptyList(); List<NodeAndData> newNodes = new ArrayList<NodeAndData>();
if (nodes != null) { if (nodes != null) {
for (String node : nodes) { for (String node : nodes) {
String nodePath = ZKUtil.joinZNode(baseNode, node); String nodePath = ZKUtil.joinZNode(baseNode, node);