[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-24 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/drill/pull/578


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-22 Thread sudheeshkatkam
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 
+#include 
+#include 
+#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).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-22 Thread sudheeshkatkam
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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;)
+throw std::runtime_error(errMsg.str().c_str());
+}
+{ // 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_pwd_secret) {
+free(m_pwd_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(context);
+
+if (SASL_CB_PASS == id) {
+*psecret = authenticator->m_pwd_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
mechanisms, exec::shared::SaslMessage& response) {
+// find and set parameters
+std::string authMechanismToUse;
+std::string serviceName;
+std::string serviceHost;
+for (size_t i = 0; i < 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-22 Thread sudheeshkatkam
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).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-22 Thread laurentgo
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
+implements RpcOutcomeListener {
+  private static final org.slf4j.Logger logger =
+  
org.slf4j.LoggerFactory.getLogger(AuthenticationOutcomeListener.class);
+
+  private static final ImmutableMap 
CHALLENGE_PROCESSORS;
+  static {
+final Map map = new 
EnumMap<>(SaslStatus.class);
+map.put(SaslStatus.SASL_IN_PROGRESS, new SaslInProgressProcessor());
+map.put(SaslStatus.SASL_SUCCESS, new SaslSuccessProcessor());
+map.put(SaslStatus.SASL_FAILED, new SaslFailedProcessor());
+CHALLENGE_PROCESSORS = Maps.immutableEnumMap(map);
+  }
+
+  private final BasicClient 
client;
+  private final R connection;
+  private final T saslRpcType;
+  private final UserGroupInformation ugi;
+  private final RpcOutcomeListener rpcOutcomeListener;
+
+  public AuthenticationOutcomeListener(BasicClient client,
+   R connection, T saslRpcType, 
UserGroupInformation ugi,
+   RpcOutcomeListener 
rpcOutcomeListener) {
+this.client = client;
+this.connection = connection;
+this.saslRpcType = saslRpcType;
+this.ugi = ugi;
+this.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()) {
+responseData = ByteString.copyFrom(evaluateChallenge(ugi, 
saslClient, new byte[0]));
+  } 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) {
+  rpcOutcomeListener.failed(RpcException.mapException(e));
+}
+  }
+
+  @Override
+  public void failed(RpcException 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-22 Thread laurentgo
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
+ * 
+ * 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 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 {
+Security.addProvider(new PlainServer.PlainServerProvider());
+  }
+
+  private final UserAuthenticator authenticator;
+
+  public PlainFactory() {
+this.authenticator = null;
+  }
+
+  public PlainFactory(final UserAuthenticator authenticator) {
+this.authenticator = authenticator;
+  }
+
+  @Override
+  public String getSimpleName() {
+return SIMPLE_NAME;
+  }
+
+  @Override
+  public UserGroupInformation createAndLoginUser(Map 
properties) throws IOException {
+final Configuration conf = new Configuration();
+UserGroupInformation.setConfiguration(conf);
+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 SaslServer createSaslServer(final UserGroupInformation ugi, final 
Map properties)
+  throws SaslException {
+return 
FastSaslServerFactory.getInstance().createSaslServer(SIMPLE_NAME, null /** 
protocol */,
+null /** serverName */, properties, new 
PlainServerCallbackHandler());
+  }
+
+  @Override
+  public SaslClient createSaslClient(final UserGroupInformation ugi, final 
Map 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[]{SIMPLE_NAME},
+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) {
+

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-22 Thread laurentgo
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 {
+super(allocator, context);
+this.handler = handler;
+  }
+
+  @Override
+  public String getName() {
+return "control"; // unused
+  }
+
+  ControlMessageHandler getMessageHandler() {
+return handler;
+  }
+
+  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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-22 Thread laurentgo
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) {
 send(queryResultHandler.getWrappedListener(resultsListener), 
RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  public void connect(RpcConnectionHandler handler, 
DrillbitEndpoint endpoint,
-  UserProperties props, UserBitShared.UserCredentials 
credentials) {
+  public CheckedFuture 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;
+  }
+
+  /**
+   * Check (after {@link #connect connecting}) if server requires 
authentication.
+   *
+   * @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},
+   * returns null. If called after {@link #connect connecting}, returns a 
list of supported mechanisms
+   * iff authentication is required.
+   *
+   * @return list of supported authentication mechanisms
+   */
+  public List getSupportedAuthenticationMechanisms() {
--- End diff --

Providing a callback for authentication seems a more robust approach 
compared to calling another method with a new set of properties...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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
--- End diff --

Fixed all the warnings, thanks for the catch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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
+ * 
+ * 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.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 implements RequestHandler {
+  private static final org.slf4j.Logger logger =
+  org.slf4j.LoggerFactory.getLogger(ServerAuthenticationHandler.class);
+
+  private static final ImmutableMap 
RESPONSE_PROCESSORS;
+
+  static {
+final Map map = new 
EnumMap<>(SaslStatus.class);
+map.put(SaslStatus.SASL_START, new SaslStartProcessor());
+map.put(SaslStatus.SASL_IN_PROGRESS, new SaslInProgressProcessor());
+map.put(SaslStatus.SASL_SUCCESS, new SaslSuccessProcessor());
+map.put(SaslStatus.SASL_FAILED, new SaslFailedProcessor());
+RESPONSE_PROCESSORS = Maps.immutableEnumMap(map);
+  }
+
+  private final RequestHandler requestHandler;
+  private final int saslRequestTypeValue;
+  private final T saslResponseType;
+
+  public ServerAuthenticationHandler(final RequestHandler 
requestHandler, final int saslRequestTypeValue,
+ final T saslResponseType) {
+this.requestHandler = requestHandler;
+this.saslRequestTypeValue = saslRequestTypeValue;
+this.saslResponseType = 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 {
+saslResponse = SaslMessage.PARSER.parseFrom(new 
ByteBufInputStream(pBody));
+  } 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 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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) {
 send(queryResultHandler.getWrappedListener(resultsListener), 
RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  public void connect(RpcConnectionHandler handler, 
DrillbitEndpoint endpoint,
-  UserProperties props, UserBitShared.UserCredentials 
credentials) {
+  public CheckedFuture 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;
+  }
+
+  /**
+   * Check (after {@link #connect connecting}) if server requires 
authentication.
+   *
+   * @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},
+   * returns null. If called after {@link #connect connecting}, returns a 
list of supported mechanisms
+   * iff authentication is required.
+   *
+   * @return list of supported authentication mechanisms
+   */
+  public List 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);
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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
+ * 
+ * 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 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).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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 {
+super(allocator, context);
+this.handler = handler;
+  }
+
+  @Override
+  public String getName() {
+return "control"; // unused
+  }
+
+  ControlMessageHandler getMessageHandler() {
+return handler;
+  }
+
+  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).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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  RpcCommand
+  getInitialCommand(final RpcCommand command) {
+if (config.getAuthMechanismToUse() == null) {
+  return super.getInitialCommand(command);
+} else {
+  return new AuthenticationCommand<>(command);
--- End diff --

Thanks for the catch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
+  final SaslClient saslClient;
+  try {
+saslClient = 
config.getAuthFactory(handshake.getAuthenticationMechanismsList())
+.createSaslClient(UserGroupInformation.getLoginUser(),
+config.getSaslClientProperties(remoteEndpoint));
+  } 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) {
+throw new RpcException(String.format("Drillbit (%s) does not 
require auth, but auth is enabled.",
+remoteEndpoint.getAddress()));
+  }
 }
   }
 
   @Override
-  protected void finalizeConnection(BitServerHandshake handshake, 
DataClientConnection connection) {
+  protected  RpcCommand
+  getInitialCommand(final RpcCommand command) {
+if (config.getAuthMechanismToUse() == null) {
+  return super.getInitialCommand(command);
+} else {
+  return new AuthenticationCommand<>(command);
--- End diff --

Yes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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
+ * 
+ * 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 javax.security.sasl.SaslServer;
+import java.io.IOException;
+
+public interface 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).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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
+implements RpcOutcomeListener {
+  private static final org.slf4j.Logger logger =
+  
org.slf4j.LoggerFactory.getLogger(AuthenticationOutcomeListener.class);
+
+  private static final ImmutableMap 
CHALLENGE_PROCESSORS;
+  static {
+final Map map = new 
EnumMap<>(SaslStatus.class);
+map.put(SaslStatus.SASL_IN_PROGRESS, new SaslInProgressProcessor());
+map.put(SaslStatus.SASL_SUCCESS, new SaslSuccessProcessor());
+map.put(SaslStatus.SASL_FAILED, new SaslFailedProcessor());
+CHALLENGE_PROCESSORS = Maps.immutableEnumMap(map);
+  }
+
+  private final BasicClient 
client;
+  private final R connection;
+  private final T saslRpcType;
+  private final UserGroupInformation ugi;
+  private final RpcOutcomeListener rpcOutcomeListener;
+
+  public AuthenticationOutcomeListener(BasicClient client,
+   R connection, T saslRpcType, 
UserGroupInformation ugi,
+   RpcOutcomeListener 
rpcOutcomeListener) {
+this.client = client;
+this.connection = connection;
+this.saslRpcType = saslRpcType;
+this.ugi = ugi;
+this.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()) {
+responseData = ByteString.copyFrom(evaluateChallenge(ugi, 
saslClient, new byte[0]));
+  } 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) {
+  rpcOutcomeListener.failed(RpcException.mapException(e));
+}
+  }
+
+  @Override
+  public void 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread sudheeshkatkam
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._


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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
+ * 
+ * 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 javax.security.sasl.SaslClient;
+
+public interface ClientConnection extends RemoteConnection {
--- End diff --

Is `ClientConnection` the right place to pass along the saslClient vs 
`AuthenticationCommand`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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
+ * 
+ * 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.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 implements RequestHandler {
+  private static final org.slf4j.Logger logger =
+  org.slf4j.LoggerFactory.getLogger(ServerAuthenticationHandler.class);
+
+  private static final ImmutableMap 
RESPONSE_PROCESSORS;
+
+  static {
+final Map map = new 
EnumMap<>(SaslStatus.class);
+map.put(SaslStatus.SASL_START, new SaslStartProcessor());
+map.put(SaslStatus.SASL_IN_PROGRESS, new SaslInProgressProcessor());
+map.put(SaslStatus.SASL_SUCCESS, new SaslSuccessProcessor());
+map.put(SaslStatus.SASL_FAILED, new SaslFailedProcessor());
+RESPONSE_PROCESSORS = Maps.immutableEnumMap(map);
+  }
+
+  private final RequestHandler requestHandler;
+  private final int saslRequestTypeValue;
+  private final T saslResponseType;
+
+  public ServerAuthenticationHandler(final RequestHandler 
requestHandler, final int saslRequestTypeValue,
+ final T saslResponseType) {
+this.requestHandler = requestHandler;
+this.saslRequestTypeValue = saslRequestTypeValue;
+this.saslResponseType = 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 {
+saslResponse = SaslMessage.PARSER.parseFrom(new 
ByteBufInputStream(pBody));
+  } 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 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
+  final SaslClient saslClient;
+  try {
+saslClient = 
config.getAuthFactory(handshake.getAuthenticationMechanismsList())
+.createSaslClient(UserGroupInformation.getLoginUser(),
+config.getSaslClientProperties(remoteEndpoint));
+  } 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) {
+throw new RpcException(String.format("Drillbit (%s) does not 
require auth, but auth is enabled.",
+remoteEndpoint.getAddress()));
+  }
 }
   }
 
   @Override
-  protected void finalizeConnection(BitServerHandshake handshake, 
DataClientConnection connection) {
+  protected  RpcCommand
+  getInitialCommand(final RpcCommand 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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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;)
+throw std::runtime_error(errMsg.str().c_str());
+}
+{ // 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_pwd_secret) {
+free(m_pwd_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(context);
+
+if (SASL_CB_PASS == id) {
+*psecret = authenticator->m_pwd_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
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(); 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 {@link java.util.HashMap}, with key 
case-insensitivity, of expected size.
--- End diff --

okay :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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) {
 send(queryResultHandler.getWrappedListener(resultsListener), 
RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  public void connect(RpcConnectionHandler handler, 
DrillbitEndpoint endpoint,
-  UserProperties props, UserBitShared.UserCredentials 
credentials) {
+  public CheckedFuture 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},
+   * returns null. If called after {@link #connect connecting}, returns a 
list of supported mechanisms
+   * iff authentication is required.
+   *
+   * @return list of supported authentication mechanisms
+   */
+  public List getSupportedAuthenticationMechanisms() {
+return serverAuthMechanisms;
+  }
+
+  /**
+   * 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
+   * authentication fails.
+   *
+   * This method uses properties provided at {@link #connect connection 
time} and override them with the
+   * given properties, if any.
+   *
+   * @param overrides parameter overrides
+   * @return result of authentication request
+   */
+  public CheckedFuture 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...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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;)
+throw std::runtime_error(errMsg.str().c_str());
+}
+{ // 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_pwd_secret) {
+free(m_pwd_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(context);
+
+if (SASL_CB_PASS == id) {
+*psecret = authenticator->m_pwd_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
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(); 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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
+ * 
+ * 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 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 {
+Security.addProvider(new PlainServer.PlainServerProvider());
+  }
+
+  private final UserAuthenticator authenticator;
+
+  public PlainFactory() {
+this.authenticator = null;
+  }
+
+  public PlainFactory(final UserAuthenticator authenticator) {
+this.authenticator = authenticator;
+  }
+
+  @Override
+  public String getSimpleName() {
+return SIMPLE_NAME;
+  }
+
+  @Override
+  public UserGroupInformation createAndLoginUser(Map 
properties) throws IOException {
+final Configuration conf = new Configuration();
+UserGroupInformation.setConfiguration(conf);
+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 SaslServer createSaslServer(final UserGroupInformation ugi, final 
Map properties)
+  throws SaslException {
+return 
FastSaslServerFactory.getInstance().createSaslServer(SIMPLE_NAME, null /** 
protocol */,
+null /** serverName */, properties, new 
PlainServerCallbackHandler());
+  }
+
+  @Override
+  public SaslClient createSaslClient(final UserGroupInformation ugi, final 
Map 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[]{SIMPLE_NAME},
+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) {
+

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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) {
 send(queryResultHandler.getWrappedListener(resultsListener), 
RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  public void connect(RpcConnectionHandler handler, 
DrillbitEndpoint endpoint,
-  UserProperties props, UserBitShared.UserCredentials 
credentials) {
+  public CheckedFuture 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;
+  }
+
+  /**
+   * Check (after {@link #connect connecting}) if server requires 
authentication.
+   *
+   * @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},
+   * returns null. If called after {@link #connect connecting}, returns a 
list of supported mechanisms
+   * iff authentication is required.
+   *
+   * @return list of supported authentication mechanisms
+   */
+  public List getSupportedAuthenticationMechanisms() {
--- End diff --

what's the use-case? debugging?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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){
 return ret;
 }
-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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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(i);
-if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) {
+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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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) {
+super(channel, 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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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
--- End diff --

Your generic type is not fully defined:
`AbstractServerConnection>`
(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>` 
instead (as we should rely on the interface instead of the abstract class)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+s_initialized = true;
+
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_secret) {
+free(m_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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 MECHANISM_MAPPING;
--- End diff --

not sure why this is public, isn't this something internal?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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;)
+throw std::runtime_error(errMsg.str().c_str());
+}
+{ // 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_pwd_secret) {
+free(m_pwd_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(context);
+
+if (SASL_CB_PASS == id) {
+*psecret = authenticator->m_pwd_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
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(); 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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) {
+return checkNotNull(primary) + "/" +
+checkNotNull(instance) + "@" +
+checkNotNull(realm);
+  }
+
+  /**
+   * Expects principal of the format primary/instance@REALM.
+   *
+   * @param principal principal
+   * @return components
+   */
+  public static String[] splitPrincipalIntoParts(final String principal) {
+final String[] components = principal.split("[/@]");
+checkState(components.length == 3);
+checkNotNull(components[0]);
+checkNotNull(components[1]);
+checkNotNull(components[2]);
+return components;
+  }
+
+  public static String canonicalizedInstanceName(String instanceName, 
final String canonicalName) {
--- End diff --

(style) for consistency, use verb present tense for methods 
(canonicalizeInstanceName)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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;)
+throw std::runtime_error(errMsg.str().c_str());
+}
+{ // 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_pwd_secret) {
+free(m_pwd_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(context);
+
+if (SASL_CB_PASS == id) {
+*psecret = authenticator->m_pwd_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
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(); 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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(i);
-if (UserSession.PASSWORD.equalsIgnoreCase(prop.getKey())) {
+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()
+  .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) {
   return handleFailure(respBuilder, 
HandshakeStatus.AUTH_FAILED, ex.getMessage(), 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)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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
+ * 
+ * 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 javax.security.sasl.SaslServer;
+import java.io.IOException;
+
+public interface 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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 
+#include 
+#include 
+#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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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
+ * 
+ * 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 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.`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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 {
+super(allocator, context);
+this.handler = handler;
+  }
+
+  @Override
+  public String getName() {
+return "control"; // unused
+  }
+
+  ControlMessageHandler getMessageHandler() {
+return handler;
+  }
+
+  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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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  RpcCommand
+  getInitialCommand(final RpcCommand command) {
+if (config.getAuthMechanismToUse() == null) {
+  return super.getInitialCommand(command);
+} else {
+  return new AuthenticationCommand<>(command);
--- End diff --

shouldn't we use `super.getInitialCommand(command)` too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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
+implements RpcOutcomeListener {
+  private static final org.slf4j.Logger logger =
+  
org.slf4j.LoggerFactory.getLogger(AuthenticationOutcomeListener.class);
+
+  private static final ImmutableMap 
CHALLENGE_PROCESSORS;
+  static {
+final Map map = new 
EnumMap<>(SaslStatus.class);
+map.put(SaslStatus.SASL_IN_PROGRESS, new SaslInProgressProcessor());
+map.put(SaslStatus.SASL_SUCCESS, new SaslSuccessProcessor());
+map.put(SaslStatus.SASL_FAILED, new SaslFailedProcessor());
+CHALLENGE_PROCESSORS = Maps.immutableEnumMap(map);
+  }
+
+  private final BasicClient 
client;
+  private final R connection;
+  private final T saslRpcType;
+  private final UserGroupInformation ugi;
+  private final RpcOutcomeListener rpcOutcomeListener;
+
+  public AuthenticationOutcomeListener(BasicClient client,
+   R connection, T saslRpcType, 
UserGroupInformation ugi,
+   RpcOutcomeListener 
rpcOutcomeListener) {
+this.client = client;
+this.connection = connection;
+this.saslRpcType = saslRpcType;
+this.ugi = ugi;
+this.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()) {
+responseData = ByteString.copyFrom(evaluateChallenge(ugi, 
saslClient, new byte[0]));
+  } 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) {
+  rpcOutcomeListener.failed(RpcException.mapException(e));
+}
+  }
+
+  @Override
+  public void failed(RpcException 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-21 Thread laurentgo
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-15 Thread superbstreak
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-14 Thread sohami
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-13 Thread bitblender
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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-13 Thread sohami
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
+ * 
+ * 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.data;
--- End diff --

Please move this file under _org.apache.drill.exec.rpc_ package.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-08 Thread sudheeshkatkam
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) {
 send(queryResultHandler.getWrappedListener(resultsListener), 
RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  public void connect(RpcConnectionHandler handler, 
DrillbitEndpoint endpoint,
-  UserProperties props, UserBitShared.UserCredentials 
credentials) {
+  public CheckedFuture 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;
+  }
+
+  /**
+   * Check (after {@link #connect connecting}) if server requires 
authentication.
+   *
+   * @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},
+   * returns null. If called after {@link #connect connecting}, returns a 
list of supported mechanisms
+   * iff authentication is required.
+   *
+   * @return list of supported authentication mechanisms
+   */
+  public List getSupportedAuthenticationMechanisms() {
--- End diff --

non-internal API


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-08 Thread sudheeshkatkam
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-08 Thread sudheeshkatkam
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){
 return ret;
 }
-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);
+} catch (std::runtime_error& e) {
--- End diff --

Suggestion for an alternative?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-08 Thread sudheeshkatkam
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 
+#include 
+#include 
+#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 
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-08 Thread sudheeshkatkam
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){
 return ret;
 }
-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.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-08 Thread sudheeshkatkam
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 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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-08 Thread sudheeshkatkam
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 
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
+  if (config.getAuthProvider() == null) {
+throw new RpcException(String.format("Drillbit (%s) requires auth, 
but auth is not configured.",
+remoteEndpoint.getAddress()));
+  }
+  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()
+.getAuthenticatorFactory(config.getAuthMechanismToUse())
+.createSaslClient(UserGroupInformation.getLoginUser(),
+config.getSaslClientProperties(remoteEndpoint));
+  } 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)).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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
+ * 
+ * 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.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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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) {
-  throw new RpcException(String.format("Invalid rpc version.  
Expected %d, actual %d.", inbound.getRpcVersion(), 
ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  
Expected %d, actual %d.",
+  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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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 {
 optional string application = 6;// example: Tableau 9.3
 }
 
+enum SaslSupport {
--- End diff --

Yes, SASL_ENC (for encryption).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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 {@link java.util.HashMap}, with key 
case-insensitivity, of expected size.
--- End diff --

The function name is `newHashMapWithExpectedSize` (following the convention 
of other functions in the class).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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("Failed to login.", cause);
+  }
+  

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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 {
+  private static final org.slf4j.Logger logger =
+  
org.slf4j.LoggerFactory.getLogger(UserClientAuthenticationHandler.class);
+
+  private static final ImmutableMap 
CHALLENGE_PROCESSORS =
+  ImmutableMap.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 settableFuture;
+
+  public UserClientAuthenticationHandler(UserClient client, 
UserGroupInformation ugi,
+ SettableFuture 
settableFuture) {
+this.client = client;
+this.ugi = ugi;
+this.settableFuture = settableFuture;
+  }
+
+  public void initiate(final String mechanismName) {
+try {
+  final ByteString responseData;
+  final SaslClient saslClient = client.getSaslClient();
+  if (saslClient.hasInitialResponse()) {
+responseData = ByteString.copyFrom(evaluateChallenge(ugi, 
saslClient, new byte[0]));
+  } 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) {
+  settableFuture.setException(e);
+}
+  }
+
+  @Override
+  public void failed(RpcException ex) {
+settableFuture.setException(new SaslException("Unexpected failure", 
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) {
+  settableFuture.setException(new SaslException("Server sent a corrupt 
message."));
+} else {
+  try {
+final SaslChallengeContext context =
+new SaslChallengeContext(value, client.getSaslClient(), ugi, 
settableFuture);
+
+final SaslMessage saslResponse = processor.process(context);
+
+if (saslResponse 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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 {
 runTest(TEST_USER_2, TEST_USER_2_PASSWORD);
   }
 
-
   @Test
   public void negativeUserAuth() throws Exception {
 negativeAuthHelper(TEST_USER_1, "blah.. blah..");
 negativeAuthHelper(TEST_USER_2, "blah.. blah..");
-negativeAuthHelper(TEST_USER_2, "");
 negativeAuthHelper("invalidUserName", "blah.. blah..");
   }
 
   @Test
+  public void emptyPassword() throws Exception {
+try {
+  runTest(TEST_USER_2, "");
+  fail("Expected an exception.");
+} 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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+s_initialized = true;
+
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_secret) {
+free(m_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
--- End diff --

Thanks for the catch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread sudheeshkatkam
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_secret) {
+free(m_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(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);
+*psecret = authenticator->m_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
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(i);
+const std::string value = m_properties->valueAt(i);
+
+if (USERPROP_SERVICE_HOST == key) {
+serviceHost 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
--- End diff --

+1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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){
 return ret;
 }
-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);
+} catch (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...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 {@link java.util.HashMap}, 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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 {
 optional string application = 6;// example: Tableau 9.3
 }
 
+enum SaslSupport {
--- End diff --

do you foresee more than two values?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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) {
-  throw new RpcException(String.format("Invalid rpc version.  
Expected %d, actual %d.", inbound.getRpcVersion(), 
ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  
Expected %d, actual %d.",
+  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)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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) {
+throw std::runtime_error("Failed to load authentication 
libraries. code: " + 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_secret) {
+free(m_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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){
 return ret;
 }
-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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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) {
+throw std::runtime_error("Failed to load authentication 
libraries. code: " + 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_secret) {
+free(m_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(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);
+*psecret = authenticator->m_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
mechanisms, exec::shared::SaslMessage& response) {
+// find and set parameters
+std::string authMechanismToUse;
+std::string serviceName;
+std::string 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 
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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
+  if (config.getAuthProvider() == null) {
+throw new RpcException(String.format("Drillbit (%s) requires auth, 
but auth is not configured.",
+remoteEndpoint.getAddress()));
+  }
+  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)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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){
 return ret;
 }
-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);
+} catch (std::runtime_error& e) {
+return handleConnError(CONN_AUTH_FAILED, e.what());
+}
+
+startMessageListener();
+initiateAuthentication();
+
+{ // block until SASL exchange is complete
+boost::mutex::scoped_lock lock(m_saslMutex);
+while (!m_saslDone) {
+

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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){
 return ret;
 }
-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(...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+// 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) {
+throw std::runtime_error("Failed to load authentication 
libraries. code: " + 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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_secret) {
+free(m_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(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);
+*psecret = authenticator->m_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
mechanisms, exec::shared::SaslMessage& response) {
+// find and set parameters
+std::string authMechanismToUse;
+std::string serviceName;
+std::string 

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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
+ * 
+ * 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.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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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 
+#include 
+#include 
+#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 
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-07 Thread laurentgo
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), ControlRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
+  if (config.getAuthProvider() == null) {
+throw new RpcException(String.format("Drillbit (%s) requires auth, 
but auth is not configured.",
+remoteEndpoint.getAddress()));
+  }
+  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()
+.getAuthenticatorFactory(config.getAuthMechanismToUse())
+.createSaslClient(UserGroupInformation.getLoginUser(),
+config.getSaslClientProperties(remoteEndpoint));
+  } 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...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-06 Thread sohami
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+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) {
+DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << i << " : " << 
mechanisms[i] << std::endl;)
+i++;
+}
+}
+s_initialized = true;
+}
+}
+}
+
+SaslAuthenticatorImpl::~SaslAuthenticatorImpl() {
+if (m_secret) {
+free(m_secret);
+}
+// may be used to negotiated security layers before disposing in the 
future
+if (m_pConnection) {
+sasl_dispose(_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(context);
+
+if ((SASL_CB_USER == id || SASL_CB_AUTHNAME == id)
+&& username != NULL) {
+*result = username->c_str();
+// *len = (unsigned int) username->length();
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::passwordCallback(sasl_conn_t *conn, void 
*context, int id, sasl_secret_t **psecret) {
+const SaslAuthenticatorImpl* const authenticator = static_cast(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);
+*psecret = authenticator->m_secret;
+}
+return SASL_OK;
+}
+
+int SaslAuthenticatorImpl::init(const std::vector& 
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(i);
+const std::string value = m_properties->valueAt(i);
+
+if (USERPROP_SERVICE_HOST == key) {
+serviceHost = value;

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-02 Thread sohami
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 
connectionManager = Maps.newConcurrentMap();
-  private final BufferAllocator dataAllocator;
+  private final ConcurrentMap 
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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-02 Thread sohami
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{
+//  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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-02 Thread sohami
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 
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-02 Thread sohami
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) {
 send(queryResultHandler.getWrappedListener(resultsListener), 
RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  public void connect(RpcConnectionHandler handler, 
DrillbitEndpoint endpoint,
-  UserProperties props, UserBitShared.UserCredentials 
credentials) {
+  public CheckedFuture 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;
+  }
+
+  /**
+   * Check (after {@link #connect connecting}) if server requires 
authentication.
+   *
+   * @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},
+   * returns null. If called after {@link #connect connecting}, returns a 
list of supported mechanisms
+   * iff authentication is required.
+   *
+   * @return list of supported authentication mechanisms
+   */
+  public List getSupportedAuthenticationMechanisms() {
+return supportedAuthMechs;
+  }
 
-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
+   * authentication fails.
+   *
+   * This method uses properties provided at {@link #connect connection 
time} and override them with the
+   * given properties, if any.
+   *
+   * @param overrides parameter overrides
+   * @return result of authentication request
+   */
+  public CheckedFuture authenticate(final 
DrillProperties overrides) {
+if (supportedAuthMechs == null) {
+  throw new IllegalStateException("Server does not require 
authentication.");
 }
+properties.merge(overrides);
+final Map propertiesMap = 
properties.stringPropertiesAsMap();
 
-
this.connectAsClient(queryResultHandler.getWrappedConnectionHandler(handler),
-hsBuilder.build(), endpoint.getAddress(), endpoint.getUserPort());
+final SettableFuture settableFuture = SettableFuture.create(); 
// future used in SASL exchange
+final CheckedFuture future =
+new AbstractCheckedFuture(settableFuture) {
+
+  @Override
+  protected SaslException mapException(Exception e) {
+if (connection != null) {
+  connection.close(); // to ensure connection is dropped
+}
+if (e instanceof ExecutionException) {
+  final Throwable cause = e.getCause();
+  if (cause instanceof SaslException) {
+return new SaslException("Authentication failed: " + 
cause.getMessage(), cause);
+  }
+}
+return new SaslException("Authentication failed 
unexpectedly.", e);
+  }
+};
+
+final AuthenticatorFactory factory;
+try {
+  factory = getAuthenticatorFactory();
+} catch (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 {
+  

[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-02 Thread sohami
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) {
-  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.", handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+  throw new RpcException(String.format("Invalid rpc version.  Expected 
%d, actual %d.",
+  handshake.getRpcVersion(), DataRpcConfig.RPC_VERSION));
+}
+
+if (handshake.getAuthenticationMechanismsCount() != 0) { // remote 
requires authentication
--- End diff --

Same as in ControlClient


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-02 Thread sohami
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) {
 send(queryResultHandler.getWrappedListener(resultsListener), 
RpcType.RUN_QUERY, query, QueryId.class);
   }
 
-  public void connect(RpcConnectionHandler handler, 
DrillbitEndpoint endpoint,
-  UserProperties props, UserBitShared.UserCredentials 
credentials) {
+  public CheckedFuture 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;
+  }
+
+  /**
+   * Check (after {@link #connect connecting}) if server requires 
authentication.
+   *
+   * @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},
+   * returns null. If called after {@link #connect connecting}, returns a 
list of supported mechanisms
+   * iff authentication is required.
+   *
+   * @return list of supported authentication mechanisms
+   */
+  public List getSupportedAuthenticationMechanisms() {
--- End diff --

not used anywhere.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #578: DRILL-4280: Kerberos Authentication

2017-02-02 Thread sohami
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 
+#include 
+#include 
+#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 
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 
lock(SaslAuthenticatorImpl::s_mutex);
+if (!s_initialized) {
+// set plugin path if provided
+if (DrillClientConfig::getSaslPluginPath()) {
+char *saslPluginPath = const_cast(DrillClientConfig::getSaslPluginPath());
+sasl_set_path(0, saslPluginPath);
+}
+
+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
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


  1   2   >