Updated Branches: refs/heads/trunk 1a70df0c0 -> 496035585
Fix build Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/49603558 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/49603558 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/49603558 Branch: refs/heads/trunk Commit: 496035585a5a9b115a23ba311459e5e08b494334 Parents: 1a70df0 Author: Sylvain Lebresne <[email protected]> Authored: Fri May 31 15:02:03 2013 +0200 Committer: Sylvain Lebresne <[email protected]> Committed: Fri May 31 15:02:03 2013 +0200 ---------------------------------------------------------------------- .../org/apache/cassandra/transport/Client.java | 4 +- .../org/apache/cassandra/transport/Message.java | 4 +- .../cassandra/transport/ServerConnection.java | 4 +- .../transport/messages/AuthChallenge.java | 67 ++++++++++ .../cassandra/transport/messages/AuthResponse.java | 99 +++++++++++++++ .../cassandra/transport/messages/AuthSuccess.java | 7 +- .../transport/messages/SaslChallenge.java | 62 --------- .../cassandra/transport/messages/SaslResponse.java | 94 -------------- 8 files changed, 178 insertions(+), 163 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/Client.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/Client.java b/src/java/org/apache/cassandra/transport/Client.java index 13e1308..8e2d765 100644 --- a/src/java/org/apache/cassandra/transport/Client.java +++ b/src/java/org/apache/cassandra/transport/Client.java @@ -41,7 +41,7 @@ import org.apache.cassandra.transport.messages.OptionsMessage; import org.apache.cassandra.transport.messages.PrepareMessage; import org.apache.cassandra.transport.messages.QueryMessage; import org.apache.cassandra.transport.messages.RegisterMessage; -import org.apache.cassandra.transport.messages.SaslResponse; +import org.apache.cassandra.transport.messages.AuthResponse; import org.apache.cassandra.transport.messages.StartupMessage; import org.apache.cassandra.utils.Hex; @@ -171,7 +171,7 @@ public class Client extends SimpleClient System.err.println("[ERROR] Authentication requires both 'username' and 'password'"); return null; } - return new SaslResponse(encodeCredentialsForSasl(credentials)); + return new AuthResponse(encodeCredentialsForSasl(credentials)); } else if (msgType.equals("REGISTER")) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/Message.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java index 1a3afa1..54da6a2 100644 --- a/src/java/org/apache/cassandra/transport/Message.java +++ b/src/java/org/apache/cassandra/transport/Message.java @@ -71,8 +71,8 @@ public abstract class Message REGISTER (11, Direction.REQUEST, RegisterMessage.codec), EVENT (12, Direction.RESPONSE, EventMessage.codec), BATCH (13, Direction.REQUEST, BatchMessage.codec), - AUTH_CHALLENGE (14, Direction.RESPONSE, SaslChallenge.codec), - AUTH_RESPONSE (15, Direction.REQUEST, SaslResponse.codec), + AUTH_CHALLENGE (14, Direction.RESPONSE, AuthChallenge.codec), + AUTH_RESPONSE (15, Direction.REQUEST, AuthResponse.codec), AUTH_SUCCESS (16, Direction.RESPONSE, AuthSuccess.codec); public final int opcode; http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/ServerConnection.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/ServerConnection.java b/src/java/org/apache/cassandra/transport/ServerConnection.java index bb43934..ec99440 100644 --- a/src/java/org/apache/cassandra/transport/ServerConnection.java +++ b/src/java/org/apache/cassandra/transport/ServerConnection.java @@ -76,7 +76,7 @@ public class ServerConnection extends Connection break; case AUTHENTICATION: // Support both SASL auth from protocol v2 and the older style Credentials auth from v1 - if (type != Message.Type.SASL_RESPONSE && type != Message.Type.CREDENTIALS) + if (type != Message.Type.AUTH_RESPONSE && type != Message.Type.CREDENTIALS) throw new ProtocolException(String.format("Unexpected message %s, expecting %s", type, version == 1 ? "CREDENTIALS" : "SASL_RESPONSE")); break; case READY: @@ -103,7 +103,7 @@ public class ServerConnection extends Connection break; case AUTHENTICATION: // Support both SASL auth from protocol v2 and the older style Credentials auth from v1 - assert requestType == Message.Type.SASL_RESPONSE || requestType == Message.Type.CREDENTIALS; + assert requestType == Message.Type.AUTH_RESPONSE || requestType == Message.Type.CREDENTIALS; if (responseType == Message.Type.READY || responseType == Message.Type.AUTH_SUCCESS) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/messages/AuthChallenge.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/messages/AuthChallenge.java b/src/java/org/apache/cassandra/transport/messages/AuthChallenge.java new file mode 100644 index 0000000..bc90dc5 --- /dev/null +++ b/src/java/org/apache/cassandra/transport/messages/AuthChallenge.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.transport.messages; + +import org.apache.cassandra.transport.CBUtil; +import org.apache.cassandra.transport.Message; +import org.jboss.netty.buffer.ChannelBuffer; + +import java.nio.ByteBuffer; + +/** + * SASL challenge sent from client to server + */ +public class AuthChallenge extends Message.Response +{ + public static final Message.Codec<AuthChallenge> codec = new Message.Codec<AuthChallenge>() + { + @Override + public AuthChallenge decode(ChannelBuffer body, int version) + { + ByteBuffer b = CBUtil.readValue(body); + byte[] token = new byte[b.remaining()]; + b.get(token); + return new AuthChallenge(token); + } + + @Override + public ChannelBuffer encode(AuthChallenge challenge) + { + return CBUtil.valueToCB(challenge.token); + } + }; + + private byte[] token; + + public AuthChallenge(byte[] token) + { + super(Message.Type.AUTH_CHALLENGE); + this.token = token; + } + + @Override + public ChannelBuffer encode() + { + return codec.encode(this); + } + + public byte[] getToken() + { + return token; + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/messages/AuthResponse.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/messages/AuthResponse.java b/src/java/org/apache/cassandra/transport/messages/AuthResponse.java new file mode 100644 index 0000000..1f8ed9f --- /dev/null +++ b/src/java/org/apache/cassandra/transport/messages/AuthResponse.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.transport.messages; + +import org.apache.cassandra.auth.AuthenticatedUser; +import org.apache.cassandra.auth.ISaslAwareAuthenticator.SaslAuthenticator; +import org.apache.cassandra.exceptions.AuthenticationException; +import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.transport.CBUtil; +import org.apache.cassandra.transport.Message; +import org.apache.cassandra.transport.ProtocolException; +import org.apache.cassandra.transport.ServerConnection; + +import org.jboss.netty.buffer.ChannelBuffer; + +import java.nio.ByteBuffer; + +/** + * A SASL token message sent from client to server. Some SASL + * mechanisms & clients may send an initial token before + * receiving a challenge from the server. + */ +public class AuthResponse extends Message.Request +{ + public static final Message.Codec<AuthResponse> codec = new Message.Codec<AuthResponse>() + { + @Override + public AuthResponse decode(ChannelBuffer body, int version) + { + if (version == 1) + throw new ProtocolException("SASL Authentication is not supported in version 1 of the protocol"); + + ByteBuffer b = CBUtil.readValue(body); + byte[] token = new byte[b.remaining()]; + b.get(token); + return new AuthResponse(token); + } + + @Override + public ChannelBuffer encode(AuthResponse response) + { + return CBUtil.valueToCB(response.token); + } + }; + + private byte[] token; + + public AuthResponse(byte[] token) + { + super(Message.Type.AUTH_RESPONSE); + this.token = token; + } + + @Override + public ChannelBuffer encode() + { + return codec.encode(this); + } + + @Override + public Response execute(QueryState queryState) + { + try + { + SaslAuthenticator authenticator = ((ServerConnection) connection).getAuthenticator(); + byte[] challenge = authenticator.evaluateResponse(token == null ? new byte[0] : token); + if (authenticator.isComplete()) + { + AuthenticatedUser user = authenticator.getAuthenticatedUser(); + queryState.getClientState().login(user); + // authentication is complete, send a ready message to the client + return new AuthSuccess(challenge); + } + else + { + return new AuthChallenge(challenge); + } + } + catch (AuthenticationException e) + { + return ErrorMessage.fromException(e); + } + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/messages/AuthSuccess.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/messages/AuthSuccess.java b/src/java/org/apache/cassandra/transport/messages/AuthSuccess.java index e923e6f..ba520bc 100644 --- a/src/java/org/apache/cassandra/transport/messages/AuthSuccess.java +++ b/src/java/org/apache/cassandra/transport/messages/AuthSuccess.java @@ -21,6 +21,8 @@ import org.apache.cassandra.transport.CBUtil; import org.apache.cassandra.transport.Message; import org.jboss.netty.buffer.ChannelBuffer; +import java.nio.ByteBuffer; + /** * Indicates to the client that authentication has succeeded. * @@ -34,7 +36,10 @@ public class AuthSuccess extends Message.Response @Override public AuthSuccess decode(ChannelBuffer body, int version) { - return new AuthSuccess(CBUtil.readValue(body)); + ByteBuffer b = CBUtil.readValue(body); + byte[] token = new byte[b.remaining()]; + b.get(token); + return new AuthSuccess(token); } @Override http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/messages/SaslChallenge.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/messages/SaslChallenge.java b/src/java/org/apache/cassandra/transport/messages/SaslChallenge.java deleted file mode 100644 index ca76c2a..0000000 --- a/src/java/org/apache/cassandra/transport/messages/SaslChallenge.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.transport.messages; - -import org.apache.cassandra.transport.CBUtil; -import org.apache.cassandra.transport.Message; -import org.jboss.netty.buffer.ChannelBuffer; - -/** - * SASL challenge sent from client to server - */ -public class SaslChallenge extends Message.Response -{ - public static final Message.Codec<SaslChallenge> codec = new Message.Codec<SaslChallenge>() - { - @Override - public SaslChallenge decode(ChannelBuffer body, int version) - { - return new SaslChallenge(CBUtil.readValue(body)); - } - - @Override - public ChannelBuffer encode(SaslChallenge challenge) - { - return CBUtil.valueToCB(challenge.token); - } - }; - - private byte[] token; - - public SaslChallenge(byte[] token) - { - super(Message.Type.AUTH_CHALLENGE); - this.token = token; - } - - @Override - public ChannelBuffer encode() - { - return codec.encode(this); - } - - public byte[] getToken() - { - return token; - } -} http://git-wip-us.apache.org/repos/asf/cassandra/blob/49603558/src/java/org/apache/cassandra/transport/messages/SaslResponse.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/transport/messages/SaslResponse.java b/src/java/org/apache/cassandra/transport/messages/SaslResponse.java deleted file mode 100644 index 9787ac5..0000000 --- a/src/java/org/apache/cassandra/transport/messages/SaslResponse.java +++ /dev/null @@ -1,94 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.cassandra.transport.messages; - -import org.apache.cassandra.auth.AuthenticatedUser; -import org.apache.cassandra.auth.ISaslAwareAuthenticator.SaslAuthenticator; -import org.apache.cassandra.exceptions.AuthenticationException; -import org.apache.cassandra.service.QueryState; -import org.apache.cassandra.transport.CBUtil; -import org.apache.cassandra.transport.Message; -import org.apache.cassandra.transport.ProtocolException; -import org.apache.cassandra.transport.ServerConnection; - -import org.jboss.netty.buffer.ChannelBuffer; - -/** - * A SASL token message sent from client to server. Some SASL - * mechanisms & clients may send an initial token before - * receiving a challenge from the server. - */ -public class SaslResponse extends Message.Request -{ - public static final Message.Codec<SaslResponse> codec = new Message.Codec<SaslResponse>() - { - @Override - public SaslResponse decode(ChannelBuffer body, int version) - { - if (version == 1) - throw new ProtocolException("SASL Authentication is not supported in version 1 of the protocol"); - - return new SaslResponse(CBUtil.readValue(body)); - } - - @Override - public ChannelBuffer encode(SaslResponse response) - { - return CBUtil.valueToCB(response.token); - } - }; - - private byte[] token; - - public SaslResponse(byte[] token) - { - super(Message.Type.AUTH_RESPONSE); - this.token = token; - } - - @Override - public ChannelBuffer encode() - { - return codec.encode(this); - } - - @Override - public Response execute(QueryState queryState) - { - try - { - SaslAuthenticator authenticator = ((ServerConnection) connection).getAuthenticator(); - byte[] challenge = authenticator.evaluateResponse(token == null ? new byte[0] : token); - if (authenticator.isComplete()) - { - AuthenticatedUser user = authenticator.getAuthenticatedUser(); - queryState.getClientState().login(user); - // authentication is complete, send a ready message to the client - return new AuthSuccess(challenge); - } - else - { - return new SaslChallenge(challenge); - } - } - catch (AuthenticationException e) - { - return ErrorMessage.fromException(e); - } - } -}
