|
@@ -20,12 +20,16 @@ package org.apache.hadoop.security;
|
|
|
|
|
|
import java.io.BufferedInputStream;
|
|
|
import java.io.BufferedOutputStream;
|
|
|
+import java.io.ByteArrayInputStream;
|
|
|
import java.io.DataInputStream;
|
|
|
import java.io.DataOutputStream;
|
|
|
+import java.io.FilterInputStream;
|
|
|
+import java.io.FilterOutputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
import java.io.OutputStream;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
@@ -485,38 +489,141 @@ public class SaslRpcClient {
|
|
|
return response;
|
|
|
}
|
|
|
|
|
|
+ private boolean useWrap() {
|
|
|
+ // getNegotiatedProperty throws if client isn't complete
|
|
|
+ String qop = (String) saslClient.getNegotiatedProperty(Sasl.QOP);
|
|
|
+ // SASL wrapping is only used if the connection has a QOP, and
|
|
|
+ // the value is not auth. ex. auth-int & auth-priv
|
|
|
+ return qop != null && !"auth".equalsIgnoreCase(qop);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
- * Get a SASL wrapped InputStream. Can be called only after saslConnect() has
|
|
|
- * been called.
|
|
|
+ * Get SASL wrapped InputStream if SASL QoP requires unwrapping,
|
|
|
+ * otherwise return original stream. Can be called only after
|
|
|
+ * saslConnect() has been called.
|
|
|
*
|
|
|
- * @param in
|
|
|
- * the InputStream to wrap
|
|
|
- * @return a SASL wrapped InputStream
|
|
|
+ * @param in - InputStream used to make the connection
|
|
|
+ * @return InputStream that may be using SASL unwrap
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
public InputStream getInputStream(InputStream in) throws IOException {
|
|
|
- if (!saslClient.isComplete()) {
|
|
|
- throw new IOException("Sasl authentication exchange hasn't completed yet");
|
|
|
+ if (useWrap()) {
|
|
|
+ in = new WrappedInputStream(in);
|
|
|
}
|
|
|
- return new SaslInputStream(in, saslClient);
|
|
|
+ return in;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Get a SASL wrapped OutputStream. Can be called only after saslConnect() has
|
|
|
- * been called.
|
|
|
+ * Get SASL wrapped OutputStream if SASL QoP requires wrapping,
|
|
|
+ * otherwise return original stream. Can be called only after
|
|
|
+ * saslConnect() has been called.
|
|
|
*
|
|
|
- * @param out
|
|
|
- * the OutputStream to wrap
|
|
|
- * @return a SASL wrapped OutputStream
|
|
|
+ * @param in - InputStream used to make the connection
|
|
|
+ * @return InputStream that may be using SASL unwrap
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
public OutputStream getOutputStream(OutputStream out) throws IOException {
|
|
|
- if (!saslClient.isComplete()) {
|
|
|
- throw new IOException("Sasl authentication exchange hasn't completed yet");
|
|
|
+ if (useWrap()) {
|
|
|
+ // the client and server negotiate a maximum buffer size that can be
|
|
|
+ // wrapped
|
|
|
+ String maxBuf = (String)saslClient.getNegotiatedProperty(Sasl.RAW_SEND_SIZE);
|
|
|
+ out = new BufferedOutputStream(new WrappedOutputStream(out),
|
|
|
+ Integer.parseInt(maxBuf));
|
|
|
+ }
|
|
|
+ return out;
|
|
|
+ }
|
|
|
+
|
|
|
+ // ideally this should be folded into the RPC decoding loop but it's
|
|
|
+ // currently split across Client and SaslRpcClient...
|
|
|
+ class WrappedInputStream extends FilterInputStream {
|
|
|
+ private ByteBuffer unwrappedRpcBuffer = ByteBuffer.allocate(0);
|
|
|
+ public WrappedInputStream(InputStream in) throws IOException {
|
|
|
+ super(in);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int read() throws IOException {
|
|
|
+ byte[] b = new byte[1];
|
|
|
+ int n = read(b, 0, 1);
|
|
|
+ return (n != -1) ? b[0] : -1;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int read(byte b[]) throws IOException {
|
|
|
+ return read(b, 0, b.length);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public int read(byte[] buf, int off, int len) throws IOException {
|
|
|
+ synchronized(unwrappedRpcBuffer) {
|
|
|
+ // fill the buffer with the next RPC message
|
|
|
+ if (unwrappedRpcBuffer.remaining() == 0) {
|
|
|
+ readNextRpcPacket();
|
|
|
+ }
|
|
|
+ // satisfy as much of the request as possible
|
|
|
+ int readLen = Math.min(len, unwrappedRpcBuffer.remaining());
|
|
|
+ unwrappedRpcBuffer.get(buf, off, readLen);
|
|
|
+ return readLen;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // all messages must be RPC SASL wrapped, else an exception is thrown
|
|
|
+ private void readNextRpcPacket() throws IOException {
|
|
|
+ LOG.debug("reading next wrapped RPC packet");
|
|
|
+ DataInputStream dis = new DataInputStream(in);
|
|
|
+ int rpcLen = dis.readInt();
|
|
|
+ byte[] rpcBuf = new byte[rpcLen];
|
|
|
+ dis.readFully(rpcBuf);
|
|
|
+
|
|
|
+ // decode the RPC header
|
|
|
+ ByteArrayInputStream bis = new ByteArrayInputStream(rpcBuf);
|
|
|
+ RpcResponseHeaderProto.Builder headerBuilder =
|
|
|
+ RpcResponseHeaderProto.newBuilder();
|
|
|
+ headerBuilder.mergeDelimitedFrom(bis);
|
|
|
+
|
|
|
+ boolean isWrapped = false;
|
|
|
+ // Must be SASL wrapped, verify and decode.
|
|
|
+ if (headerBuilder.getCallId() == AuthProtocol.SASL.callId) {
|
|
|
+ RpcSaslProto.Builder saslMessage = RpcSaslProto.newBuilder();
|
|
|
+ saslMessage.mergeDelimitedFrom(bis);
|
|
|
+ if (saslMessage.getState() == SaslState.WRAP) {
|
|
|
+ isWrapped = true;
|
|
|
+ byte[] token = saslMessage.getToken().toByteArray();
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("unwrapping token of length:" + token.length);
|
|
|
+ }
|
|
|
+ token = saslClient.unwrap(token, 0, token.length);
|
|
|
+ unwrappedRpcBuffer = ByteBuffer.wrap(token);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (!isWrapped) {
|
|
|
+ throw new SaslException("Server sent non-wrapped response");
|
|
|
+ }
|
|
|
}
|
|
|
- return new SaslOutputStream(out, saslClient);
|
|
|
}
|
|
|
|
|
|
+ class WrappedOutputStream extends FilterOutputStream {
|
|
|
+ public WrappedOutputStream(OutputStream out) throws IOException {
|
|
|
+ super(out);
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public void write(byte[] buf, int off, int len) throws IOException {
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("wrapping token of length:" + len);
|
|
|
+ }
|
|
|
+ buf = saslClient.wrap(buf, off, len);
|
|
|
+ RpcSaslProto saslMessage = RpcSaslProto.newBuilder()
|
|
|
+ .setState(SaslState.WRAP)
|
|
|
+ .setToken(ByteString.copyFrom(buf, 0, buf.length))
|
|
|
+ .build();
|
|
|
+ RpcRequestMessageWrapper request =
|
|
|
+ new RpcRequestMessageWrapper(saslHeader, saslMessage);
|
|
|
+ DataOutputStream dob = new DataOutputStream(out);
|
|
|
+ dob.writeInt(request.getLength());
|
|
|
+ request.write(dob);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/** Release resources used by wrapped saslClient */
|
|
|
public void dispose() throws SaslException {
|
|
|
if (saslClient != null) {
|
|
@@ -572,4 +679,4 @@ public class SaslRpcClient {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-}
|
|
|
+}
|