Description
Drill should support Kerberos based authentication from clients. This means that both the ODBC and JDBC drivers as well as the web/REST interfaces should support inbound Kerberos. For Web this would most likely be SPNEGO while for ODBC and JDBC this will be more generic Kerberos.
Since Hive and much of Hadoop supports Kerberos there is a potential for a lot of reuse of ideas if not implementation.
Note that this is related to but not the same as https://issues.apache.org/jira/browse/DRILL-3584
Attachments
Issue Links
- is related to
-
CALCITE-1159 Support Kerberos-authenticated clients using SPNEGO
- Closed
-
DRILL-5425 Support HTTP Kerberos auth using SPNEGO
- Resolved
- is required by
-
DRILL-3584 Drill Kerberos HDFS Support / Documentation
- Closed
- links to
Activity
Hi Diego , Kerberos SPNEGO support was not included as part of the patch, and so Drill cannot be enabled only with Kerberos if web server is required. PLAIN also needs to be enabled. Accordingly, I opened DRILL-5425.
Hi,
I'm using drill 1.10.0 and I enabled kerberos on drill-override.conf as described in the documentation
drill.exec {
security:
}
however, when starting the drillbit process, the webserver is not starting and is showing this warning:
[main] WARN o.a.drill.exec.server.rest.WebServer - Not starting web server. Currently Drill supports web authentication only through username/password. But PLAIN mechanism is not configured.
2017-04-07 13:41:18,513 [main] INFO o.a.d.exec.server.BootStrapContext - Process user name: 'user' and logged in successfully as '<username>/<FQDN>@<REALM>.COM'
2017-04-07 13:41:18,526 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 2 classes for org.apache.drill.exec.rpc.security.AuthenticatorFactory took 2ms
2017-04-07 13:41:18,527 [main] INFO o.a.d.e.r.s.AuthenticatorProviderImpl - Configured authentication mechanisms: [kerberos]
2017-04-07 13:41:18,877 [main] INFO o.a.d.e.s.s.PersistentStoreRegistry - Using the configured PStoreProvider class: 'org.apache.drill.exec.store.sys.store.provider.ZookeeperPersistentStoreProvider'.
2017-04-07 13:41:19,123 [main] INFO o.a.d.e.r.user.UserConnectionConfig - Configured all user connections to require authentication using: [kerberos]
2017-04-07 13:41:19,129 [main] INFO o.apache.drill.exec.server.Drillbit - Construction completed (1177 ms).
2017-04-07 13:41:19,416 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 16 classes for org.apache.drill.common.logical.data.LogicalOperator took 8ms
2017-04-07 13:41:19,424 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 10 classes for org.apache.drill.common.logical.StoragePluginConfig took 5ms
2017-04-07 13:41:19,427 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 2ms
2017-04-07 13:41:19,576 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 63 classes for org.apache.drill.exec.physical.base.PhysicalOperator took 66ms
2017-04-07 13:41:19,682 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 37 classes for org.apache.drill.exec.physical.impl.BatchCreator took 25ms
2017-04-07 13:41:19,687 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 5 classes for org.apache.drill.exec.physical.impl.RootCreator took 2ms
2017-04-07 13:41:20,266 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 1 classes for org.apache.drill.exec.expr.fn.PluggableFunctionRegistry took 4ms
...
2017-04-07 13:41:21,041 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 114 classes for org.apache.hadoop.hive.ql.udf.generic.GenericUDF took 65ms
2017-04-07 13:41:21,405 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 68 classes for org.apache.hadoop.hive.ql.exec.UDF took 180ms
2017-04-07 13:41:21,426 [main] INFO o.a.d.e.e.f.FunctionImplementationRegistry - Function registry loaded. 433 functions loaded in 1706 ms.
2017-04-07 13:41:21,434 [main] INFO o.a.d.e.e.f.FunctionImplementationRegistry - Created and validated local udf directory [/tmp/drill/drillbits/udf/udf/local]
2017-04-07 13:41:21,437 [main] INFO o.a.drill.exec.compile.CodeCompiler - Plain java code generation preferred: false
2017-04-07 13:41:21,612 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 9 classes for org.apache.drill.exec.store.StoragePlugin took 21ms
2017-04-07 13:41:21,760 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 6 classes for org.apache.drill.exec.store.dfs.FormatPlugin took 21ms
2017-04-07 13:41:21,775 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 0ms
2017-04-07 13:41:21,797 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 6 classes for org.apache.drill.exec.store.dfs.FormatPlugin took 0ms
2017-04-07 13:41:21,799 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 0ms
2017-04-07 13:41:21,799 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 0ms
2017-04-07 13:41:21,799 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 0ms
2017-04-07 13:41:21,820 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 6 classes for org.apache.drill.exec.store.dfs.FormatPlugin took 0ms
2017-04-07 13:41:21,823 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 0ms
2017-04-07 13:41:21,823 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 0ms
2017-04-07 13:41:21,823 [main] INFO o.a.d.c.s.persistence.ScanResult - loading 7 classes for org.apache.drill.common.logical.FormatPluginConfig took 0ms
2017-04-07 13:41:21,986 [main] INFO o.a.d.e.e.f.r.RemoteFunctionRegistry - Preparing three remote udf areas: staging, registry and tmp.
2017-04-07 13:41:22,018 [main] INFO o.a.d.e.e.f.r.RemoteFunctionRegistry - Created remote udf area [/drill/drillbits/udf/registry] on file system file:///
2017-04-07 13:41:22,022 [main] INFO o.a.d.e.e.f.r.RemoteFunctionRegistry - Created remote udf area [/drill/drillbits/udf/staging] on file system file:///
2017-04-07 13:41:22,027 [main] INFO o.a.d.e.e.f.r.RemoteFunctionRegistry - Created remote udf area [/drill/drillbits/udf/tmp] on file system file:///
2017-04-07 13:41:22,046 [main] WARN o.a.drill.exec.server.rest.WebServer - Not starting web server. Currently Drill supports web authentication only through username/password. But PLAIN mechanism is not configured.
2017-04-07 13:41:22,047 [main] INFO o.apache.drill.exec.server.Drillbit - Startup completed (2918 ms).
If kerberos is configured (o.a.d.e.r.s.AuthenticatorProviderImpl - Configured authentication mechanisms: [kerberos]), should the web server ask for PLAIN?
This bug should cover the following:
"Drill should support Kerberos based authentication from clients. This means that both the ODBC and JDBC drivers as well as the web/REST interfaces should support inbound Kerberos. For Web this would most likely be SPNEGO while for ODBC and JDBC this will be more generic Kerberos."
Testing on all area: web/REST, SPNEGO, ODBC and JDBC are on going.
Github user sudheeshkatkam commented on the issue:
https://github.com/apache/drill/pull/578
Pushed the latest changes. The only thing left is the C++ build instructions for Cyrus SASL, which, as noted previously, I will commit as another patch.
Thanks to all the reviewers!
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102606250
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +129,183 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms() {-
- End diff –
-
Will defer to future. For now, I will remove this method, and make auth part of `connect`.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102605579
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationOutcomeListener.java —
@@ -0,0 +1,238 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.proto.UserBitShared.SaslMessage;
+import org.apache.drill.exec.proto.UserBitShared.SaslStatus;
+import org.apache.drill.exec.rpc.BasicClient;
+import org.apache.drill.exec.rpc.ClientConnection;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.EnumMap;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AuthenticationOutcomeListener<T extends EnumLite, R extends ClientConnection,
+ HANDSHAKE_SEND extends MessageLite, HANDSHAKE_RESPONSE extends MessageLite>
+ implements RpcOutcomeListener<SaslMessage> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationOutcomeListener.class);
+
+ private static final ImmutableMap<SaslStatus, SaslChallengeProcessor> CHALLENGE_PROCESSORS;
+ static
+
+ private final BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client;
+ private final R connection;
+ private final T saslRpcType;
+ private final UserGroupInformation ugi;
+ private final RpcOutcomeListener<?> rpcOutcomeListener;
+
+ public AuthenticationOutcomeListener(BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client,
+ R connection, T saslRpcType, UserGroupInformation ugi,
+ RpcOutcomeListener<?> rpcOutcomeListener)
+
+ public void initiate(final String mechanismName) {
+ logger.trace("Initiating SASL exchange.");
+ try {
+ final ByteString responseData;
+ final SaslClient saslClient = connection.getSaslClient();
+ if (saslClient.hasInitialResponse())
else
{ + responseData = ByteString.EMPTY; + } + client.send(new AuthenticationOutcomeListener<>(client, connection, saslRpcType, ugi, rpcOutcomeListener),
+ connection,
+ saslRpcType,
+ SaslMessage.newBuilder()
+ .setMechanism(mechanismName)
+ .setStatus(SaslStatus.SASL_START)
+ .setData(responseData)
+ .build(),
+ SaslMessage.class,
+ true /** the connection will not be backed up at this point */);
+ logger.trace("Initiated SASL exchange.");
+ } catch (final Exception e)
+ }
+
+ @Override
+ public void failed(RpcException ex)
+
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ final SaslChallengeProcessor processor = CHALLENGE_PROCESSORS.get(value.getStatus());
+ if (processor == null)
else {
+ try {
+ final SaslChallengeContext context = new SaslChallengeContext(value, connection.getSaslClient(), ugi);
+
+ final SaslMessage saslResponse = processor.process(context);
+
+ if (saslResponse != null)
else
{ + // success + rpcOutcomeListener.success(null, null); + }+ } catch (final Exception e)
{ + rpcOutcomeListener.failed(RpcException.mapException(e)); + } + }
+ }
+
+ @Override
+ public void interrupted(InterruptedException e)
+
+ private static class SaslChallengeContext {
+
+ final SaslMessage challenge;
+ final SaslClient saslClient;
+ final UserGroupInformation ugi;
+
+ public SaslChallengeContext(SaslMessage challenge, SaslClient saslClient, UserGroupInformation ugi)
+ }
+
+ private interface SaslChallengeProcessor
+
+ private static class SaslInProgressProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+
+ final byte[] responseBytes = evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray());
+
+ final boolean isComplete = context.saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_SUCCESS : SaslStatus.SASL_IN_PROGRESS);
+ return response.build();
+ }
+ }
+
+ private static class SaslSuccessProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
+ } else {
+
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray()); // discard response
+
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
— End diff –
Looks like DRILL-4335 is requesting for TLS. But encryption via SASL is also possible using [wrap and unwrap](https://docs.oracle.com/javase/7/docs/api/javax/security/sasl/SaslServer.html#unwrap(byte[],%20int,%20int)) functions, if "quality of protection" is "auth-conf" (confidentiality). The differences are [nicely noted here](http://stackoverflow.com/questions/11347304/security-authentication-ssl-vs-sasl).
I do not see any open tickets for encryption using SASL, but that should be a sub-task of DRILL-291(https://issues.apache.org/jira/browse/DRILL-291).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102555281
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_pwd_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_pwd_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
— End diff –
+ To return err code.
+ `init` is invoked as part of `validateHandshake` (after construction). Properties should not released, as expected by `validateHandshake`.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102572925
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
— End diff –
Addressed in another comment (for simpler names).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102572556
— Diff: contrib/native/client/cmakeModules/FindSASL.cmake —
@@ -0,0 +1,55 @@
+#
+# 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.
+#
+
+# - Try to find Cyrus SASL
+
+if (MSVC)
— End diff –
I will do this (as another commit that @bitblender worked on).
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102483298
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationOutcomeListener.java —
@@ -0,0 +1,238 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.proto.UserBitShared.SaslMessage;
+import org.apache.drill.exec.proto.UserBitShared.SaslStatus;
+import org.apache.drill.exec.rpc.BasicClient;
+import org.apache.drill.exec.rpc.ClientConnection;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.EnumMap;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AuthenticationOutcomeListener<T extends EnumLite, R extends ClientConnection,
+ HANDSHAKE_SEND extends MessageLite, HANDSHAKE_RESPONSE extends MessageLite>
+ implements RpcOutcomeListener<SaslMessage> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationOutcomeListener.class);
+
+ private static final ImmutableMap<SaslStatus, SaslChallengeProcessor> CHALLENGE_PROCESSORS;
+ static
+
+ private final BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client;
+ private final R connection;
+ private final T saslRpcType;
+ private final UserGroupInformation ugi;
+ private final RpcOutcomeListener<?> rpcOutcomeListener;
+
+ public AuthenticationOutcomeListener(BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client,
+ R connection, T saslRpcType, UserGroupInformation ugi,
+ RpcOutcomeListener<?> rpcOutcomeListener)
+
+ public void initiate(final String mechanismName) {
+ logger.trace("Initiating SASL exchange.");
+ try {
+ final ByteString responseData;
+ final SaslClient saslClient = connection.getSaslClient();
+ if (saslClient.hasInitialResponse())
else
{ + responseData = ByteString.EMPTY; + } + client.send(new AuthenticationOutcomeListener<>(client, connection, saslRpcType, ugi, rpcOutcomeListener),
+ connection,
+ saslRpcType,
+ SaslMessage.newBuilder()
+ .setMechanism(mechanismName)
+ .setStatus(SaslStatus.SASL_START)
+ .setData(responseData)
+ .build(),
+ SaslMessage.class,
+ true /** the connection will not be backed up at this point */);
+ logger.trace("Initiated SASL exchange.");
+ } catch (final Exception e)
+ }
+
+ @Override
+ public void failed(RpcException ex)
+
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ final SaslChallengeProcessor processor = CHALLENGE_PROCESSORS.get(value.getStatus());
+ if (processor == null)
else {
+ try {
+ final SaslChallengeContext context = new SaslChallengeContext(value, connection.getSaslClient(), ugi);
+
+ final SaslMessage saslResponse = processor.process(context);
+
+ if (saslResponse != null)
else
{ + // success + rpcOutcomeListener.success(null, null); + }+ } catch (final Exception e)
{ + rpcOutcomeListener.failed(RpcException.mapException(e)); + } + }
+ }
+
+ @Override
+ public void interrupted(InterruptedException e)
+
+ private static class SaslChallengeContext {
+
+ final SaslMessage challenge;
+ final SaslClient saslClient;
+ final UserGroupInformation ugi;
+
+ public SaslChallengeContext(SaslMessage challenge, SaslClient saslClient, UserGroupInformation ugi)
+ }
+
+ private interface SaslChallengeProcessor
+
+ private static class SaslInProgressProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+
+ final byte[] responseBytes = evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray());
+
+ final boolean isComplete = context.saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_SUCCESS : SaslStatus.SASL_IN_PROGRESS);
+ return response.build();
+ }
+ }
+
+ private static class SaslSuccessProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
+ } else {
+
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray()); // discard response
+
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
— End diff –
is it DRILL-4335 or another issue? I'm curious to see how it would compare with TLS.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102483189
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainFactory.java —
@@ -0,0 +1,166 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.security.plain;
+
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.rpc.security.AuthenticatorFactory;
+import org.apache.drill.exec.rpc.security.FastSaslServerFactory;
+import org.apache.drill.exec.rpc.security.FastSaslClientFactory;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticationException;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+import java.security.Security;
+import java.util.Map;
+
+public class PlainFactory implements AuthenticatorFactory {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainFactory.class);
+
+ public static final String SIMPLE_NAME = PlainServer.MECHANISM_NAME;
+
+ static
+
+ private final UserAuthenticator authenticator;
+
+ public PlainFactory()
+
+ public PlainFactory(final UserAuthenticator authenticator)
+
+ @Override
+ public String getSimpleName()
+
+ @Override
+ public UserGroupInformation createAndLoginUser(Map<String, ?> properties) throws IOException {
+ final Configuration conf = new Configuration();
+ UserGroupInformation.setConfiguration(conf);
+ try
catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login. ", cause);
+ }
+ }
+
+ @Override
+ public SaslServer createSaslServer(final UserGroupInformation ugi, final Map<String, ?> properties)
+ throws SaslException
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi, final Map<String, ?> properties)
+ throws SaslException {
+ final String userName = (String) properties.get(DrillProperties.USER);
+ final String password = (String) properties.get(DrillProperties.PASSWORD);
+
+ return FastSaslClientFactory.getInstance().createSaslClient(new String[]
,
+ null /** authorization ID */, null, null, properties, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
+ if (callback instanceof PasswordCallback)
{ + PasswordCallback.class.cast(callback).setPassword(password.toCharArray()); + continue; + } + throw new UnsupportedCallbackException(callback);
+ }
+ }
+ });
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (authenticator != null)
+ }
+
+ private class PlainServerCallbackHandler implements CallbackHandler {
+
+ @Override
+ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ NameCallback nameCallback = null;
+ PasswordCallback passwordCallback = null;
+ AuthorizeCallback authorizeCallback = null;
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
else if (callback instanceof PasswordCallback)
{ + passwordCallback = PasswordCallback.class.cast(callback); + }else if (callback instanceof AuthorizeCallback)
{ + authorizeCallback = AuthorizeCallback.class.cast(callback); + }else
{ + throw new UnsupportedCallbackException(callback); + } + }
+
+ if (nameCallback == null || passwordCallback == null || authorizeCallback == null)
+
+ try {
+ assert authenticator != null;
+ authenticator.authenticate(nameCallback.getName(), new String(passwordCallback.getPassword()));
— End diff –
I believe this is because `PlainFactory` relies on `FastSasl*Factory` which in turn uses `javax.security.sasl.Sasl` java class, which instantiates factories using the default constructor.
but you could directly instantiate `PlainServer` from `PlainFactory` without using reflection too (if someone where to use `Sasl#createSaslServer` directly, it would not work either because you also need to use the right callback class...)
I don't think the `Closable` is an issue: objects returned by PlainFactory should be considered invalid if the factory is closed (it's already the case).
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102475172
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnectionConfig.java —
@@ -0,0 +1,59 @@
+/**
+ * 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.drill.exec.rpc.control;
+
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.BitConnectionConfig;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
+
+// package private
+class ControlConnectionConfig extends BitConnectionConfig {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlConnectionConfig.class);
+
+ private final ControlMessageHandler handler;
+
+ private DrillbitEndpoint localEndpoint;
+
+ ControlConnectionConfig(BufferAllocator allocator, BootStrapContext context, ControlMessageHandler handler)
+ throws DrillbitStartupException
+
+ @Override
+ public String getName()
+
+ ControlMessageHandler getMessageHandler()
+
+ void setLocalEndpoint(DrillbitEndpoint endpoint) {
— End diff –
Previous approach was to directly inject the endpoint from the server into the registry I believe:
https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ConnectionManagerRegistry.java#L66
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102473811
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +129,183 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms() {-
- End diff –
-
Providing a callback for authentication seems a more robust approach compared to calling another method with a new set of properties...
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102395784
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +129,183 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms() {-
- End diff –
-
Applications (maybe in future) can choose a mechanism. Example:
```
userClient.connect(props);
list = userClient.getSupportedAuthenticationMechanisms();
// pick one from 'list', maybe through callback to user
userClient.authenticate(props2);
```
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102394939
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainFactory.java —
@@ -0,0 +1,166 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.security.plain;
+
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.rpc.security.AuthenticatorFactory;
+import org.apache.drill.exec.rpc.security.FastSaslServerFactory;
+import org.apache.drill.exec.rpc.security.FastSaslClientFactory;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticationException;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+import java.security.Security;
+import java.util.Map;
+
+public class PlainFactory implements AuthenticatorFactory {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainFactory.class);
+
+ public static final String SIMPLE_NAME = PlainServer.MECHANISM_NAME;
+
+ static
+
+ private final UserAuthenticator authenticator;
+
+ public PlainFactory()
+
+ public PlainFactory(final UserAuthenticator authenticator)
+
+ @Override
+ public String getSimpleName()
+
+ @Override
+ public UserGroupInformation createAndLoginUser(Map<String, ?> properties) throws IOException {
+ final Configuration conf = new Configuration();
+ UserGroupInformation.setConfiguration(conf);
+ try
catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login. ", cause);
+ }
+ }
+
+ @Override
+ public SaslServer createSaslServer(final UserGroupInformation ugi, final Map<String, ?> properties)
+ throws SaslException
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi, final Map<String, ?> properties)
+ throws SaslException {
+ final String userName = (String) properties.get(DrillProperties.USER);
+ final String password = (String) properties.get(DrillProperties.PASSWORD);
+
+ return FastSaslClientFactory.getInstance().createSaslClient(new String[]
,
+ null /** authorization ID */, null, null, properties, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
+ if (callback instanceof PasswordCallback)
{ + PasswordCallback.class.cast(callback).setPassword(password.toCharArray()); + continue; + } + throw new UnsupportedCallbackException(callback);
+ }
+ }
+ });
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (authenticator != null)
+ }
+
+ private class PlainServerCallbackHandler implements CallbackHandler {
+
+ @Override
+ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ NameCallback nameCallback = null;
+ PasswordCallback passwordCallback = null;
+ AuthorizeCallback authorizeCallback = null;
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
else if (callback instanceof PasswordCallback)
{ + passwordCallback = PasswordCallback.class.cast(callback); + }else if (callback instanceof AuthorizeCallback)
{ + authorizeCallback = AuthorizeCallback.class.cast(callback); + }else
{ + throw new UnsupportedCallbackException(callback); + } + }
+
+ if (nameCallback == null || passwordCallback == null || authorizeCallback == null)
+
+ try {
+ assert authenticator != null;
+ authenticator.authenticate(nameCallback.getName(), new String(passwordCallback.getPassword()));
— End diff –
I agree the handler is misused, but seems like the check cannot reside elsewhere. As you mentioned, this should be in `PlainServer`, but `UserAuthenticator` relies on `ScanResult` and is `Closeable`, and those changes maybe uglier. I followed other implementations ([quick search](https://www.google.com/search?q=plainsaslserver)). I will add a comment in `PlainFactory` and make `PlainServer` package private to reduce visibility.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102396712
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -105,8 +128,83 @@ protected void finalizeConnection(BitControlHandshake handshake, ControlConnecti
connection.setEndpoint(handshake.getEndpoint());
}
- public ControlConnection getConnection() {
- return this.connection;
+ @Override
+ protected <M extends MessageLite> RpcCommand<M, ControlConnection>
+ getInitialCommand(final RpcCommand<M, ControlConnection> command) {
+ if (config.getAuthMechanismToUse() == null) { + return super.getInitialCommand(command); + }else {
+ return new AuthenticationCommand<>(command);-
- End diff –
-
Thanks for the catch.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102391859
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationOutcomeListener.java —
@@ -0,0 +1,238 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.proto.UserBitShared.SaslMessage;
+import org.apache.drill.exec.proto.UserBitShared.SaslStatus;
+import org.apache.drill.exec.rpc.BasicClient;
+import org.apache.drill.exec.rpc.ClientConnection;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.EnumMap;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AuthenticationOutcomeListener<T extends EnumLite, R extends ClientConnection,
+ HANDSHAKE_SEND extends MessageLite, HANDSHAKE_RESPONSE extends MessageLite>
+ implements RpcOutcomeListener<SaslMessage> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationOutcomeListener.class);
+
+ private static final ImmutableMap<SaslStatus, SaslChallengeProcessor> CHALLENGE_PROCESSORS;
+ static
+
+ private final BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client;
+ private final R connection;
+ private final T saslRpcType;
+ private final UserGroupInformation ugi;
+ private final RpcOutcomeListener<?> rpcOutcomeListener;
+
+ public AuthenticationOutcomeListener(BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client,
+ R connection, T saslRpcType, UserGroupInformation ugi,
+ RpcOutcomeListener<?> rpcOutcomeListener)
+
+ public void initiate(final String mechanismName) {
+ logger.trace("Initiating SASL exchange.");
+ try {
+ final ByteString responseData;
+ final SaslClient saslClient = connection.getSaslClient();
+ if (saslClient.hasInitialResponse())
else
{ + responseData = ByteString.EMPTY; + } + client.send(new AuthenticationOutcomeListener<>(client, connection, saslRpcType, ugi, rpcOutcomeListener),
+ connection,
+ saslRpcType,
+ SaslMessage.newBuilder()
+ .setMechanism(mechanismName)
+ .setStatus(SaslStatus.SASL_START)
+ .setData(responseData)
+ .build(),
+ SaslMessage.class,
+ true /** the connection will not be backed up at this point */);
+ logger.trace("Initiated SASL exchange.");
+ } catch (final Exception e)
+ }
+
+ @Override
+ public void failed(RpcException ex)
+
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ final SaslChallengeProcessor processor = CHALLENGE_PROCESSORS.get(value.getStatus());
+ if (processor == null)
else {
+ try {
+ final SaslChallengeContext context = new SaslChallengeContext(value, connection.getSaslClient(), ugi);
+
+ final SaslMessage saslResponse = processor.process(context);
+
+ if (saslResponse != null)
else
{ + // success + rpcOutcomeListener.success(null, null); + }+ } catch (final Exception e)
{ + rpcOutcomeListener.failed(RpcException.mapException(e)); + } + }
+ }
+
+ @Override
+ public void interrupted(InterruptedException e)
+
+ private static class SaslChallengeContext {
+
+ final SaslMessage challenge;
+ final SaslClient saslClient;
+ final UserGroupInformation ugi;
+
+ public SaslChallengeContext(SaslMessage challenge, SaslClient saslClient, UserGroupInformation ugi)
+ }
+
+ private interface SaslChallengeProcessor
+
+ private static class SaslInProgressProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+
+ final byte[] responseBytes = evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray());
+
+ final boolean isComplete = context.saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_SUCCESS : SaslStatus.SASL_IN_PROGRESS);
+ return response.build();
+ }
+ }
+
+ private static class SaslSuccessProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
+ } else {
+
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray()); // discard response
+
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
— End diff –
After authentication, the saslClient object can be used for encryption and integrity checks for future requests and responses on the connection (@sohami is working on this feature).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102389973
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -89,14 +90,36 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
+ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { + connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender); }
@Override
protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
+ final SaslClient saslClient;
— End diff –
The saslClient is instantiated differently in user->bit comm. and bit->bit comm.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102390094
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,76 @@
+/**
+ * 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.drill.common;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+public final class KerberosUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
+
+ public static final String KERBEROS_SASL_NAME = "GSSAPI";
+
+ public static final String KERBEROS_SIMPLE_NAME = "KERBEROS";
— End diff –
+ For a user, auth=KERBEROS seems easier than auth=GSSAPI, and so I thought why not for admin as well (just like other Hadoop projects).
+ [This link](http://docs.oracle.com/javase/jndi/tutorial/ldap/security/gssapi.html) says GSS-API SASL mechanism was retrofitted to mean only Kerberos v5.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102393799
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/ServerAuthenticationHandler.java —
@@ -0,0 +1,269 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.InvalidProtocolBufferException;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import org.apache.drill.exec.proto.UserBitShared.SaslMessage;
+import org.apache.drill.exec.proto.UserBitShared.SaslStatus;
+import org.apache.drill.exec.rpc.RequestHandler;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.ResponseSender;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.ServerConnection;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.EnumMap;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class ServerAuthenticationHandler<C extends ServerConnection, T extends EnumLite> implements RequestHandler<C> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(ServerAuthenticationHandler.class);
+
+ private static final ImmutableMap<SaslStatus, SaslResponseProcessor> RESPONSE_PROCESSORS;
+
+ static
+
+ private final RequestHandler<C> requestHandler;
+ private final int saslRequestTypeValue;
+ private final T saslResponseType;
+
+ public ServerAuthenticationHandler(final RequestHandler<C> requestHandler, final int saslRequestTypeValue,
+ final T saslResponseType)
+
+ @Override
+ public void handle(C connection, int rpcType, ByteBuf pBody, ByteBuf dBody, ResponseSender sender)
+ throws RpcException {
+ final String remoteAddress = connection.getRemoteAddress().toString();
+
+ // exchange involves server "challenges" and client "responses" (initiated by client)
+ if (saslRequestTypeValue == rpcType) {
+ final SaslMessage saslResponse;
+ try
catch (final InvalidProtocolBufferException e)
{ + handleAuthFailure(connection, remoteAddress, sender, e, saslResponseType); + return; + } +
+ logger.trace("Received SASL message {} from {}", saslResponse.getStatus(), remoteAddress);
+ final SaslResponseProcessor processor = RESPONSE_PROCESSORS.get(saslResponse.getStatus());
+ if (processor == null) {
+ logger.info("Unknown message type from client from {}. Will stop authentication.", remoteAddress);
+ handleAuthFailure(connection, remoteAddress, sender, new SaslException("Received unexpected message"),
+ saslResponseType);
+ return;
+ }
+
+ final SaslResponseContext<C, T> context = new SaslResponseContext<>(saslResponse, connection, remoteAddress,
+ sender, requestHandler, saslResponseType);
+ try
catch (final Exception e)
{ + handleAuthFailure(connection, remoteAddress, sender, e, saslResponseType); + }+ } else
{ + + // this handler only handles messages of SASL_MESSAGE_VALUE type + + // drop connection + connection.close(); + + // the response type for this request type is likely known from UserRpcConfig, + // but the client should not be making any requests before authenticating. + throw new UnsupportedOperationException( + String.format("Request of type %d is not allowed without authentication. " + + "Client on %s must authenticate before making requests. Connection dropped.", + rpcType, remoteAddress)); + } + }
+
+ private static class SaslResponseContext<C extends ServerConnection, T extends EnumLite> {
+
+ final SaslMessage saslResponse;
+ final C connection;
+ final String remoteAddress;
+ final ResponseSender sender;
+ final RequestHandler<C> requestHandler;
+ final T saslResponseType;
+
+ SaslResponseContext(SaslMessage saslResponse, C connection, String remoteAddress, ResponseSender sender,
+ RequestHandler<C> requestHandler, T saslResponseType)
+ }
+
+ private interface SaslResponseProcessor {
+
+ /**
+ * Process response from client, and if there are no exceptions, send response using
+ *
. Otherwise, throw the exception.
+ *
+ * @param context response context
+ */
+ void process(SaslResponseContext context) throws Exception;
+
+ }
+
+ private static class SaslStartProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception
+ }
+
+ private static class SaslInProgressProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception {
+ final SaslMessage.Builder challenge = SaslMessage.newBuilder();
+ final SaslServer saslServer = context.connection.getSaslServer();
+
+ final byte[] challengeBytes = evaluateResponse(saslServer, context.saslResponse.getData().toByteArray());
+
+ if (saslServer.isComplete()) {
+ challenge.setStatus(SaslStatus.SASL_SUCCESS);
+ if (challengeBytes != null)
+
+ handleSuccess(context, challenge, saslServer);
+ } else
+ }
+ }
+
+ // only when client succeeds first
+ private static class SaslSuccessProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception {
+ // at this point, #isComplete must be false; so try once, fail otherwise
+ final SaslServer saslServer = context.connection.getSaslServer();
+
+ evaluateResponse(saslServer, context.saslResponse.getData().toByteArray()); // discard challenge
+
+ if (saslServer.isComplete())
else {
+ logger.info("Failed to authenticate client from {}", context.remoteAddress);
+ throw new SaslException("Client allegedly succeeded authentication, but server did not. Suspicious?");
+ }
+ }
+ }
+
+ private static class SaslFailedProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception {
+ logger.info("Client from {} failed authentication graciously, and does not want to continue.",
+ context.remoteAddress);
+ throw new SaslException("Client graciously failed authentication");
+ }
+ }
+
+ private static byte[] evaluateResponse(final SaslServer saslServer,
+ final byte[] responseBytes) throws SaslException {
+ try {
+ return UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<byte[]>() {
+ @Override
+ public byte[] run() throws Exception
+ });
+ } catch (final UndeclaredThrowableException e)
catch (final IOException | InterruptedException e) {
+ if (e instanceof SaslException)
else
{ + throw new SaslException(String.format("Unexpected failure trying to authenticate using %s", + saslServer.getMechanismName()), e); + } + }
+ }
+
+ @SuppressWarnings("unchecked")
+ private static void handleSuccess(final SaslResponseContext context, final SaslMessage.Builder challenge,
+ final SaslServer saslServer) throws IOException {
+ context.connection.changeHandlerTo(context.requestHandler);
+ context.connection.finalizeSaslSession();
+ context.sender.send(new Response(context.saslResponseType, challenge.build()));
+
+ // setup security layers here..
— End diff –
Same comment as one of the above (saslServer will be used for future messages on the connection).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102396526
— Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/ServerConnection.java —
@@ -0,0 +1,37 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc;
+
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+
+public interface ServerConnection<C extends ServerConnection> extends RemoteConnection {
+
+ // init only once
+ void initSaslServer(String mechanismName) throws IOException;
— End diff –
Same comment as one of the above (saslServer will be used for future messages on the connection).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102396769
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java —
@@ -75,27 +87,118 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ protected void handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { throw new UnsupportedOperationException("DataClient is unidirectional by design."); }
BufferAllocator getAllocator()
{ - return allocator; + return config.getAllocator(); } @Override
protected void validateHandshake(BitServerHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != DataRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
+ final SaslClient saslClient;
+ try
catch (final IOException e)
{ + throw new RpcException(String.format("Failed to initiate authenticate to %s", remoteEndpoint.getAddress()), e); + }+ if (saslClient == null)
{ + throw new RpcException("Unexpected failure. Could not initiate SASL exchange."); + } + connection.setSaslClient(saslClient);
+ } else {
+ if (config.getAuthMechanismToUse() != null)
}
}
@Override
- protected void finalizeConnection(BitServerHandshake handshake, DataClientConnection connection) {
+ protected <M extends MessageLite> RpcCommand<M, DataClientConnection>
+ getInitialCommand(final RpcCommand<M, DataClientConnection> command) {
+ if (config.getAuthMechanismToUse() == null) { + return super.getInitialCommand(command); + }else {
+ return new AuthenticationCommand<>(command);-
- End diff –
-
Yes.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102393352
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractServerConnection.java —
@@ -0,0 +1,121 @@
+/**
+ * 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.drill.exec.rpc;
+
+import io.netty.channel.socket.SocketChannel;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+
+public abstract class AbstractServerConnection<C extends AbstractServerConnection>
— End diff –
Fixed all the warnings, thanks for the catch.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102396422
— Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/ClientConnection.java —
@@ -0,0 +1,30 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc;
+
+import javax.security.sasl.SaslClient;
+
+public interface ClientConnection extends RemoteConnection {
— End diff –
Same comment as one of the above (saslClient will be used for future messages on the connection).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102391399
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnectionConfig.java —
@@ -0,0 +1,59 @@
+/**
+ * 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.drill.exec.rpc.control;
+
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.BitConnectionConfig;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
+
+// package private
+class ControlConnectionConfig extends BitConnectionConfig {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlConnectionConfig.class);
+
+ private final ControlMessageHandler handler;
+
+ private DrillbitEndpoint localEndpoint;
+
+ ControlConnectionConfig(BufferAllocator allocator, BootStrapContext context, ControlMessageHandler handler)
+ throws DrillbitStartupException
+
+ @Override
+ public String getName()
+
+ ControlMessageHandler getMessageHandler()
+
+ void setLocalEndpoint(DrillbitEndpoint endpoint) {
— End diff –
Hmm what was the previous approach?
I only moved this code from another class. Immutability would be nice, but I am not sure what the change entails (maybe a little involved).
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102336062
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,76 @@
+/**
+ * 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.drill.common;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+public final class KerberosUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
+
+ public static final String KERBEROS_SASL_NAME = "GSSAPI";
+
+ public static final String KERBEROS_SIMPLE_NAME = "KERBEROS";
— End diff –
why not using the SASL name? (in theory, gssapi can work with other systems than kerberos)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101123168
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
+
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 1;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
— End diff –
smart pointers can be used too here
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102295865
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java —
@@ -75,27 +87,118 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ protected void handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { throw new UnsupportedOperationException("DataClient is unidirectional by design."); }
BufferAllocator getAllocator()
{ - return allocator; + return config.getAllocator(); } @Override
protected void validateHandshake(BitServerHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != DataRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
+ final SaslClient saslClient;
+ try
catch (final IOException e)
{ + throw new RpcException(String.format("Failed to initiate authenticate to %s", remoteEndpoint.getAddress()), e); + }+ if (saslClient == null)
{ + throw new RpcException("Unexpected failure. Could not initiate SASL exchange."); + } + connection.setSaslClient(saslClient);
+ } else {
+ if (config.getAuthMechanismToUse() != null)
}
}
@Override
- protected void finalizeConnection(BitServerHandshake handshake, DataClientConnection connection) {
+ protected <M extends MessageLite> RpcCommand<M, DataClientConnection>
+ getInitialCommand(final RpcCommand<M, DataClientConnection> command) {
+ if (config.getAuthMechanismToUse() == null) { + return super.getInitialCommand(command); + }else {
+ return new AuthenticationCommand<>(command);-
- End diff –
-
shouldn't we use `super.getInitialCommand(command)` here too?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101631177
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,76 @@
+/**
+ * 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.drill.common;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+public final class KerberosUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
+
+ public static final String KERBEROS_SASL_NAME = "GSSAPI";
+
+ public static final String KERBEROS_SIMPLE_NAME = "KERBEROS";
+
+ public static final String HOSTNAME_PATTERN = "_HOST";
+
+ /**
+ * Returns principal of format primary/instance@REALM.
+ *
+ * @param primary non-null primary component
+ * @param instance non-null instance component
+ * @param realm non-null realm component
+ * @return principal of format primary/instance@REALM
+ */
+ public static String getPrincipalFromParts(final String primary, final String instance, final String realm)
+
+ /**
+ * Expects principal of the format primary/instance@REALM.
+ *
+ * @param principal principal
+ * @return components
+ */
+ public static String[] splitPrincipalIntoParts(final String principal)
+
+ public static String canonicalizedInstanceName(String instanceName, final String canonicalName) {
— End diff –
(style) for consistency, use verb present tense for methods (canonicalizeInstanceName)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101599452
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
— End diff –
Why not keeping the SASL names since they are well-defined? (and also uppercased)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101601121
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_pwd_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_pwd_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
— End diff –
since DrillUserProperties was not copied, it might be possible that this has been released already (from an API point of view). Why not doing it directly into the constructor?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102303605
— Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/ServerConnection.java —
@@ -0,0 +1,37 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc;
+
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+
+public interface ServerConnection<C extends ServerConnection> extends RemoteConnection {
+
+ // init only once
+ void initSaslServer(String mechanismName) throws IOException;
— End diff –
should the server connection exposes SASL protocol directly? isn't ServerAuthenticationHandler a better place to manage the saslServer instance?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102346707
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +129,183 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms() {-
- End diff –
-
what's the use-case? debugging?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102307826
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -358,25 +291,59 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
return handleFailure(respBuilder, HandshakeStatus.RPC_VERSION_MISMATCH, errMsg, null);
}
- if (authenticator != null) {
+ connection.setHandshake(inbound);
+
+ if (!config.isAuthEnabled()) { + connection.finalizeSession(inbound.getCredentials().getUserName()); + respBuilder.setStatus(HandshakeStatus.SUCCESS); + return respBuilder.build(); + }+
+ final boolean clientSupportsSasl = inbound.hasSaslSupport() &&
+ (inbound.getSaslSupport().ordinal() >= SaslSupport.SASL_AUTH.ordinal());
+ if (!clientSupportsSasl) { // for backward compatibility < 1.10
+ final String userName = inbound.getCredentials().getUserName();
+ if (logger.isTraceEnabled()) {
+ logger.trace("User {} on connection {} is likely using an older client.",
+ userName, connection.getRemoteAddress());
+ }
try {
String password = "";
final UserProperties props = inbound.getProperties();
for (int i = 0; i < props.getPropertiesCount(); i++) {
Property prop = props.getProperties; - if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey()))
Unknown macro: { + if (DrillProperties.PASSWORD.equalsIgnoreCase(prop.getKey())) { password = prop.getValue(); break; } }
- authenticator.authenticate(inbound.getCredentials().getUserName(), password);
+ final PlainFactory plainFactory = config.getAuthProvider().getPlainFactory();-
- End diff –
-
instead of keeping a deprecated methods, why not looking for `PLAIN` auth mechanism, and instead of calling authenticate, do an internal sasl session?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101598438
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_pwd_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_pwd_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::string key = m_properties->keyAt;
+ const std::string value = m_properties->valueAt;
+
+ if (USERPROP_SERVICE_HOST == key)
else if (USERPROP_SERVICE_NAME == key)
{ + serviceName = value; + }else if (USERPROP_PASSWORD == key)
{ + const size_t length = value.length(); + m_pwd_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + length); + std::memcpy(m_pwd_secret->data, value.c_str(), length); + m_pwd_secret->len = length; + authMechanismToUse = PLAIN_NAME; + }else if (USERPROP_USERNAME == key)
{ + m_username = value; + }else if (USERPROP_AUTH_MECHANISM == key)
{ + authMechanismToUse = value; + } + }
+ if (authMechanismToUse.empty()) return SASL_NOMECH;
+
+ // check if requested mechanism is supported by server
+ boost::algorithm::to_lower(authMechanismToUse);
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
+ std::string mechanism = mechanisms[i];
+ boost::algorithm::to_lower(mechanism);
— End diff –
maybe we don't need to lower the mechanism here but instead when it is added to the vector?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102279444
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractServerConnection.java —
@@ -0,0 +1,121 @@
+/**
+ * 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.drill.exec.rpc;
+
+import io.netty.channel.socket.SocketChannel;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.slf4j.Logger;
+
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+
+public abstract class AbstractServerConnection<C extends AbstractServerConnection>
— End diff –
Your generic type is not fully defined:
`AbstractServerConnection<C extends AbstractServerConnection<C>>`
(it happens all over the place btw. I suspect this is an IntelliJ thing, as Eclipse is printing out lots of warnings. If you can change your IDE settings, and get all the rawtypes warning introduced by your patches, it would be nice!)
Although the type definition looks correct, I also wonder if the type to be used shouldn't be `AbstractServerConnection<C extends ServerConnection<C>>` instead (as we should rely on the interface instead of the abstract class)?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102347409
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +124,178 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }
- if (props != null) {
- hsBuilder.setProperties(props);
+ /**
+ * Check (after {@link #connect connecting}) if server requires authentication.
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return serverAuthMechanisms != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
{ + return serverAuthMechanisms; + }
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms()+
{@link CheckedFuture#checkedGet results}
+ /**
+ * Authenticate to the server asynchronously. Returns a future that+ * in null if authentication succeeds, or throws a
{@link SaslException}with relevant message if
{@link #connect connection time}
+ * authentication fails.
+ *
+ * This method uses properties provided atand override them with the
+ * given properties, if any.
+ *
+ * @param overrides parameter overrides
+ * @return result of authentication request
+ */
+ public CheckedFuture<Void, SaslException> authenticate(final DrillProperties overrides) {-
- End diff –
-
is there any need (other than for testing?) to not include authentication in the connection process?
From my point of view, it should be included since the user already provided all the needed properties (overrides is NULL in DrillClient), and the user cannot do anything until authenticated anyway...
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102278730
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractClientConnection.java —
@@ -0,0 +1,61 @@
+/**
+ * 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.drill.exec.rpc;
+
+import io.netty.channel.socket.SocketChannel;
+import org.slf4j.Logger;
+
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+
+public abstract class AbstractClientConnection extends AbstractRemoteConnection implements ClientConnection {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractClientConnection.class);
+
+ private SaslClient saslClient;
+
+ public AbstractClientConnection(SocketChannel channel, String name)
+
+ protected abstract Logger getLogger();
+
+ @Override
+ public void setSaslClient(final SaslClient saslClient) {
+ assert this.saslClient == null;
+ this.saslClient = saslClient;
— End diff –
According to the interface comment, it's supposed to be only set one, but this is not checked.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101116414
— Diff: common/src/main/java/org/apache/drill/common/map/CaseInsensitiveMap.java —
@@ -55,6 +55,18 @@
}
/**
+ * Returns a new instance of
, with key case-insensitivity, of expected size.
— End diff –
okay
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101598598
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_pwd_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_pwd_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::string key = m_properties->keyAt;
+ const std::string value = m_properties->valueAt;
+
+ if (USERPROP_SERVICE_HOST == key)
else if (USERPROP_SERVICE_NAME == key)
{ + serviceName = value; + }else if (USERPROP_PASSWORD == key)
{ + const size_t length = value.length(); + m_pwd_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + length); + std::memcpy(m_pwd_secret->data, value.c_str(), length); + m_pwd_secret->len = length; + authMechanismToUse = PLAIN_NAME; + }else if (USERPROP_USERNAME == key)
{ + m_username = value; + }else if (USERPROP_AUTH_MECHANISM == key)
{ + authMechanismToUse = value; + } + }
+ if (authMechanismToUse.empty()) return SASL_NOMECH;
+
+ // check if requested mechanism is supported by server
+ boost::algorithm::to_lower(authMechanismToUse);
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
— End diff –
assuming that the mechanisms are already lower case, we can use std::vector::find instead...
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101602793
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_pwd_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_pwd_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::string key = m_properties->keyAt;
+ const std::string value = m_properties->valueAt;
+
+ if (USERPROP_SERVICE_HOST == key)
else if (USERPROP_SERVICE_NAME == key)
{ + serviceName = value; + }else if (USERPROP_PASSWORD == key)
{ + const size_t length = value.length(); + m_pwd_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + length); + std::memcpy(m_pwd_secret->data, value.c_str(), length); + m_pwd_secret->len = length; + authMechanismToUse = PLAIN_NAME; + }else if (USERPROP_USERNAME == key)
{ + m_username = value; + }else if (USERPROP_AUTH_MECHANISM == key)
{ + authMechanismToUse = value; + } + }
+ if (authMechanismToUse.empty()) return SASL_NOMECH;
+
+ // check if requested mechanism is supported by server
+ boost::algorithm::to_lower(authMechanismToUse);
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
+ std::string mechanism = mechanisms[i];
+ boost::algorithm::to_lower(mechanism);
+ if (authMechanismToUse == mechanism)
+ }
+ if (!isSupportedByServer) return SASL_NOMECH;
+
+ // find the SASL name
+ const std::map<std::string, std::string>::const_iterator it =
+ SaslAuthenticatorImpl::MECHANISM_MAPPING.find(authMechanismToUse);
+ if (it == SaslAuthenticatorImpl::MECHANISM_MAPPING.end()) return SASL_NOMECH;
+ const std::string saslMechanismToUse = it->second;
+
+ // setup callbacks and parameters
+ const sasl_callback_t callbacks[] = {
+
,
+
,
+
,
+
+ };
+ if (serviceName.empty()) serviceName = DEFAULT_SERVICE_NAME;
+
+ // create SASL client
+ int saslResult = sasl_client_new(serviceName.c_str(), serviceHost.c_str(), NULL /** iplocalport */,
+ NULL /** ipremoteport /, callbacks, 0 /* sec flags */, &m_pConnection);
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SaslAuthenticatorImpl::init: sasl_client_new code: "
+ << saslResult << std::endl
+ if (saslResult != SASL_OK) return saslResult;
+
+ // initiate; for now, pass in only one mechanism
— End diff –
Starting from this point, I guess this is where the init really starts (aned everything above should probably be done at construction time)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102294263
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlConnectionConfig.java —
@@ -0,0 +1,59 @@
+/**
+ * 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.drill.exec.rpc.control;
+
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.BitConnectionConfig;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
+
+// package private
+class ControlConnectionConfig extends BitConnectionConfig {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControlConnectionConfig.class);
+
+ private final ControlMessageHandler handler;
+
+ private DrillbitEndpoint localEndpoint;
+
+ ControlConnectionConfig(BufferAllocator allocator, BootStrapContext context, ControlMessageHandler handler)
+ throws DrillbitStartupException
+
+ @Override
+ public String getName()
+
+ ControlMessageHandler getMessageHandler()
+
+ void setLocalEndpoint(DrillbitEndpoint endpoint) {
— End diff –
Can we avoid modifying the original config to return the endpoint created by the server? it seems to me that the previous approach sounded safer as config is usually considered immutable.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102299705
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -89,14 +90,36 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
+ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { + connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender); }
@Override
protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
+ final SaslClient saslClient;
— End diff –
shouldn't the SASL client instantiation happens in `AuthenticationCommand` (which in turns provide it to `AuthenticationOutcomeRpcListener`). It looks like it would be more contained.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102330939
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainFactory.java —
@@ -0,0 +1,166 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.security.plain;
+
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.rpc.security.AuthenticatorFactory;
+import org.apache.drill.exec.rpc.security.FastSaslServerFactory;
+import org.apache.drill.exec.rpc.security.FastSaslClientFactory;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticationException;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+import java.security.Security;
+import java.util.Map;
+
+public class PlainFactory implements AuthenticatorFactory {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainFactory.class);
+
+ public static final String SIMPLE_NAME = PlainServer.MECHANISM_NAME;
+
+ static
+
+ private final UserAuthenticator authenticator;
+
+ public PlainFactory()
+
+ public PlainFactory(final UserAuthenticator authenticator)
+
+ @Override
+ public String getSimpleName()
+
+ @Override
+ public UserGroupInformation createAndLoginUser(Map<String, ?> properties) throws IOException {
+ final Configuration conf = new Configuration();
+ UserGroupInformation.setConfiguration(conf);
+ try
catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login. ", cause);
+ }
+ }
+
+ @Override
+ public SaslServer createSaslServer(final UserGroupInformation ugi, final Map<String, ?> properties)
+ throws SaslException
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi, final Map<String, ?> properties)
+ throws SaslException {
+ final String userName = (String) properties.get(DrillProperties.USER);
+ final String password = (String) properties.get(DrillProperties.PASSWORD);
+
+ return FastSaslClientFactory.getInstance().createSaslClient(new String[]
,
+ null /** authorization ID */, null, null, properties, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
+ if (callback instanceof PasswordCallback)
{ + PasswordCallback.class.cast(callback).setPassword(password.toCharArray()); + continue; + } + throw new UnsupportedCallbackException(callback);
+ }
+ }
+ });
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (authenticator != null)
+ }
+
+ private class PlainServerCallbackHandler implements CallbackHandler {
+
+ @Override
+ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ NameCallback nameCallback = null;
+ PasswordCallback passwordCallback = null;
+ AuthorizeCallback authorizeCallback = null;
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
else if (callback instanceof PasswordCallback)
{ + passwordCallback = PasswordCallback.class.cast(callback); + }else if (callback instanceof AuthorizeCallback)
{ + authorizeCallback = AuthorizeCallback.class.cast(callback); + }else
{ + throw new UnsupportedCallbackException(callback); + } + }
+
+ if (nameCallback == null || passwordCallback == null || authorizeCallback == null)
+
+ try {
+ assert authenticator != null;
+ authenticator.authenticate(nameCallback.getName(), new String(passwordCallback.getPassword()));
— End diff –
I don't believe callback is the right place to handle authentication. Unless I'm mistaken about the security API, callbacks are designed to gather information, but the actual authentication should happen inside the server `evaluateResponse` method.
In theory, someone could use the wrong `CallbackHandler` instance when calling `createSaslServer`, which would result in no validation...
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101605131
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -435,9 +427,14 @@ public synchronized boolean reconnect() {
}
private void connect(DrillbitEndpoint endpoint) throws RpcException {
- final FutureHandler f = new FutureHandler();
- client.connect(f, endpoint, props, getUserCredentials());
- f.checkedGet();
+ client.connect(endpoint, properties, getUserCredentials()).checkedGet();
+ if (client.serverRequiresAuthentication()) {
+ try {
+ client.authenticate(null).checkedGet();-
- End diff –
-
shouldn't this be done directly by the rpc client?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102348994
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -358,25 +291,59 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
return handleFailure(respBuilder, HandshakeStatus.RPC_VERSION_MISMATCH, errMsg, null);
}
- if (authenticator != null) {
+ connection.setHandshake(inbound);
+
+ if (!config.isAuthEnabled()) { + connection.finalizeSession(inbound.getCredentials().getUserName()); + respBuilder.setStatus(HandshakeStatus.SUCCESS); + return respBuilder.build(); + }+
+ final boolean clientSupportsSasl = inbound.hasSaslSupport() &&
+ (inbound.getSaslSupport().ordinal() >= SaslSupport.SASL_AUTH.ordinal());
+ if (!clientSupportsSasl) { // for backward compatibility < 1.10
+ final String userName = inbound.getCredentials().getUserName();
+ if (logger.isTraceEnabled()) {
+ logger.trace("User {} on connection {} is likely using an older client.",
+ userName, connection.getRemoteAddress());
+ }
try {
String password = "";
final UserProperties props = inbound.getProperties();
for (int i = 0; i < props.getPropertiesCount(); i++) {
Property prop = props.getProperties; - if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey()))
Unknown macro: { + if (DrillProperties.PASSWORD.equalsIgnoreCase(prop.getKey())) { password = prop.getValue(); break; } }
- authenticator.authenticate(inbound.getCredentials().getUserName(), password);
+ final PlainFactory plainFactory = config.getAuthProvider().getPlainFactory();
+ if (plainFactory == null) { + throw new UserAuthenticationException("The server no longer supports username/password" + + " based authentication. Please talk to your system administrator."); + }+ plainFactory.getAuthenticator()
{ return handleFailure(respBuilder, HandshakeStatus.AUTH_FAILED, ex.getMessage(), ex); }
+ .authenticate(userName, password);
+ connection.changeHandlerTo(config.getMessageHandler());
+ connection.finalizeSession(userName);
+ respBuilder.setStatus(HandshakeStatus.SUCCESS);
+ if (logger.isTraceEnabled()) {
+ logger.trace("Authenticated {} successfully using PLAIN from {}", userName,
+ connection.getRemoteAddress());
+ }
+ return respBuilder.build();
} catch (UserAuthenticationException ex)}
- connection.setUser(inbound);
- return respBuilder.setStatus(HandshakeStatus.SUCCESS).build();
+ // mention server's authentication capabilities
+ respBuilder.addAllAuthenticationMechanisms(config.getAuthProvider().getAllFactoryNames());
+
+ // for now, this means PLAIN credentials will be sent over twice-
- End diff –
-
just a thought if this is something you're worried about. What prevents the server to detect the plain credentials from the user, and try to perform the authentication directly (if plain is supported)?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101600545
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_pwd_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err) {
+ std::stringstream errMsg;
+ errMsg << "Failed to load authentication libraries. code: " << err;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << errMsg << std::endl
— End diff –
or inline errMsg directly into the log statement?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101600048
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.hpp —
@@ -0,0 +1,65 @@
+/*
+ * 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.
+ */
+
+#ifndef DRILLCLIENT_SASLAUTHENTICATORIMPL_HPP
+#define DRILLCLIENT_SASLAUTHENTICATORIMPL_HPP
+
+#include <string>
+#include <map>
+#include <vector>
+#include "drill/drillClient.hpp"
+#include "UserBitShared.pb.h"
+
+#include "sasl/sasl.h"
+#include "sasl/saslplug.h"
+
+namespace Drill {
+
+class SaslAuthenticatorImpl {
+
+public:
+
+ static const std::map<std::string, std::string> MECHANISM_MAPPING;
— End diff –
not sure why this is public, isn't this something internal?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102344235
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationOutcomeListener.java —
@@ -0,0 +1,238 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.proto.UserBitShared.SaslMessage;
+import org.apache.drill.exec.proto.UserBitShared.SaslStatus;
+import org.apache.drill.exec.rpc.BasicClient;
+import org.apache.drill.exec.rpc.ClientConnection;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.EnumMap;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AuthenticationOutcomeListener<T extends EnumLite, R extends ClientConnection,
+ HANDSHAKE_SEND extends MessageLite, HANDSHAKE_RESPONSE extends MessageLite>
+ implements RpcOutcomeListener<SaslMessage> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationOutcomeListener.class);
+
+ private static final ImmutableMap<SaslStatus, SaslChallengeProcessor> CHALLENGE_PROCESSORS;
+ static
+
+ private final BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client;
+ private final R connection;
+ private final T saslRpcType;
+ private final UserGroupInformation ugi;
+ private final RpcOutcomeListener<?> rpcOutcomeListener;
+
+ public AuthenticationOutcomeListener(BasicClient<T, R, HANDSHAKE_SEND, HANDSHAKE_RESPONSE> client,
+ R connection, T saslRpcType, UserGroupInformation ugi,
+ RpcOutcomeListener<?> rpcOutcomeListener)
+
+ public void initiate(final String mechanismName) {
+ logger.trace("Initiating SASL exchange.");
+ try {
+ final ByteString responseData;
+ final SaslClient saslClient = connection.getSaslClient();
+ if (saslClient.hasInitialResponse())
else
{ + responseData = ByteString.EMPTY; + } + client.send(new AuthenticationOutcomeListener<>(client, connection, saslRpcType, ugi, rpcOutcomeListener),
+ connection,
+ saslRpcType,
+ SaslMessage.newBuilder()
+ .setMechanism(mechanismName)
+ .setStatus(SaslStatus.SASL_START)
+ .setData(responseData)
+ .build(),
+ SaslMessage.class,
+ true /** the connection will not be backed up at this point */);
+ logger.trace("Initiated SASL exchange.");
+ } catch (final Exception e)
+ }
+
+ @Override
+ public void failed(RpcException ex)
+
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ final SaslChallengeProcessor processor = CHALLENGE_PROCESSORS.get(value.getStatus());
+ if (processor == null)
else {
+ try {
+ final SaslChallengeContext context = new SaslChallengeContext(value, connection.getSaslClient(), ugi);
+
+ final SaslMessage saslResponse = processor.process(context);
+
+ if (saslResponse != null)
else
{ + // success + rpcOutcomeListener.success(null, null); + }+ } catch (final Exception e)
{ + rpcOutcomeListener.failed(RpcException.mapException(e)); + } + }
+ }
+
+ @Override
+ public void interrupted(InterruptedException e)
+
+ private static class SaslChallengeContext {
+
+ final SaslMessage challenge;
+ final SaslClient saslClient;
+ final UserGroupInformation ugi;
+
+ public SaslChallengeContext(SaslMessage challenge, SaslClient saslClient, UserGroupInformation ugi)
+ }
+
+ private interface SaslChallengeProcessor
+
+ private static class SaslInProgressProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+
+ final byte[] responseBytes = evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray());
+
+ final boolean isComplete = context.saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_SUCCESS : SaslStatus.SASL_IN_PROGRESS);
+ return response.build();
+ }
+ }
+
+ private static class SaslSuccessProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
+ } else {
+
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray()); // discard response
+
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ // setup security layers here..
+ return null;
— End diff –
shouldn't we dispose of the saslClient (`SaslClient#dispose()`) to remove sensitive informations (probably to be done in case of failure too)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101121400
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -412,37 +427,155 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
if(ret!=CONN_SUCCESS)
- if(this->m_handshakeStatus != exec::user::SUCCESS){
- switch(this->m_handshakeStatus){
- case exec::user::RPC_VERSION_MISMATCH:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected "
- << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl
- return handleConnError(CONN_BAD_RPC_VER,
- getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION,
- m_handshakeVersion,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::AUTH_FAILED:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl
- return handleConnError(CONN_AUTH_FAILED,
- getMessage(ERR_CONN_AUTHFAIL,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::UNKNOWN_FAILURE:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl
- return handleConnError(CONN_HANDSHAKE_FAILED,
- getMessage(ERR_CONN_UNKNOWN_ERR,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- default:
- break;
+
+ switch(this->m_handshakeStatus) { + case exec::user::SUCCESS: + // reset io_service after handshake is validated before running queries + m_io_service.reset(); + return CONN_SUCCESS; + case exec::user::RPC_VERSION_MISMATCH: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected " + << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;) + return handleConnError(CONN_BAD_RPC_VER, getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION, + m_handshakeVersion, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_FAILED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl;) + return handleConnError(CONN_AUTH_FAILED, getMessage(ERR_CONN_AUTHFAIL, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::UNKNOWN_FAILURE: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_REQUIRED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl;) + return handleAuthentication(properties); + default: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown return status." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + }+}
+
+connectionStatus_t DrillClientImpl::handleAuthentication(const DrillUserProperties *userProperties) {
+ try {
+ m_saslAuthenticator = new SaslAuthenticatorImpl(userProperties);-
- End diff –
-
- maybe we should use smart pointers
- I'm not strongly opinionated against exceptions in C++, but this is not a common pattern in drill code base
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102358541
— Diff: contrib/native/client/cmakeModules/FindSASL.cmake —
@@ -0,0 +1,55 @@
+#
+# 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.
+#
+
+# - Try to find Cyrus SASL
+
+if (MSVC)
— End diff –
Please also update the readme txt files on how to get/install SASL.
On Mac 10.11 and higher, it looks like using Apple SASL implementation is deprecated (I get deprecation warnings from the compiler)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102342473
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
— End diff –
these defines should probably be (static) constant instead.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102344639
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/ServerAuthenticationHandler.java —
@@ -0,0 +1,269 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.InvalidProtocolBufferException;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import org.apache.drill.exec.proto.UserBitShared.SaslMessage;
+import org.apache.drill.exec.proto.UserBitShared.SaslStatus;
+import org.apache.drill.exec.rpc.RequestHandler;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.ResponseSender;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.ServerConnection;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.EnumMap;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class ServerAuthenticationHandler<C extends ServerConnection, T extends EnumLite> implements RequestHandler<C> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(ServerAuthenticationHandler.class);
+
+ private static final ImmutableMap<SaslStatus, SaslResponseProcessor> RESPONSE_PROCESSORS;
+
+ static
+
+ private final RequestHandler<C> requestHandler;
+ private final int saslRequestTypeValue;
+ private final T saslResponseType;
+
+ public ServerAuthenticationHandler(final RequestHandler<C> requestHandler, final int saslRequestTypeValue,
+ final T saslResponseType)
+
+ @Override
+ public void handle(C connection, int rpcType, ByteBuf pBody, ByteBuf dBody, ResponseSender sender)
+ throws RpcException {
+ final String remoteAddress = connection.getRemoteAddress().toString();
+
+ // exchange involves server "challenges" and client "responses" (initiated by client)
+ if (saslRequestTypeValue == rpcType) {
+ final SaslMessage saslResponse;
+ try
catch (final InvalidProtocolBufferException e)
{ + handleAuthFailure(connection, remoteAddress, sender, e, saslResponseType); + return; + } +
+ logger.trace("Received SASL message {} from {}", saslResponse.getStatus(), remoteAddress);
+ final SaslResponseProcessor processor = RESPONSE_PROCESSORS.get(saslResponse.getStatus());
+ if (processor == null) {
+ logger.info("Unknown message type from client from {}. Will stop authentication.", remoteAddress);
+ handleAuthFailure(connection, remoteAddress, sender, new SaslException("Received unexpected message"),
+ saslResponseType);
+ return;
+ }
+
+ final SaslResponseContext<C, T> context = new SaslResponseContext<>(saslResponse, connection, remoteAddress,
+ sender, requestHandler, saslResponseType);
+ try
catch (final Exception e)
{ + handleAuthFailure(connection, remoteAddress, sender, e, saslResponseType); + }+ } else
{ + + // this handler only handles messages of SASL_MESSAGE_VALUE type + + // drop connection + connection.close(); + + // the response type for this request type is likely known from UserRpcConfig, + // but the client should not be making any requests before authenticating. + throw new UnsupportedOperationException( + String.format("Request of type %d is not allowed without authentication. " + + "Client on %s must authenticate before making requests. Connection dropped.", + rpcType, remoteAddress)); + } + }
+
+ private static class SaslResponseContext<C extends ServerConnection, T extends EnumLite> {
+
+ final SaslMessage saslResponse;
+ final C connection;
+ final String remoteAddress;
+ final ResponseSender sender;
+ final RequestHandler<C> requestHandler;
+ final T saslResponseType;
+
+ SaslResponseContext(SaslMessage saslResponse, C connection, String remoteAddress, ResponseSender sender,
+ RequestHandler<C> requestHandler, T saslResponseType)
+ }
+
+ private interface SaslResponseProcessor {
+
+ /**
+ * Process response from client, and if there are no exceptions, send response using
+ *
. Otherwise, throw the exception.
+ *
+ * @param context response context
+ */
+ void process(SaslResponseContext context) throws Exception;
+
+ }
+
+ private static class SaslStartProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception
+ }
+
+ private static class SaslInProgressProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception {
+ final SaslMessage.Builder challenge = SaslMessage.newBuilder();
+ final SaslServer saslServer = context.connection.getSaslServer();
+
+ final byte[] challengeBytes = evaluateResponse(saslServer, context.saslResponse.getData().toByteArray());
+
+ if (saslServer.isComplete()) {
+ challenge.setStatus(SaslStatus.SASL_SUCCESS);
+ if (challengeBytes != null)
+
+ handleSuccess(context, challenge, saslServer);
+ } else
+ }
+ }
+
+ // only when client succeeds first
+ private static class SaslSuccessProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception {
+ // at this point, #isComplete must be false; so try once, fail otherwise
+ final SaslServer saslServer = context.connection.getSaslServer();
+
+ evaluateResponse(saslServer, context.saslResponse.getData().toByteArray()); // discard challenge
+
+ if (saslServer.isComplete())
else {
+ logger.info("Failed to authenticate client from {}", context.remoteAddress);
+ throw new SaslException("Client allegedly succeeded authentication, but server did not. Suspicious?");
+ }
+ }
+ }
+
+ private static class SaslFailedProcessor implements SaslResponseProcessor {
+
+ @Override
+ public void process(final SaslResponseContext context) throws Exception {
+ logger.info("Client from {} failed authentication graciously, and does not want to continue.",
+ context.remoteAddress);
+ throw new SaslException("Client graciously failed authentication");
+ }
+ }
+
+ private static byte[] evaluateResponse(final SaslServer saslServer,
+ final byte[] responseBytes) throws SaslException {
+ try {
+ return UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<byte[]>() {
+ @Override
+ public byte[] run() throws Exception
+ });
+ } catch (final UndeclaredThrowableException e)
catch (final IOException | InterruptedException e) {
+ if (e instanceof SaslException)
else
{ + throw new SaslException(String.format("Unexpected failure trying to authenticate using %s", + saslServer.getMechanismName()), e); + } + }
+ }
+
+ @SuppressWarnings("unchecked")
+ private static void handleSuccess(final SaslResponseContext context, final SaslMessage.Builder challenge,
+ final SaslServer saslServer) throws IOException {
+ context.connection.changeHandlerTo(context.requestHandler);
+ context.connection.finalizeSaslSession();
+ context.sender.send(new Response(context.saslResponseType, challenge.build()));
+
+ // setup security layers here..
— End diff –
we should probably dispose of the saslServer (`saslServer#dispose()`)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102344797
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/ClientAuthenticatorProvider.java —
@@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.security;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.exec.rpc.security.kerberos.KerberosFactory;
+import org.apache.drill.exec.rpc.security.plain.PlainFactory;
+
+import javax.security.sasl.SaslException;
+import java.util.Map;
+import java.util.Set;
+
+public class ClientAuthenticatorProvider implements AuthenticatorProvider {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(ClientAuthenticatorProvider.class);
+
+ private static final String customFactories = System.getProperty("customAuthFactories");
— End diff –
shouldn't this property be prefixed with `drill.`?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102292416
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -105,8 +128,83 @@ protected void finalizeConnection(BitControlHandshake handshake, ControlConnecti
connection.setEndpoint(handshake.getEndpoint());
}
- public ControlConnection getConnection() {
- return this.connection;
+ @Override
+ protected <M extends MessageLite> RpcCommand<M, ControlConnection>
+ getInitialCommand(final RpcCommand<M, ControlConnection> command) {
+ if (config.getAuthMechanismToUse() == null) { + return super.getInitialCommand(command); + }else {
+ return new AuthenticationCommand<>(command);-
- End diff –
-
shouldn't we use `super.getInitialCommand(command)` too?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r102304062
— Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/ClientConnection.java —
@@ -0,0 +1,30 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc;
+
+import javax.security.sasl.SaslClient;
+
+public interface ClientConnection extends RemoteConnection {
— End diff –
Is `ClientConnection` the right place to pass along the saslClient vs `AuthenticationCommand`?
Github user parthchandra commented on the issue:
https://github.com/apache/drill/pull/578
+1
Thanks for the work guys, especially the extensive review comments.
I am aiming for this to be part of the next release, which maybe soon, given the poll on the dev list recently. So comments sooner rather than later would be helpful (I am gonna be on-and-off on vacation after next week).
BTW thank you for your initial comments.
I'm still reviewing the code. I thought I'd have it done sooner, but unfortunately, I was now able to give it my full attention until now. Let me know if there's hard deadlines for this.
Github user superbstreak commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101409917
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,211 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_pwd_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err) {
+ std::stringstream errMsg;
+ errMsg << "Failed to load authentication libraries. code: " << err;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << errMsg << std::endl
— End diff –
This would not compile. Try errMsg.str()
On windows, Im getting:
Error 443 error C2678: binary '<<' : no operator found which takes a left-hand operand of type 'std::ostream' (or there is no acceptable conversion)
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r101175920
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -435,9 +427,14 @@ public synchronized boolean reconnect() {
}
private void connect(DrillbitEndpoint endpoint) throws RpcException {
- final FutureHandler f = new FutureHandler();
- client.connect(f, endpoint, props, getUserCredentials());
- f.checkedGet();
+ client.connect(endpoint, properties, getUserCredentials()).checkedGet();
+ if (client.serverRequiresAuthentication()) {
+ try { + client.authenticate(null).checkedGet(); + }catch (SaslException e)
{ + throw new RpcException(e); + }-
- End diff –
-
Originally if client fail's to connect to server because of Authentication failure, then it doesn't retry since `InvalidConnectionInfoException` is thrown and handled separately.
But If authentication using SASL fails then we are always throwing `RpcException` and we will end up doing retry. We should have a check for exception thrown by SASL authentication failure and doesn't do retry in that scenario as well.
Github user bitblender commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100938180
— Diff: contrib/native/client/src/clientlib/drillClientImpl.hpp —
@@ -362,6 +363,7 @@ class DrillClientImpl : public DrillClientImplBase{
m_handshakeVersion(0),
m_handshakeStatus(exec::user::SUCCESS),
m_bIsConnected(false),
+ m_saslDone(false),
— End diff –
m_saslAuthenticator should be NULL initialized
Github user sohami commented on the issue:
https://github.com/apache/drill/pull/578
Just one last comment regarding moving BitConnectionConfig file. Apart from that LGTM.
+1 from my side.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100907447
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/BitConnectionConfig.java —
@@ -0,0 +1,106 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.data;
— End diff –
Please move this file under org.apache.drill.exec.rpc package.
Github user sudheeshkatkam commented on the issue:
https://github.com/apache/drill/pull/578
@sohami @laurentgo any more comments?
(I need to push one minor change in config)
Github user sudheeshkatkam commented on the issue:
https://github.com/apache/drill/pull/578
+ Addressed review comments
+ Updated commit messages
+ Rebased on latest master
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100152148
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +129,183 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms() {-
- End diff –
-
non-internal API
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100131255
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java —
@@ -75,27 +85,106 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ protected void handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { throw new UnsupportedOperationException("DataClient is unidirectional by design."); }
BufferAllocator getAllocator()
{ - return allocator; + return config.getAllocator(); } @Override
protected void validateHandshake(BitServerHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != DataRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
— End diff –
Correct me if I am wrong, but both your intentions are different.
I've addressed Sorabh's comment, as in, "check for the case if Authentication is enabled on this client and for some reason server is sending empty list of mechanisms list (may be wrong config) then we should throw exception"
But regarding Laurent's comment, the "code" is the "same as in ControlClient", the objects are all different (handshake, connection, config). That refactoring would require a lot more changes to BasicClient. I'll open a ticket once this PR is merged.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100129942
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -407,37 +422,155 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
if(ret!=CONN_SUCCESS)
- if(this->m_handshakeStatus != exec::user::SUCCESS){
- switch(this->m_handshakeStatus){
- case exec::user::RPC_VERSION_MISMATCH:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected "
- << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl
- return handleConnError(CONN_BAD_RPC_VER,
- getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION,
- m_handshakeVersion,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::AUTH_FAILED:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl
- return handleConnError(CONN_AUTH_FAILED,
- getMessage(ERR_CONN_AUTHFAIL,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::UNKNOWN_FAILURE:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl
- return handleConnError(CONN_HANDSHAKE_FAILED,
- getMessage(ERR_CONN_UNKNOWN_ERR,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- default:
- break;
+
+ switch(this->m_handshakeStatus) { + case exec::user::SUCCESS: + // reset io_service after handshake is validated before running queries + m_io_service.reset(); + return CONN_SUCCESS; + case exec::user::RPC_VERSION_MISMATCH: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected " + << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;) + return handleConnError(CONN_BAD_RPC_VER, getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION, + m_handshakeVersion, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_FAILED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl;) + return handleConnError(CONN_AUTH_FAILED, getMessage(ERR_CONN_AUTHFAIL, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::UNKNOWN_FAILURE: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_REQUIRED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl;) + return authenticate(properties); + default: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown return status." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + }+}
{ + m_saslAuthenticator = new SaslAuthenticatorImpl(userProperties); + }
+
+connectionStatus_t DrillClientImpl::authenticate(const DrillUserProperties* userProperties) {
+ trycatch (std::runtime_error& e) {
-
- End diff –
-
Suggestion for an alternative?
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100129719
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,210 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
— End diff –
This allows for lazy init in case auth is not used.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100122930
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -407,37 +422,155 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
if(ret!=CONN_SUCCESS)
- if(this->m_handshakeStatus != exec::user::SUCCESS){
- switch(this->m_handshakeStatus){
- case exec::user::RPC_VERSION_MISMATCH:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected "
- << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl
- return handleConnError(CONN_BAD_RPC_VER,
- getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION,
- m_handshakeVersion,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::AUTH_FAILED:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl
- return handleConnError(CONN_AUTH_FAILED,
- getMessage(ERR_CONN_AUTHFAIL,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::UNKNOWN_FAILURE:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl
- return handleConnError(CONN_HANDSHAKE_FAILED,
- getMessage(ERR_CONN_UNKNOWN_ERR,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- default:
- break;
+
+ switch(this->m_handshakeStatus) { + case exec::user::SUCCESS: + // reset io_service after handshake is validated before running queries + m_io_service.reset(); + return CONN_SUCCESS; + case exec::user::RPC_VERSION_MISMATCH: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected " + << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;) + return handleConnError(CONN_BAD_RPC_VER, getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION, + m_handshakeVersion, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_FAILED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl;) + return handleConnError(CONN_AUTH_FAILED, getMessage(ERR_CONN_AUTHFAIL, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::UNKNOWN_FAILURE: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_REQUIRED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl;) + return authenticate(properties); + default: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown return status." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + }+}
+
+connectionStatus_t DrillClientImpl::authenticate(const DrillUserProperties* userProperties) {
+ try {
+ m_saslAuthenticator = new SaslAuthenticatorImpl(userProperties);-
- End diff –
-
I don't think so. (I am not sure how else this could be done.)
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100122717
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -568,19 +565,13 @@ public void runQuery(QueryType type, List<PlanFragment> planFragments, UserResul
- Helper method to generate the UserCredentials message from the properties.
*/
private UserBitShared.UserCredentials getUserCredentials() {
- // If username is not propagated as one of the properties
- String userName = "anonymous";
- - if (props != null) {
- for (Property property: props.getPropertiesList()) {
- if (property.getKey().equalsIgnoreCase("user") && !Strings.isNullOrEmpty(property.getValue())) { - userName = property.getValue(); - break; - }
- }
+ String userName = properties.getProperty(DrillProperties.USER);
+ if (Strings.isNullOrEmpty(userName)) {
+ userName = "anonymous"; // if username is not propagated as one of the properties-
- End diff –
-
To keep the functionality as is, not sure was the original intent was.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r100122562
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ConnectionManagerRegistry.java —
@@ -32,24 +29,19 @@
private final ConcurrentMap<DrillbitEndpoint, ControlConnectionManager> registry = Maps.newConcurrentMap();
- private final ControlMessageHandler handler;
- private final BootStrapContext context;
- private volatile DrillbitEndpoint localEndpoint;
- private final BufferAllocator allocator;
+ private final BitConnectionConfigImpl config;
- public ConnectionManagerRegistry(BufferAllocator allocator, ControlMessageHandler handler, BootStrapContext context) {
- super();
- this.handler = handler;
- this.context = context;
- this.allocator = allocator;
+ public ConnectionManagerRegistry(BitConnectionConfigImpl config) {-
- End diff –
-
Each impl of ConnectionConfig is package private. There are subtle differences among them, which is why references using impl are used with within the package. But in generic classes (e.g.AbstractServerConnection), interface is used.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99952254
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -89,14 +90,48 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
+ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { + connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender); }
@Override
protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
+ if (config.getAuthProvider() == null)
+ if (!handshake.getAuthenticationMechanismsList().contains(config.getAuthMechanismToUse()))
{ + throw new RpcException(String.format("Drillbit (%s) does not support %s", remoteEndpoint.getAddress(), + config.getAuthMechanismToUse())); + } +
+ final SaslClient saslClient;
+ try
catch (final SaslException e)
{ + throw new RpcException("Failed to create SaslClient.", e); + }catch (final IOException e)
{ + throw new RpcException("Unexpected failure trying to login.", e); + } + if (saslClient == null) {
— End diff –
Yes, per the [SaslClientFactory API](https://docs.oracle.com/javase/7/docs/api/javax/security/sasl/SaslClientFactory.html#createSaslClient(java.lang.String[],%20java.lang.String,%20java.lang.String,%20java.lang.String,%20java.util.Map,%20javax.security.auth.callback.CallbackHandler)).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99951513
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/BitConnectionConfigImpl.java —
@@ -0,0 +1,118 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.data;
+
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.rpc.BitConnectionConfig;
+import org.apache.drill.exec.rpc.security.AuthenticatorProvider;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.Map;
+
+// package private
+class BitConnectionConfigImpl implements BitConnectionConfig {
— End diff –
will refactor.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99951476
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java —
@@ -84,23 +87,33 @@ public ControlConnection initRemoteConnection(SocketChannel channel) {
public MessageLite getHandshakeResponse(BitControlHandshake inbound) throws Exception {
// logger.debug("Handling handshake from other bit. {}", inbound);
if (inbound.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
- if (!inbound.hasEndpoint() || inbound.getEndpoint().getAddress().isEmpty() || inbound.getEndpoint().getControlPort() < 1) {
- throw new RpcException(String.format("RPC didn't provide valid counter endpoint information. Received %s.", inbound.getEndpoint()));
+ if (!inbound.hasEndpoint() ||
+ inbound.getEndpoint().getAddress().isEmpty() ||
+ inbound.getEndpoint().getControlPort() < 1) { + throw new RpcException(String.format("RPC didn't provide valid counter endpoint information. Received %s.", + inbound.getEndpoint())); }connection.setEndpoint(inbound.getEndpoint());
// add the
ControlConnectionManager manager = connectionRegistry.getConnectionManager(inbound.getEndpoint());
// update the close handler.
- proxyCloseHandler.setHandler(manager.getCloseHandlerCreator().getHandler(connection, proxyCloseHandler.getHandler()));
+ proxyCloseHandler.setHandler(manager.getCloseHandlerCreator().getHandler(connection,
+ proxyCloseHandler.getHandler()));
// add to the connection manager.
manager.addExternalConnection(connection);
- return BitControlHandshake.newBuilder().setRpcVersion(ControlRpcConfig.RPC_VERSION).build();
+ final BitControlHandshake.Builder builder = BitControlHandshake.newBuilder();
+ builder.setRpcVersion(ControlRpcConfig.RPC_VERSION);
+ if (config.getAuthProvider() != null) {-
- End diff –
-
There is only one authProvider per drillbit, but auth can be enabled user to bit, but not bit to bit, which means relying on being empty is not sufficient.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99900908
— Diff: protocol/src/main/protobuf/User.proto —
@@ -72,6 +79,7 @@ message UserToBitHandshake {
optional bool support_complex_types = 6 [default = false];
optional bool support_timeout = 7 [default = false];
optional RpcEndpointInfos client_infos = 8;
+ optional SaslSupport sasl_support = 9;
— End diff –
Meant more like what level of "sasl support", since there could be more than two.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99900608
— Diff: protocol/src/main/protobuf/User.proto —
@@ -63,6 +65,11 @@ message RpcEndpointInfos
+enum SaslSupport {
— End diff –
Yes, SASL_ENC (for encryption).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99900504
— Diff: common/src/main/java/org/apache/drill/common/map/CaseInsensitiveMap.java —
@@ -55,6 +55,18 @@
}
/**
+ * Returns a new instance of
, with key case-insensitivity, of expected size.
— End diff –
The function name is `newHashMapWithExpectedSize` (following the convention of other functions in the class).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99898556
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserAuthenticationUtil.java —
@@ -0,0 +1,255 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.ConnectionParameters;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.annotation.Nullable;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.AccessController;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Set;
+
+public final class UserAuthenticationUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserAuthenticationUtil.class);
+
+ private static final String PLAIN_MECHANISM = "PLAIN";
+
+ private static final String DEFAULT_SERVICE_NAME = System.getProperty("service.name.primary", "drill");
+
+ private static final String DEFAULT_REALM_NAME = System.getProperty("service.name.realm", "default");
+
+ public enum ClientAuthenticationProvider {
+
+ KERBEROS {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
+ UserGroupInformation.AuthenticationMethod.KERBEROS.toString());
+ UserGroupInformation.setConfiguration(conf);
+
+ final String keytab = parameters.getParameter(ConnectionParameters.KEYTAB);
+ final boolean assumeSubject = parameters.getParameter(ConnectionParameters.KERBEROS_FROM_SUBJECT) != null &&
+ Boolean.parseBoolean(parameters.getParameter(ConnectionParameters.KERBEROS_FROM_SUBJECT));
+ try {
+ final UserGroupInformation ugi;
+ if (assumeSubject) {
+ ugi = UserGroupInformation.getUGIFromSubject(Subject.getSubject(AccessController.getContext()));
+ logger.debug("Assuming subject for {}.", ugi.getShortUserName());
+ } else {
+ if (keytab != null) {
+ ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+ parameters.getParameter(ConnectionParameters.USER), keytab);
+ logger.debug("Logged in {} using keytab.", ugi.getShortUserName());
+ } else {
+ // includes Kerberos ticket login
+ ugi = UserGroupInformation.getCurrentUser();
+ logger.debug("Logged in {} using ticket.", ugi.getShortUserName());
+ }
+ }
+ return ugi;
+ } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login.", cause);
+ }
+ }
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi,
+ final ConnectionParameters parameters) throws SaslException {
+ final String servicePrincipal = getServicePrincipal(parameters);
+
+ final String parts[] = KerberosUtil.splitPrincipalIntoParts(servicePrincipal);
+ final String serviceName = parts[0];
+ final String serviceHostName = parts[1];
+ // ignore parts[2]; GSSAPI gets the realm info from the ticket
+ try {
+ final SaslClient saslClient = ugi.doAs(new PrivilegedExceptionAction<SaslClient>() {
+
+ @Override
+ public SaslClient run() throws Exception {
+ return Sasl.createSaslClient(new String[]{KerberosUtil.KERBEROS_SASL_NAME},
+ null /** authorization ID */, serviceName, serviceHostName,
+ null /** properties; default QOP is auth */, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks)
+ throws IOException, UnsupportedCallbackException { + throw new UnsupportedCallbackException(callbacks[0]); + }
+ });
+ }
+ });
+ logger.debug("GSSAPI SaslClient created to authenticate to {} running on {}",
+ serviceName, serviceHostName);
+ return saslClient;
+ } catch (final UndeclaredThrowableException e) { + throw new SaslException(String.format("Unexpected failure trying to authenticate to %s using GSSAPI", + serviceHostName), e.getCause()); + } catch (final IOException | InterruptedException e) {
+ if (e instanceof SaslException) { + throw (SaslException) e; + }
+ throw new SaslException(String.format("Unexpected failure trying to authenticate to %s using GSSAPI",
+ serviceHostName), e);
+ }
+ }
+ },
+
+ PLAIN {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ try { + return UserGroupInformation.getCurrentUser(); + } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException) { + throw new SaslException("Failed to login.", cause); + }
+ throw new SaslException("Unexpected failure trying to login. ", cause);
+ }
+ }
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi,
+ final ConnectionParameters parameters) throws SaslException {
+ final String userName = parameters.getParameter(ConnectionParameters.USER);
+ final String password = parameters.getParameter(ConnectionParameters.PASSWORD);
+
+ return Sasl.createSaslClient(new String[]
, null /** authorization ID */,
+ null, null, null /** properties; default QOP is auth */, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
+ if (callback instanceof PasswordCallback)
{ + PasswordCallback.class.cast(callback).setPassword(password.toCharArray()); + continue; + } + throw new UnsupportedCallbackException(callback);
+ }
+ }
+ });
+ }
+ };
+
+ public abstract UserGroupInformation login(ConnectionParameters parameters) throws SaslException;
+
+ public abstract SaslClient createSaslClient(UserGroupInformation ugi, ConnectionParameters parameters)
— End diff –
Done.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99898478
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClientAuthenticationHandler.java —
@@ -0,0 +1,229 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.protobuf.ByteString;
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.proto.UserProtos.RpcType;
+import org.apache.drill.exec.proto.UserProtos.SaslMessage;
+import org.apache.drill.exec.proto.UserProtos.SaslStatus;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+// package private
+class UserClientAuthenticationHandler implements RpcOutcomeListener<SaslMessage> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(UserClientAuthenticationHandler.class);
+
+ private static final ImmutableMap<SaslStatus, SaslChallengeProcessor> CHALLENGE_PROCESSORS =
+ ImmutableMap.<SaslStatus, SaslChallengeProcessor>builder()
+ .put(SaslStatus.SASL_IN_PROGRESS, new SaslInProgressProcessor())
+ .put(SaslStatus.SASL_SUCCESS, new SaslSuccessProcessor())
+ .put(SaslStatus.SASL_FAILED, new SaslFailedProcessor())
+ .build();
+
+ private final UserClient client;
+ private final UserGroupInformation ugi;
+ private final SettableFuture<Void> settableFuture;
+
+ public UserClientAuthenticationHandler(UserClient client, UserGroupInformation ugi,
+ SettableFuture<Void> settableFuture)
+
+ public void initiate(final String mechanismName) {
+ try {
+ final ByteString responseData;
+ final SaslClient saslClient = client.getSaslClient();
+ if (saslClient.hasInitialResponse())
else
{ + responseData = ByteString.EMPTY; + } + client.send(new UserClientAuthenticationHandler(client, ugi, settableFuture),
+ RpcType.SASL_MESSAGE,
+ SaslMessage.newBuilder()
+ .setMechanism(mechanismName)
+ .setStatus(SaslStatus.SASL_START)
+ .setData(responseData)
+ .build(),
+ SaslMessage.class);
+ logger.trace("Initiated SASL exchange.");
+ } catch (final Exception e)
+ }
+
+ @Override
+ public void failed(RpcException ex)
+
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ final SaslChallengeProcessor processor = CHALLENGE_PROCESSORS.get(value.getStatus());
+ if (processor == null)
else {
+ try {
+ final SaslChallengeContext context =
+ new SaslChallengeContext(value, client.getSaslClient(), ugi, settableFuture);
+
+ final SaslMessage saslResponse = processor.process(context);
+
+ if (saslResponse != null)
else
{ + // success + client.disposeSaslClient(); + settableFuture.set(null); + } + } catch (final Exception e) {
+ try
catch (Exception ignored)
{ + //ignored + } + settableFuture.setException(e);
+ }
+ }
+ }
+
+ @Override
+ public void interrupted(InterruptedException e)
+
+ private static class SaslChallengeContext {
+
+ final SaslMessage challenge;
+ final SaslClient saslClient;
+ final UserGroupInformation ugi;
+ final SettableFuture<Void> settableFuture;
+
+ public SaslChallengeContext(SaslMessage challenge, SaslClient saslClient, UserGroupInformation ugi,
+ SettableFuture<Void> settableFuture)
+ }
+
+ private interface SaslChallengeProcessor
+
+ private static class SaslInProgressProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+
+ final byte[] responseBytes = evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray());
+
+ final boolean isComplete = context.saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_SUCCESS : SaslStatus.SASL_IN_PROGRESS);
+ return response.build();
+ }
+ }
+
+ private static class SaslSuccessProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ return null;
+ } else {
+
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray()); // discard response
+
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ return null;
+ } else
+ }
+ }
+ }
+
+ private static class SaslFailedProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ throw new SaslException("Authentication failed. Incorrect credentials?");
— End diff –
Typically the client is not notified why auth failed, except that something went wrong.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99898177
— Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java —
@@ -56,16 +68,27 @@ public void positiveUserAuth() throws Exception
-
@Test
public void negativeUserAuth() throws Exception
@Test
+ public void emptyPassword() throws Exception {
+ try
catch (RpcException e) {
+ final String exMsg = e.getMessage();
+ assertThat(exMsg, containsString("Insufficient credentials"));
— End diff –
Not part of Kerberos testing, I just followed how the tests were written previously.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99897983
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
+
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 1;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
— End diff –
Yes, as shown in Cyrus SASL examples.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99897855
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -89,14 +89,42 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
+ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { + connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender); }
@Override
protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
— End diff –
Thanks for the catch.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99897718
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,206 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::string key = m_properties->keyAt;
+ const std::string value = m_properties->valueAt;
+
+ if (USERPROP_SERVICE_HOST == key)
else if (USERPROP_SERVICE_NAME == key)
{ + serviceName = value; + }else if (USERPROP_PASSWORD == key)
{ + m_password = value; + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = PLAIN_NAME; + }else if (USERPROP_USERNAME == key)
{ + m_username = value; + }else if (USERPROP_AUTH_MECHANISM == key)
{ + authMechanismToUse = value; + } + }
+ if (authMechanismToUse.empty()) return SASL_NOMECH;
— End diff –
Same logic available for Plain here as well, but not for Kerberos. The C++ client library is primarily used through ODBC which typically provides parts of the principal (service_name, service_host).
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99879420
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,210 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::string key = m_properties->keyAt;
+ const std::string value = m_properties->valueAt;
+
+ if (USERPROP_SERVICE_HOST == key)
else if (USERPROP_SERVICE_NAME == key)
{ + serviceName = value; + }else if (USERPROP_PASSWORD == key)
{ + m_password = value; + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = PLAIN_NAME; + }else if (USERPROP_USERNAME == key)
{ + m_username = value; + }else if (USERPROP_AUTH_MECHANISM == key)
{ + authMechanismToUse = value; + } + }
+ if (authMechanismToUse.empty()) return SASL_NOMECH;
+
+ // check if requested mechanism is supported by server
+ boost::algorithm::to_lower(authMechanismToUse);
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
+ std::string mechanism = mechanisms[i];
+ boost::algorithm::to_lower(mechanism);
+ if (authMechanismToUse == mechanism)
+ }
+ if (!isSupportedByServer) return SASL_NOMECH;
+
+ // find the SASL name
+ const std::map<std::string, std::string>::const_iterator it =
+ SaslAuthenticatorImpl::MECHANISM_MAPPING.find(authMechanismToUse);
+ if (it == SaslAuthenticatorImpl::MECHANISM_MAPPING.end()) return SASL_NOMECH;
+ const std::string saslMechanismToUse = it->second;
+
+ // setup callbacks and parameters
+ const sasl_callback_t callbacks[] = {
+
,
— End diff –
if you can keep using c++ cast, it would be great
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99871911
— Diff: protocol/src/main/protobuf/User.proto —
@@ -72,6 +79,7 @@ message UserToBitHandshake {
optional bool support_complex_types = 6 [default = false];
optional bool support_timeout = 7 [default = false];
optional RpcEndpointInfos client_infos = 8;
+ optional SaslSupport sasl_support = 9;
— End diff –
maybe to be renamed support_sasl_auth (like support_complex_types and support_timeout)?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99881850
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,210 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
— End diff –
shouldn't this be in a static member? we already have a place where the driver does a bunch of static initialization anyway...
Then the object could just check if the library was successfully initialized, and if not, any attempt at initialization would fail.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99884385
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,210 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::string key = m_properties->keyAt;
+ const std::string value = m_properties->valueAt;
+
+ if (USERPROP_SERVICE_HOST == key)
else if (USERPROP_SERVICE_NAME == key)
{ + serviceName = value; + } else if (USERPROP_PASSWORD == key) {
+ m_password = value;
+ m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length());
— End diff –
maybe set the secret here (since this doesn't change)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99885146
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -568,19 +565,13 @@ public void runQuery(QueryType type, List<PlanFragment> planFragments, UserResul
- Helper method to generate the UserCredentials message from the properties.
*/
private UserBitShared.UserCredentials getUserCredentials() {
- // If username is not propagated as one of the properties
- String userName = "anonymous";
- - if (props != null) {
- for (Property property: props.getPropertiesList()) {
- if (property.getKey().equalsIgnoreCase("user") && !Strings.isNullOrEmpty(property.getValue())) { - userName = property.getValue(); - break; - }
- }
+ String userName = properties.getProperty(DrillProperties.USER);
+ if (Strings.isNullOrEmpty(userName)) {
+ userName = "anonymous"; // if username is not propagated as one of the properties-
- End diff –
-
why not getting the system user then?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99890434
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java —
@@ -75,27 +85,106 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ protected void handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { throw new UnsupportedOperationException("DataClient is unidirectional by design."); }
BufferAllocator getAllocator()
{ - return allocator; + return config.getAllocator(); } @Override
protected void validateHandshake(BitServerHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != DataRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
— End diff –
+1
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99887378
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -89,14 +90,48 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
+ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { + connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender); }
@Override
protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
+ if (config.getAuthProvider() == null)
+ if (!handshake.getAuthenticationMechanismsList().contains(config.getAuthMechanismToUse()))
{ + throw new RpcException(String.format("Drillbit (%s) does not support %s", remoteEndpoint.getAddress(), + config.getAuthMechanismToUse())); + } +
+ final SaslClient saslClient;
+ try
catch (final SaslException e)
{ + throw new RpcException("Failed to create SaslClient.", e); + }catch (final IOException e)
{ + throw new RpcException("Unexpected failure trying to login.", e); + } + if (saslClient == null) {
— End diff –
is it even possible? I'm assuming that createSaslClient would throw in case of error...
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99888092
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -89,14 +90,48 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
+ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { + connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender); }
@Override
protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
+ if (config.getAuthProvider() == null)
+ if (!handshake.getAuthenticationMechanismsList().contains(config.getAuthMechanismToUse()))
{ + throw new RpcException(String.format("Drillbit (%s) does not support %s", remoteEndpoint.getAddress(), + config.getAuthMechanismToUse())); + } +
+ final SaslClient saslClient;
+ try {
+ saslClient = config.getAuthProvider()
— End diff –
why the config doesn't return directly the factory? it would make the code a bit simpler I guess (and probably no need then to expose the mechanism used from outside the config)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99889252
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java —
@@ -84,23 +87,33 @@ public ControlConnection initRemoteConnection(SocketChannel channel) {
public MessageLite getHandshakeResponse(BitControlHandshake inbound) throws Exception {
// logger.debug("Handling handshake from other bit. {}", inbound);
if (inbound.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
- if (!inbound.hasEndpoint() || inbound.getEndpoint().getAddress().isEmpty() || inbound.getEndpoint().getControlPort() < 1) {
- throw new RpcException(String.format("RPC didn't provide valid counter endpoint information. Received %s.", inbound.getEndpoint()));
+ if (!inbound.hasEndpoint() ||
+ inbound.getEndpoint().getAddress().isEmpty() ||
+ inbound.getEndpoint().getControlPort() < 1) { + throw new RpcException(String.format("RPC didn't provide valid counter endpoint information. Received %s.", + inbound.getEndpoint())); }connection.setEndpoint(inbound.getEndpoint());
// add the
ControlConnectionManager manager = connectionRegistry.getConnectionManager(inbound.getEndpoint());
// update the close handler.
- proxyCloseHandler.setHandler(manager.getCloseHandlerCreator().getHandler(connection, proxyCloseHandler.getHandler()));
+ proxyCloseHandler.setHandler(manager.getCloseHandlerCreator().getHandler(connection,
+ proxyCloseHandler.getHandler()));
// add to the connection manager.
manager.addExternalConnection(connection);
- return BitControlHandshake.newBuilder().setRpcVersion(ControlRpcConfig.RPC_VERSION).build();
+ final BitControlHandshake.Builder builder = BitControlHandshake.newBuilder();
+ builder.setRpcVersion(ControlRpcConfig.RPC_VERSION);
+ if (config.getAuthProvider() != null) {-
- End diff –
-
instead, maybe have a method to get all mechanisms. If empty, it means that authentication is disabled (should match protobuf behavior of returning empty list by default)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99871673
— Diff: protocol/src/main/protobuf/User.proto —
@@ -63,6 +65,11 @@ message RpcEndpointInfos
+enum SaslSupport {
— End diff –
do you foresee more than two values?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99874906
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -407,37 +422,155 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
if(ret!=CONN_SUCCESS)
- if(this->m_handshakeStatus != exec::user::SUCCESS){
- switch(this->m_handshakeStatus){
- case exec::user::RPC_VERSION_MISMATCH:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected "
- << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl
- return handleConnError(CONN_BAD_RPC_VER,
- getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION,
- m_handshakeVersion,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::AUTH_FAILED:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl
- return handleConnError(CONN_AUTH_FAILED,
- getMessage(ERR_CONN_AUTHFAIL,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::UNKNOWN_FAILURE:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl
- return handleConnError(CONN_HANDSHAKE_FAILED,
- getMessage(ERR_CONN_UNKNOWN_ERR,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- default:
- break;
+
+ switch(this->m_handshakeStatus) { + case exec::user::SUCCESS: + // reset io_service after handshake is validated before running queries + m_io_service.reset(); + return CONN_SUCCESS; + case exec::user::RPC_VERSION_MISMATCH: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected " + << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;) + return handleConnError(CONN_BAD_RPC_VER, getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION, + m_handshakeVersion, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_FAILED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl;) + return handleConnError(CONN_AUTH_FAILED, getMessage(ERR_CONN_AUTHFAIL, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::UNKNOWN_FAILURE: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_REQUIRED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl;) + return authenticate(properties); + default: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown return status." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + }+}
{ + m_saslAuthenticator = new SaslAuthenticatorImpl(userProperties); + }
+
+connectionStatus_t DrillClientImpl::authenticate(const DrillUserProperties* userProperties) {
+ trycatch (std::runtime_error& e)
{ + return handleConnError(CONN_AUTH_FAILED, e.what()); + }+
+ startMessageListener();
+ initiateAuthentication();
+
+Unknown macro: { // block until SASL exchange is complete + boost} - // reset io_service after handshake is validated before running queries
- m_io_service.reset();
- return CONN_SUCCESS;
+
+ if (SASL_OK == m_saslResultCode) { + DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "DrillClientImpl::authenticate: Successfully authenticated!" + << std::endl;) + + // in future, negotiated security layers are known here.. + + m_io_service.reset(); + return CONN_SUCCESS; + }else
{ + DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "DrillClientImpl::authenticate: Authentication failed: " + << m_saslResultCode << std::endl;) + // shuts down socket as well + return handleConnError(CONN_AUTH_FAILED, "User authentication failed. Check connection parameters?"); + }}
+void DrillClientImpl::initiateAuthentication() {
+ exec::shared::SaslMessage response;
+ m_saslResultCode = m_saslAuthenticator->init(m_serverAuthMechanisms, response);
+
+
+ switch (m_saslResultCode) {
+ case SASL_CONTINUE:
+ case SASL_OK: {
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::initiateAuthentication: inititated. " << std::endl
— End diff –
typo: inititaed -> initiated
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99890259
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/BitConnectionConfigImpl.java —
@@ -0,0 +1,118 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.drill.exec.rpc.data;
+
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.rpc.BitConnectionConfig;
+import org.apache.drill.exec.rpc.security.AuthenticatorProvider;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.Map;
+
+// package private
+class BitConnectionConfigImpl implements BitConnectionConfig {
— End diff –
isn't this class almost the same as the one in rpc.control package (and btw super confusing to reuse the same name even in different packages)? Maybe we can make a generic version of it.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99880759
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,210 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ // loads all the available mechanism and factories in the sasl_lib referenced by the path
+ const int err = sasl_client_init(NULL);
+ if (0 != err)
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL) {
+ *result = username->c_str();
+ // *len = (unsigned int) username->length();
— End diff –
not needed?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99886887
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ConnectionManagerRegistry.java —
@@ -32,24 +29,19 @@
private final ConcurrentMap<DrillbitEndpoint, ControlConnectionManager> registry = Maps.newConcurrentMap();
- private final ControlMessageHandler handler;
- private final BootStrapContext context;
- private volatile DrillbitEndpoint localEndpoint;
- private final BufferAllocator allocator;
+ private final BitConnectionConfigImpl config;
- public ConnectionManagerRegistry(BufferAllocator allocator, ControlMessageHandler handler, BootStrapContext context) {
- super();
- this.handler = handler;
- this.context = context;
- this.allocator = allocator;
+ public ConnectionManagerRegistry(BitConnectionConfigImpl config) {-
- End diff –
-
shouldn't the interface be used instead?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99874669
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -407,37 +422,155 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
if(ret!=CONN_SUCCESS)
- if(this->m_handshakeStatus != exec::user::SUCCESS){
- switch(this->m_handshakeStatus){
- case exec::user::RPC_VERSION_MISMATCH:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected "
- << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl
- return handleConnError(CONN_BAD_RPC_VER,
- getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION,
- m_handshakeVersion,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::AUTH_FAILED:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl
- return handleConnError(CONN_AUTH_FAILED,
- getMessage(ERR_CONN_AUTHFAIL,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::UNKNOWN_FAILURE:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl
- return handleConnError(CONN_HANDSHAKE_FAILED,
- getMessage(ERR_CONN_UNKNOWN_ERR,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- default:
- break;
+
+ switch(this->m_handshakeStatus) { + case exec::user::SUCCESS: + // reset io_service after handshake is validated before running queries + m_io_service.reset(); + return CONN_SUCCESS; + case exec::user::RPC_VERSION_MISMATCH: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected " + << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;) + return handleConnError(CONN_BAD_RPC_VER, getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION, + m_handshakeVersion, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_FAILED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl;) + return handleConnError(CONN_AUTH_FAILED, getMessage(ERR_CONN_AUTHFAIL, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::UNKNOWN_FAILURE: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_REQUIRED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl;) + return authenticate(properties); + default: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown return status." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + }+}
+
+connectionStatus_t DrillClientImpl::authenticate(const DrillUserProperties* userProperties) {
+ try {
+ m_saslAuthenticator = new SaslAuthenticatorImpl(userProperties);-
- End diff –
-
Can we avoid dynamic allocation?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99875648
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -407,37 +422,155 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
if(ret!=CONN_SUCCESS)
- if(this->m_handshakeStatus != exec::user::SUCCESS){
- switch(this->m_handshakeStatus){
- case exec::user::RPC_VERSION_MISMATCH:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected "
- << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl
- return handleConnError(CONN_BAD_RPC_VER,
- getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION,
- m_handshakeVersion,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::AUTH_FAILED:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl
- return handleConnError(CONN_AUTH_FAILED,
- getMessage(ERR_CONN_AUTHFAIL,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::UNKNOWN_FAILURE:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl
- return handleConnError(CONN_HANDSHAKE_FAILED,
- getMessage(ERR_CONN_UNKNOWN_ERR,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- default:
- break;
+
+ switch(this->m_handshakeStatus) { + case exec::user::SUCCESS: + // reset io_service after handshake is validated before running queries + m_io_service.reset(); + return CONN_SUCCESS; + case exec::user::RPC_VERSION_MISMATCH: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected " + << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;) + return handleConnError(CONN_BAD_RPC_VER, getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION, + m_handshakeVersion, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_FAILED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl;) + return handleConnError(CONN_AUTH_FAILED, getMessage(ERR_CONN_AUTHFAIL, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::UNKNOWN_FAILURE: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_REQUIRED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl;) + return authenticate(properties); + default: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown return status." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + }+}
{ + m_saslAuthenticator = new SaslAuthenticatorImpl(userProperties); + }
+
+connectionStatus_t DrillClientImpl::authenticate(const DrillUserProperties* userProperties) {
+ trycatch (std::runtime_error& e) {
-
- End diff –
-
exceptions in C++ are tricky, I'm not against personally, but I wonder if this is needed (especially since here, it's because static initialization failed...)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99871395
— Diff: protocol/src/main/protobuf/User.proto —
@@ -111,6 +115,21 @@ message BitToUserHandshake {
optional string errorId = 4;
optional string errorMessage = 5;
optional RpcEndpointInfos server_infos = 6;
+ repeated string authenticationMechanisms = 7;
— End diff –
repeated fields are optional
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99873776
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -407,37 +422,155 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
if(ret!=CONN_SUCCESS)
- if(this->m_handshakeStatus != exec::user::SUCCESS){
- switch(this->m_handshakeStatus){
- case exec::user::RPC_VERSION_MISMATCH:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected "
- << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl
- return handleConnError(CONN_BAD_RPC_VER,
- getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION,
- m_handshakeVersion,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::AUTH_FAILED:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl
- return handleConnError(CONN_AUTH_FAILED,
- getMessage(ERR_CONN_AUTHFAIL,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- case exec::user::UNKNOWN_FAILURE:
- DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl
- return handleConnError(CONN_HANDSHAKE_FAILED,
- getMessage(ERR_CONN_UNKNOWN_ERR,
- this->m_handshakeErrorId.c_str(),
- this->m_handshakeErrorMsg.c_str()));
- default:
- break;
+
+ switch(this->m_handshakeStatus) { + case exec::user::SUCCESS: + // reset io_service after handshake is validated before running queries + m_io_service.reset(); + return CONN_SUCCESS; + case exec::user::RPC_VERSION_MISMATCH: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Invalid rpc version. Expected " + << DRILL_RPC_VERSION << ", actual "<< m_handshakeVersion << "." << std::endl;) + return handleConnError(CONN_BAD_RPC_VER, getMessage(ERR_CONN_BAD_RPC_VER, DRILL_RPC_VERSION, + m_handshakeVersion, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_FAILED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Authentication failed." << std::endl;) + return handleConnError(CONN_AUTH_FAILED, getMessage(ERR_CONN_AUTHFAIL, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::UNKNOWN_FAILURE: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown error during handshake." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + case exec::user::AUTH_REQUIRED: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl;) + return authenticate(properties); + default: + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Unknown return status." << std::endl;) + return handleConnError(CONN_HANDSHAKE_FAILED, getMessage(ERR_CONN_UNKNOWN_ERR, + this->m_handshakeErrorId.c_str(), + this->m_handshakeErrorMsg.c_str())); + }+}
+
+connectionStatus_t DrillClientImpl::authenticate(const DrillUserProperties* userProperties) {-
- End diff –
-
let's keep the convention of using handleXXX: handleAuthentication(...)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99869783
— Diff: common/src/main/java/org/apache/drill/common/map/CaseInsensitiveMap.java —
@@ -55,6 +55,18 @@
}
/**
+ * Returns a new instance of
, with key case-insensitivity, of expected size.
— End diff –
Better just say: Returns a new case insensitive map of expected size...
use of HashMap is just an implementation detail
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99678863
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,206 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 0;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ s_initialized = true;
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = static_cast<const std::string* const>(context);
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = static_cast<const SaslAuthenticatorImpl* const>(context);
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector<std::string>& mechanisms, exec::shared::SaslMessage& response) {
+ // find and set parameters
+ std::string authMechanismToUse;
+ std::string serviceName;
+ std::string serviceHost;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::string key = m_properties->keyAt;
+ const std::string value = m_properties->valueAt;
+
+ if (USERPROP_SERVICE_HOST == key)
else if (USERPROP_SERVICE_NAME == key)
{ + serviceName = value; + }else if (USERPROP_PASSWORD == key)
{ + m_password = value; + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = PLAIN_NAME; + }else if (USERPROP_USERNAME == key)
{ + m_username = value; + }else if (USERPROP_AUTH_MECHANISM == key)
{ + authMechanismToUse = value; + } + }
+ if (authMechanismToUse.empty()) return SASL_NOMECH;
— End diff –
For cpp client we always expect *authMechanism* to be mentioned as parameter ? Since on java side we look for other Kerberos/Plain mechanism related parameters and use it in case a particular mechanism is not specified on client side.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99265626
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,206 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
— End diff –
Not sure if earlier comment got deleted, re-writing again. `sasl_client_init` can throw below error which we should check and handle.
```
SASL_OK
Success
SASL_BADVERS
Mechanism version mismatch
SASL_BADPARAM
Error in config file
SASL_NOMEM
Not enough memory to complete operation
```
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99257059
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java —
@@ -75,27 +85,106 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ protected void handle(DataClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { throw new UnsupportedOperationException("DataClient is unidirectional by design."); }
BufferAllocator getAllocator()
{ - return allocator; + return config.getAllocator(); } @Override
protected void validateHandshake(BitServerHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != DataRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
— End diff –
Same as in ControlClient
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99261445
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +129,183 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms() {-
- End diff –
-
not used anywhere.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99257662
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/BitConnectionConfigImpl.java —
@@ -0,0 +1,121 @@
+/**
+ * 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.drill.exec.rpc.control;
+
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.config.DrillProperties;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.BitConnectionConfig;
+import org.apache.drill.exec.rpc.security.AuthenticatorProvider;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.ControlMessageHandler;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.Map;
+
+// package private
+class BitConnectionConfigImpl implements BitConnectionConfig {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnectionConfigImpl.class);
+
+ private final BufferAllocator allocator;
+ private final BootStrapContext context;
+ private final ControlMessageHandler handler;
+
+ private final AuthenticatorProvider authProvider;
+ private final String authMechanismToUse;
+ private final String clusterId;
+
+ private DrillbitEndpoint localEndpoint;
+
+ BitConnectionConfigImpl(BufferAllocator allocator, BootStrapContext context, ControlMessageHandler handler) {
+ this.allocator = allocator;
+ this.context = context;
+ this.handler = handler;
+
+ final DrillConfig config = context.getConfig();
+ this.authProvider = config.getBoolean(ExecConstants.BIT_AUTHENTICATION_ENABLED)
+ ? context.getAuthProvider()
+ : null;
+ this.authMechanismToUse = config.getBoolean(ExecConstants.BIT_AUTHENTICATION_ENABLED)
+ ? config.getString(ExecConstants.BIT_AUTHENTICATION_MECHANISM)
+ : null;
+ this.clusterId = config.getBoolean(ExecConstants.USE_CLUSTER_ID_AS_KERBEROS_INSTANCE_NAME)
— End diff –
Seems like we are missing this check when authentication is enabled. Same for Data implementation
```
if (authProvider.getAllFactoryNames().size() == 0)
```
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99261692
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -88,22 +129,183 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, DrillProperties parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) .setCredentials(credentials) - .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)); + .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName)) + .setSaslSupport(SaslSupport.SASL_AUTH) + .setProperties(parameters.serializeForServer()); + this.properties = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
{ + return supportedAuthMechs; + }
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms()
- if (props != null) {
- hsBuilder.setProperties(props);
+ /**
+ * Authenticate to the server asynchronously. Returns a future that {@link CheckedFuture#checkedGet results}+ * in null if authentication succeeds, or throws a
{@link SaslException}with relevant message if
{@link #connect connection time}
+ * authentication fails.
+ *
+ * This method uses properties provided atand override them with the
{ + throw new IllegalStateException("Server does not require authentication."); }
+ * given properties, if any.
+ *
+ * @param overrides parameter overrides
+ * @return result of authentication request
+ */
+ public CheckedFuture<Void, SaslException> authenticate(final DrillProperties overrides) {
+ if (supportedAuthMechs == null)+ properties.merge(overrides);
+ final Map<String, String> propertiesMap = properties.stringPropertiesAsMap();
- this.connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler),
- hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
+ final SettableFuture<Void> settableFuture = SettableFuture.create(); // future used in SASL exchange
+ final CheckedFuture<Void, SaslException> future =
+ new AbstractCheckedFuture<Void, SaslException>(settableFuture) {
+
+ @Override
+ protected SaslException mapException(Exception e) {
+ if (connection != null) { + connection.close(); // to ensure connection is dropped + }+ if (e instanceof ExecutionException)
Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + return new SaslException("Authentication failed: " + cause.getMessage(), cause); + } + }+ return new SaslException("Authentication failed unexpectedly.", e);
{ + factory = getAuthenticatorFactory(); + }
+ }
+ };
+
+ final AuthenticatorFactory factory;
+ trycatch (final SaslException e)
{ + settableFuture.setException(e); + return future; + }
+
+ final String mechanismName = factory.getSimpleName();
+ logger.trace("Will try to login for {} mechanism.", mechanismName);
+ final UserGroupInformation ugi;
+ try { + ugi = factory.createAndLoginUser(propertiesMap); + } catch (final IOException e) { + settableFuture.setException(e); + return future; + }+
{ + saslClient = factory.createSaslClient(ugi, propertiesMap); + connection.setSaslClient(saslClient); + }
+ logger.trace("Will try to authenticate to server using {} mechanism.", mechanismName);
+ final SaslClient saslClient;
+ trycatch (final SaslException e)
{ + settableFuture.setException(e); + return future; + }+
{ + settableFuture.setException(new SaslException("Cannot initiate authentication. Insufficient credentials?")); + return future; + }
+ if (saslClient == null)+
+ logger.trace("Initiating SASL exchange.");
+ new AuthenticationOutcomeListener<>(this, connection, RpcType.SASL_MESSAGE, ugi, new RpcOutcomeListener<Void>()Unknown macro: { + + @Override + public void failed(RpcException ex) { + settableFuture.setException(ex); + } + + @Override + public void success(Void value, ByteBuf buffer) { + authComplete = true; + settableFuture.set(null); + } + + @Override + public void interrupted(InterruptedException e) { + settableFuture.setException(e); + } + })
+ .initiate(mechanismName);-
- End diff –
-
please move this up.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99256160
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java —
@@ -89,14 +89,42 @@ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
}
@Override
- protected Response handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(connection, rpcType, pBody, dBody);
+ protected void handle(ControlConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException { + connection.getCurrentHandler().handle(connection, rpcType, pBody, dBody, sender); }
@Override
protected void validateHandshake(BitControlHandshake handshake) throws RpcException {
if (handshake.getRpcVersion() != ControlRpcConfig.RPC_VERSION)
+
+ if (handshake.getAuthenticationMechanismsCount() != 0) { // remote requires authentication
— End diff –
I think we should also have a check for the case if Authentication is enabled on this *client* and for some reason *server* is sending empty list of mechanisms list (may be wrong config) then we should throw exception.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99256832
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControllerImpl.java —
@@ -39,36 +39,33 @@
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ControllerImpl.class);
private volatile ControlServer server;
- private final ControlMessageHandler handler;
private final BootStrapContext context;
private final ConnectionManagerRegistry connectionRegistry; - private final boolean allowPortHunting;
private final CustomHandlerRegistry handlerRegistry;
+ private final BitConnectionConfigImpl config;
- public ControllerImpl(BootStrapContext context, ControlMessageHandler handler, BufferAllocator allocator,
- boolean allowPortHunting) {
- super();
- this.handler = handler;
+ public ControllerImpl(BootStrapContext context, BufferAllocator allocator, ControlMessageHandler handler) { this.context = context; - this.connectionRegistry = new ConnectionManagerRegistry(allocator, handler, context); - this.allowPortHunting = allowPortHunting; + config = new BitConnectionConfigImpl(allocator, context, handler); + this.connectionRegistry = new ConnectionManagerRegistry(config); this.handlerRegistry = handler.getHandlerRegistry(); }
@Override
- public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws DrillbitStartupException {
- server = new ControlServer(handler, context, connectionRegistry);
+ public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint, final boolean allowPortHunting)
+ throws DrillbitStartupException {-
- End diff –
-
nothing inside the constructor throws *`DrillbitStartupException`*
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99257914
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionManager.java —
@@ -21,30 +21,28 @@
import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.UserBitShared.RpcChannel;
import org.apache.drill.exec.rpc.ReconnectingConnection;
-import org.apache.drill.exec.server.BootStrapContext;
public class DataConnectionManager extends ReconnectingConnection<DataClientConnection, BitClientHandshake>{
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataConnectionManager.class);
— End diff –
Please remove this. There are few other places too with this commented lines.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99260686
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,175 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
+
+ private static final String UTF_8_NULL = "\u0000";
+
+ public static final String MECHANISM_NAME = "PLAIN";
+
+ public static class PlainServerFactory implements SaslServerFactory {
+
+ @Override
+ public SaslServer createSaslServer(final String mechanism, final String protocol, final String serverName,
+ final Map<String, ?> props, final CallbackHandler cbh)
+ throws SaslException
+
+ @Override
+ public String[] getMechanismNames(final Map<String, ?> props) {
+ return props == null || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT))
— End diff –
we should change this check as well like above to be consistent. Since right now if `props` is not null and POLICY_NOPLAINTEXT property is absent then we will return empty string.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99257539
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java —
@@ -37,40 +37,28 @@
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataConnectionCreator.class);
private volatile DataServer server;
- private final BootStrapContext context;
- private final WorkEventBus workBus;
- private final WorkerBee bee;
- private final boolean allowPortHunting;
- private ConcurrentMap<DrillbitEndpoint, DataConnectionManager> connectionManager = Maps.newConcurrentMap();
- private final BufferAllocator dataAllocator;
+ private final ConcurrentMap<DrillbitEndpoint, DataConnectionManager> connectionManager = Maps.newConcurrentMap();
- public DataConnectionCreator(
- BootStrapContext context,
- BufferAllocator allocator,
- WorkEventBus workBus,
- WorkerBee bee,
- boolean allowPortHunting) {
- super();
- this.context = context;
- this.workBus = workBus;
- this.bee = bee;
- this.allowPortHunting = allowPortHunting;
- this.dataAllocator = allocator;
+ private final BitConnectionConfigImpl config;
+
+ public DataConnectionCreator(BootStrapContext context, BufferAllocator allocator, WorkEventBus workBus,
+ WorkerBee bee) { + config = new BitConnectionConfigImpl(allocator, context, new DataServerRequestHandler(workBus, bee)); }
- public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws DrillbitStartupException {
- server = new DataServer(context, dataAllocator, workBus, bee);
+ public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint, boolean allowPortHunting)
+ throws DrillbitStartupException {-
- End diff –
-
Same as ControlServer
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r99247679
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ConnectionManagerRegistry.java —
@@ -32,24 +29,19 @@
private final ConcurrentMap<DrillbitEndpoint, ControlConnectionManager> registry = Maps.newConcurrentMap();
- private final ControlMessageHandler handler;
- private final BootStrapContext context;
- private volatile DrillbitEndpoint localEndpoint;
- private final BufferAllocator allocator;
+ private final BitConnectionConfigImpl config;
- public ConnectionManagerRegistry(BufferAllocator allocator, ControlMessageHandler handler, BootStrapContext context) {
- super();
- this.handler = handler;
- this.context = context;
- this.allocator = allocator;
+ public ConnectionManagerRegistry(BitConnectionConfigImpl config) { + this.config = config; }
- public ControlConnectionManager getConnectionManager(DrillbitEndpoint endpoint) {
- assert localEndpoint != null : "DrillbitEndpoint must be set before a connection manager can be retrieved";
- ControlConnectionManager m = registry.get(endpoint);
+ public ControlConnectionManager getConnectionManager(DrillbitEndpoint remoteEndpoint) {
+ assert config.getLocalEndpoint() != null :
+ "DrillbitEndpoint must be set before a connection manager can be retrieved";
+ ControlConnectionManager m = registry.get(remoteEndpoint);
if (m == null) { - m = new ControlConnectionManager(allocator, endpoint, localEndpoint, handler, context);
- ControlConnectionManager m2 = registry.putIfAbsent(endpoint, m);
+ m = new ControlConnectionManager(config, remoteEndpoint);
+ ControlConnectionManager m2 = registry.putIfAbsent(remoteEndpoint, m);-
- End diff –
-
can be final.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r98513830
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
+
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 1;
— End diff –
My bad, it should be 0. It was 0 but I changed it.
Github user bitblender commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r98512248
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
+
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 1;
— End diff –
Why does this index start from 1 ?
Github user bitblender commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r97848365
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
— End diff –
This is incorrect. s_initialized, set to true, is published for other threads to observe, even before the initialization is complete. This should be moved to the end of initialization.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94700671
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserAuthenticationUtil.java —
@@ -0,0 +1,255 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.ConnectionParameters;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.annotation.Nullable;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.AccessController;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Set;
+
+public final class UserAuthenticationUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserAuthenticationUtil.class);
+
+ private static final String PLAIN_MECHANISM = "PLAIN";
+
+ private static final String DEFAULT_SERVICE_NAME = System.getProperty("service.name.primary", "drill");
+
+ private static final String DEFAULT_REALM_NAME = System.getProperty("service.name.realm", "default");
+
+ public enum ClientAuthenticationProvider {
+
+ KERBEROS {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
+ UserGroupInformation.AuthenticationMethod.KERBEROS.toString());
+ UserGroupInformation.setConfiguration(conf);
+
+ final String keytab = parameters.getParameter(ConnectionParameters.KEYTAB);
+ final boolean assumeSubject = parameters.getParameter(ConnectionParameters.KERBEROS_FROM_SUBJECT) != null &&
+ Boolean.parseBoolean(parameters.getParameter(ConnectionParameters.KERBEROS_FROM_SUBJECT));
+ try {
+ final UserGroupInformation ugi;
+ if (assumeSubject) {
+ ugi = UserGroupInformation.getUGIFromSubject(Subject.getSubject(AccessController.getContext()));
+ logger.debug("Assuming subject for {}.", ugi.getShortUserName());
+ } else {
+ if (keytab != null) {
+ ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+ parameters.getParameter(ConnectionParameters.USER), keytab);
+ logger.debug("Logged in {} using keytab.", ugi.getShortUserName());
+ } else {
+ // includes Kerberos ticket login
+ ugi = UserGroupInformation.getCurrentUser();
+ logger.debug("Logged in {} using ticket.", ugi.getShortUserName());
+ }
+ }
+ return ugi;
+ } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login.", cause);
+ }
+ }
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi,
+ final ConnectionParameters parameters) throws SaslException {
+ final String servicePrincipal = getServicePrincipal(parameters);
+
+ final String parts[] = KerberosUtil.splitPrincipalIntoParts(servicePrincipal);
+ final String serviceName = parts[0];
+ final String serviceHostName = parts[1];
+ // ignore parts[2]; GSSAPI gets the realm info from the ticket
+ try {
+ final SaslClient saslClient = ugi.doAs(new PrivilegedExceptionAction<SaslClient>() {
+
+ @Override
+ public SaslClient run() throws Exception {
+ return Sasl.createSaslClient(new String[]{KerberosUtil.KERBEROS_SASL_NAME},
+ null /** authorization ID */, serviceName, serviceHostName,
+ null /** properties; default QOP is auth */, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks)
+ throws IOException, UnsupportedCallbackException { + throw new UnsupportedCallbackException(callbacks[0]); + }
+ });
+ }
+ });
+ logger.debug("GSSAPI SaslClient created to authenticate to {} running on {}",
+ serviceName, serviceHostName);
+ return saslClient;
+ } catch (final UndeclaredThrowableException e) { + throw new SaslException(String.format("Unexpected failure trying to authenticate to %s using GSSAPI", + serviceHostName), e.getCause()); + } catch (final IOException | InterruptedException e) {
+ if (e instanceof SaslException) { + throw (SaslException) e; + }
+ throw new SaslException(String.format("Unexpected failure trying to authenticate to %s using GSSAPI",
+ serviceHostName), e);
+ }
+ }
+ },
+
+ PLAIN {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ try { + return UserGroupInformation.getCurrentUser(); + } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException) { + throw new SaslException("Failed to login.", cause); + }
+ throw new SaslException("Unexpected failure trying to login. ", cause);
+ }
+ }
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi,
+ final ConnectionParameters parameters) throws SaslException {
+ final String userName = parameters.getParameter(ConnectionParameters.USER);
+ final String password = parameters.getParameter(ConnectionParameters.PASSWORD);
+
+ return Sasl.createSaslClient(new String[]
, null /** authorization ID */,
+ null, null, null /** properties; default QOP is auth */, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
+ if (callback instanceof PasswordCallback)
{ + PasswordCallback.class.cast(callback).setPassword(password.toCharArray()); + continue; + } + throw new UnsupportedCallbackException(callback);
+ }
+ }
+ });
+ }
+ };
+
+ public abstract UserGroupInformation login(ConnectionParameters parameters) throws SaslException;
+
+ public abstract SaslClient createSaslClient(UserGroupInformation ugi, ConnectionParameters parameters)
+ throws SaslException;
+
+ }
+
+ public static ClientAuthenticationProvider getClientAuthenticationProvider(
+ final ConnectionParameters parameters, final List<String> supportedAuthMechanisms) throws SaslException {
+ // canonicalization
+ final Set<String> supportedAuthMechanismSet = ImmutableSet.copyOf(
+ Iterators.transform(supportedAuthMechanisms.iterator(), new Function<String, String>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable String input)
+ }));
+
+ // first, check if a certain mechanism must be used
+ String authMechanism = parameters.getParameter(ConnectionParameters.AUTH_MECHANISM);
+ if (authMechanism != null) {
+ authMechanism = authMechanism.toUpperCase();
+ final ClientAuthenticationProvider authenticator;
+ try
catch (final IllegalArgumentException e)
{ + throw new SaslException(String.format("Unknown mechanism: %s", authMechanism)); + } +
+ if (!supportedAuthMechanismSet.contains(authMechanism))
+ return authenticator;
+ }
+
+ // check if Kerberos is supported, and the service principal is provided
+ if (supportedAuthMechanismSet.contains(KerberosUtil.KERBEROS_SIMPLE_NAME) &&
+ parameters.getParameter(ConnectionParameters.SERVICE_PRINCIPAL) != null)
+
+ // check if username/password is supported, and username/password are provided
+ if (supportedAuthMechanismSet.contains(PLAIN_MECHANISM) &&
+ parameters.getParameter(ConnectionParameters.USER) != null &&
+ !Strings.isNullOrEmpty(parameters.getParameter(ConnectionParameters.PASSWORD)))
+
+ throw new SaslException(String.format("Server requires authentication using %s. Insufficient credentials?",
— End diff –
Exception message - Insufficient parameters (instead of credentials) in connection strings ?
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94880411
— Diff: contrib/native/client/src/clientlib/drillClientImpl.hpp —
@@ -522,6 +534,13 @@ class DrillClientImpl : public DrillClientImplBase{
exec::user::RpcEndpointInfos m_serverInfos;
bool m_bIsConnected;
+ std::vector<std::string> m_mechanisms;
— End diff –
Please change it to be "m_supportedMechanisms" as in Java client or "m_serverMechanisms" since these are list of mechanisms supported by server not client.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94866946
— Diff: exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java —
@@ -17,7 +17,10 @@
*/
package org.apache.drill.exec.server;
+import com.google.common.collect.Lists;
+import com.typesafe.config.ConfigValueFactory;
import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.config.ConnectionParameters;
import org.apache.drill.common.config.DrillConfig;
— End diff –
It would be great to add negative. Some of which I can think of are as below. Not sure if they are already covered.
1) Server not configured for any mechanism.
2) Server configured for mechanism but client requesting some other mechanism for authentication
3) Client getting service ticket for one host but sending message to different host.
4) Client trying to authenticate with kdc using wrong creds.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94862522
— Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/TestSaslExchange.java —
@@ -0,0 +1,63 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.collect.Lists;
+import com.typesafe.config.ConfigValueFactory;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.config.ConnectionParameters;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.security.Security;
+import java.util.Properties;
+
+public class TestSaslExchange extends BaseTestQuery {
+
+ @BeforeClass
+ public static void setup() {
+ Security.addProvider(new SimpleProvider());
+ FastSaslServerFactory.reload();
+
+ final Properties props = cloneDefaultTestConfigProperties();
+ final DrillConfig newConfig = new DrillConfig(DrillConfig.create(props)
+ .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
+ ConfigValueFactory.fromAnyRef("true"))
+ .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
+ ConfigValueFactory.fromIterable(Lists.newArrayList(SimpleMechanism.MECHANISM_NAME))),
+ false);
+
+ final Properties connectionProps = new Properties();
+ connectionProps.setProperty(ConnectionParameters.PASSWORD, "anything works!");
+ updateTestCluster(3, newConfig, connectionProps);
— End diff –
How does client knows to instantiate "SimpleMechanism" in this case? I don't see getClientAuthenticationHandler generating authHandler for "SimpleMechanism". Also we are not setting "auth" in connection string.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94879981
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
+
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
— End diff –
It will help if we can put a comment here that sasl_client_init loads all the available mechanism and factories in the sasl_lib referenced by the path
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94825267
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClientAuthenticationHandler.java —
@@ -0,0 +1,229 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.protobuf.ByteString;
+import io.netty.buffer.ByteBuf;
+import org.apache.drill.exec.proto.UserProtos.RpcType;
+import org.apache.drill.exec.proto.UserProtos.SaslMessage;
+import org.apache.drill.exec.proto.UserProtos.SaslStatus;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+// package private
+class UserClientAuthenticationHandler implements RpcOutcomeListener<SaslMessage> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(UserClientAuthenticationHandler.class);
+
+ private static final ImmutableMap<SaslStatus, SaslChallengeProcessor> CHALLENGE_PROCESSORS =
+ ImmutableMap.<SaslStatus, SaslChallengeProcessor>builder()
+ .put(SaslStatus.SASL_IN_PROGRESS, new SaslInProgressProcessor())
+ .put(SaslStatus.SASL_SUCCESS, new SaslSuccessProcessor())
+ .put(SaslStatus.SASL_FAILED, new SaslFailedProcessor())
+ .build();
+
+ private final UserClient client;
+ private final UserGroupInformation ugi;
+ private final SettableFuture<Void> settableFuture;
+
+ public UserClientAuthenticationHandler(UserClient client, UserGroupInformation ugi,
+ SettableFuture<Void> settableFuture)
+
+ public void initiate(final String mechanismName) {
+ try {
+ final ByteString responseData;
+ final SaslClient saslClient = client.getSaslClient();
+ if (saslClient.hasInitialResponse())
else
{ + responseData = ByteString.EMPTY; + } + client.send(new UserClientAuthenticationHandler(client, ugi, settableFuture),
+ RpcType.SASL_MESSAGE,
+ SaslMessage.newBuilder()
+ .setMechanism(mechanismName)
+ .setStatus(SaslStatus.SASL_START)
+ .setData(responseData)
+ .build(),
+ SaslMessage.class);
+ logger.trace("Initiated SASL exchange.");
+ } catch (final Exception e)
+ }
+
+ @Override
+ public void failed(RpcException ex)
+
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ final SaslChallengeProcessor processor = CHALLENGE_PROCESSORS.get(value.getStatus());
+ if (processor == null)
else {
+ try {
+ final SaslChallengeContext context =
+ new SaslChallengeContext(value, client.getSaslClient(), ugi, settableFuture);
+
+ final SaslMessage saslResponse = processor.process(context);
+
+ if (saslResponse != null)
else
{ + // success + client.disposeSaslClient(); + settableFuture.set(null); + } + } catch (final Exception e) {
+ try
catch (Exception ignored)
{ + //ignored + } + settableFuture.setException(e);
+ }
+ }
+ }
+
+ @Override
+ public void interrupted(InterruptedException e)
+
+ private static class SaslChallengeContext {
+
+ final SaslMessage challenge;
+ final SaslClient saslClient;
+ final UserGroupInformation ugi;
+ final SettableFuture<Void> settableFuture;
+
+ public SaslChallengeContext(SaslMessage challenge, SaslClient saslClient, UserGroupInformation ugi,
+ SettableFuture<Void> settableFuture)
+ }
+
+ private interface SaslChallengeProcessor
+
+ private static class SaslInProgressProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+
+ final byte[] responseBytes = evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray());
+
+ final boolean isComplete = context.saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_SUCCESS : SaslStatus.SASL_IN_PROGRESS);
+ return response.build();
+ }
+ }
+
+ private static class SaslSuccessProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ return null;
+ } else {
+
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(context.ugi, context.saslClient,
+ context.challenge.getData().toByteArray()); // discard response
+
+ if (context.saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", context.saslClient.getMechanismName());
+ return null;
+ } else
+ }
+ }
+ }
+
+ private static class SaslFailedProcessor implements SaslChallengeProcessor {
+
+ @Override
+ public SaslMessage process(SaslChallengeContext context) throws Exception {
+ throw new SaslException("Authentication failed. Incorrect credentials?");
— End diff –
I see SASL_FAILED status code can be send in multiple scenario like InvalidByteBuff (can happen over network / bad client), null processor (which again can be because of above 2 reasons if message is tampered) and in case when actually SASL exchange failed. First 2 cases doesn't look like related to bad creds. Is it possible to segregate those and throw proper exception? This can help client to know more about the problem rather than struggling with creds which might not be an actual issue.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94864855
— Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestCustomUserAuthenticator.java —
@@ -56,16 +68,27 @@ public void positiveUserAuth() throws Exception
-
@Test
public void negativeUserAuth() throws Exception
@Test
+ public void emptyPassword() throws Exception {
+ try
catch (RpcException e) {
+ final String exMsg = e.getMessage();
+ assertThat(exMsg, containsString("Insufficient credentials"));
— End diff –
We should check the "cause" rather than message string since that can change over time ? Same in negativeAuthHelper.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94880079
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
+
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath()) {
— End diff –
What happens if sasl plugin path is not provided ? As per sas_client_init doc it will return the result code like below. We should handle the return code properly.
/* initialize the SASL client drivers
- callbacks – base callbacks for all client connections
- returns:
- SASL_OK – Success
- SASL_NOMEM – Not enough memory
- SASL_BADVERS – Mechanism version mismatch
- SASL_BADPARAM – error in config file
- SASL_NOMECH – No mechanisms available
- ...
*/
int sasl_client_init(const sasl_callback_t *callbacks)
{
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94849762
— Diff: protocol/src/main/protobuf/User.proto —
@@ -111,6 +115,21 @@ message BitToUserHandshake {
optional string errorId = 4;
optional string errorMessage = 5;
optional RpcEndpointInfos server_infos = 6;
+ repeated string authenticationMechanisms = 7;
— End diff –
shouldn't this be optional ? I am not sure when new server builds this message and send back to older client then how this field is ignored on client side without being optional.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94867052
— Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestKerberosSaslAuthentication.java —
@@ -0,0 +1,239 @@
+/**
+ * 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.drill.exec.rpc.user.security;
+
+import com.google.common.collect.Lists;
+import com.typesafe.config.ConfigValueFactory;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.config.ConnectionParameters;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl;
+import org.apache.drill.exec.security.impl.LoginManagerImpl;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.hadoop.security.authentication.util.KerberosUtil;
+import org.apache.kerby.kerberos.kerb.KrbException;
+import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import sun.security.krb5.Config;
+
+import javax.security.auth.Subject;
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.net.ServerSocket;
+import java.nio.file.Files;
+import java.security.PrivilegedExceptionAction;
+import java.util.Properties;
+
+@Ignore("Expects users to exist. Set SERVER_SHORT_NAME to current user name to run the tests.")
+public class TestKerberosSaslAuthentication extends BaseTestQuery {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(TestKerberosSaslAuthentication.class);
+
+ private static File workspace;
+
+ private static File kdcDir;
+ private static SimpleKdcServer kdc;
+ private static int kdcPort;
+
+ private static final String HOSTNAME = "localhost";
+ private static final String REALM = "EXAMPLE.COM";
+
+ private static final String CLIENT_SHORT_NAME = "client";
+ private static final String CLIENT_PRINCIPAL = CLIENT_SHORT_NAME + "@" + REALM;
+ private static final String SERVER_SHORT_NAME = "server";
+ private static final String SERVER_PRINCIPAL = SERVER_SHORT_NAME + "/" + HOSTNAME + "@" + REALM;
+
+ private static File keytabDir;
+ private static File clientKeytab;
+ private static File serverKeytab;
+
+ private static boolean kdcStarted;
+
+ @BeforeClass
+ public static void setupKdc() throws Exception {
+ kdc = new SimpleKdcServer();
+ workspace = new File(getTempDir("kerberos_target"));
+
+ kdcDir = new File(workspace, TestKerberosSaslAuthentication.class.getSimpleName());
+ kdcDir.mkdirs();
+ kdc.setWorkDir(kdcDir);
+
+ kdc.setKdcHost(HOSTNAME);
+ kdcPort = getFreePort();
+ kdc.setAllowTcp(true);
+ kdc.setAllowUdp(false);
+ kdc.setKdcTcpPort(kdcPort);
+
+ logger.debug("Starting KDC server at {}:{}", HOSTNAME, kdcPort);
+
+ kdc.init();
+ kdc.start();
+ kdcStarted = true;
+
+
+ keytabDir = new File(workspace, TestKerberosSaslAuthentication.class.getSimpleName()
+ + "_keytabs");
+ keytabDir.mkdirs();
+ setupUsers(keytabDir);
+
+ // Kerby sets "java.security.krb5.conf" for us!
+ System.clearProperty("java.security.auth.login.config");
+ System.setProperty("javax.security.auth.useSubjectCredsOnly", "false");
+ // Uncomment the following lines for debugging.
+ // System.setProperty("sun.security.spnego.debug", "true");
+ // System.setProperty("sun.security.krb5.debug", "true");
+
+ // Create a new DrillConfig which has user authentication enabled and authenticator set to
+ // UserAuthenticatorTestImpl.
+ final DrillConfig newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
+ .withValue(ExecConstants.USER_AUTHENTICATION_ENABLED,
+ ConfigValueFactory.fromAnyRef(true))
+ .withValue(ExecConstants.USER_AUTHENTICATOR_IMPL,
+ ConfigValueFactory.fromAnyRef(UserAuthenticatorTestImpl.TYPE))
+ .withValue(LoginManagerImpl.SERVICE_PRINCIPAL,
+ ConfigValueFactory.fromAnyRef(SERVER_PRINCIPAL))
+ .withValue(LoginManagerImpl.SERVICE_KEYTAB_LOCATION,
+ ConfigValueFactory.fromAnyRef(serverKeytab.toString()))
+ .withValue(ExecConstants.AUTHENTICATION_MECHANISMS,
+ ConfigValueFactory.fromIterable(Lists.newArrayList("plain", "kerberos"))),
+ false);
+
+ final Properties connectionProps = new Properties();
+ connectionProps.setProperty(ConnectionParameters.USER, "anonymous");
+ connectionProps.setProperty(ConnectionParameters.PASSWORD, "anything works!");
+
+ // ADD A NOTE EXPLAINING THIS MAGIC
— End diff –
Forget the add note ?
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94696573
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,175 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
+
+ private static final String UTF_8_NULL = "\u0000";
+
+ public static final String MECHANISM_NAME = "PLAIN";
+
+ public static class PlainServerFactory implements SaslServerFactory {
+
+ @Override
+ public SaslServer createSaslServer(final String mechanism, final String protocol, final String serverName,
+ final Map<String, ?> props, final CallbackHandler cbh)
+ throws SaslException {
+ return MECHANISM_NAME.equals(mechanism) ?
— End diff –
please indent the ternary operator properly to make it readable.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94700422
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserAuthenticationUtil.java —
@@ -0,0 +1,255 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.ConnectionParameters;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.annotation.Nullable;
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.AccessController;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Set;
+
+public final class UserAuthenticationUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserAuthenticationUtil.class);
+
+ private static final String PLAIN_MECHANISM = "PLAIN";
+
+ private static final String DEFAULT_SERVICE_NAME = System.getProperty("service.name.primary", "drill");
+
+ private static final String DEFAULT_REALM_NAME = System.getProperty("service.name.realm", "default");
+
+ public enum ClientAuthenticationProvider {
+
+ KERBEROS {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
+ UserGroupInformation.AuthenticationMethod.KERBEROS.toString());
+ UserGroupInformation.setConfiguration(conf);
+
+ final String keytab = parameters.getParameter(ConnectionParameters.KEYTAB);
+ final boolean assumeSubject = parameters.getParameter(ConnectionParameters.KERBEROS_FROM_SUBJECT) != null &&
+ Boolean.parseBoolean(parameters.getParameter(ConnectionParameters.KERBEROS_FROM_SUBJECT));
+ try {
+ final UserGroupInformation ugi;
+ if (assumeSubject) {
+ ugi = UserGroupInformation.getUGIFromSubject(Subject.getSubject(AccessController.getContext()));
+ logger.debug("Assuming subject for {}.", ugi.getShortUserName());
+ } else {
+ if (keytab != null) {
+ ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+ parameters.getParameter(ConnectionParameters.USER), keytab);
+ logger.debug("Logged in {} using keytab.", ugi.getShortUserName());
+ } else {
+ // includes Kerberos ticket login
+ ugi = UserGroupInformation.getCurrentUser();
+ logger.debug("Logged in {} using ticket.", ugi.getShortUserName());
+ }
+ }
+ return ugi;
+ } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login.", cause);
+ }
+ }
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi,
+ final ConnectionParameters parameters) throws SaslException {
+ final String servicePrincipal = getServicePrincipal(parameters);
+
+ final String parts[] = KerberosUtil.splitPrincipalIntoParts(servicePrincipal);
+ final String serviceName = parts[0];
+ final String serviceHostName = parts[1];
+ // ignore parts[2]; GSSAPI gets the realm info from the ticket
+ try {
+ final SaslClient saslClient = ugi.doAs(new PrivilegedExceptionAction<SaslClient>() {
+
+ @Override
+ public SaslClient run() throws Exception {
+ return Sasl.createSaslClient(new String[]{KerberosUtil.KERBEROS_SASL_NAME},
+ null /** authorization ID */, serviceName, serviceHostName,
+ null /** properties; default QOP is auth */, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks)
+ throws IOException, UnsupportedCallbackException { + throw new UnsupportedCallbackException(callbacks[0]); + }
+ });
+ }
+ });
+ logger.debug("GSSAPI SaslClient created to authenticate to {} running on {}",
+ serviceName, serviceHostName);
+ return saslClient;
+ } catch (final UndeclaredThrowableException e) { + throw new SaslException(String.format("Unexpected failure trying to authenticate to %s using GSSAPI", + serviceHostName), e.getCause()); + } catch (final IOException | InterruptedException e) {
+ if (e instanceof SaslException) { + throw (SaslException) e; + }
+ throw new SaslException(String.format("Unexpected failure trying to authenticate to %s using GSSAPI",
+ serviceHostName), e);
+ }
+ }
+ },
+
+ PLAIN {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ try { + return UserGroupInformation.getCurrentUser(); + } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException) { + throw new SaslException("Failed to login.", cause); + }
+ throw new SaslException("Unexpected failure trying to login. ", cause);
+ }
+ }
+
+ @Override
+ public SaslClient createSaslClient(final UserGroupInformation ugi,
+ final ConnectionParameters parameters) throws SaslException {
+ final String userName = parameters.getParameter(ConnectionParameters.USER);
+ final String password = parameters.getParameter(ConnectionParameters.PASSWORD);
+
+ return Sasl.createSaslClient(new String[]
, null /** authorization ID */,
+ null, null, null /** properties; default QOP is auth */, new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
+ if (callback instanceof PasswordCallback)
{ + PasswordCallback.class.cast(callback).setPassword(password.toCharArray()); + continue; + } + throw new UnsupportedCallbackException(callback);
+ }
+ }
+ });
+ }
+ };
+
+ public abstract UserGroupInformation login(ConnectionParameters parameters) throws SaslException;
+
+ public abstract SaslClient createSaslClient(UserGroupInformation ugi, ConnectionParameters parameters)
— End diff –
Can we change the signature to include input param - (Map<String, ?> properties) as in SaslServer ? For authentication case it will be null from caller. But for encryption it will contain proper QOP parameter which will be passed from caller.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94701486
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -246,28 +163,78 @@ protected void handle(UserClientConnectionImpl connection, int rpcType, ByteBuf
public class UserClientConnectionImpl extends RemoteConnection implements UserClientConnection {
private UserSession session;
+ private SaslServer saslServer;
+ private RequestHandler<UserClientConnectionImpl> currentHandler;
+ private UserToBitHandshake inbound;
public UserClientConnectionImpl(SocketChannel channel)
{ super(channel, "user client"); + currentHandler = authFactory == null ? handler : new UserServerAuthenticationHandler(handler, loginManager); }void disableReadTimeout()
{ getChannel().pipeline().remove(BasicServer.TIMEOUT_HANDLER); }- void setUser(final UserToBitHandshake inbound) throws IOException {
+ void setHandshake(final UserToBitHandshake inbound) { + this.inbound = inbound; + }+
{ + throw new IllegalStateException("SASL server already initialized."); + }
+ void initSaslServer(final String mechanismName, final Map<String, ?> properties)
+ throws IllegalStateException, IllegalArgumentException, SaslException {
+ if (saslServer != null)+
+ this.saslServer = authFactory.getMechanism(mechanismName)
+ .createSaslServer(properties);
+ if (saslServer == null) {
+ throw new SaslException("Server could not initiate authentication. Insufficient parameters?");-
- End diff –
-
Shouldn't the exception message here be "Failed to initialize Sasl Server", since that's what the function is doing ?
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94881310
— Diff: contrib/native/client/src/clientlib/saslAuthenticatorImpl.cpp —
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+#include <vector>
+#include <boost/algorithm/string.hpp>
+#include <boost/assign.hpp>
+#include "saslAuthenticatorImpl.hpp"
+
+#include "drillClientImpl.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+
+#define DEFAULT_SERVICE_NAME "drill"
+
+#define KERBEROS_SIMPLE_NAME "kerberos"
+#define KERBEROS_SASL_NAME "gssapi"
+#define PLAIN_NAME "plain"
+
+const std::map<std::string, std::string> SaslAuthenticatorImpl::MECHANISM_MAPPING = boost::assign::map_list_of
+ (KERBEROS_SIMPLE_NAME, KERBEROS_SASL_NAME)
+ (PLAIN_NAME, PLAIN_NAME)
+;
+
+boost::mutex SaslAuthenticatorImpl::s_mutex;
+bool SaslAuthenticatorImpl::s_initialized = false;
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL) {
+
+ if (!s_initialized) {
+ boost::lock_guard<boost::mutex> lock(SaslAuthenticatorImpl::s_mutex);
+ if (!s_initialized) {
+ s_initialized = true;
+
+ // set plugin path if provided
+ if (DrillClientConfig::getSaslPluginPath())
+
+ sasl_client_init(NULL);
+ { // for debugging purposes
+ const char **mechanisms = sasl_global_listmech();
+ int i = 1;
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "SASL mechanisms available on client: " << std::endl
+ while (mechanisms[i] != NULL)
+ }
+ }
+ }
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be used to negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
— End diff –
Is memory for "context" object released during "sasl_dispose" of m_pConnection ?
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94689336
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,62 @@
+/**
+ * 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.drill.common;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.base.Preconditions.checkState;
+
+public final class KerberosUtil {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
— End diff –
Please remove the commented line.
Github user sohami commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r94695872
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,175 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
— End diff –
Remove this
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r86268011
— Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/SimpleServer.java —
@@ -0,0 +1,138 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.primitives.Ints;
+
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.util.Map;
+
+public class SimpleServer implements SaslServer {
+
+ private boolean completed;
+ private String authorizationId;
+ private final int total;
+ private int count = 0;
+
+ SimpleServer(final int total)
+
+ @Override
+ public String getMechanismName()
+
+ @Override
+ public byte[] evaluateResponse(byte[] response) throws SaslException {
+ if (completed)
+ if (response == null || response.length < 1)
{ + throw new SaslException("Received challenge is empty when secret expected"); + } +
+ if (count == 0) { // first expect authorization ID
+ //This SaslServer simply permits a client to authenticate according to whatever username
+ //was supplied in client's response[]
+ authorizationId = new String(response);
— End diff –
Fixed.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r86267336
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -246,28 +154,80 @@ protected void handle(UserClientConnectionImpl connection, int rpcType, ByteBuf
public class UserClientConnectionImpl extends RemoteConnection implements UserClientConnection {
private UserSession session;
+ private SaslServer saslServer;
+ private RequestHandler<UserClientConnectionImpl> currentHandler;
+ private UserToBitHandshake inbound;
public UserClientConnectionImpl(SocketChannel channel)
{ super(channel, "user client"); + currentHandler = authFactory == null ? handler : new UserServerAuthenticationHandler(handler); }void disableReadTimeout()
{ getChannel().pipeline().remove(BasicServer.TIMEOUT_HANDLER); }- void setUser(final UserToBitHandshake inbound) throws IOException {
+ void setHandshake(final UserToBitHandshake inbound) throws IOException { + this.inbound = inbound; + }+
{ + throw new IllegalStateException("SASL server already initialized."); + }
+ void initSaslServer(final String mechanismName, final Map<String, ?> properties)
+ throws IllegalStateException, SaslException {
+ if (saslServer != null)+ this.saslServer = authFactory.getMechanism(mechanismName)
-
- End diff –
-
Makes sense as an enhancement. The current properties are for SASL library.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r86267130
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -308,33 +312,57 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
connection.setHandshake(inbound);
try {
+ // TODO(SUDHEESH): MUST FIX THIS VERSION CHECK FIRST BEFORE THE CHECK BELOW
if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
- if (authenticator != null) {
- try {
- String password = "";
- final UserProperties props = inbound.getProperties();
- for (int i = 0; i < props.getPropertiesCount(); i++) {
- Property prop = props.getProperties;
- if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) {
- password = prop.getValue();
- break;
+ connection.setHandshake(inbound);
+
+ if (authFactory != null) {
+ if (inbound.getRpcVersion() <= 5) { // for backward compatibility <= 1.8
+ final String userName = inbound.getCredentials().getUserName();
+ if (logger.isTraceEnabled()) {
+ logger.trace("User {} on connection {} is using an older client (Drill version <= 1.8).",
+ userName, connection.getRemoteAddress());
+ }
+ try {
+ String password = "";
+ final UserProperties props = inbound.getProperties();
+ for (int i = 0; i < props.getPropertiesCount(); i++)Unknown macro: { + Property prop = props.getProperties(i); + if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) { + password = prop.getValue(); + break; + } + }+ final PlainMechanism plainMechanism = authFactory.getPlainMechanism();
{ + throw new UserAuthenticationException("The server no longer supports username/password" + + " based authentication. Please talk to your system administrator."); }
+ if (plainMechanism == null)+ plainMechanism.getAuthenticator().authenticate(userName, password);
+ connection.changeHandlerTo(handler);
+ connection.finalizeSession(userName);
+ respBuilder.setStatus(HandshakeStatus.SUCCESS);-
- End diff –
-
Successful auth is logged elsewhere.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r86267064
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -308,33 +312,57 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
connection.setHandshake(inbound);
try {
+ // TODO(SUDHEESH): MUST FIX THIS VERSION CHECK FIRST BEFORE THE CHECK BELOW
if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
- if (authenticator != null) {
- try {
- String password = "";
- final UserProperties props = inbound.getProperties();
- for (int i = 0; i < props.getPropertiesCount(); i++) {
- Property prop = props.getProperties;
- if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) {
- password = prop.getValue();
- break;
+ connection.setHandshake(inbound);
+
+ if (authFactory != null) {
+ if (inbound.getRpcVersion() <= 5) { // for backward compatibility <= 1.8-
- End diff –
-
Resolved all compatibility problems.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r86267015
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,182 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.kerberos.KerberosMechanism;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ private final Map<String, AuthenticationMechanism> mechanisms = CaseInsensitiveMap.newHashMapWithExpectedSize(5);
+
+ @SuppressWarnings("unchecked")
+ public AuthenticationMechanismFactory(final ScanResult scan, final DrillConfig config,
+ final List<String> configuredMechanisms)
+ throws DrillbitStartupException {
+ logger.debug("Configuring authentication mechanisms: {}", configuredMechanisms);
+ // transform all names to uppercase
+ final Set<String> configuredMechanismsSet = Sets.newHashSet(Iterators.transform(configuredMechanisms.iterator(),
+ new Function<String, String>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable String input)
+ }));
+
+ // First, load Drill provided out-of-box mechanisms
+ if (configuredMechanismsSet.contains(PlainMechanism.SIMPLE_NAME))
+
+ if (configuredMechanismsSet.contains(KerberosMechanism.SIMPLE_NAME)) {
+ logger.trace("Kerberos mechanism enabled.");
+ final String servicePrincipal = config.getString("drill.exec.security.auth.principal");
+ final String keytab = config.getString("drill.exec.security.auth.keytab");
+
+ try { // Kerberos mechanism requires a service to login
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "KERBEROS");
+ // To parse non-typical principal name, uncomment below line
+ // CommonConfigurationKeys.HADOOP_SECURITY_AUTH_TO_LOCAL, rules);
+ UserGroupInformation.setConfiguration(conf);
+ UserGroupInformation.loginUserFromKeytab(servicePrincipal, keytab);
+ logger.trace("Login successful for user: {}", servicePrincipal);
+ } catch (IOException e) {
+ throw new DrillbitStartupException("Drillbit service login failed", e);
— End diff –
The code is now moved to LoginManagerImpl.
Drill as a service has to login to Kerberos (AS), so this failure condition is required.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853861
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -246,28 +160,75 @@ protected void handle(UserClientConnectionImpl connection, int rpcType, ByteBuf
public class UserClientConnectionImpl extends RemoteConnection implements UserClientConnection {
private UserSession session;
+ private SaslServer saslServer;
+ private RequestHandler<UserClientConnectionImpl> currentHandler;
— End diff –
No, the handler on a connection can change. See #changeHandlerTo.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853933
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -308,33 +312,57 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
connection.setHandshake(inbound);
— End diff –
Fixed.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85876652
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServerAuthenticationHandler.java —
@@ -0,0 +1,194 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import org.apache.drill.exec.proto.UserProtos.RpcType;
+import org.apache.drill.exec.proto.UserProtos.SaslMessage;
+import org.apache.drill.exec.proto.UserProtos.SaslStatus;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.ResponseSender;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RequestHandler;
+import org.apache.drill.exec.security.LoginManager;
+
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+
+// package private
+class UserServerAuthenticationHandler implements RequestHandler<UserServer.UserClientConnectionImpl> {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(UserServerAuthenticationHandler.class);
+
+ private final UserServerRequestHandler requestHandler;
+ private final LoginManager loginManager;
+
+ public UserServerAuthenticationHandler(final UserServerRequestHandler requestHandler,
+ final LoginManager loginManager)
+
+ @Override
+ public void handle(UserServer.UserClientConnectionImpl connection, int rpcType, ByteBuf pBody, ByteBuf dBody,
+ ResponseSender sender) throws RpcException {
+ final String remoteAddress = connection.getRemoteAddress().toString();
+ switch (rpcType) {
— End diff –
Refactored this `switch case` statement.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85854058
— Diff: protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java —
@@ -47,157 +47,165 @@ public static void registerAllExtensions(
*/
GOODBYE(2, 2),
/**
+ * <code>SASL_MESSAGE = 24;</code>
+ *
+ * <pre>
+ * user to bit and bit to user
+ * </pre>
+ */
+ SASL_MESSAGE(3, 24),
— End diff –
None. Will move to end.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85852510
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,169 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.drill.exec.security.LoginManager;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
+
+ // Mapping: SASL name -> mechanism
+ // See AuthenticationMechanism#getMechanismName
+ private final Map<String, AuthenticationMechanism> mechanisms = CaseInsensitiveMap.newHashMapWithExpectedSize(5);
+
+ @SuppressWarnings("unchecked")
+ public AuthenticationMechanismFactory(final DrillConfig config, final ScanResult scan,
+ final LoginManager loginManager) throws DrillbitStartupException {
+ if (!config.hasPath(ExecConstants.AUTHENTICATION_MECHANISMS))
+
+ final List<String> configuredMechanisms = config.getStringList(ExecConstants.AUTHENTICATION_MECHANISMS);
+ logger.debug("Configuring authentication mechanisms: {}", configuredMechanisms);
+ // transform all names to uppercase
+ final Set<String> configuredMechanismsSet = Sets.newHashSet(Iterators.transform(configuredMechanisms.iterator(),
+ new Function<String, String>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable String input)
+ }));
+
+ // PLAIN mechanism need special handling due to UserAuthenticator
+ if (configuredMechanismsSet.contains(PlainMechanism.MECHANISM_NAME)) {
+ // instantiated here, but closed in PlainMechanism#close
+ final UserAuthenticator userAuthenticator = UserAuthenticatorFactory.createAuthenticator(config, scan);
+ final PlainMechanism mechanism = new PlainMechanism(userAuthenticator);
+ mechanisms.put(PlainMechanism.MECHANISM_NAME, mechanism);
+ configuredMechanismsSet.remove(PlainMechanism.MECHANISM_NAME);
— End diff –
Done.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85848455
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,40 @@
+/**
+ * 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.drill.common;
+
+public final class KerberosUtil {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
+
+ public static final String KERBEROS_SASL_NAME = "GSSAPI";
+
+ public static final String KERBEROS_SIMPLE_NAME = "KERBEROS";
+
+ // primary/instance@REALM
+ public static String getPrincipalFromParts(final String primary, final String instance, final String realm)
+
+ // primary/instance@REALM
+ public static String[] splitPrincipalIntoParts(final String principal) {
— End diff –
That's the conventional format for a service principal, as mentioned in the design doc. Will add doc and preconditions.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85852541
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,169 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.drill.exec.security.LoginManager;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
+
+ // Mapping: SASL name -> mechanism
+ // See AuthenticationMechanism#getMechanismName
+ private final Map<String, AuthenticationMechanism> mechanisms = CaseInsensitiveMap.newHashMapWithExpectedSize(5);
+
+ @SuppressWarnings("unchecked")
+ public AuthenticationMechanismFactory(final DrillConfig config, final ScanResult scan,
+ final LoginManager loginManager) throws DrillbitStartupException {
+ if (!config.hasPath(ExecConstants.AUTHENTICATION_MECHANISMS))
+
+ final List<String> configuredMechanisms = config.getStringList(ExecConstants.AUTHENTICATION_MECHANISMS);
+ logger.debug("Configuring authentication mechanisms: {}", configuredMechanisms);
+ // transform all names to uppercase
+ final Set<String> configuredMechanismsSet = Sets.newHashSet(Iterators.transform(configuredMechanisms.iterator(),
+ new Function<String, String>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable String input)
+ }));
+
+ // PLAIN mechanism need special handling due to UserAuthenticator
+ if (configuredMechanismsSet.contains(PlainMechanism.MECHANISM_NAME))
+
+ // Then, load other mechanisms, if any
+ if (!configuredMechanismsSet.isEmpty()) {
+ final Collection<Class<? extends AuthenticationMechanism>> mechanismImpls =
+ scan.getImplementations(AuthenticationMechanism.class);
+ logger.debug("Found AuthenticationMechanism implementations: {}", mechanismImpls);
+
+ for (Class<? extends AuthenticationMechanism> clazz : mechanismImpls) {
+ final SaslMechanism annotation = clazz.getAnnotation(SaslMechanism.class);
+ if (annotation == null) {
+ logger.warn("{} doesn't have {} annotation. Skipping.", clazz.getCanonicalName(), SaslMechanism.class);
+ continue;
+ }
+
+ final String annotatedName = annotation.name();
+ if (Strings.isNullOrEmpty(annotatedName)) {
+ logger.warn("Authentication mechanism {} does not have a proper {} annotation. Skipping.",
— End diff –
Done.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853020
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,174 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
— End diff –
`completed` flag is set to true regardless of failure, which does not follow the #isComplete contract.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85849993
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -88,21 +84,22 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-import com.google.common.util.concurrent.AbstractCheckedFuture;
+
import com.google.common.util.concurrent.SettableFuture;
+import javax.security.sasl.SaslException;
+
/**
- Thin wrapper around a UserClient that handles connect/close and transforms
- String into ByteBuf.
*/
public class DrillClient implements Closeable, ConnectionThrottle {
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class);
- private static final ObjectMapper objectMapper = new ObjectMapper();
+ protected static final ObjectMapper objectMapper = new ObjectMapper();-
- End diff –
-
Will revert.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85848383
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,40 @@
+/**
+ * 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.drill.common;
+
+public final class KerberosUtil {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
+
+ public static final String KERBEROS_SASL_NAME = "GSSAPI";
+
+ public static final String KERBEROS_SIMPLE_NAME = "KERBEROS";
+
+ // primary/instance@REALM
+ public static String getPrincipalFromParts(final String primary, final String instance, final String realm) {
— End diff –
Parameters are expected to be non-null. I'll add doc and preconditions.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85850377
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -591,7 +584,7 @@ public void runQuery(QueryType type, String plan, UserResultsListener resultsLis
client.submitQuery(resultsListener, newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build());
}
- private class ListHoldingResultsListener implements UserResultsListener {
+ protected class ListHoldingResultsListener implements UserResultsListener {-
- End diff –
-
Will revert.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853900
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -299,6 +260,20 @@ public ChannelFuture getChannelClosureFuture() {
public SocketAddress getRemoteAddress()
+
+ @Override
+ public void close() {
+ super.close();
— End diff –
Maybe not. Fixed.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85852461
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -603,33 +596,16 @@ public ListHoldingResultsListener(UserProtos.RunQuery query) {
@Override
public void submissionFailed(UserException ex) {
- // or !client.isActive()
- if (ex.getCause() instanceof ChannelClosedException) {
- if (reconnect()) {
-
- End diff –
-
Will revert.
I think there is a missing VisibleForTesting annotation for runQuery that uses this listener.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853194
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserAuthenticationUtil.java —
@@ -0,0 +1,248 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.ConnectionParameters;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.annotation.Nullable;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Set;
+
+public final class UserAuthenticationUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserAuthenticationUtil.class);
+
+ private static final String PLAIN_MECHANISM = "PLAIN";
+
+ private static final String DEFAULT_SERVICE_NAME = System.getProperty("service.name.primary", "drill");
+
+ private static final String DEFAULT_REALM_NAME = System.getProperty("service.name.realm", "default");
+
+ public enum ClientAuthenticationProvider {
+
+ KERBEROS {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
+ UserGroupInformation.AuthenticationMethod.KERBEROS.toString());
+ UserGroupInformation.setConfiguration(conf);
+
+ final String keytab = parameters.getParameter(ConnectionParameters.KEYTAB);
+ try {
+ final UserGroupInformation ugi;
+ if (keytab != null)
else
{ + // includes Kerberos ticket login + ugi = UserGroupInformation.getCurrentUser(); + logger.debug("Logged in using ticket."); + } + return ugi;
+ } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login: " + cause.getMessage());
— End diff –
Done.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853989
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -328,44 +304,73 @@ protected void consumeHandshake(ChannelHandlerContext ctx, UserToBitHandshake in
public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
logger.trace("Handling handshake from user to bit. {}", inbound);
-
// if timeout is unsupported or is set to false, disable timeout.
if (!inbound.hasSupportTimeout() || !inbound.getSupportTimeout())
- BitToUserHandshake.Builder respBuilder = BitToUserHandshake.newBuilder()
+ final BitToUserHandshake.Builder respBuilder = BitToUserHandshake.newBuilder()
.setRpcVersion(UserRpcConfig.RPC_VERSION);
try {
- if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) {
- final String errMsg = String.format("Invalid rpc version. Expected %d, actual %d.",
- UserRpcConfig.RPC_VERSION, inbound.getRpcVersion());
+ if (!SUPPORTED_RPC_VERSIONS.contains(inbound.getRpcVersion())) { + final String errMsg = String.format("Invalid rpc version. Expected %s, actual %d.", + SUPPORTED_RPC_VERSIONS, inbound.getRpcVersion()); return handleFailure(respBuilder, HandshakeStatus.RPC_VERSION_MISMATCH, errMsg, null); }
- if (authenticator != null) {
+ connection.setHandshake(inbound);
+
+ if (authFactory == null) { // authentication is disabled + connection.finalizeSession(inbound.getCredentials().getUserName()); + respBuilder.setStatus(HandshakeStatus.SUCCESS); + return respBuilder.build(); + }+
+ if (inbound.getRpcVersion() == NON_SASL_RPC_VERSION_SUPPORTED) { // for backward compatibility
+ final String userName = inbound.getCredentials().getUserName();
+ if (logger.isTraceEnabled()) {
+ logger.trace("User {} on connection {} is using an older client (Drill version <= 1.8).",
+ userName, connection.getRemoteAddress());
+ }
try {
String password = "";
final UserProperties props = inbound.getProperties();
for (int i = 0; i < props.getPropertiesCount(); i++) {
Property prop = props.getProperties; - if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey()))
Unknown macro: { + if (ConnectionParameters.PASSWORD.equalsIgnoreCase(prop.getKey())) { password = prop.getValue(); break; } }
- authenticator.authenticate(inbound.getCredentials().getUserName(), password);
+ final PlainMechanism plainMechanism = authFactory.getPlainMechanism();
+ if (plainMechanism == null) { + throw new UserAuthenticationException("The server no longer supports username/password" + + " based authentication. Please talk to your system administrator."); + }+ plainMechanism.getAuthenticator()
+ .authenticate(userName, password);
+ connection.changeHandlerTo(handler);
+ connection.finalizeSession(userName);
+ respBuilder.setStatus(HandshakeStatus.SUCCESS);-
- End diff –
-
I remember making that change. Missed this PR somehow. Fixed.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853162
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,174 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
+
+ public static class PlainServerFactory implements SaslServerFactory {
+
+ @Override
+ public SaslServer createSaslServer(final String mechanism, final String protocol, final String serverName,
+ final Map<String, ?> props, final CallbackHandler cbh)
+ throws SaslException
+
+ @Override
+ public String[] getMechanismNames(final Map<String, ?> props) {
+ return props == null || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT)) ?
+ new String[]
:
+ new String[0];
+ }
+ }
+
+ @SuppressWarnings("serial")
+ public static class PlainServerProvider extends Provider {
+
+ public PlainServerProvider()
+ }
+
+ private CallbackHandler cbh;
+ private boolean completed;
+ private String authorizationID;
+
+ PlainServer(final CallbackHandler cbh) throws SaslException {
+ if (cbh == null)
+ this.cbh = cbh;
+ }
+
+ @Override
+ public String getMechanismName()
+
+ @Override
+ public byte[] evaluateResponse(byte[] response) throws SaslException {
+ if (completed)
+
+ if (response == null)
+
+ completed = true;
+ final String payload;
+ try
catch (final Exception e)
{ + throw new SaslException("Received corrupt response", e); + } +
+ // Separator defined in PlainClient is 0
+ // three parts: [ authorizationID, authenticationID, password ]
+ final String[] parts = payload.split("\u0000", 3);
+ if (parts.length != 3)
+ if (parts[0].isEmpty())
{ + parts[0] = parts[1]; // authorizationID = authenticationID + } +
+ final NameCallback nc = new NameCallback("PLAIN authentication ID: ");
+ nc.setName(parts[1]);
+ final PasswordCallback pc = new PasswordCallback("PLAIN password: ", false);
+ pc.setPassword(parts[2].toCharArray());
+
+ final AuthorizeCallback ac = new AuthorizeCallback(parts[1], parts[0]);
+ try {
+ cbh.handle(new Callback[]
);
+ } catch (final UnsupportedCallbackException | IOException e)
+ authorizationID = ac.getAuthorizedID();
+ return null;
+ }
+
+ @Override
+ public boolean isComplete()
+
+ @Override
+ public String getAuthorizationID() {
+ if (completed)
+ throw new IllegalStateException("PLAIN authentication not completed");
+ }
+
+ @Override
+ public Object getNegotiatedProperty(String propName) {
+ if (completed)
+ throw new IllegalStateException("PLAIN authentication not completed");
+ }
+
+ @Override
+ public byte[] wrap(byte[] outgoing, int offset, int len) throws SaslException {
+ if (completed) {
— End diff –
Per the contract (see [PlainClient](http://www.docjar.com/html/api/com/sun/security/sasl/PlainClient.java.html)).
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853794
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -78,21 +101,241 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, ConnectionParameters parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) - .setCredentials(credentials); + .setCredentials(credentials) + .setProperties(parameters.serializeForServer()); + this.parameters = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
{ + return supportedAuthMechs; + }
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms()+
{@link CheckedFuture#checkedGet results}
+ /**
+ * Authenticate to the server asynchronously. Returns a future that+ * in null if authentication succeeds, or throws a
{@link SaslException}with relevant message if
{@link #connect connection time}
+ * authentication fails.
+ *
+ * This method uses parameters provided atand override them with the
{ + throw new IllegalStateException("Server does not require authentication."); + }
+ * given parameters, if any.
+ *
+ * @param overrides parameter overrides
+ * @return result of authentication request
+ */
+ public CheckedFuture<Void, SaslException> authenticate(final ConnectionParameters overrides) {
+ if (supportedAuthMechs == null)+ parameters.merge(overrides);
{ + connection.close(); // to ensure connection is dropped + }
+
+ final SettableFuture<Void> settableFuture = SettableFuture.create(); // future used in SASL exchange
+ final CheckedFuture<Void, SaslException> future =
+ new AbstractCheckedFuture<Void, SaslException>(settableFuture) {
+
+ @Override
+ protected SaslException mapException(Exception e) {
+ if (connection != null)+ if (e instanceof ExecutionException)
Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + return new SaslException("Authentication failed: " + cause.getMessage(), cause); + } + }+ return new SaslException("Authentication failed unexpectedly.", e);
+ }
+ };
- if (props != null) {
- hsBuilder.setProperties(props);
+ final ClientAuthenticationProvider authenticationProvider;
+ try { + authenticationProvider = + UserAuthenticationUtil.getClientAuthenticationProvider(parameters, supportedAuthMechs); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; }
- this.connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler),
- hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
+ final String providerName = authenticationProvider.name();
+ logger.trace("Will try to login for {} mechanism.", providerName);
+ final UserGroupInformation ugi;
+ try { + ugi = authenticationProvider.login(parameters); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; + }
+
+ logger.trace("Will try to authenticate to server using {} mechanism.", providerName);
+ try { + saslClient = authenticationProvider.createSaslClient(ugi, parameters); + } catch (final SaslException e) { + settableFuture.setException(e); + return future; + }+
{ + settableFuture.setException(new SaslException("Cannot initiate authentication. Insufficient credentials?")); + return future; + }
+ if (saslClient == null)+ logger.trace("Initiating SASL exchange.");
+
+ tryUnknown macro: { + final ByteString responseData; + if (saslClient.hasInitialResponse()) { + responseData = ByteString.copyFrom(evaluateChallenge(ugi, saslClient, new byte[0])); + } else { + responseData = ByteString.EMPTY; + } + send(new SaslChallengeHandler(ugi, settableFuture), + RpcType.SASL_MESSAGE, + SaslMessage.newBuilder() + .setMechanism(providerName) + .setStatus(SaslStatus.SASL_START) + .setData(responseData) + .build(), + SaslMessage.class); + logger.trace("Initiated SASL exchange."); + }catch (final SaslException e)
{ + settableFuture.setException(e); + }+ return future;
+ }
+
+ private static byte[] evaluateChallenge(final UserGroupInformation ugi, final SaslClient saslClient,
+ final byte[] challenge) throws SaslException {
+ try {
+ return ugi.doAs(new PrivilegedExceptionAction<byte[]>()Unknown macro: { + @Override + public byte[] run() throws Exception { + return saslClient.evaluateChallenge(challenge); + } + });
+ } catch (final UndeclaredThrowableException e)Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + throw (SaslException) cause; + } else { + throw new SaslException( + String.format("Unexpected failure (%s)", saslClient.getMechanismName()), cause); + } + }catch (final IOException | InterruptedException e)
{ + throw new SaslException(String.format("Unexpected failure (%s)", saslClient.getMechanismName()), e); + }+ }
{ + this.ugi = ugi; + this.future = future; + }
+
+ // handles SASL message exchange
+ private class SaslChallengeHandler implements RpcOutcomeListener<SaslMessage> {
+
+ private final UserGroupInformation ugi;
+ private final SettableFuture<Void> future;
+
+ public SaslChallengeHandler(UserGroupInformation ugi, SettableFuture<Void> future)+
{ + future.setException(new SaslException("Unexpected failure", ex)); + }
+ @Override
+ public void failed(RpcException ex)+
{ + future.setException(e); + }
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ switch (value.getStatus()) {
+ case SASL_AUTH_IN_PROGRESS: {
+ try {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+ final byte[] responseBytes = evaluateChallenge(ugi, saslClient, value.getData().toByteArray());
+ final boolean isComplete = saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}. Sending response to server.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_AUTH_SUCCESS : SaslStatus.SASL_AUTH_IN_PROGRESS);
+ send(new SaslChallengeHandler(ugi, future),
+ connection,
+ RpcType.SASL_MESSAGE,
+ response.build(),
+ SaslMessage.class,
+ true // the connection will not be backed up at this point
+ );
+ } catch (Exception e)
+ break;
+ }
+ case SASL_AUTH_SUCCESS: {
+ try {
+ if (saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", saslClient.getMechanismName());
+ saslClient.dispose();
+ saslClient = null;
+ future.set(null); // success
+ } else {
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(ugi, saslClient, value.getData().toByteArray()); // discard response
+ if (saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", saslClient.getMechanismName());
+ saslClient.dispose();
+ saslClient = null;
+ future.set(null); // success
+ } else { + future.setException( + new SaslException("Server allegedly succeeded authentication, but client did not. Suspicious?")); + }
+ }
+ } catch (Exception e) { + future.setException(e); + }+ break;
+ }
+ case SASL_AUTH_FAILED:Unknown macro: { + future.setException(new SaslException("Incorrect credentials?")); + try { + saslClient.dispose(); + } catch (final SaslException ignored) { + // ignored + } + saslClient = null; + break; + }+ default:
+ future.setException(new SaslException("Server sent a corrupt message."));-
- End diff –
-
Done.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85849922
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -224,19 +222,15 @@ public synchronized void connect(String connect, Properties props) throws RpcExc
}
final ArrayList<DrillbitEndpoint> endpoints = new ArrayList<>(clusterCoordinator.getAvailableEndpoints());
- checkState(!endpoints.isEmpty(), "No DrillbitEndpoint can be found");
+ if (endpoints.isEmpty()) {-
- End diff –
-
Will revert.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853038
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,175 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
+
+ public static class PlainServerFactory implements SaslServerFactory {
+
+ @Override
+ public SaslServer createSaslServer(final String mechanism, final String protocol, final String serverName,
+ final Map<String, ?> props, final CallbackHandler cbh)
+ throws SaslException
+
+ @Override
+ public String[] getMechanismNames(final Map<String, ?> props) {
+ return props == null || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT)) ?
+ new String[]
:
+ new String[0];
+ }
+ }
+
+ @SuppressWarnings("serial")
+ public static class PlainServerProvider extends Provider {
+
+ public PlainServerProvider()
+ }
+
+ private CallbackHandler cbh;
+ private boolean completed = false;
+ private String authorizationID;
+
+ PlainServer(final CallbackHandler cbh) throws SaslException {
+ if (cbh == null)
+ this.cbh = cbh;
+ }
+
+ @Override
+ public String getMechanismName()
+
+ @Override
+ public byte[] evaluateResponse(byte[] response) throws SaslException {
+ if (completed)
+
+ if (response == null)
+
+ final String payload;
+ try
catch (final Exception e) {
— End diff –
Done.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85848714
— Diff: common/src/main/java/org/apache/drill/common/config/ConnectionParameters.java —
@@ -0,0 +1,153 @@
+/**
+ * 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.drill.common.config;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.drill.exec.proto.UserProtos.Property;
+import org.apache.drill.exec.proto.UserProtos.UserProperties;
+
+import java.util.Properties;
+
+public final class ConnectionParameters {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConnectionParameters.class);
+
+ // PROPERTY KEYS
+ // definitions should be in lowercase
+
+ public static final String ZOOKEEPER_CONNECTION = "zk";
+
+ public static final String DRILLBIT_CONNECTION = "drillbit";
+
+ public static final String SCHEMA = "schema";
+
+ public static final String USER = "user";
+
+ public static final String PASSWORD = "password";
+
+ public static final String IMPERSONATION_TARGET = "impersonation_target";
+
+ public static final String AUTH_MECHANISM = "auth";
+
+ public static final String SERVICE_PRINCIPAL = "principal";
+
+ public static final String SERVICE_NAME = "service_name";
+
+ public static final String SERVICE_HOST = "service_host";
+
+ public static final String REALM = "realm";
+
+ public static final String KEYTAB = "keytab";
+
+ // CONVENIENCE SETS OF PROPERTIES
+
+ public static final ImmutableSet<String> ALLOWED_BY_CLIENT =
+ ImmutableSet.of(ZOOKEEPER_CONNECTION, DRILLBIT_CONNECTION, SCHEMA, USER, PASSWORD, IMPERSONATION_TARGET,
+ AUTH_MECHANISM, SERVICE_PRINCIPAL, SERVICE_NAME, SERVICE_HOST, REALM, KEYTAB);
+
+ public static final ImmutableSet<String> ACCEPTED_BY_SERVER = ImmutableSet.of(SCHEMA, IMPERSONATION_TARGET);
+
+ private final Properties properties; // keys must be lower case
+
+ private ConnectionParameters(Properties properties) {
+ this.properties = properties;
— End diff –
Constructor is already private. Static factory methods below.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85853323
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -78,21 +101,241 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, ConnectionParameters parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) - .setCredentials(credentials); + .setCredentials(credentials) + .setProperties(parameters.serializeForServer()); + this.parameters = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
{ + return supportedAuthMechs; + }
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms()+
{@link CheckedFuture#checkedGet results}
+ /**
+ * Authenticate to the server asynchronously. Returns a future that+ * in null if authentication succeeds, or throws a
{@link SaslException}with relevant message if
{@link #connect connection time}
+ * authentication fails.
+ *
+ * This method uses parameters provided atand override them with the
{ + throw new IllegalStateException("Server does not require authentication."); + }
+ * given parameters, if any.
+ *
+ * @param overrides parameter overrides
+ * @return result of authentication request
+ */
+ public CheckedFuture<Void, SaslException> authenticate(final ConnectionParameters overrides) {
+ if (supportedAuthMechs == null)+ parameters.merge(overrides);
{ + connection.close(); // to ensure connection is dropped + }
+
+ final SettableFuture<Void> settableFuture = SettableFuture.create(); // future used in SASL exchange
+ final CheckedFuture<Void, SaslException> future =
+ new AbstractCheckedFuture<Void, SaslException>(settableFuture) {
+
+ @Override
+ protected SaslException mapException(Exception e) {
+ if (connection != null)+ if (e instanceof ExecutionException)
Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + return new SaslException("Authentication failed: " + cause.getMessage(), cause); + } + }+ return new SaslException("Authentication failed unexpectedly.", e);
+ }
+ };
- if (props != null) {
- hsBuilder.setProperties(props);
+ final ClientAuthenticationProvider authenticationProvider;
+ try { + authenticationProvider = + UserAuthenticationUtil.getClientAuthenticationProvider(parameters, supportedAuthMechs); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; }
- this.connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler),
- hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
+ final String providerName = authenticationProvider.name();
+ logger.trace("Will try to login for {} mechanism.", providerName);
+ final UserGroupInformation ugi;
+ try { + ugi = authenticationProvider.login(parameters); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; + }
+
+ logger.trace("Will try to authenticate to server using {} mechanism.", providerName);
+ try { + saslClient = authenticationProvider.createSaslClient(ugi, parameters); + } catch (final SaslException e) { + settableFuture.setException(e); + return future; + }+
{ + settableFuture.setException(new SaslException("Cannot initiate authentication. Insufficient credentials?")); + return future; + }
+ if (saslClient == null)+ logger.trace("Initiating SASL exchange.");
+
+ tryUnknown macro: { + final ByteString responseData; + if (saslClient.hasInitialResponse()) { + responseData = ByteString.copyFrom(evaluateChallenge(ugi, saslClient, new byte[0])); + } else { + responseData = ByteString.EMPTY; + } + send(new SaslChallengeHandler(ugi, settableFuture), + RpcType.SASL_MESSAGE, + SaslMessage.newBuilder() + .setMechanism(providerName) + .setStatus(SaslStatus.SASL_START) + .setData(responseData) + .build(), + SaslMessage.class); + logger.trace("Initiated SASL exchange."); + }catch (final SaslException e)
{ + settableFuture.setException(e); + }+ return future;
+ }
+
+ private static byte[] evaluateChallenge(final UserGroupInformation ugi, final SaslClient saslClient,
+ final byte[] challenge) throws SaslException {
+ try {
+ return ugi.doAs(new PrivilegedExceptionAction<byte[]>()Unknown macro: { + @Override + public byte[] run() throws Exception { + return saslClient.evaluateChallenge(challenge); + } + });
+ } catch (final UndeclaredThrowableException e)Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + throw (SaslException) cause; + } else { + throw new SaslException( + String.format("Unexpected failure (%s)", saslClient.getMechanismName()), cause); + } + }catch (final IOException | InterruptedException e)
{ + throw new SaslException(String.format("Unexpected failure (%s)", saslClient.getMechanismName()), e); + }+ }
{ + this.ugi = ugi; + this.future = future; + }
+
+ // handles SASL message exchange
+ private class SaslChallengeHandler implements RpcOutcomeListener<SaslMessage> {
+
+ private final UserGroupInformation ugi;
+ private final SettableFuture<Void> future;
+
+ public SaslChallengeHandler(UserGroupInformation ugi, SettableFuture<Void> future)+
{ + future.setException(new SaslException("Unexpected failure", ex)); + }
+ @Override
+ public void failed(RpcException ex)+
{ + future.setException(e); + }
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ switch (value.getStatus()) {
+ case SASL_AUTH_IN_PROGRESS: {
+ try {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+ final byte[] responseBytes = evaluateChallenge(ugi, saslClient, value.getData().toByteArray());
+ final boolean isComplete = saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}. Sending response to server.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_AUTH_SUCCESS : SaslStatus.SASL_AUTH_IN_PROGRESS);
+ send(new SaslChallengeHandler(ugi, future),
+ connection,
+ RpcType.SASL_MESSAGE,
+ response.build(),
+ SaslMessage.class,
+ true // the connection will not be backed up at this point
+ );
+ } catch (Exception e)+ break;
+ }
+ case SASL_AUTH_SUCCESS: {
+ try {
+ if (saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", saslClient.getMechanismName());
+ saslClient.dispose();-
- End diff –
-
Will update PR with the latest changes that include refactoring this `switch case` statement.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85852489
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,169 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.drill.exec.security.LoginManager;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
+
+ // Mapping: SASL name -> mechanism
+ // See AuthenticationMechanism#getMechanismName
— End diff –
Fixed doc.
Github user sudheeshkatkam commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85850267
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -285,9 +279,14 @@ public synchronized boolean reconnect() {
}
private void connect(DrillbitEndpoint endpoint) throws RpcException {
- final FutureHandler f = new FutureHandler();
- client.connect(f, endpoint, props, getUserCredentials());
- f.checkedGet();
+ client.connect(endpoint, parameters, getUserCredentials()).checkedGet();
+ if (client.serverRequiresAuthentication()) {-
- End diff –
-
Yes, notice the checkedGet() call to connect above (and if required to authenticate, the checkedGet() call to authenticate).
Or am I missing something?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85817951
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserAuthenticationUtil.java —
@@ -0,0 +1,248 @@
+/**
+ * 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.drill.exec.rpc.user;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+import org.apache.drill.common.KerberosUtil;
+import org.apache.drill.common.config.ConnectionParameters;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.annotation.Nullable;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.LoginException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Set;
+
+public final class UserAuthenticationUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserAuthenticationUtil.class);
+
+ private static final String PLAIN_MECHANISM = "PLAIN";
+
+ private static final String DEFAULT_SERVICE_NAME = System.getProperty("service.name.primary", "drill");
+
+ private static final String DEFAULT_REALM_NAME = System.getProperty("service.name.realm", "default");
+
+ public enum ClientAuthenticationProvider {
+
+ KERBEROS {
+ @Override
+ public UserGroupInformation login(final ConnectionParameters parameters) throws SaslException {
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION,
+ UserGroupInformation.AuthenticationMethod.KERBEROS.toString());
+ UserGroupInformation.setConfiguration(conf);
+
+ final String keytab = parameters.getParameter(ConnectionParameters.KEYTAB);
+ try {
+ final UserGroupInformation ugi;
+ if (keytab != null)
else
{ + // includes Kerberos ticket login + ugi = UserGroupInformation.getCurrentUser(); + logger.debug("Logged in using ticket."); + } + return ugi;
+ } catch (final IOException e) {
+ logger.debug("Login failed.", e);
+ final Throwable cause = e.getCause();
+ if (cause instanceof LoginException)
+ throw new SaslException("Unexpected failure trying to login: " + cause.getMessage());
— End diff –
should we chain the original exception?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85803203
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -603,33 +596,16 @@ public ListHoldingResultsListener(UserProtos.RunQuery query) {
@Override
public void submissionFailed(UserException ex) {
- // or !client.isActive()
- if (ex.getCause() instanceof ChannelClosedException) {
- if (reconnect()) {
-
- End diff –
-
does it mean there won't be any reconnection attempt?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85803810
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,169 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.drill.exec.security.LoginManager;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
+
+ // Mapping: SASL name -> mechanism
+ // See AuthenticationMechanism#getMechanismName
— End diff –
this method doesn't exist!
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85819136
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -78,21 +101,241 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, ConnectionParameters parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) - .setCredentials(credentials); + .setCredentials(credentials) + .setProperties(parameters.serializeForServer()); + this.parameters = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
{ + return supportedAuthMechs; + }
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms()+
{@link CheckedFuture#checkedGet results}
+ /**
+ * Authenticate to the server asynchronously. Returns a future that+ * in null if authentication succeeds, or throws a
{@link SaslException}with relevant message if
{@link #connect connection time}
+ * authentication fails.
+ *
+ * This method uses parameters provided atand override them with the
{ + throw new IllegalStateException("Server does not require authentication."); + }
+ * given parameters, if any.
+ *
+ * @param overrides parameter overrides
+ * @return result of authentication request
+ */
+ public CheckedFuture<Void, SaslException> authenticate(final ConnectionParameters overrides) {
+ if (supportedAuthMechs == null)+ parameters.merge(overrides);
{ + connection.close(); // to ensure connection is dropped + }
+
+ final SettableFuture<Void> settableFuture = SettableFuture.create(); // future used in SASL exchange
+ final CheckedFuture<Void, SaslException> future =
+ new AbstractCheckedFuture<Void, SaslException>(settableFuture) {
+
+ @Override
+ protected SaslException mapException(Exception e) {
+ if (connection != null)+ if (e instanceof ExecutionException)
Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + return new SaslException("Authentication failed: " + cause.getMessage(), cause); + } + }+ return new SaslException("Authentication failed unexpectedly.", e);
+ }
+ };
- if (props != null) {
- hsBuilder.setProperties(props);
+ final ClientAuthenticationProvider authenticationProvider;
+ try { + authenticationProvider = + UserAuthenticationUtil.getClientAuthenticationProvider(parameters, supportedAuthMechs); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; }
- this.connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler),
- hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
+ final String providerName = authenticationProvider.name();
+ logger.trace("Will try to login for {} mechanism.", providerName);
+ final UserGroupInformation ugi;
+ try { + ugi = authenticationProvider.login(parameters); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; + }
+
+ logger.trace("Will try to authenticate to server using {} mechanism.", providerName);
+ try { + saslClient = authenticationProvider.createSaslClient(ugi, parameters); + } catch (final SaslException e) { + settableFuture.setException(e); + return future; + }+
{ + settableFuture.setException(new SaslException("Cannot initiate authentication. Insufficient credentials?")); + return future; + }
+ if (saslClient == null)+ logger.trace("Initiating SASL exchange.");
+
+ tryUnknown macro: { + final ByteString responseData; + if (saslClient.hasInitialResponse()) { + responseData = ByteString.copyFrom(evaluateChallenge(ugi, saslClient, new byte[0])); + } else { + responseData = ByteString.EMPTY; + } + send(new SaslChallengeHandler(ugi, settableFuture), + RpcType.SASL_MESSAGE, + SaslMessage.newBuilder() + .setMechanism(providerName) + .setStatus(SaslStatus.SASL_START) + .setData(responseData) + .build(), + SaslMessage.class); + logger.trace("Initiated SASL exchange."); + }catch (final SaslException e)
{ + settableFuture.setException(e); + }+ return future;
+ }
+
+ private static byte[] evaluateChallenge(final UserGroupInformation ugi, final SaslClient saslClient,
+ final byte[] challenge) throws SaslException {
+ try {
+ return ugi.doAs(new PrivilegedExceptionAction<byte[]>()Unknown macro: { + @Override + public byte[] run() throws Exception { + return saslClient.evaluateChallenge(challenge); + } + });
+ } catch (final UndeclaredThrowableException e)Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + throw (SaslException) cause; + } else { + throw new SaslException( + String.format("Unexpected failure (%s)", saslClient.getMechanismName()), cause); + } + }catch (final IOException | InterruptedException e)
{ + throw new SaslException(String.format("Unexpected failure (%s)", saslClient.getMechanismName()), e); + }+ }
{ + this.ugi = ugi; + this.future = future; + }
+
+ // handles SASL message exchange
+ private class SaslChallengeHandler implements RpcOutcomeListener<SaslMessage> {
+
+ private final UserGroupInformation ugi;
+ private final SettableFuture<Void> future;
+
+ public SaslChallengeHandler(UserGroupInformation ugi, SettableFuture<Void> future)+
{ + future.setException(new SaslException("Unexpected failure", ex)); + }
+ @Override
+ public void failed(RpcException ex)+
{ + future.setException(e); + }
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ switch (value.getStatus()) {
+ case SASL_AUTH_IN_PROGRESS: {
+ try {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+ final byte[] responseBytes = evaluateChallenge(ugi, saslClient, value.getData().toByteArray());
+ final boolean isComplete = saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}. Sending response to server.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_AUTH_SUCCESS : SaslStatus.SASL_AUTH_IN_PROGRESS);
+ send(new SaslChallengeHandler(ugi, future),
+ connection,
+ RpcType.SASL_MESSAGE,
+ response.build(),
+ SaslMessage.class,
+ true // the connection will not be backed up at this point
+ );
+ } catch (Exception e)+ break;
+ }
+ case SASL_AUTH_SUCCESS: {
+ try {
+ if (saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", saslClient.getMechanismName());
+ saslClient.dispose();-
- End diff –
-
should all the cleaning of saslClient for every case be factored out?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85823003
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -328,44 +304,73 @@ protected void consumeHandshake(ChannelHandlerContext ctx, UserToBitHandshake in
public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
logger.trace("Handling handshake from user to bit. {}", inbound);
-
// if timeout is unsupported or is set to false, disable timeout.
if (!inbound.hasSupportTimeout() || !inbound.getSupportTimeout())
- BitToUserHandshake.Builder respBuilder = BitToUserHandshake.newBuilder()
+ final BitToUserHandshake.Builder respBuilder = BitToUserHandshake.newBuilder()
.setRpcVersion(UserRpcConfig.RPC_VERSION);
try {
- if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) {
- final String errMsg = String.format("Invalid rpc version. Expected %d, actual %d.",
- UserRpcConfig.RPC_VERSION, inbound.getRpcVersion());
+ if (!SUPPORTED_RPC_VERSIONS.contains(inbound.getRpcVersion())) { + final String errMsg = String.format("Invalid rpc version. Expected %s, actual %d.", + SUPPORTED_RPC_VERSIONS, inbound.getRpcVersion()); return handleFailure(respBuilder, HandshakeStatus.RPC_VERSION_MISMATCH, errMsg, null); }
- if (authenticator != null) {
+ connection.setHandshake(inbound);
+
+ if (authFactory == null) { // authentication is disabled + connection.finalizeSession(inbound.getCredentials().getUserName()); + respBuilder.setStatus(HandshakeStatus.SUCCESS); + return respBuilder.build(); + }+
+ if (inbound.getRpcVersion() == NON_SASL_RPC_VERSION_SUPPORTED) { // for backward compatibility
+ final String userName = inbound.getCredentials().getUserName();
+ if (logger.isTraceEnabled()) {
+ logger.trace("User {} on connection {} is using an older client (Drill version <= 1.8).",
+ userName, connection.getRemoteAddress());
+ }
try {
String password = "";
final UserProperties props = inbound.getProperties();
for (int i = 0; i < props.getPropertiesCount(); i++) {
Property prop = props.getProperties; - if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey()))
Unknown macro: { + if (ConnectionParameters.PASSWORD.equalsIgnoreCase(prop.getKey())) { password = prop.getValue(); break; } }
- authenticator.authenticate(inbound.getCredentials().getUserName(), password);
+ final PlainMechanism plainMechanism = authFactory.getPlainMechanism();
+ if (plainMechanism == null) { + throw new UserAuthenticationException("The server no longer supports username/password" + + " based authentication. Please talk to your system administrator."); + }+ plainMechanism.getAuthenticator()
+ .authenticate(userName, password);
+ connection.changeHandlerTo(handler);
+ connection.finalizeSession(userName);
+ respBuilder.setStatus(HandshakeStatus.SUCCESS);-
- End diff –
-
should the rpc version of the response be changed to `NON_SASL_RPC_VERSION_SUPPORTED` in that specific case?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85804286
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,169 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.drill.exec.security.LoginManager;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
+
+ // Mapping: SASL name -> mechanism
+ // See AuthenticationMechanism#getMechanismName
+ private final Map<String, AuthenticationMechanism> mechanisms = CaseInsensitiveMap.newHashMapWithExpectedSize(5);
+
+ @SuppressWarnings("unchecked")
+ public AuthenticationMechanismFactory(final DrillConfig config, final ScanResult scan,
+ final LoginManager loginManager) throws DrillbitStartupException {
+ if (!config.hasPath(ExecConstants.AUTHENTICATION_MECHANISMS))
+
+ final List<String> configuredMechanisms = config.getStringList(ExecConstants.AUTHENTICATION_MECHANISMS);
+ logger.debug("Configuring authentication mechanisms: {}", configuredMechanisms);
+ // transform all names to uppercase
+ final Set<String> configuredMechanismsSet = Sets.newHashSet(Iterators.transform(configuredMechanisms.iterator(),
+ new Function<String, String>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable String input)
+ }));
+
+ // PLAIN mechanism need special handling due to UserAuthenticator
+ if (configuredMechanismsSet.contains(PlainMechanism.MECHANISM_NAME)) {
+ // instantiated here, but closed in PlainMechanism#close
+ final UserAuthenticator userAuthenticator = UserAuthenticatorFactory.createAuthenticator(config, scan);
+ final PlainMechanism mechanism = new PlainMechanism(userAuthenticator);
+ mechanisms.put(PlainMechanism.MECHANISM_NAME, mechanism);
+ configuredMechanismsSet.remove(PlainMechanism.MECHANISM_NAME);
— End diff –
you can do it in the if clause:
`if (configuredMechanismSet.remove(PlainMechanism.MECHANISM_NAME))`
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85802232
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret) {
— End diff –
you can use a smart pointer to guard this object and free it automatically
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85819216
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java —
@@ -78,21 +101,241 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query)
- public void connect(RpcConnectionHandler<ServerConnection> handler, DrillbitEndpoint endpoint,
- UserProperties props, UserBitShared.UserCredentials credentials) {
+ public CheckedFuture<Void, RpcException> connect(DrillbitEndpoint endpoint, ConnectionParameters parameters,
+ UserCredentials credentials) { + final FutureHandler handler = new FutureHandler(); UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder() .setRpcVersion(UserRpcConfig.RPC_VERSION) .setSupportListening(true) .setSupportComplexTypes(supportComplexTypes) .setSupportTimeout(true) - .setCredentials(credentials); + .setCredentials(credentials) + .setProperties(parameters.serializeForServer()); + this.parameters = parameters; + + connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler), + hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort()); + return handler; + }+
{@link #connect connecting}) if server requires authentication.
+ /**
+ * Check (after
+ *
+ * @return true if server requires authentication
+ */
+ public boolean serverRequiresAuthentication() { + return supportedAuthMechs != null; + }
+
+ /**
+ * Returns a list of supported authentication mechanism. If called before {@link #connect connecting},
{@link #connect connecting}
+ * returns null. If called after, returns a list of supported mechanisms
{ + return supportedAuthMechs; + }
+ * iff authentication is required.
+ *
+ * @return list of supported authentication mechanisms
+ */
+ public List<String> getSupportedAuthenticationMechanisms()+
{@link CheckedFuture#checkedGet results}
+ /**
+ * Authenticate to the server asynchronously. Returns a future that+ * in null if authentication succeeds, or throws a
{@link SaslException}with relevant message if
{@link #connect connection time}
+ * authentication fails.
+ *
+ * This method uses parameters provided atand override them with the
{ + throw new IllegalStateException("Server does not require authentication."); + }
+ * given parameters, if any.
+ *
+ * @param overrides parameter overrides
+ * @return result of authentication request
+ */
+ public CheckedFuture<Void, SaslException> authenticate(final ConnectionParameters overrides) {
+ if (supportedAuthMechs == null)+ parameters.merge(overrides);
{ + connection.close(); // to ensure connection is dropped + }
+
+ final SettableFuture<Void> settableFuture = SettableFuture.create(); // future used in SASL exchange
+ final CheckedFuture<Void, SaslException> future =
+ new AbstractCheckedFuture<Void, SaslException>(settableFuture) {
+
+ @Override
+ protected SaslException mapException(Exception e) {
+ if (connection != null)+ if (e instanceof ExecutionException)
Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + return new SaslException("Authentication failed: " + cause.getMessage(), cause); + } + }+ return new SaslException("Authentication failed unexpectedly.", e);
+ }
+ };
- if (props != null) {
- hsBuilder.setProperties(props);
+ final ClientAuthenticationProvider authenticationProvider;
+ try { + authenticationProvider = + UserAuthenticationUtil.getClientAuthenticationProvider(parameters, supportedAuthMechs); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; }
- this.connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler),
- hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
+ final String providerName = authenticationProvider.name();
+ logger.trace("Will try to login for {} mechanism.", providerName);
+ final UserGroupInformation ugi;
+ try { + ugi = authenticationProvider.login(parameters); + }catch (final SaslException e)
{ + settableFuture.setException(e); + return future; + }
+
+ logger.trace("Will try to authenticate to server using {} mechanism.", providerName);
+ try { + saslClient = authenticationProvider.createSaslClient(ugi, parameters); + } catch (final SaslException e) { + settableFuture.setException(e); + return future; + }+
{ + settableFuture.setException(new SaslException("Cannot initiate authentication. Insufficient credentials?")); + return future; + }
+ if (saslClient == null)+ logger.trace("Initiating SASL exchange.");
+
+ tryUnknown macro: { + final ByteString responseData; + if (saslClient.hasInitialResponse()) { + responseData = ByteString.copyFrom(evaluateChallenge(ugi, saslClient, new byte[0])); + } else { + responseData = ByteString.EMPTY; + } + send(new SaslChallengeHandler(ugi, settableFuture), + RpcType.SASL_MESSAGE, + SaslMessage.newBuilder() + .setMechanism(providerName) + .setStatus(SaslStatus.SASL_START) + .setData(responseData) + .build(), + SaslMessage.class); + logger.trace("Initiated SASL exchange."); + }catch (final SaslException e)
{ + settableFuture.setException(e); + }+ return future;
+ }
+
+ private static byte[] evaluateChallenge(final UserGroupInformation ugi, final SaslClient saslClient,
+ final byte[] challenge) throws SaslException {
+ try {
+ return ugi.doAs(new PrivilegedExceptionAction<byte[]>()Unknown macro: { + @Override + public byte[] run() throws Exception { + return saslClient.evaluateChallenge(challenge); + } + });
+ } catch (final UndeclaredThrowableException e)Unknown macro: { + final Throwable cause = e.getCause(); + if (cause instanceof SaslException) { + throw (SaslException) cause; + } else { + throw new SaslException( + String.format("Unexpected failure (%s)", saslClient.getMechanismName()), cause); + } + }catch (final IOException | InterruptedException e)
{ + throw new SaslException(String.format("Unexpected failure (%s)", saslClient.getMechanismName()), e); + }+ }
{ + this.ugi = ugi; + this.future = future; + }
+
+ // handles SASL message exchange
+ private class SaslChallengeHandler implements RpcOutcomeListener<SaslMessage> {
+
+ private final UserGroupInformation ugi;
+ private final SettableFuture<Void> future;
+
+ public SaslChallengeHandler(UserGroupInformation ugi, SettableFuture<Void> future)+
{ + future.setException(new SaslException("Unexpected failure", ex)); + }
+ @Override
+ public void failed(RpcException ex)+
{ + future.setException(e); + }
+ @Override
+ public void success(SaslMessage value, ByteBuf buffer) {
+ logger.trace("Server responded with message of type: {}", value.getStatus());
+ switch (value.getStatus()) {
+ case SASL_AUTH_IN_PROGRESS: {
+ try {
+ final SaslMessage.Builder response = SaslMessage.newBuilder();
+ final byte[] responseBytes = evaluateChallenge(ugi, saslClient, value.getData().toByteArray());
+ final boolean isComplete = saslClient.isComplete();
+ logger.trace("Evaluated challenge. Completed? {}. Sending response to server.", isComplete);
+ response.setData(responseBytes != null ? ByteString.copyFrom(responseBytes) : ByteString.EMPTY);
+ // if isComplete, the client will get one more response from server
+ response.setStatus(isComplete ? SaslStatus.SASL_AUTH_SUCCESS : SaslStatus.SASL_AUTH_IN_PROGRESS);
+ send(new SaslChallengeHandler(ugi, future),
+ connection,
+ RpcType.SASL_MESSAGE,
+ response.build(),
+ SaslMessage.class,
+ true // the connection will not be backed up at this point
+ );
+ } catch (Exception e)
+ break;
+ }
+ case SASL_AUTH_SUCCESS: {
+ try {
+ if (saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", saslClient.getMechanismName());
+ saslClient.dispose();
+ saslClient = null;
+ future.set(null); // success
+ } else {
+ // server completed before client; so try once, fail otherwise
+ evaluateChallenge(ugi, saslClient, value.getData().toByteArray()); // discard response
+ if (saslClient.isComplete()) {
+ logger.trace("Successfully authenticated to server using {}", saslClient.getMechanismName());
+ saslClient.dispose();
+ saslClient = null;
+ future.set(null); // success
+ } else { + future.setException( + new SaslException("Server allegedly succeeded authentication, but client did not. Suspicious?")); + }
+ }
+ } catch (Exception e) { + future.setException(e); + }+ break;
+ }
+ case SASL_AUTH_FAILED:Unknown macro: { + future.setException(new SaslException("Incorrect credentials?")); + try { + saslClient.dispose(); + } catch (final SaslException ignored) { + // ignored + } + saslClient = null; + break; + }+ default:
+ future.setException(new SaslException("Server sent a corrupt message."));-
- End diff –
-
no call to saslClient.dispose()?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85802469
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
+ std::string &chosenMech,
+ const char **out,
+ unsigned *outlen) {
+ // set params
+ std::string authMechanismToUse = NULL;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::map<std::string, uint32_t>::const_iterator it =
+ DrillUserProperties::USER_PROPERTIES.find(m_properties->keyAt);
+ if (it == DrillUserProperties::USER_PROPERTIES.end())
+ if (IS_BITSET((*it).second, USERPROP_FLAGS_USERNAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting name" << std::endl;) + m_username = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_PASSWORD))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting password" << std::endl;) + m_password = m_properties->valueAt(i); + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = "plain"; + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_AUTH_MECHANISM))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + authMechanismToUse = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_NAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + m_servicename = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_HOST))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service host" << std::endl;) + m_servicehost = m_properties->valueAt(i); + } + }
+ if (authMechanismToUse == NULL)
+
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
+ std::string mechanism = mechanisms[i];
+ if (authMechanismToUse.compare(mechanism) == 0) { + isSupportedByServer = true; + }
+ }
+ if (!isSupportedByServer) { + return SASL_NOMECH; + }
+ boost::algorithm::to_lower(authMechanismToUse);
+ chosenMech = authMechanismToUse;
+ std::string sasMechanismToUse = NULL;
+ if (authMechanismToUse.compare("plain") == 0) {
— End diff –
"plain"/"gssapi" should probably be represented by constants
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85819718
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -299,6 +260,20 @@ public ChannelFuture getChannelClosureFuture() {
public SocketAddress getRemoteAddress()
+
+ @Override
+ public void close() {
+ super.close();
— End diff –
if the super method throws, saslServer won't be cleaned. Is that okay?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85804609
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,169 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.drill.exec.security.LoginManager;
+
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ public static final String AUTHENTICATION_MECHANISMS = "drill.exec.security.auth.mechanisms";
+
+ // Mapping: SASL name -> mechanism
+ // See AuthenticationMechanism#getMechanismName
+ private final Map<String, AuthenticationMechanism> mechanisms = CaseInsensitiveMap.newHashMapWithExpectedSize(5);
+
+ @SuppressWarnings("unchecked")
+ public AuthenticationMechanismFactory(final DrillConfig config, final ScanResult scan,
+ final LoginManager loginManager) throws DrillbitStartupException {
+ if (!config.hasPath(ExecConstants.AUTHENTICATION_MECHANISMS))
+
+ final List<String> configuredMechanisms = config.getStringList(ExecConstants.AUTHENTICATION_MECHANISMS);
+ logger.debug("Configuring authentication mechanisms: {}", configuredMechanisms);
+ // transform all names to uppercase
+ final Set<String> configuredMechanismsSet = Sets.newHashSet(Iterators.transform(configuredMechanisms.iterator(),
+ new Function<String, String>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable String input)
+ }));
+
+ // PLAIN mechanism need special handling due to UserAuthenticator
+ if (configuredMechanismsSet.contains(PlainMechanism.MECHANISM_NAME))
+
+ // Then, load other mechanisms, if any
+ if (!configuredMechanismsSet.isEmpty()) {
+ final Collection<Class<? extends AuthenticationMechanism>> mechanismImpls =
+ scan.getImplementations(AuthenticationMechanism.class);
+ logger.debug("Found AuthenticationMechanism implementations: {}", mechanismImpls);
+
+ for (Class<? extends AuthenticationMechanism> clazz : mechanismImpls) {
+ final SaslMechanism annotation = clazz.getAnnotation(SaslMechanism.class);
+ if (annotation == null) {
+ logger.warn("{} doesn't have {} annotation. Skipping.", clazz.getCanonicalName(), SaslMechanism.class);
+ continue;
+ }
+
+ final String annotatedName = annotation.name();
+ if (Strings.isNullOrEmpty(annotatedName)) {
+ logger.warn("Authentication mechanism {} does not have a proper {} annotation. Skipping.",
— End diff –
maybe the message should be more explicit about the missing name attribute?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85802665
— Diff: contrib/native/client/src/include/drill/common.hpp —
@@ -34,7 +34,7 @@
#include <vector>
#include <boost/shared_ptr.hpp>
-#define DRILL_RPC_VERSION 5
+#define DRILL_RPC_VERSION 6
— End diff –
does it mean the client won't work with an older server version?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85800948
— Diff: contrib/native/client/src/clientlib/drillClientImpl.hpp —
@@ -564,6 +570,34 @@ class ZookeeperImpl
;
+class SaslAuthenticatorImpl {
+
+ public:
+
+ SaslAuthenticatorImpl(const DrillUserProperties* const properties);
+
+ ~SaslAuthenticatorImpl();
+
+ int init(std::vector<std::string> mechanisms, std::string &chosenMech,
— End diff –
use const reference as much as possible.
The API also mix up C and C++ types, maybe this class should only expose an API based on C++ types, and manage the conversion from/to C types as a better encapsulation model
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85819519
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -246,28 +160,75 @@ protected void handle(UserClientConnectionImpl connection, int rpcType, ByteBuf
public class UserClientConnectionImpl extends RemoteConnection implements UserClientConnection {
private UserSession session;
+ private SaslServer saslServer;
+ private RequestHandler<UserClientConnectionImpl> currentHandler;
— End diff –
this field can be final I guess
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85802018
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection) {
— End diff –
you can guard if you store the sasl connection into a smart pointer (where you specify sasl_dispose as the destructor)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85801077
— Diff: contrib/native/client/src/clientlib/drillClientImpl.hpp —
@@ -564,6 +570,34 @@ class ZookeeperImpl
;
+class SaslAuthenticatorImpl {
+
+ public:
+
+ SaslAuthenticatorImpl(const DrillUserProperties* const properties);
+
+ ~SaslAuthenticatorImpl();
+
+ int init(std::vector<std::string> mechanisms, std::string &chosenMech,
+ const char **out, unsigned *outlen);
+
+ int step(const char* const in, const unsigned inlen, const char **out, unsigned *outlen) const;
+
+ static int passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret);
— End diff –
you don't need to expose the callbacks I believe
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85802137
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret) {
+ free(m_secret);
— End diff –
since this object was created by the SASL library, should sasl_dispose be called instead?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85802847
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -224,19 +222,15 @@ public synchronized void connect(String connect, Properties props) throws RpcExc
}
final ArrayList<DrillbitEndpoint> endpoints = new ArrayList<>(clusterCoordinator.getAvailableEndpoints());
- checkState(!endpoints.isEmpty(), "No DrillbitEndpoint can be found");
+ if (endpoints.isEmpty()) {-
- End diff –
-
why stop using checkState()?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85806353
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,175 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
+
+ public static class PlainServerFactory implements SaslServerFactory {
+
+ @Override
+ public SaslServer createSaslServer(final String mechanism, final String protocol, final String serverName,
+ final Map<String, ?> props, final CallbackHandler cbh)
+ throws SaslException
+
+ @Override
+ public String[] getMechanismNames(final Map<String, ?> props) {
+ return props == null || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT)) ?
+ new String[]
:
+ new String[0];
+ }
+ }
+
+ @SuppressWarnings("serial")
+ public static class PlainServerProvider extends Provider {
+
+ public PlainServerProvider()
+ }
+
+ private CallbackHandler cbh;
+ private boolean completed = false;
+ private String authorizationID;
+
+ PlainServer(final CallbackHandler cbh) throws SaslException {
+ if (cbh == null)
+ this.cbh = cbh;
+ }
+
+ @Override
+ public String getMechanismName()
+
+ @Override
+ public byte[] evaluateResponse(byte[] response) throws SaslException {
+ if (completed)
+
+ if (response == null)
+
+ final String payload;
+ try
catch (final Exception e) {
— End diff –
use `new String(response, java.nio.charset.StandardChaset.UTF_8)`: the behaviour of that call is well-defined (and doesn't throw exception)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85800113
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
+ std::string &chosenMech,
+ const char **out,
+ unsigned *outlen) {
+ // set params
+ std::string authMechanismToUse = NULL;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::map<std::string, uint32_t>::const_iterator it =
+ DrillUserProperties::USER_PROPERTIES.find(m_properties->keyAt);
+ if (it == DrillUserProperties::USER_PROPERTIES.end())
+ if (IS_BITSET((*it).second, USERPROP_FLAGS_USERNAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting name" << std::endl;) + m_username = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_PASSWORD))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting password" << std::endl;) + m_password = m_properties->valueAt(i); + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = "plain"; + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_AUTH_MECHANISM))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + authMechanismToUse = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_NAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + m_servicename = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_HOST))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service host" << std::endl;) + m_servicehost = m_properties->valueAt(i); + } + }
+ if (authMechanismToUse == NULL) {
— End diff –
tricky/incorrect... llvm compiler doesn't allow it for example
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85800898
— Diff: contrib/native/client/src/clientlib/drillClientImpl.hpp —
@@ -564,6 +570,34 @@ class ZookeeperImpl
;
+class SaslAuthenticatorImpl {
— End diff –
I suggest to move it to a separate unit, as the class is fairly complex, and this is an helper class.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85798045
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id) {
+ const std::string password = authenticator->m_password;
+ const size_t length = password.length();
+ authenticator->m_secret->len = length;
+ std::memcpy(authenticator->m_secret->data, password.c_str(), length);
— End diff –
isn't the context outliving the callback? (meaning we would not need like to copy the string, like for usernameCallback?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85799157
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
+ std::string &chosenMech,
+ const char **out,
+ unsigned *outlen) {
+ // set params
+ std::string authMechanismToUse = NULL;
— End diff –
like previously authMechanismToUse is not a pointer. You can remove the = NULL part.
I actually tried on Mac, and seems like the C++ lib doesn't like std::string(NULL) (it segfaults)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85798863
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
+ std::string &chosenMech,
+ const char **out,
+ unsigned *outlen) {
+ // set params
+ std::string authMechanismToUse = NULL;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::map<std::string, uint32_t>::const_iterator it =
+ DrillUserProperties::USER_PROPERTIES.find(m_properties->keyAt);
+ if (it == DrillUserProperties::USER_PROPERTIES.end())
+ if (IS_BITSET((*it).second, USERPROP_FLAGS_USERNAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting name" << std::endl;) + m_username = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_PASSWORD))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting password" << std::endl;) + m_password = m_properties->valueAt(i); + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = "plain"; + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_AUTH_MECHANISM))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + authMechanismToUse = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_NAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + m_servicename = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_HOST))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service host" << std::endl;) + m_servicehost = m_properties->valueAt(i); + } + }
+ if (authMechanismToUse == NULL)
+
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
+ std::string mechanism = mechanisms[i];
— End diff –
you can use a const reference
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85798339
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
+ std::string &chosenMech,
+ const char **out,
+ unsigned *outlen) {
+ // set params
+ std::string authMechanismToUse = NULL;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::map<std::string, uint32_t>::const_iterator it =
+ DrillUserProperties::USER_PROPERTIES.find(m_properties->keyAt);
+ if (it == DrillUserProperties::USER_PROPERTIES.end())
+ if (IS_BITSET((*it).second, USERPROP_FLAGS_USERNAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting name" << std::endl;) + m_username = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_PASSWORD))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting password" << std::endl;) + m_password = m_properties->valueAt(i); + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = "plain"; + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_AUTH_MECHANISM))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + authMechanismToUse = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_NAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + m_servicename = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_HOST))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service host" << std::endl;) + m_servicehost = m_properties->valueAt(i); + } + }
+ if (authMechanismToUse == NULL)
+
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
+ std::string mechanism = mechanisms[i];
+ if (authMechanismToUse.compare(mechanism) == 0) { + isSupportedByServer = true; + }
+ }
+ if (!isSupportedByServer) { + return SASL_NOMECH; + }
+ boost::algorithm::to_lower(authMechanismToUse);
+ chosenMech = authMechanismToUse;
+ std::string sasMechanismToUse = NULL;
+ if (authMechanismToUse.compare("plain") == 0)
else if (authMechanismToUse.compare("kerberos") == 0)
{ + sasMechanismToUse = "gssapi"; + }else
{ + return SASL_NOMECH; + } +
+ // create
+ const sasl_callback_t callbacks[] = {
+ {
+ SASL_CB_USER, (sasl_callback_proc_t) &userNameCallback, (void *) &m_username
— End diff –
maybe you can use the same callback function for all of them
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85798971
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
+ std::string &chosenMech,
+ const char **out,
+ unsigned *outlen) {
+ // set params
+ std::string authMechanismToUse = NULL;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::map<std::string, uint32_t>::const_iterator it =
+ DrillUserProperties::USER_PROPERTIES.find(m_properties->keyAt);
+ if (it == DrillUserProperties::USER_PROPERTIES.end())
+ if (IS_BITSET((*it).second, USERPROP_FLAGS_USERNAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting name" << std::endl;) + m_username = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_PASSWORD))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting password" << std::endl;) + m_password = m_properties->valueAt(i); + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = "plain"; + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_AUTH_MECHANISM))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + authMechanismToUse = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_NAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + m_servicename = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_HOST))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service host" << std::endl;) + m_servicehost = m_properties->valueAt(i); + } + }
+ if (authMechanismToUse == NULL)
+
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
— End diff –
you can use the find function...
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85800423
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
+ std::string &chosenMech,
+ const char **out,
+ unsigned *outlen) {
+ // set params
+ std::string authMechanismToUse = NULL;
+ for (size_t i = 0; i < m_properties->size(); i++) {
+ const std::map<std::string, uint32_t>::const_iterator it =
+ DrillUserProperties::USER_PROPERTIES.find(m_properties->keyAt);
+ if (it == DrillUserProperties::USER_PROPERTIES.end())
+ if (IS_BITSET((*it).second, USERPROP_FLAGS_USERNAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting name" << std::endl;) + m_username = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_PASSWORD))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting password" << std::endl;) + m_password = m_properties->valueAt(i); + m_secret = (sasl_secret_t *) malloc(sizeof(sasl_secret_t) + m_password.length()); + authMechanismToUse = "plain"; + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_AUTH_MECHANISM))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + authMechanismToUse = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_NAME))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service name" << std::endl;) + m_servicename = m_properties->valueAt(i); + continue; + }+ if (IS_BITSET((*it).second, USERPROP_FLAGS_SERVICE_HOST))
{ + DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Setting service host" << std::endl;) + m_servicehost = m_properties->valueAt(i); + } + }
+ if (authMechanismToUse == NULL)
+
+ bool isSupportedByServer = false;
+ for (size_t i = 0; i < mechanisms.size(); i++) {
+ std::string mechanism = mechanisms[i];
+ if (authMechanismToUse.compare(mechanism) == 0) { + isSupportedByServer = true; + }
+ }
+ if (!isSupportedByServer) { + return SASL_NOMECH; + }
+ boost::algorithm::to_lower(authMechanismToUse);
+ chosenMech = authMechanismToUse;
+ std::string sasMechanismToUse = NULL;
+ if (authMechanismToUse.compare("plain") == 0) {
— End diff –
you can just use == or != to compare two strings in C++
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85798619
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
+
+ if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+ && username != NULL)
+ return SASL_OK;
+}
+
+static int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void *context, int id, sasl_secret_t **psecret) {
+ const SaslAuthenticatorImpl* const authenticator = (const SaslAuthenticatorImpl* const) context;
+
+ if (SASL_CB_PASS == id)
+ return SASL_OK;
+}
+
+SaslAuthenticatorImpl::SaslAuthenticatorImpl(const DrillUserProperties* const properties) :
+ m_properties(properties), m_pConnection(NULL), m_secret(NULL), m_servicename(NULL), m_servicehost(NULL)
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+ if (m_secret)
+ // may be to use negotiated security layers before disposing in the future
+ if (m_pConnection)
+ m_pConnection = NULL;
+}
+
+int SaslAuthenticatorImpl::init(std::vector<std::string> mechanisms,
— End diff –
use a const ref for mechanism
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85797710
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
— End diff –
do you have to make it part of SaslAuthenticatorImpl interface? (maybe it could be a simple function in an anonymous namespace)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85797062
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -427,6 +511,121 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
getMessage(ERR_CONN_AUTHFAIL,
this->m_handshakeErrorId.c_str(),
this->m_handshakeErrorMsg.c_str()));
+ case exec::user::AUTH_REQUIRED: {
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl
+ SaslAuthenticatorImpl saslAuthenticator(properties);
+ int saslResult = 0;
+ std::string chosenMech;
+ const char *out;
+ unsigned outlen;
+ saslResult = saslAuthenticator.init(m_mechanisms, chosenMech, &out, &outlen);
+ if (saslResult != SASL_OK)
+ if (NULL == out)
{ + out = (&::google::protobuf::internal::kEmptyString)->c_str(); + } + // send initial response
+ {
+ exec::user::SaslMessage response;
+ response.set_data(out, outlen);
+ response.set_mechanism(chosenMech[0]);
+ response.set_status(exec::user::SaslStatus::SASL_START);
+
+ }
+
+ bool done = false;
+ while (saslResult == SASL_OK || saslResult == SASL_CONTINUE) {
+ if (done)
+ // receive challenge
+ InBoundRpcMessage inboundMessage;
+ readMessage(inboundMessage);
+ if (m_pError)
+ exec::user::SaslMessage challenge;
+ challenge.ParseFromArray(inboundMessage.m_pbody.data(), inboundMessage.m_pbody.size());
— End diff –
you should check the return value for parsing errors
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85796780
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -427,6 +511,121 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
getMessage(ERR_CONN_AUTHFAIL,
this->m_handshakeErrorId.c_str(),
this->m_handshakeErrorMsg.c_str()));
+ case exec::user::AUTH_REQUIRED: {
— End diff –
this case block is too big I think, you might want to extract it (or part of it) into methods or helper functions
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85794422
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -102,18 +106,33 @@ connectionStatus_t DrillClientImpl::connect(const char* connStr)
zook.close();
- m_bIsDirectConnection=true;
+ m_bIsDirectConnection=true;
}else if(!strcmp(protocol.c_str(), "local")) { boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant char tempStr[MAX_CONNECT_STR+1]; strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0; host=strtok(tempStr, ":"); port=strtok(NULL, ""); - m_bIsDirectConnection=false; + m_bIsDirectConnection=false; }else
{ return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str())); }DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl
+ std::string servicename = NULL;-
- End diff –
-
servicename is a value, not a pointer. If you don't want to set a value, simply remove the assignment part (otherwise, it's like calling std::string(NULL))
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85795087
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -248,7 +267,7 @@ connectionStatus_t DrillClientImpl::recvHandshake(){
}
m_io_service.reset();
- if (DrillClientConfig::getHandshakeTimeout() > 0){
+ if (false){-
- End diff –
-
is it for testing purposes?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85796563
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -427,6 +511,121 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
getMessage(ERR_CONN_AUTHFAIL,
this->m_handshakeErrorId.c_str(),
this->m_handshakeErrorMsg.c_str()));
+ case exec::user::AUTH_REQUIRED: {
+ DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Server requires SASL authentication." << std::endl
+ SaslAuthenticatorImpl saslAuthenticator(properties);
+ int saslResult = 0;
+ std::string chosenMech;
+ const char *out;
+ unsigned outlen;
+ saslResult = saslAuthenticator.init(m_mechanisms, chosenMech, &out, &outlen);
+ if (saslResult != SASL_OK)
+ if (NULL == out) {
+ out = (&::google::protobuf::internal::kEmptyString)->c_str();
— End diff –
we should not use protobuf internal constants. Either use the empty string "" here (it might be okay if some of the functions you call out with are not keeping reference on it after returning), or create a special constant.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85645966
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -591,7 +584,7 @@ public void runQuery(QueryType type, String plan, UserResultsListener resultsLis
client.submitQuery(resultsListener, newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build());
}
- private class ListHoldingResultsListener implements UserResultsListener {
+ protected class ListHoldingResultsListener implements UserResultsListener {-
- End diff –
-
why the change of visibility?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85792599
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,40 @@
+/**
+ * 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.drill.common;
+
+public final class KerberosUtil {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
+
+ public static final String KERBEROS_SASL_NAME = "GSSAPI";
+
+ public static final String KERBEROS_SIMPLE_NAME = "KERBEROS";
+
+ // primary/instance@REALM
+ public static String getPrincipalFromParts(final String primary, final String instance, final String realm) {
— End diff –
what if instance is null (seems to be allowed to only have primary@REALM)
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85645935
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -88,21 +84,22 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Strings;
-import com.google.common.util.concurrent.AbstractCheckedFuture;
+
import com.google.common.util.concurrent.SettableFuture;
+import javax.security.sasl.SaslException;
+
/**
- Thin wrapper around a UserClient that handles connect/close and transforms
- String into ByteBuf.
*/
public class DrillClient implements Closeable, ConnectionThrottle {
private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class);
- private static final ObjectMapper objectMapper = new ObjectMapper();
+ protected static final ObjectMapper objectMapper = new ObjectMapper();-
- End diff –
-
why this change? I was not able to find a change in the diff which requires the change of visibility
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85645840
— Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp —
@@ -1849,4 +2048,150 @@ void ZookeeperImpl:: debugPrint(){
}
}
+typedef int (*sasl_callback_proc_t)(void); // see sasl_callback_ft
+
+static int SaslAuthenticatorImpl::userNameCallback(void *context, int id, const char **result, unsigned *len) {
+ const std::string* const username = (const std::string* const) context;
— End diff –
avoid C-style cast
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85645751
— Diff: contrib/native/client/cmakeModules/FindSASL.cmake —
@@ -0,0 +1,55 @@
+#
+# 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.
+#
+
+# - Try to find Cyrus SASL
+
+if (MSVC)
+ if(${CMAKE_BUILD_TYPE} MATCHES "Debug")
+ set(SASL_BuildOutputDir "Debug")
+ else()
+ set(SASL_BuildOutputDir "Release")
+ endif()
+ if("${SASL_HOME}" MATCHES "^$")
+ message(" ")
+ message("- Please set the cache variable SASL_HOME to point to the directory with the Cyrus SASL source.")
+ message("- CMAKE will look for Cyrus SASL include files in $SASL_HOME/src/c/include.")
— End diff –
those are the instructions for zookeeper, but are they the same for Cyrus SASL? A quick look at the source don't show any src/c/include directory.
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85792866
— Diff: common/src/main/java/org/apache/drill/common/config/ConnectionParameters.java —
@@ -0,0 +1,153 @@
+/**
+ * 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.drill.common.config;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.drill.exec.proto.UserProtos.Property;
+import org.apache.drill.exec.proto.UserProtos.UserProperties;
+
+import java.util.Properties;
+
+public final class ConnectionParameters {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ConnectionParameters.class);
+
+ // PROPERTY KEYS
+ // definitions should be in lowercase
+
+ public static final String ZOOKEEPER_CONNECTION = "zk";
+
+ public static final String DRILLBIT_CONNECTION = "drillbit";
+
+ public static final String SCHEMA = "schema";
+
+ public static final String USER = "user";
+
+ public static final String PASSWORD = "password";
+
+ public static final String IMPERSONATION_TARGET = "impersonation_target";
+
+ public static final String AUTH_MECHANISM = "auth";
+
+ public static final String SERVICE_PRINCIPAL = "principal";
+
+ public static final String SERVICE_NAME = "service_name";
+
+ public static final String SERVICE_HOST = "service_host";
+
+ public static final String REALM = "realm";
+
+ public static final String KEYTAB = "keytab";
+
+ // CONVENIENCE SETS OF PROPERTIES
+
+ public static final ImmutableSet<String> ALLOWED_BY_CLIENT =
+ ImmutableSet.of(ZOOKEEPER_CONNECTION, DRILLBIT_CONNECTION, SCHEMA, USER, PASSWORD, IMPERSONATION_TARGET,
+ AUTH_MECHANISM, SERVICE_PRINCIPAL, SERVICE_NAME, SERVICE_HOST, REALM, KEYTAB);
+
+ public static final ImmutableSet<String> ACCEPTED_BY_SERVER = ImmutableSet.of(SCHEMA, IMPERSONATION_TARGET);
+
+ private final Properties properties; // keys must be lower case
+
+ private ConnectionParameters(Properties properties) {
+ this.properties = properties;
— End diff –
for safety, shouldn't properties be copied or constructor made private (so that caller doesn't change it by mistake)?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85646140
— Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/security/SimpleServer.java —
@@ -0,0 +1,138 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.primitives.Ints;
+
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.util.Map;
+
+public class SimpleServer implements SaslServer {
+
+ private boolean completed;
+ private String authorizationId;
+ private final int total;
+ private int count = 0;
+
+ SimpleServer(final int total)
+
+ @Override
+ public String getMechanismName()
+
+ @Override
+ public byte[] evaluateResponse(byte[] response) throws SaslException {
+ if (completed)
+ if (response == null || response.length < 1)
{ + throw new SaslException("Received challenge is empty when secret expected"); + } +
+ if (count == 0) { // first expect authorization ID
+ //This SaslServer simply permits a client to authenticate according to whatever username
+ //was supplied in client's response[]
+ authorizationId = new String(response);
— End diff –
it's probably risky to rely on the default encoding to convert the byte array into a string
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85645955
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java —
@@ -285,9 +279,14 @@ public synchronized boolean reconnect() {
}
private void connect(DrillbitEndpoint endpoint) throws RpcException {
- final FutureHandler f = new FutureHandler();
- client.connect(f, endpoint, props, getUserCredentials());
- f.checkedGet();
+ client.connect(endpoint, parameters, getUserCredentials()).checkedGet();
+ if (client.serverRequiresAuthentication()) {-
- End diff –
-
what if the server doesn't require authentication, shouldn't the client wait too?
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85645696
— Diff: contrib/native/client/cmakeModules/FindSASL.cmake —
@@ -0,0 +1,55 @@
+#
+# 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.
+#
+
+# - Try to find Cyrus SASL
+
+if (MSVC)
+ if(${CMAKE_BUILD_TYPE} MATCHES "Debug")
+ set(SASL_BuildOutputDir "Debug")
+ else()
+ set(SASL_BuildOutputDir "Release")
+ endif()
+ if("${SASL_HOME}" MATCHES "^$")
+ message(" ")
+ message("- Please set the cache variable SASL_HOME to point to the directory with the Cyrus SASL source.")
+ message("- CMAKE will look for Cyrus SASL include files in $SASL_HOME/src/c/include.")
+ message("- CMAKE will look for Cyrus SASL library files in $SASL_HOME/src/c/Debug or $SASL_HOME/src/c/Release.")
+ else()
+ FILE(TO_CMAKE_PATH ${SASL_HOME} SASL_HomePath)
+ set(SASL_LIB_PATHS ${SASL_HomePath}/src/c/${SASL_BuildOutputDir} ${SASL_HomePath}/src/c/x64/${SASL_BuildOutputDir} )
+
+ find_path(SASL_INCLUDE_DIR sasl.h ${Zookeeper_HomePath}/src/c/include)
— End diff –
Zookeeper reference
Github user laurentgo commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r85792190
— Diff: common/src/main/java/org/apache/drill/common/KerberosUtil.java —
@@ -0,0 +1,40 @@
+/**
+ * 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.drill.common;
+
+public final class KerberosUtil {
+// private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(KerberosUtil.class);
+
+ public static final String KERBEROS_SASL_NAME = "GSSAPI";
+
+ public static final String KERBEROS_SIMPLE_NAME = "KERBEROS";
+
+ // primary/instance@REALM
+ public static String getPrincipalFromParts(final String primary, final String instance, final String realm)
+
+ // primary/instance@REALM
+ public static String[] splitPrincipalIntoParts(final String principal) {
— End diff –
what about malformed principal strings? or it seems that instance might be optional too.
You might want to add unit tests to cover multiple cases
I'd like to see associated C++ changes for this before we consider merging this. We shouldn't treat one of the clients as second class.
Github user chunhui-shi commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r78798093
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -246,28 +154,80 @@ protected void handle(UserClientConnectionImpl connection, int rpcType, ByteBuf
public class UserClientConnectionImpl extends RemoteConnection implements UserClientConnection {
private UserSession session;
+ private SaslServer saslServer;
+ private RequestHandler<UserClientConnectionImpl> currentHandler;
+ private UserToBitHandshake inbound;
public UserClientConnectionImpl(SocketChannel channel)
{ super(channel, "user client"); + currentHandler = authFactory == null ? handler : new UserServerAuthenticationHandler(handler); }void disableReadTimeout()
{ getChannel().pipeline().remove(BasicServer.TIMEOUT_HANDLER); }- void setUser(final UserToBitHandshake inbound) throws IOException {
+ void setHandshake(final UserToBitHandshake inbound) throws IOException { + this.inbound = inbound; + }+
{ + throw new IllegalStateException("SASL server already initialized."); + }
+ void initSaslServer(final String mechanismName, final Map<String, ?> properties)
+ throws IllegalStateException, SaslException {
+ if (saslServer != null)+ this.saslServer = authFactory.getMechanism(mechanismName)
-
- End diff –
-
It should be the responsibility of class AuthenticationMechanismFactory to decide what mechanism to provide. Right? So AuthenticationMechanismFactory may also need 'properties' to decide what mechanism to ask for. E.g. Some company may want connections from out of VPN or in VPN to use different mechanisms.
Github user chunhui-shi commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r78791984
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,174 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
+
+ public static class PlainServerFactory implements SaslServerFactory {
+
+ @Override
+ public SaslServer createSaslServer(final String mechanism, final String protocol, final String serverName,
+ final Map<String, ?> props, final CallbackHandler cbh)
+ throws SaslException
+
+ @Override
+ public String[] getMechanismNames(final Map<String, ?> props) {
+ return props == null || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT)) ?
+ new String[]
:
+ new String[0];
+ }
+ }
+
+ @SuppressWarnings("serial")
+ public static class PlainServerProvider extends Provider {
+
+ public PlainServerProvider()
+ }
+
+ private CallbackHandler cbh;
+ private boolean completed;
+ private String authorizationID;
+
+ PlainServer(final CallbackHandler cbh) throws SaslException {
+ if (cbh == null)
+ this.cbh = cbh;
+ }
+
+ @Override
+ public String getMechanismName()
+
+ @Override
+ public byte[] evaluateResponse(byte[] response) throws SaslException {
+ if (completed)
+
+ if (response == null)
+
+ completed = true;
— End diff –
'completed' should be set at the end of the whole authentication since it is the critical flag to decide if the user was authenticated successfully.
Github user chunhui-shi commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r78791283
— Diff: protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java —
@@ -47,157 +47,165 @@ public static void registerAllExtensions(
*/
GOODBYE(2, 2),
/**
+ * <code>SASL_MESSAGE = 24;</code>
+ *
+ * <pre>
+ * user to bit and bit to user
+ * </pre>
+ */
+ SASL_MESSAGE(3, 24),
— End diff –
What is the reason we inject SASL_MESSAGE here and we have to update all definitions below?
Github user chunhui-shi commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r78790827
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,174 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
— End diff –
Is there any reason we have to implement our own PlainServer? What about hadoop's SaslPlainServer?
Github user gparai commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r77429753
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -308,33 +312,57 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
connection.setHandshake(inbound);
try {
+ // TODO(SUDHEESH): MUST FIX THIS VERSION CHECK FIRST BEFORE THE CHECK BELOW
if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
- if (authenticator != null) {
- try {
- String password = "";
- final UserProperties props = inbound.getProperties();
- for (int i = 0; i < props.getPropertiesCount(); i++) {
- Property prop = props.getProperties;
- if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) {
- password = prop.getValue();
- break;
+ connection.setHandshake(inbound);
+
+ if (authFactory != null) {
+ if (inbound.getRpcVersion() <= 5) { // for backward compatibility <= 1.8
+ final String userName = inbound.getCredentials().getUserName();
+ if (logger.isTraceEnabled()) {
+ logger.trace("User {} on connection {} is using an older client (Drill version <= 1.8).",
+ userName, connection.getRemoteAddress());
+ }
+ try {
+ String password = "";
+ final UserProperties props = inbound.getProperties();
+ for (int i = 0; i < props.getPropertiesCount(); i++)Unknown macro: { + Property prop = props.getProperties(i); + if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) { + password = prop.getValue(); + break; + } + }+ final PlainMechanism plainMechanism = authFactory.getPlainMechanism();
{ + throw new UserAuthenticationException("The server no longer supports username/password" + + " based authentication. Please talk to your system administrator."); }
+ if (plainMechanism == null)+ plainMechanism.getAuthenticator().authenticate(userName, password);
+ connection.changeHandlerTo(handler);
+ connection.finalizeSession(userName);
+ respBuilder.setStatus(HandshakeStatus.SUCCESS);-
- End diff –
-
Would it be useful to add log the successful mechanism (where we have `respBuilder.setStatus(HandshakeStatus.SUCCESS);`)?
Github user gparai commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r77429640
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -308,33 +312,57 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
connection.setHandshake(inbound);
try {
+ // TODO(SUDHEESH): MUST FIX THIS VERSION CHECK FIRST BEFORE THE CHECK BELOW
if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
- if (authenticator != null) {
- try {
- String password = "";
- final UserProperties props = inbound.getProperties();
- for (int i = 0; i < props.getPropertiesCount(); i++) {
- Property prop = props.getProperties;
- if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) {
- password = prop.getValue();
- break;
+ connection.setHandshake(inbound);
+
+ if (authFactory != null) {
+ if (inbound.getRpcVersion() <= 5) { // for backward compatibility <= 1.8-
- End diff –
-
Can we use something like MIN_COMPATIBLE_RPC_VERSION for 5?
Github user gparai commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r77429619
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -308,33 +312,57 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
connection.setHandshake(inbound);
— End diff –
Why do we call `connection.setHandshake(inbound);` again below?
Github user gparai commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r77429494
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java —
@@ -308,33 +312,57 @@ public BitToUserHandshake getHandshakeResponse(UserToBitHandshake inbound) throw
connection.setHandshake(inbound);
try {
+ // TODO(SUDHEESH): MUST FIX THIS VERSION CHECK FIRST BEFORE THE CHECK BELOW
— End diff –
Does this need to be fixed? If not, then we should remove the comment.
Github user gparai commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r77429451
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/AuthenticationMechanismFactory.java —
@@ -0,0 +1,182 @@
+/**
+ * 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.drill.exec.rpc.security;
+
+import com.google.common.base.Function;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.map.CaseInsensitiveMap;
+import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.rpc.security.kerberos.KerberosMechanism;
+import org.apache.drill.exec.rpc.security.plain.PlainMechanism;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class AuthenticationMechanismFactory implements AutoCloseable {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(AuthenticationMechanismFactory.class);
+
+ private final Map<String, AuthenticationMechanism> mechanisms = CaseInsensitiveMap.newHashMapWithExpectedSize(5);
+
+ @SuppressWarnings("unchecked")
+ public AuthenticationMechanismFactory(final ScanResult scan, final DrillConfig config,
+ final List<String> configuredMechanisms)
+ throws DrillbitStartupException {
+ logger.debug("Configuring authentication mechanisms: {}", configuredMechanisms);
+ // transform all names to uppercase
+ final Set<String> configuredMechanismsSet = Sets.newHashSet(Iterators.transform(configuredMechanisms.iterator(),
+ new Function<String, String>() {
+ @Nullable
+ @Override
+ public String apply(@Nullable String input)
+ }));
+
+ // First, load Drill provided out-of-box mechanisms
+ if (configuredMechanismsSet.contains(PlainMechanism.SIMPLE_NAME))
+
+ if (configuredMechanismsSet.contains(KerberosMechanism.SIMPLE_NAME)) {
+ logger.trace("Kerberos mechanism enabled.");
+ final String servicePrincipal = config.getString("drill.exec.security.auth.principal");
+ final String keytab = config.getString("drill.exec.security.auth.keytab");
+
+ try { // Kerberos mechanism requires a service to login
+ final Configuration conf = new Configuration();
+ conf.set(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION, "KERBEROS");
+ // To parse non-typical principal name, uncomment below line
+ // CommonConfigurationKeys.HADOOP_SECURITY_AUTH_TO_LOCAL, rules);
+ UserGroupInformation.setConfiguration(conf);
+ UserGroupInformation.loginUserFromKeytab(servicePrincipal, keytab);
+ logger.trace("Login successful for user: {}", servicePrincipal);
+ } catch (IOException e) {
+ throw new DrillbitStartupException("Drillbit service login failed", e);
— End diff –
Rather than failing with an exception can we simply not add Kerberos to the list of mechanisms i.e. we do not error out but rather add valid mechanisms.
Github user gparai commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r77429414
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/security/plain/PlainServer.java —
@@ -0,0 +1,174 @@
+/**
+ * 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.drill.exec.rpc.security.plain;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslException;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslServerFactory;
+import java.io.IOException;
+import java.security.Provider;
+import java.util.Map;
+
+/**
+ * Plain SaslServer implementation. See https://tools.ietf.org/html/rfc4616
+ */
+public class PlainServer implements SaslServer {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlainServer.class);
+
+ public static class PlainServerFactory implements SaslServerFactory {
+
+ @Override
+ public SaslServer createSaslServer(final String mechanism, final String protocol, final String serverName,
+ final Map<String, ?> props, final CallbackHandler cbh)
+ throws SaslException
+
+ @Override
+ public String[] getMechanismNames(final Map<String, ?> props) {
+ return props == null || "false".equals(props.get(Sasl.POLICY_NOPLAINTEXT)) ?
+ new String[]
:
+ new String[0];
+ }
+ }
+
+ @SuppressWarnings("serial")
+ public static class PlainServerProvider extends Provider {
+
+ public PlainServerProvider()
+ }
+
+ private CallbackHandler cbh;
+ private boolean completed;
+ private String authorizationID;
+
+ PlainServer(final CallbackHandler cbh) throws SaslException {
+ if (cbh == null)
+ this.cbh = cbh;
+ }
+
+ @Override
+ public String getMechanismName()
+
+ @Override
+ public byte[] evaluateResponse(byte[] response) throws SaslException {
+ if (completed)
+
+ if (response == null)
+
+ completed = true;
+ final String payload;
+ try
catch (final Exception e)
{ + throw new SaslException("Received corrupt response", e); + } +
+ // Separator defined in PlainClient is 0
+ // three parts: [ authorizationID, authenticationID, password ]
+ final String[] parts = payload.split("\u0000", 3);
+ if (parts.length != 3)
+ if (parts[0].isEmpty())
{ + parts[0] = parts[1]; // authorizationID = authenticationID + } +
+ final NameCallback nc = new NameCallback("PLAIN authentication ID: ");
+ nc.setName(parts[1]);
+ final PasswordCallback pc = new PasswordCallback("PLAIN password: ", false);
+ pc.setPassword(parts[2].toCharArray());
+
+ final AuthorizeCallback ac = new AuthorizeCallback(parts[1], parts[0]);
+ try {
+ cbh.handle(new Callback[]
);
+ } catch (final UnsupportedCallbackException | IOException e)
+ authorizationID = ac.getAuthorizedID();
+ return null;
+ }
+
+ @Override
+ public boolean isComplete()
+
+ @Override
+ public String getAuthorizationID() {
+ if (completed)
+ throw new IllegalStateException("PLAIN authentication not completed");
+ }
+
+ @Override
+ public Object getNegotiatedProperty(String propName) {
+ if (completed)
+ throw new IllegalStateException("PLAIN authentication not completed");
+ }
+
+ @Override
+ public byte[] wrap(byte[] outgoing, int offset, int len) throws SaslException {
+ if (completed) {
— End diff –
Do we need to wait for 'completed' before throwing the error?
Github user gparai commented on a diff in the pull request:
https://github.com/apache/drill/pull/578#discussion_r77403393
— Diff: exec/java-exec/src/main/java/org/apache/drill/exec/client/AuthenticationUtil.java —
@@ -0,0 +1,157 @@
+/**
+ * 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.drill.exec.client;
+
+import org.apache.drill.common.config.ConnectionParams;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.security.PrivilegedExceptionAction;
+
+public final class AuthenticationUtil {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AuthenticationUtil.class);
+
+ private static final String PLAIN_NAME = "PLAIN";
+
+ private static final String KERBEROS_NAME = "GSSAPI";
+
+
+ public static String getMechanismFromParams(final ConnectionParams params) {
+ if (params.getParam(ConnectionParams.AUTH_MECHANISM) != null)
+ if (params.getParam(ConnectionParams.PASSWORD) != null)
{ + return PLAIN_NAME; + } + if (params.getParam(ConnectionParams.PRINCIPAL) != null ||
+ (params.getParam(ConnectionParams.SERVICE_HOST) != null &&
+ params.getParam(ConnectionParams.SERVICE_NAME) != null))
+ return null;
+ }
+
+ public static SaslClient getPlainSaslClient(final String userName, final String password) throws SaslException {
+ return Sasl.createSaslClient(new String[]
, null /* authorizationID */, null, null, null,
+ new CallbackHandler() {
+ @Override
+ public void handle(final Callback[] callbacks) throws IOException, UnsupportedCallbackException {
+ for (final Callback callback : callbacks) {
+ if (callback instanceof NameCallback)
+ if (callback instanceof PasswordCallback)
{ + PasswordCallback.class.cast(callback).setPassword(password.toCharArray()); + continue; + } + throw new UnsupportedCallbackException(callback);
+ }
+ }
+ });
+ }
+
+ public static String deriveKerberosName(final ConnectionParams params) {
+ final String principal = params.getParam(ConnectionParams.PRINCIPAL);
+ if (principal != null)
+
+ final StringBuilder principalBuilder = new StringBuilder();
+ final String serviceNameProp = params.getParam(ConnectionParams.SERVICE_NAME);
+ if (serviceNameProp != null)
else {
+ principalBuilder.append(System.getProperty("drill.service.name", "drill"));
— End diff –
Can this not be null?
GitHub user sudheeshkatkam opened a pull request:
https://github.com/apache/drill/pull/578
DRILL-4280: Kerberos Authentication
I am posting these changes from review. There are four commits in this pull request. All changes squashed are in [this branch](https://github.com/sudheeshkatkam/drill/tree/DRILL-4280-squashed). There are 4 more commits that are in the squashed branch but not in this PR (tests, client changes including C++, etc.)
The last commit (not here yet) will move forward the RPC version that will starting using SASL for authentication.
Please refer to the design doc for details.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/sudheeshkatkam/drill DRILL-4280-PR
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/drill/pull/578.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #578
commit e4b0c6afdc358123fc6b6e911cf37ec347bda242
Author: Sudheesh Katkam <skatkam@maprtech.com>
Date: 2016-07-25T21:48:15Z
DRILL-4280: HYGIENE
+ Pass references of BootstrapContext to ServiceEngine and down
commit 27a1638e2f2eb3aaf582c3d3398960d6dcee979b
Author: Sudheesh Katkam <skatkam@maprtech.com>
Date: 2016-07-25T22:47:37Z
DRILL-4280: CORE
+ Define SaslStatus and SaslMessage messages in protocol
+ Add new "authenticationMechanisms" field to BitToUserHandshake
commit 409318de6b7b73467c8d1052c7e5eacb72cbeb07
Author: Sudheesh Katkam <skatkam@maprtech.com>
Date: 2016-07-26T21:42:49Z
DRILL-4280: CORE
+ Add new RequestHandler interface, and two implementations used in
UserServer to handle authentication first and then query requests
+ UserAuthenticationHandler handles SASL messages on server side
+ Move UserServer#handle logic to UserServerRequestHandler
+ Add authenticate method in UserClient
commit 692755b3991c33a501cc36238d9f9c04f66fe068
Author: Sudheesh Katkam <skatkam@maprtech.com>
Date: 2016-08-31T17:40:53Z
DRILL-4280: CORE
+ Add AuthenticationMechanism interface
+ Kerberos implementation
+ includes SaslServer and SaslClient wrappers
+ Plain implementation
+ PlainServer implements SaslServer (unavailable in Java)
for username/password based authentication
+ retrofit user authenticator
+ add logic for backward compatibility
+ Custom SASL mechanisms are discovered through the SaslMechanism
annotation
+ FastSaslServerFactory caches SaslServer factories
If the main (only?) use case is client-to-drill communication, rather than drill-to-drill communication, and since Avatica already does Kerberos (see CALCITE-1159) it seems to me that a client based on Avatica would be an alternative. This would solve DRILL-4791 as well.
What are the pros and cons of Avatica?
Kerberos SPNEGO is tracked separately. Close this one.