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:
parent
f66532d9dc
commit
66772ce043
|
@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.security.TokenInfo;
|
|||
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 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 {
|
||||
UserGroupInformation loginUser = UserGroupInformation.getLoginUser();
|
||||
|
|
|
@ -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);
|
||||
|
|
Loading…
Reference in New Issue