You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by parthchandra <gi...@git.apache.org> on 2017/09/20 17:53:36 UTC

[GitHub] drill pull request #950: Drill 5431: SSL Support

GitHub user parthchandra opened a pull request:

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

    Drill 5431: SSL Support

    Add support for SSL between Java/C++ clients and Drillbits. 

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/parthchandra/drill DRILL-5431-0

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/950.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #950
    
----
commit dd22a5a6630ebd87ecf35fb61fc44fcea830a4fa
Author: Sudheesh Katkam <su...@apache.org>
Date:   2017-05-16T21:48:57Z

    DRILL-5431: Upgrade Netty to 4.0.47

commit a34ca452e391d88f64213fccc69e42f1fca91633
Author: Parth Chandra <pa...@apache.org>
Date:   2017-06-20T21:13:53Z

    DRILL-5431: SSL Support (Java) - Update DrillConfig to merge properties passed in from the client command line

commit 13f32d581fa01bc53d7580092ef3d1bbb500f4df
Author: Parth Chandra <pa...@apache.org>
Date:   2017-07-25T16:21:02Z

    DRILL-5431: SSL Support (Java) - Add test certificates, keys, keystore, and truststore.

commit f073001bfbbcf3bec20aae93636c139b7d98f6ec
Author: Parth Chandra <pa...@apache.org>
Date:   2017-08-28T17:08:15Z

    DRILL-5698: Revert unnecessary changes to C++ client

commit 759b5b201a9725f4b377590f48db30e0d5d58856
Author: Parth Chandra <pa...@apache.org>
Date:   2017-06-16T23:49:45Z

    DRILL-5431: Update POM to upgrade to Netty 4.0.48 and add exclusions to all modules that included older versions of Netty

commit 2f3b504e56fa0df704d8153b9c104da18e81d41d
Author: Parth Chandra <pa...@apache.org>
Date:   2017-06-07T18:09:10Z

    DRILL-5431: SSL Support (C++) - Refactoring of C++ client.
    
        Move classes out of drillclient to their own files
        Fix build on MacOS to suppress warnings from boost code
        Refactoring of user properties to use a map

commit 999da4d9c063157aec8d5bd3583d4776652960c3
Author: Parth Chandra <pa...@apache.org>
Date:   2017-06-10T05:03:59Z

    DRILL-5431: SSL Support (Java) - Java client server SSL implementation

commit 9329306abed5b351226b0f25bf8a7f2ce5304679
Author: Parth Chandra <pa...@apache.org>
Date:   2017-08-29T19:04:57Z

    DRILL-5431: SSL Support (Java) - Enable OpenSSL support

commit ee75133198167c685e00183d3d34eca65fa43b09
Author: Parth Chandra <pa...@apache.org>
Date:   2017-07-11T00:19:12Z

    DRILL-5431: SSL Support (C++) - Add boost example code for ssl (small change to the code to pick up the certificate and key files from the test dir). Useful to test the ssl environment.

commit 95f609aa33e30d621108b8594360b9538374694e
Author: Parth Chandra <pa...@apache.org>
Date:   2017-07-24T19:55:02Z

    DRILL-5431: SSL Support (C++) - Update DrillClientImpl to use Channel implementation

commit 6d38f2dc0b4607727a77f491373d93ca9706724e
Author: Parth Chandra <pa...@apache.org>
Date:   2017-07-25T16:22:23Z

    DRILL-5431: SSL Support (C++) - Add (Netty like) socket abstraction that encapsulates a TCP socket or a SSL Stream on TCP.
    
    The testSSL program tests the client connection against a drillbit by sending a drill handshake.

commit 23aac62331a9eb900fb5e6ca5e62ca62438ed9ec
Author: Parth Chandra <pa...@apache.org>
Date:   2017-07-31T20:28:24Z

    DRILL-5431: SSL Support (C++) - Fix Sasl on Windows to build from source (instead of install) directory

----


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141730279
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            connectionStatus_t validateConnectionString();
    --- End diff --
    
    Cause there isn't one. Removed the declaration.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140621875
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java ---
    @@ -203,6 +226,9 @@ public void close() throws IOException {
           if (elapsed > 500) {
             logger.info("closed eventLoopGroup " + eventLoopGroup + " in " + elapsed + " ms");
           }
    +      if(isSslEnabled()) {
    --- End diff --
    
    But its safer when another derived class implementation comes along and forgets to check.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142821900
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -302,6 +310,12 @@ int main(int argc, char* argv[]) {
             std::string serviceHost=qsOptionValues["service_host"];
             std::string serviceName=qsOptionValues["service_name"];
             std::string auth=qsOptionValues["auth"];
    +        std::string enableSSL=qsOptionValues["enableSSL"];
    +        std::string tlsProtocol=qsOptionValues["TLSProtocol"];
    +        std::string certFilePath=qsOptionValues["certFilePath"];
    +        std::string disableHostnameVerification=qsOptionValues["disableHostnameVerification"];
    +        std::string disableCertVerification=qsOptionValues["disableCertVerification"];
    +		std::string useSystemTrustStore = qsOptionValues["useSystemTrustStore"];
    --- End diff --
    
    done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141725643
  
    --- Diff: contrib/native/client/src/clientlib/streamSocket.hpp ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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 STREAMSOCKET_HPP
    +#define STREAMSOCKET_HPP
    +
    +#include "drill/common.hpp"
    +#include "env.h"
    +#include "wincert.ipp"
    +
    +#include <boost/asio.hpp>
    +#include <boost/asio/ssl.hpp>
    +
    +namespace Drill {
    +
    +typedef boost::asio::ip::tcp::socket::lowest_layer_type streamSocket_t;
    +typedef boost::asio::ssl::stream<boost::asio::ip::tcp::socket> sslTCPSocket_t;
    +typedef boost::asio::ip::tcp::socket basicTCPSocket_t;
    +
    +
    +// Some helper typedefs to define the highly templatized boost::asio methods
    +typedef boost::asio::const_buffers_1 ConstBufferSequence; 
    +typedef boost::asio::mutable_buffers_1 MutableBufferSequence;
    +
    +// ReadHandlers have different possible signatures.
    +//
    +// As a standard C-type callback
    +//    typedef void (*ReadHandler)(const boost::system::error_code& ec, std::size_t bytes_transferred);
    +//
    +// Or as a C++ functor
    +//    struct ReadHandler {
    +//        virtual void operator()(
    +//                const boost::system::error_code& ec,
    +//                std::size_t bytes_transferred) = 0;
    +//};
    +//
    +// We need a different signature though, since we need to pass in a member function of a drill client 
    +// class (which is C++), as a functor generated by boost::bind as a ReadHandler
    +// 
    +typedef boost::function<void (const boost::system::error_code& ec, std::size_t bytes_transferred) > ReadHandler;
    +
    +class AsioStreamSocket{
    +    public:
    +        virtual ~AsioStreamSocket(){};
    +        virtual streamSocket_t& getInnerSocket() = 0;
    +
    +        virtual std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        virtual std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        //
    +        // boost::asio::async_read has the signature 
    +        // template<
    +        //     typename AsyncReadStream,
    +        //     typename MutableBufferSequence,
    +        //     typename ReadHandler>
    +        // void-or-deduced async_read(
    +        //     AsyncReadStream & s,
    +        //     const MutableBufferSequence & buffers,
    +        //     ReadHandler handler);
    +        //
    +        // For our use case, the derived class will have an instance of a concrete type for AsyncReadStream which 
    +        // will implement the requirements for the AsyncReadStream type. We need not pass that in as a parameter 
    +        // since the class already has the value
    +        // The method is templatized since the ReadHandler type is dependent on the class implementing the read 
    +        // handler (basically the class using the asio stream)
    +        //
    +        virtual void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler) = 0;
    +
    +        // call the underlying protocol's handshake method.
    +        // if the useSystemConfig flag is true, then use properties read
    +        // from the underlying operating system
    +        virtual void protocolHandshake(bool useSystemConfig) = 0;
    +        virtual void protocolClose() = 0;
    +};
    +
    +class Socket: 
    +    public AsioStreamSocket, 
    +    public basicTCPSocket_t{
    +
    +    public:
    +        Socket(boost::asio::io_service& ioService) : basicTCPSocket_t(ioService) {
    +            }
    +
    +        ~Socket(){
    +            boost::system::error_code ignorederr;
    +            this->shutdown(boost::asio::ip::tcp::socket::shutdown_both, ignorederr);
    +            this->close();
    +        };
    +
    +        basicTCPSocket_t& getSocketStream(){ return *this;}
    +
    +        streamSocket_t& getInnerSocket(){ return this->lowest_layer();}
    +
    +        std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->write_some(buffers, ec);
    +        }
    +
    +        std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->read_some(buffers, ec);
    +        }
    +
    +        void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler){
    +            return async_read(*this, buffers, handler);
    +        }
    +
    +        void protocolHandshake(bool useSystemConfig){}; //nothing to do
    +        void protocolClose(){ 
    +            // shuts down the socket!
    +            boost::system::error_code ignorederr;
    +            ((basicTCPSocket_t*)this)->shutdown(boost::asio::ip::tcp::socket::shutdown_both,
    +                ignorederr
    +                );         
    +        } 
    --- End diff --
    
    Boost is not too clear about the semantics of shutdown versus close. Close closes the socket, while shutdown releases all resources so the socket will no longer receive or send. So what's the difference? I'm not sure. 
    Anyway, I chose to interpret that shutdown means I may possibly want to reuse the socket (though I don't think it is really possible), while close means we will never reuse this.



---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142719061
  
    --- Diff: contrib/native/client/readme.linux ---
    @@ -84,6 +84,21 @@ OR
         ln -svf libboost_filesystem.a libboost_filesystem-mt.a
         ln -svf libboost_date_time.a libboost_date_time-mt.a
     
    +5) Install or  build Cyrus SASL 
    +   To Install 
    +       yum install cyrus-sasl-devel
    +       yum install cyrus-sasl-gssapi
    +   libs are installed in /usr/lib64/sasl2
    +   includes are installed in /usr/include
    +
    +   To build your own 
    +   See readme.sasl for instructions
    +
    +6) Install OpenSSL
    +   yum install openssl
    --- End diff --
    
    might not be necessary as it is a dependency of openssl-devel


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142824662
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    --- End diff --
    
    Agree that the outdated, insecure versions should not be supported. The Java version documents that we will support TLS 1, TLS 1.2, and TLS 1.2,  so I left those three in there with TLS1.2 as the default.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140134229
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -120,6 +125,25 @@ protected void initChannel(SocketChannel ch) throws Exception {
         // }
       }
     
    +  // Adds a SSL handler if enabled. Required only for client and server communications, so
    +  // a real implementation is only available for UserClient
    +  protected void setupSSL(ChannelPipeline pipe, ConnectionMultiListener.SSLHandshakeListener sslHandshakeListener) {
    +    // Do nothing
    --- End diff --
    
    we can throw UnSupportedException to catch any error in flows where this is called in context of Data/control server.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141246382
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            connectionStatus_t validateConnectionString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    +                    return boost::asio::ssl::context::sslv23;
    +                } else if (version == "tlsv1") {
    +                    return boost::asio::ssl::context::tlsv1;
    +                } else if (version == "sslv3") {
    +                    return boost::asio::ssl::context::sslv3;
    +                } else {
    +                    return boost::asio::ssl::context::tlsv12;
    +                }
    +            }
    +
    +        SSLChannelContext(DrillUserProperties *props, boost::asio::ssl::context::method tlsVersion, boost::asio::ssl::verify_mode verifyMode) :
    +                ChannelContext(props),
    +                m_SSLContext(tlsVersion) {
    +                m_SSLContext.set_default_verify_paths();
    +                m_SSLContext.set_options(
    +                        boost::asio::ssl::context::default_workarounds
    +                        | boost::asio::ssl::context::no_sslv2
    +                        | boost::asio::ssl::context::single_dh_use
    +                        );
    +                m_SSLContext.set_verify_mode(verifyMode);
    +            };
    +            ~SSLChannelContext(){};
    +            boost::asio::ssl::context& getSslContext(){ return m_SSLContext;}
    +        private:
    +            boost::asio::ssl::context m_SSLContext;
    +    };
    +
    +    typedef ChannelContext ChannelContext_t; 
    +    typedef SSLChannelContext SSLChannelContext_t; 
    +
    +    class ChannelContextFactory{
    +        public:
    +            static ChannelContext_t* getChannelContext(channelType_t t, DrillUserProperties* props);
    +    };
    +
    +    /***
    +     * The Channel class encapsulates a connection to a drillbit. Based on 
    +     * the connection string and the options, the connection will be either 
    +     * a simple socket or a socket using an ssl stream. The class also encapsulates
    +     * connecting to a drillbit directly or thru zookeeper.
    +     * The channel class owns the socket and the io_service that the applications
    +     * will use to communicate with the server.
    +     ***/
    +    class Channel{
    +        public: 
    +            Channel(const char* connStr);
    +            Channel(const char* host, const char* port);
    +            Channel(boost::asio::io_service& ioService, const char* connStr);
    +            Channel(boost::asio::io_service& ioService, const char* host, const char* port);
    +            virtual ~Channel();
    +            virtual connectionStatus_t init(ChannelContext_t* context)=0;
    +            connectionStatus_t connect();
    +            connectionStatus_t protocolClose();
    +            template <typename SettableSocketOption> void setOption(SettableSocketOption& option);
    +            DrillClientError* getError(){ return m_pError;}
    +            void close(){ 
    +                if(m_state==CHANNEL_INITIALIZED||m_state==CHANNEL_CONNECTED){
    +                    m_pSocket->protocolClose();
    +                    m_state=CHANNEL_CLOSED;
    +                }
    +            } // Not OK to use the channel after this call. 
    +
    +            boost::asio::io_service& getIOService(){
    +                return m_ioService;
    +            }
    +
    +            // returns a reference to the underlying socket 
    +            // This access should really be removed and encapsulated in calls that 
    +            // manage async_send and async_recv 
    +            // Until then we will let DrillClientImpl have direct access
    +            streamSocket_t& getInnerSocket(){
    +                return m_pSocket->getInnerSocket();
    +            }
    +            
    +            AsioStreamSocket& getSocketStream(){
    +                return *m_pSocket;
    +            }
    +
    +            ConnectionEndpoint* getEndpoint(){return m_pEndpoint;}
    +
    +        protected:
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            boost::asio::io_service& m_ioService;
    +            boost::asio::io_service m_ioServiceFallback; // used if m_ioService is not provided
    +            AsioStreamSocket* m_pSocket;
    +            ConnectionEndpoint *m_pEndpoint;
    +            ChannelContext_t *m_pContext;
    +
    +        private:
    +            typedef enum channelState{ 
    +                CHANNEL_UNINITIALIZED=1, 
    +                CHANNEL_INITIALIZED, 
    +                CHANNEL_CONNECTED, 
    +                CHANNEL_CLOSED       
    +            } channelState_t;
    +            
    +            connectionStatus_t connectInternal();
    +            connectionStatus_t protocolHandshake(bool useSystemConfig){
    +                connectionStatus_t status = CONN_SUCCESS;
    +                try{
    +                    m_pSocket->protocolHandshake(useSystemConfig);
    +                } catch (boost::system::system_error e) {
    +                    status = handleError(CONN_HANDSHAKE_FAILED, e.what());
    +                }
    +                return status;
    +            }
    +
    +            channelState_t m_state;
    +            DrillClientError* m_pError;
    +            bool m_ownIoService;
    --- End diff --
    
    why is this flag needed ? 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141677219
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,91 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl) {
    +    HCERTSTORE hStore;
    +    PCCERT_CONTEXT pContext = NULL;
    +    X509 *x509;
    +	char* stores[] = {
    +	    "CA",
    +		"MY",
    +		"ROOT",
    +		"SPC"
    +	};
    +     
    +    SSL_CTX * ctx = SSL_get_SSL_CTX(ssl);
    +    X509_STORE *store = SSL_CTX_get_cert_store(ctx);
    +
    +	for(int i=0; i<4; i++){
    +    hStore = CertOpenSystemStore(NULL, stores[i]);
    +
    +    if (!hStore)
    +        return 1;
    --- End diff --
    
    BTW, if not certificates exist in the store (unlikely), then the certificate validation itself will fail. 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r143246761
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSL.java ---
    @@ -201,6 +202,7 @@ public void testSSLQuery() throws Exception {
         test("SELECT * FROM cp.`region.json`");
       }
     
    +  @Ignore
    --- End diff --
    
    Will do.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140608922
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    +
    +  protected String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    +    value = hadoopConfig.get(name, "");
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getConfigParamWithDefault(String name, String defaultValue) {
    +    String value = "";
    +    if (config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    if (value.isEmpty()) {
    +      value = defaultValue;
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String resolveHadoopPropertyName(String nameTemplate, SSLFactory.Mode mode) {
    +    return MessageFormat.format(nameTemplate, mode.toString().toLowerCase());
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  public abstract SslContext initSslContext() throws DrillException;
    +
    +  public abstract SSLContext initSSLContext() throws DrillException;
    +
    +  public abstract boolean isUserSslEnabled();
    +
    +  public abstract boolean isHttpsEnabled();
    +
    +  public abstract String getKeyStoreType();
    +
    +  public abstract String getKeyStorePath();
    +
    +  public abstract String getKeyStorePassword();
    +
    +  public abstract String getKeyPassword();
    +
    +  public abstract String getTrustStoreType();
    +
    +  public abstract boolean hasTrustStorePath();
    +
    +  public abstract String getTrustStorePath();
    +
    +  public abstract boolean hasTrustStorePassword();
    +
    +  public abstract String getTrustStorePassword();
    +
    +  public abstract String getProtocol();
    +
    +  public abstract SslProvider getProvider();
    +
    +  public abstract int getHandshakeTimeout();
    +
    +  public abstract SSLFactory.Mode getMode();
    +
    +  public abstract boolean disableHostVerification();
    +
    +  public abstract boolean disableCertificateVerification();
    +
    +  public abstract boolean useSystemTrustStore();
    +
    +  public abstract boolean isSslValid();
    +
    +  public SslContext getNettySslContext() {
    +    return nettySslContext;
    +  }
    +
    +  public TrustManagerFactory initializeTrustManagerFactory() throws DrillException {
    +    TrustManagerFactory tmf;
    +    KeyStore ts = null;
    +    //Support Windows/MacOs system trust store
    +    try {
    +      String trustStoreType = getTrustStoreType();
    +      if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +        // This is valid for MS-Windows and MacOs
    +        logger.debug("Initializing System truststore.");
    +        ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +        ts.load(null, null);
    +      } else if (!getTrustStorePath().isEmpty()) {
    +          // if truststore is not provided then we will use the default. Note that the default depends on
    +          // the TrustManagerFactory that in turn depends on the Security Provider.
    +          // Use null as the truststore which will result in the default truststore being picked up
    +          logger.debug("Initializing truststore {}.", getTrustStorePath());
    +          ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +          InputStream tsStream = new FileInputStream(getTrustStorePath());
    +          ts.load(tsStream, getTrustStorePassword().toCharArray());
    +      } else {
    +        logger.debug("Initializing default truststore.");
    +      }
    +      if (disableCertificateVerification()) {
    +        tmf = InsecureTrustManagerFactory.INSTANCE;
    +      } else {
    +        tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
    +      }
    +      tmf.init(ts);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the truststore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return tmf;
    +  }
    +
    +  public KeyManagerFactory initializeKeyManagerFactory() throws DrillException {
    +    KeyManagerFactory kmf;
    +    String keyStorePath = getKeyStorePath();
    +    String keyStorePassword = getKeyStorePassword();
    +    String keyStoreType = getKeyStoreType();
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      KeyStore ks =
    +          KeyStore.getInstance(!keyStoreType.isEmpty() ? keyStoreType : KeyStore.getDefaultType());
    +      //initialize the key manager factory
    +      // Will throw an exception if the file is not found/accessible.
    +      InputStream ksStream = new FileInputStream(keyStorePath);
    +      // A key password CANNOT be null or an empty string.
    +      if (keyStorePassword.isEmpty()) {
    +        throw new DrillException("The Keystore password cannot be empty.");
    +      }
    +      ks.load(ksStream, keyStorePassword.toCharArray());
    +      // Empty Keystore. (Remarkably, it is possible to do this).
    +      if (ks.size() == 0) {
    +        throw new DrillException("The Keystore has no entries.");
    +      }
    +      kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
    +      kmf.init(ks, getKeyPassword().toCharArray());
    +
    +    } catch (Exception e) {
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the keystore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return kmf;
    +  }
    +
    +  public void initContext() throws DrillException {
    +    if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +      initSSLContext();
    +      logger.debug("Initialized Windows SSL context using JDK.");
    --- End diff --
    
    OK


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142863375
  
    --- Diff: exec/java-exec/pom.xml ---
    @@ -589,6 +669,17 @@
     
       <build>
     
    +    <extensions>
    +      <!--
    +      Include the os-maven-plugin to get os.detected.classifier
    +      -->
    +      <extension>
    +        <groupId>kr.motd.maven</groupId>
    +        <artifactId>os-maven-plugin</artifactId>
    +        <version>1.5.0.Final</version>
    --- End diff --
    
    Fix has helped, now project builds on my centos.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140121083
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -110,6 +110,18 @@
       String HASHAGG_FALLBACK_ENABLED_KEY = "drill.exec.hashagg.fallback.enabled";
       BooleanValidator HASHAGG_FALLBACK_ENABLED_VALIDATOR = new BooleanValidator(HASHAGG_FALLBACK_ENABLED_KEY);
     
    +  String SSL_PROVIDER = "drill.exec.ssl.provider"; // valid values are "JDK", "OPENSSL" // default JDK
    +  String SSL_PROTOCOL = "drill.exec.ssl.protocol"; // valid values are SSL, SSLV2, SSLV3, TLS, TLSV1, TLSv1.1, TLSv1.2(default)
    +  String SSL_KEYSTORE_TYPE = "drill.exec.ssl.keyStoreType";
    +  String SSL_KEYSTORE_PATH = "drill.exec.ssl.keyStorePath";     // path to keystore. default : $JRE_HOME/lib/security/keystore.jks
    +  String SSL_KEYSTORE_PASSWORD = "drill.exec.ssl.keyStorePassword"; // default: changeit
    +  String SSL_KEY_PASSWORD = "drill.exec.ssl.keyPassword"; //
    +  String SSL_TRUSTSTORE_TYPE = "drill.exec.ssl.trustStoreType"; // valid values are jks(default), jceks, pkcs12
    +  String SSL_TRUSTSTORE_PATH = "drill.exec.ssl.trustStorePath"; // path to keystore. default : $JRE_HOME/lib/security/cacerts.jks
    +  String SSL_TRUSTSTORE_PASSWORD = "drill.exec.ssl.trustStorePassword"; // default: changeit
    +  String SSL_USE_HADOOP_CONF = "drill.exec.ssl.useHadoopConfig"; // Initialize ssl params from hadoop if not provided by drill. default: true
    +  String SSL_HANDSHAKE_TIMEOUT = "drill.exec.security.user.encryption.ssl.handshakeTimeout"; // Default 10 seconds
    --- End diff --
    
    General convention for naming the config properties looks to be lowercase. I would suggest that we stick to that for ease of admin configuring the cluster.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142727180
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    --- End diff --
    
    shouldn't port be an int?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142821870
  
    --- Diff: contrib/native/client/CMakeLists.txt ---
    @@ -93,7 +94,7 @@ else()
         #    of boost. Arbirtarily, we choose the new namspace to be drill_boost.
         #    See the instructions in the readme for linux/macos and rebuild boost. Then
         #    uncomment the line below to build
    -        set(Boost_NAMESPACE drill_boost)
    +    #    set(Boost_NAMESPACE drill_boost)
    --- End diff --
    
    Not really. I undid what was probably a temporary change. The default has always been to not have to use a custom boost build.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142827859
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java ---
    @@ -49,12 +64,19 @@ public void testMissingKeystorePassword() throws Exception {
         ConfigBuilder config = new ConfigBuilder();
         config.put(ExecConstants.HTTP_KEYSTORE_PATH, "/root");
         config.put(ExecConstants.HTTP_KEYSTORE_PASSWORD, "");
    +    config.put(ExecConstants.SSL_USE_HADOOP_CONF, false);
    +    config.put(ExecConstants.USER_SSL_ENABLED, true);
         try {
    -      SSLConfig sslv = new SSLConfig(config.build());
    +      SSLConfig sslv = new SSLConfigBuilder()
    +          .config(config.build())
    +          .mode(SSLFactory.Mode.SERVER)
    +          .initializeSSLContext(false)
    +          .validateKeyStore(true)
    +          .build();
           fail();
           //Expected
         } catch (Exception e) {
    -      assertTrue(e instanceof DrillException);
    +
    --- End diff --
    
    If the hadoop config is invalid the build method will throw an exception. So we are looking to make sure the build method does, in fact, throw an exception. If it does the test passes, otherwise it fails.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142718085
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -302,6 +310,12 @@ int main(int argc, char* argv[]) {
             std::string serviceHost=qsOptionValues["service_host"];
             std::string serviceName=qsOptionValues["service_name"];
             std::string auth=qsOptionValues["auth"];
    +        std::string enableSSL=qsOptionValues["enableSSL"];
    +        std::string tlsProtocol=qsOptionValues["TLSProtocol"];
    +        std::string certFilePath=qsOptionValues["certFilePath"];
    +        std::string disableHostnameVerification=qsOptionValues["disableHostnameVerification"];
    +        std::string disableCertVerification=qsOptionValues["disableCertVerification"];
    +		std::string useSystemTrustStore = qsOptionValues["useSystemTrustStore"];
    --- End diff --
    
    (style) indentation issue


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142731277
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,448 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                DRILL_LOG(LOG_INFO) << "Failed to get endpoint from zk" << std::endl;
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(((.*):([0-9]+),?)+)(/.+)?");
    +    boost::cmatch matched;
    +
    +    if(boost::regex_match(m_connectString.c_str(), matched, connStrExpr)){
    +        m_protocol.assign(matched[1].first, matched[1].second);
    +        if(isDirectConnection()){
    +            m_host.assign(matched[4].first, matched[4].second);
    --- End diff --
    
    looks like you can also do `m_host = matched[4].str()`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141498641
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/ConnectionMultiListener.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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 com.google.protobuf.MessageLite;
    +import io.netty.buffer.ByteBuf;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.util.concurrent.Future;
    +import io.netty.util.concurrent.GenericFutureListener;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.slf4j.Logger;
    +
    +import java.net.SocketAddress;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * @param <CC> Client Connection Listener
    + * @param <HS> Outbound handshake message type
    + * @param <HR> Inbound handshake message type
    + * @param <BC> BasicClient type
    + *             <p>
    + *             Implements a wrapper class that allows a client connection to associate different behaviours after
    + *             establishing a connection with the server. The client can choose to send an application handshake, or
    + *             in the case of SSL, wait for a SSL handshake completion and then send an application handshake.
    + */
    +
    +public class ConnectionMultiListener<CC extends ClientConnection, HS extends MessageLite, HR extends MessageLite, BC extends BasicClient> {
    +
    +  private static final Logger logger = org.slf4j.LoggerFactory.getLogger(ConnectionMultiListener.class);
    +
    +  private final RpcConnectionHandler<CC> connectionListener;
    +  private final HS handshakeValue;
    +  private final BC parent;
    +
    +  private ConnectionMultiListener(RpcConnectionHandler<CC> connectionListener, HS handshakeValue,
    +      BC basicClient) {
    +    assert connectionListener != null;
    +    assert handshakeValue != null;
    +
    +    this.connectionListener = connectionListener;
    +    this.handshakeValue = handshakeValue;
    +    this.parent = basicClient;
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  public static <CC extends ClientConnection, HS extends MessageLite, BC extends BasicClient> Builder
    +  newBuilder(RpcConnectionHandler<CC> connectionListener, HS handshakeValue,
    +      BC basicClient) {
    +    return new Builder(connectionListener, handshakeValue, basicClient);
    +  }
    +
    +  public ConnectionHandler connectionHandler = null;
    +  public HandshakeSendHandler handshakeSendHandler = null;
    +  public SSLConnectionHandler sslConnectionHandler = null;
    +
    +  /**
    +   * Manages connection establishment outcomes.
    +   */
    +  private class ConnectionHandler implements GenericFutureListener<ChannelFuture> {
    +
    +    @Override public void operationComplete(ChannelFuture future) throws Exception {
    +      boolean isInterrupted = false;
    +
    +      // We want to wait for at least 120 secs when interrupts occur. Establishing a connection fails/succeeds quickly,
    +      // So there is no point propagating the interruption as failure immediately.
    +      long remainingWaitTimeMills = 120000;
    +      long startTime = System.currentTimeMillis();
    +      // logger.debug("Connection operation finished.  Success: {}", future.isSuccess());
    +      while (true) {
    +        try {
    +          future.get(remainingWaitTimeMills, TimeUnit.MILLISECONDS);
    +          if (future.isSuccess()) {
    +            SocketAddress remote = future.channel().remoteAddress();
    +            SocketAddress local = future.channel().localAddress();
    +            parent.setAddresses(remote, local);
    +            // if SSL is enabled send the handshake after the ssl handshake is completed, otherwise send it
    +            // now
    +            if(!parent.isSslEnabled()) {
    +              // send a handshake on the current thread. This is the only time we will send from within the event thread.
    +              // We can do this because the connection will not be backed up.
    +              parent.send(handshakeSendHandler, handshakeValue, true);
    +            }
    +          } else {
    +            connectionListener.connectionFailed(RpcConnectionHandler.FailureType.CONNECTION,
    +                new RpcException("General connection failure."));
    +          }
    +          // logger.debug("Handshake queued for send.");
    +          break;
    +        } catch (final InterruptedException interruptEx) {
    +          remainingWaitTimeMills -= (System.currentTimeMillis() - startTime);
    +          startTime = System.currentTimeMillis();
    +          isInterrupted = true;
    +          if (remainingWaitTimeMills < 1) {
    +            connectionListener.connectionFailed(RpcConnectionHandler.FailureType.CONNECTION, interruptEx);
    +            break;
    +          }
    +          // Ignore the interrupt and continue to wait until we elapse remainingWaitTimeMills.
    +        } catch (final Exception ex) {
    +          logger.error("Failed to establish connection", ex);
    +          connectionListener.connectionFailed(RpcConnectionHandler.FailureType.CONNECTION, ex);
    +          break;
    +        }
    +      }
    +
    +      if (isInterrupted) {
    +        // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
    +        // interruption and respond to it if it wants to.
    +        Thread.currentThread().interrupt();
    +      }
    +    }
    +  }
    +
    +  private class SSLConnectionHandler implements GenericFutureListener<Future<Channel>> {
    +    @Override public void operationComplete(Future<Channel> future) throws Exception {
    +      // send the handshake
    +      parent.send(handshakeSendHandler, handshakeValue, true);
    +    }
    +  }
    +
    +  /**
    +   * manages handshake outcomes.
    +   */
    +  private class HandshakeSendHandler implements RpcOutcomeListener<HR> {
    +
    +    @Override public void failed(RpcException ex) {
    +      logger.debug("Failure while initiating handshake", ex);
    +      connectionListener.connectionFailed(RpcConnectionHandler.FailureType.HANDSHAKE_COMMUNICATION, ex);
    +    }
    +
    +    @Override public void success(HR value, ByteBuf buffer) {
    +      // logger.debug("Handshake received. {}", value);
    +      try {
    +        parent.validateHandshake(value);
    +        parent.finalizeConnection(value, parent.connection);
    +        connectionListener.connectionSucceeded((CC) parent.connection);
    +        // logger.debug("Handshake completed succesfully.");
    +      } catch (Exception ex) {
    +        logger.debug("Failure while validating handshake", ex);
    +        connectionListener.connectionFailed(RpcConnectionHandler.FailureType.HANDSHAKE_VALIDATION, ex);
    +      }
    +    }
    +
    +    @Override public void interrupted(final InterruptedException ex) {
    +      logger.warn("Interrupted while waiting for handshake response", ex);
    +      connectionListener.connectionFailed(RpcConnectionHandler.FailureType.HANDSHAKE_COMMUNICATION, ex);
    +    }
    +  }
    +
    +  /*
    +    The SSL Handshake listener is special in that it is needed at the time of initializing an SSL
    +    enabled pipeline and so is instantiated before the instance of the outer class may be needed.
    +    We create an instance and set a reference back to the outer class instance when it is created
    +    at the time of connection.
    +   */
    +  public static class SSLHandshakeListener implements GenericFutureListener<Future<Channel>> {
    +    ConnectionMultiListener parent;
    +    public SSLHandshakeListener() {
    +    }
    +
    +    public void setParent(ConnectionMultiListener cml){
    +      this.parent = cml;
    +    }
    +
    +    @Override public void operationComplete(Future<Channel> future) throws Exception {
    +      if(parent != null){
    +        if(future.isSuccess()) {
    +          Channel c = future.get();
    +          parent.sslConnectionHandler.operationComplete(future);
    +          parent.parent.setSslChannel(c);
    +        } else {
    +          throw new DrillException("SSL handshake failed.", future.cause());
    +        }
    +      } else {
    +        throw new RpcException("RPC Setup error. SSL handshake complete handler is not set up.");
    +      }
    +      return;
    +    }
    +  }
    +
    +
    +  public static class Builder<CC extends ClientConnection, HS extends MessageLite, HR extends MessageLite, BC extends BasicClient> {
    +
    +    private RpcConnectionHandler<CC> connectionListener;
    +    private HS handshakeValue;
    +    private BC basicClient;
    +    private ConnectionMultiListener cml;
    +
    +    private Builder(RpcConnectionHandler<CC> connectionListener, HS handshakeValue, BC basicClient) {
    +      this.connectionListener = connectionListener;
    +      this.handshakeValue = handshakeValue;
    +      this.basicClient = basicClient;
    +      this.cml = new ConnectionMultiListener(connectionListener, handshakeValue, basicClient);
    +    }
    +
    +    public Builder enableSSL() {
    +      cml.connectionHandler = cml.new ConnectionHandler();
    +      cml.sslConnectionHandler = cml.new SSLConnectionHandler();
    +      return this;
    +    }
    +
    +    public Builder enablePlain() {
    +      cml.connectionHandler = cml.new ConnectionHandler();
    +      return this;
    +    }
    +
    +    public Builder enableHandshake() {
    +      cml.handshakeSendHandler = cml.new HandshakeSendHandler();
    +      return this;
    +    }
    +
    +    public ConnectionMultiListener build() {
    +      //always enable handshake
    +      if (cml.handshakeSendHandler == null) {
    +        enableHandshake();
    +      }
    +      if (cml.connectionHandler == null && cml.sslConnectionHandler == null) {
    +        enablePlain();
    +      }
    +      return cml;
    --- End diff --
    
    Agree that I can defer creation of ConnectionMultiListener to the build() method. 
    I thought that we should let the caller decide what handler it wants to enable. However, I'm clearly overriding that in the build()method, so perhaps you're right.


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/950
  
    @arina-ielchiieva  The `TestSSLConfig.testMissingKeystorePassword` test will fail if the exception is not thrown. The test is checking that the exception is in fact being thrown. Anyway, I added back the assertTrue check as well. It doesn't hurt to have it there.
    Re the failing `TestUserBitSSL.testClientConfigHostnameVerification`. The test generates a self signed certificate with the local machine host name
    ```
          String fqdn = InetAddress.getLocalHost().getHostName();
          SelfSignedCertificate certificate = new SelfSignedCertificate(fqdn);
    ```
    It then connects to the Drillbit which, one assumes, should have the same hostname so we can test that the certificate's hostname is being validated.  
    I really don't know why this would fail unless you have your test machine hostname not set or set incorrectly. It should work for all cases where the hostname is the default localhost.
    It does appear that this test might cause angst down the road and it might be best to disable it for the general case.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141247355
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,91 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl) {
    +    HCERTSTORE hStore;
    +    PCCERT_CONTEXT pContext = NULL;
    +    X509 *x509;
    +	char* stores[] = {
    +	    "CA",
    +		"MY",
    +		"ROOT",
    +		"SPC"
    +	};
    +     
    +    SSL_CTX * ctx = SSL_get_SSL_CTX(ssl);
    +    X509_STORE *store = SSL_CTX_get_cert_store(ctx);
    +
    +	for(int i=0; i<4; i++){
    +    hStore = CertOpenSystemStore(NULL, stores[i]);
    +
    +    if (!hStore)
    +        return 1;
    --- End diff --
    
    This means we will return with failure while opening any of the 4 system store. Should we instead try all 4 system stores and log the ones for which failure happened (by appending the names to string param suggested in above comment) but still succeed if anyone store was successfully opened ? 
    
    But then I think we should also check if there is atleast one certificate which was added to X509 store out of these system store ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140608891
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    +
    +  protected String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    +    value = hadoopConfig.get(name, "");
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getConfigParamWithDefault(String name, String defaultValue) {
    +    String value = "";
    +    if (config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    if (value.isEmpty()) {
    +      value = defaultValue;
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String resolveHadoopPropertyName(String nameTemplate, SSLFactory.Mode mode) {
    +    return MessageFormat.format(nameTemplate, mode.toString().toLowerCase());
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  public abstract SslContext initSslContext() throws DrillException;
    +
    +  public abstract SSLContext initSSLContext() throws DrillException;
    --- End diff --
    
    Good idea. This part is somewhat confusing.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142734246
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    +                    return boost::asio::ssl::context::sslv23;
    +                } else if (version == "tlsv1") {
    +                    return boost::asio::ssl::context::tlsv1;
    +                } else if (version == "sslv3") {
    +                    return boost::asio::ssl::context::sslv3;
    +                } else {
    +                    return boost::asio::ssl::context::tlsv12;
    +                }
    +            }
    +
    +        SSLChannelContext(DrillUserProperties *props, boost::asio::ssl::context::method tlsVersion, boost::asio::ssl::verify_mode verifyMode) :
    +                ChannelContext(props),
    +                m_SSLContext(tlsVersion) {
    +                m_SSLContext.set_default_verify_paths();
    +                m_SSLContext.set_options(
    +                        boost::asio::ssl::context::default_workarounds
    +                        | boost::asio::ssl::context::no_sslv2
    +                        | boost::asio::ssl::context::single_dh_use
    +                        );
    +                m_SSLContext.set_verify_mode(verifyMode);
    +            };
    +            ~SSLChannelContext(){};
    +            boost::asio::ssl::context& getSslContext(){ return m_SSLContext;}
    +        private:
    +            boost::asio::ssl::context m_SSLContext;
    +    };
    +
    +    typedef ChannelContext ChannelContext_t; 
    +    typedef SSLChannelContext SSLChannelContext_t; 
    +
    +    class ChannelContextFactory{
    --- End diff --
    
    I wonder about having one factory for context and one for channel, vs one factory for a given channel type (to create both channel and socket)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140357288
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigClient.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.ssl;
    +
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslContextBuilder;
    +import io.netty.handler.ssl.SslProvider;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.TrustManagerFactory;
    +
    +public class SSLConfigClient extends SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfigClient.class);
    +
    +  private final SSLFactory.Mode mode; // Let's reuse Hadoop's SSLFactory.Mode to distinguish client/server
    --- End diff --
    
    _SSLFactory.Mode mode_ can be moved to base class


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142828440
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSL.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc.user.security;
    +
    +import com.typesafe.config.ConfigValueFactory;
    +import io.netty.handler.ssl.util.SelfSignedCertificate;
    +import junit.framework.TestCase;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.exec.ExecConstants;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.net.InetAddress;
    +import java.security.KeyStore;
    +import java.util.Properties;
    +
    +import static junit.framework.TestCase.fail;
    +import static org.junit.Assert.assertEquals;
    +
    +public class TestUserBitSSL extends BaseTestQuery {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(TestUserBitSSL.class);
    +
    +  private static DrillConfig newConfig;
    +  private static Properties initProps; // initial client properties
    +  private static ClassLoader classLoader;
    +  private static String ksPath;
    +  private static String tsPath;
    +  private static String emptyTSPath;
    +  private static String unknownKsPath;
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +
    +    // Create a new DrillConfig
    +    classLoader = TestUserBitSSL.class.getClassLoader();
    +    ksPath = new File(classLoader.getResource("ssl/keystore.ks").getFile()).getAbsolutePath();
    +    unknownKsPath = new File(classLoader.getResource("ssl/unknownkeystore.ks").getFile()).getAbsolutePath();
    +    tsPath = new File(classLoader.getResource("ssl/truststore.ks").getFile()).getAbsolutePath();
    +    emptyTSPath = new File(classLoader.getResource("ssl/emptytruststore.ks").getFile()).getAbsolutePath();
    +    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
    +        .withValue(ExecConstants.SSL_USE_HADOOP_CONF,
    +            ConfigValueFactory.fromAnyRef(false))
    +        .withValue(ExecConstants.USER_SSL_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.SSL_KEYSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(ksPath))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(tsPath))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_PROTOCOL,
    +            ConfigValueFactory.fromAnyRef("TLSv1.2")),
    +      false);
    +
    +    initProps = new Properties();
    +    initProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    initProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +
    +    // Start an SSL enabled cluster
    +    updateTestCluster(1, newConfig, initProps);
    +  }
    +
    +  @AfterClass
    +  public static void cleanTest() throws Exception {
    +    DrillConfig restoreConfig =
    +        new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties()), false);
    +    updateTestCluster(1, restoreConfig);
    +  }
    +
    +  @Test
    +  public void testSSLConnection() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionWithKeystore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ksPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailBadTrustStore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ""); // NO truststore
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailBadPassword() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "bad_password");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailEmptyTrustStore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, emptyTSPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testSSLQuery() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +    test("SELECT * FROM cp.`region.json`");
    +  }
    +
    +  @Test
    +  public void testClientConfigHostnameVerification() {
    +    String password = "test_password";
    +    String trustStoreFileName = "drillTestTrustStore";
    +    String keyStoreFileName = "drillTestKeyStore";
    +    KeyStore ts, ks;
    +    File tempFile1, tempFile2;
    +    String trustStorePath;
    +    String keyStorePath;
    +
    +    try {
    +      String fqdn = InetAddress.getLocalHost().getHostName();
    +      SelfSignedCertificate certificate = new SelfSignedCertificate(fqdn);
    +
    +      tempFile1 = File.createTempFile(trustStoreFileName, ".ks");
    +      tempFile1.deleteOnExit();
    +      trustStorePath = tempFile1.getAbsolutePath();
    +      //generate a truststore.
    +      ts = KeyStore.getInstance(KeyStore.getDefaultType());
    +      ts.load(null, password.toCharArray());
    +      ts.setCertificateEntry("drillTest", certificate.cert());
    +      // Store away the truststore.
    +      FileOutputStream fos1 = new FileOutputStream(tempFile1);
    +      ts.store(fos1, password.toCharArray());
    +      fos1.close();
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142861999
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSL.java ---
    @@ -223,9 +223,12 @@ public void testClientConfigHostnameVerification() {
           ts.load(null, password.toCharArray());
           ts.setCertificateEntry("drillTest", certificate.cert());
           // Store away the truststore.
    -      FileOutputStream fos1 = new FileOutputStream(tempFile1);
    -      ts.store(fos1, password.toCharArray());
    -      fos1.close();
    +      try (FileOutputStream fos1 = new FileOutputStream(tempFile1);) {
    +        ts.store(fos1, password.toCharArray());
    +        fos1.close();
    --- End diff --
    
    No need to close stream. It will be closed automatically.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142821887
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -50,7 +49,14 @@ struct Option{
         {"service_host", "Service host for Kerberos", false},
         {"service_name", "Service name for Kerberos", false},
         {"auth", "Authentication mechanism to use", false},
    -    {"sasl_encrypt", "Negotiate for encrypted connection", false}
    +    {"sasl_encrypt", "Negotiate for encrypted connection", false},
    +    {"enableSSL", "Enable SSL", false},
    +    {"TLSProtocol", "TLS protocol version", false},
    +    {"certFilePath", "Path to SSL certificate file", false},
    +    {"disableHostnameVerification", "disable host name verification", false},
    +    {"disableCertVerification", "disable certificate verification", false},
    +	{"useSystemTrustStore", "[Windows only]. Use the system truststore.", false }
    --- End diff --
    
    done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141728836
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,452 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(.*):([0-9]+)(?:/(.+))?");
    +    boost::cmatch matched;
    +
    +    if(boost::regex_match(m_connectString.c_str(), matched, connStrExpr)){
    +        m_protocol.assign(matched[1].first, matched[1].second);
    +        std::string host, port;
    +        host.assign(matched[2].first, matched[2].second);
    +        port.assign(matched[3].first, matched[3].second);
    +        if(isDirectConnection()){
    +            // if the connection is to a zookeeper, 
    +            // we will get the host and the port only after connecting to the Zookeeper
    +            m_host=host;
    +            m_port=port;
    +        }
    +        m_hostPortStr=host+std::string(":")+port;
    +        std::string pathToDrill;
    +        if(matched.size()==5){
    +            pathToDrill.assign(matched[4].first, matched[4].second);
    +            if(!pathToDrill.empty()){
    +                m_pathToDrill=std::string("/")+pathToDrill;
    +            }
    +        }
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) 
    +                << "Conn str: "<< m_connectString 
    +                << ";  protocol: " << m_protocol 
    +                << ";  host: " << host 
    +                << "; port: " << port 
    +                << ";  path to drill: " << m_pathToDrill 
    +                << std::endl;)
    +    } else {
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Invalid connect string. Regexp did not match" << std::endl;)
    +    }
    +
    +    return;
    +}
    +
    +bool ConnectionEndpoint::isDirectConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "local") || !strcmp(m_protocol.c_str(), "drillbit"));
    +}
    +
    +bool ConnectionEndpoint::isZookeeperConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "zk"));
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpointFromZk(){
    +    ZookeeperClient zook(m_pathToDrill);
    +    assert(!m_hostPortStr.empty());
    +    std::vector<std::string> drillbits;
    +    if(zook.getAllDrillbits(m_hostPortStr.c_str(), drillbits)!=0){
    +        return handleError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    +    }
    +    if (drillbits.empty()){
    +        return handleError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    +    }
    +    Utils::shuffle(drillbits);
    +    exec::DrillbitEndpoint endpoint;
    +    int err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    +    if(!err){
    +        m_host=boost::lexical_cast<std::string>(endpoint.address());
    +        m_port=boost::lexical_cast<std::string>(endpoint.user_port());
    +    }
    +    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    +        zook.close();
    +    return CONN_SUCCESS;
    +}
    +
    +connectionStatus_t ConnectionEndpoint::handleError(connectionStatus_t status, std::string msg){
    +    DrillClientError* pErr = new DrillClientError(status, DrillClientError::CONN_ERROR_START+status, msg);
    +    if(m_pError!=NULL){ delete m_pError; m_pError=NULL;}
    +    m_pError=pErr;
    +    return status;
    +}
    +
    +/****************************
    + * Channel Context Factory
    + ****************************/
    +ChannelContext* ChannelContextFactory::getChannelContext(channelType_t t, DrillUserProperties* props){
    +    ChannelContext* pChannelContext=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannelContext=new ChannelContext(props);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM: {
    +
    +            std::string protocol;
    +            props->getProp(USERPROP_TLSPROTOCOL, protocol);
    +            boost::asio::ssl::context::method tlsVersion = SSLChannelContext::getTlsVersion(protocol);
    +
    +            std::string noVerifyCert;
    +            props->getProp(USERPROP_DISABLE_CERTVERIFICATION, noVerifyCert);
    +            boost::asio::ssl::context::verify_mode verifyMode = boost::asio::ssl::context::verify_peer;
    +            if (noVerifyCert == "true") {
    +                verifyMode = boost::asio::ssl::context::verify_none;
    +            }
    +
    +            pChannelContext = new SSLChannelContext(props, tlsVersion, verifyMode);
    +        }
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannelContext;
    +} 
    +
    +/*******************
    + *  ChannelFactory
    + * *****************/
    +Channel* ChannelFactory::getChannel(channelType_t t, const char* connStr){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(connStr);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(connStr);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +Channel* ChannelFactory::getChannel(channelType_t t, const char* host, const char* port){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(host, port);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(host, port);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +Channel* ChannelFactory::getChannel(channelType_t t, boost::asio::io_service& ioService, const char* connStr){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(ioService, connStr);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(ioService, connStr);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +Channel* ChannelFactory::getChannel(channelType_t t, boost::asio::io_service& ioService, const char* host, const char* port){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(ioService, host, port);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(ioService, host, port);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +/*******************
    + *  Channel
    + * *****************/
    +
    +Channel::Channel(const char* connStr) : m_ioService(m_ioServiceFallback){
    +    m_pEndpoint=new ConnectionEndpoint(connStr);
    +    m_ownIoService = true;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::Channel(const char* host, const char* port) : m_ioService(m_ioServiceFallback){
    +    m_pEndpoint=new ConnectionEndpoint(host, port);
    +    m_ownIoService = true;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::Channel(boost::asio::io_service& ioService, const char* connStr):m_ioService(ioService){
    +    m_pEndpoint=new ConnectionEndpoint(connStr);
    +    m_ownIoService = false;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::Channel(boost::asio::io_service& ioService, const char* host, const char* port) : m_ioService(ioService){
    +    m_pEndpoint=new ConnectionEndpoint(host, port);
    +    m_ownIoService = true;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::~Channel(){
    +    if(m_pEndpoint!=NULL){
    +        delete m_pEndpoint; m_pEndpoint=NULL;
    +    }
    +    if(m_pSocket!=NULL){
    +        delete m_pSocket; m_pSocket=NULL;
    +    }
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +template <typename SettableSocketOption> void Channel::setOption(SettableSocketOption& option){
    +    //May be useful some day. 
    +    //At the moment, we only need to set some well known options after we connect.
    +    assert(0); 
    +}
    +
    +connectionStatus_t Channel::init(ChannelContext_t* pContext){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    this->m_state=CHANNEL_INITIALIZED;
    +    this->m_pContext = pContext;
    +    return ret;
    +}
    +
    +connectionStatus_t Channel::connect(){
    +    connectionStatus_t ret=CONN_FAILURE;
    +    if(this->m_state==CHANNEL_INITIALIZED){
    +        ret=m_pEndpoint->getDrillbitEndpoint();
    +        if(ret==CONN_SUCCESS){
    +            DRILL_LOG(LOG_TRACE) << "Connecting to drillbit: " 
    +                << m_pEndpoint->getHost() 
    +                << ":" << m_pEndpoint->getPort() 
    +                << "." << std::endl;
    +            ret=this->connectInternal();
    +        }else{
    +            handleError(ret, m_pEndpoint->getError()->msg);
    +        }
    +    }
    +    this->m_state=(ret==CONN_SUCCESS)?CHANNEL_CONNECTED:this->m_state;
    +    return ret;
    +}
    +
    +connectionStatus_t Channel::handleError(connectionStatus_t status, std::string msg){
    +    DrillClientError* pErr = new DrillClientError(status, DrillClientError::CONN_ERROR_START+status, msg);
    +    if(m_pError!=NULL){ delete m_pError; m_pError=NULL;}
    +    m_pError=pErr;
    +    return status;
    +}
    +
    +connectionStatus_t Channel::connectInternal() {
    +    using boost::asio::ip::tcp;
    +    tcp::endpoint endpoint;
    +    const char *host = m_pEndpoint->getHost().c_str();
    +    const char *port = m_pEndpoint->getPort().c_str();
    +    try {
    +        tcp::resolver resolver(m_ioService);
    +        tcp::resolver::query query(tcp::v4(), host, port);
    +        tcp::resolver::iterator iter = resolver.resolve(query);
    +        tcp::resolver::iterator end;
    +        while (iter != end) {
    +            endpoint = *iter++;
    +            DRILL_LOG(LOG_TRACE) << endpoint << std::endl;
    +        }
    +        boost::system::error_code ec;
    +        m_pSocket->getInnerSocket().connect(endpoint, ec);
    +        if (ec) {
    +            return handleError(CONN_FAILURE, getMessage(ERR_CONN_FAILURE, host, port, ec.message().c_str()));
    +        }
    +    } catch (std::exception e) {
    +        // Handle case when the hostname cannot be resolved. "resolve" is hard-coded in boost asio resolver.resolve
    +        if (!strcmp(e.what(), "resolve")) {
    +            return handleError(CONN_HOSTNAME_RESOLUTION_ERROR, getMessage(ERR_CONN_EXCEPT, e.what()));
    +        }
    +        return handleError(CONN_FAILURE, getMessage(ERR_CONN_EXCEPT, e.what()));
    +    }
    +
    +    // set socket keep alive
    +    boost::asio::socket_base::keep_alive keepAlive(true);
    +    m_pSocket->getInnerSocket().set_option(keepAlive);
    +    // set no_delay
    +    boost::asio::ip::tcp::no_delay noDelay(true);
    +    m_pSocket->getInnerSocket().set_option(noDelay);
    +    // set reuse addr
    +    boost::asio::socket_base::reuse_address reuseAddr(true);
    +    m_pSocket->getInnerSocket().set_option(reuseAddr);
    +
    +    std::string useSystemTrustStore;
    +    m_pContext->getUserProperties()->getProp(USERPROP_USESYSTEMTRUSTSTORE, useSystemTrustStore);
    +
    +    return this->protocolHandshake(useSystemTrustStore=="true");
    +
    +}
    +
    +connectionStatus_t SocketChannel::init(ChannelContext_t* pContext){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    m_pSocket=new Socket(m_ioService);
    +    if(m_pSocket!=NULL){
    +        ret=Channel::init(pContext);
    +    }else{
    +        DRILL_LOG(LOG_ERROR) << "Channel initialization failure. " << std::endl;
    +        handleError(CONN_NOSOCKET, getMessage(ERR_CONN_NOSOCKET));
    +        ret=CONN_FAILURE;
    +    }
    +    return ret;
    +}
    +
    +#if defined(IS_SSL_ENABLED)
    +connectionStatus_t SSLStreamChannel::init(ChannelContext_t* pContext){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +
    +    const DrillUserProperties* props = pContext->getUserProperties();
    +	std::string useSystemTrustStore;
    +	props->getProp(USERPROP_USESYSTEMTRUSTSTORE, useSystemTrustStore);
    +	if (useSystemTrustStore != "true"){
    +		std::string certFile;
    +		props->getProp(USERPROP_CERTFILEPATH, certFile);
    +		try{
    +			((SSLChannelContext_t*)pContext)->getSslContext().load_verify_file(certFile);
    +		}
    +		catch (boost::system::system_error e){
    +			DRILL_LOG(LOG_ERROR) << "Channel initialization failure. Certificate file  "
    +				<< certFile
    +				<< " could not be loaded."
    +				<< std::endl;
    +			handleError(CONN_SSLERROR, getMessage(ERR_CONN_SSLCERTFAIL, certFile.c_str(), e.what()));
    +			ret = CONN_FAILURE;
    +		}
    +	}
    +
    +    std::string disableHostVerification;
    +    props->getProp(USERPROP_DISABLE_HOSTVERIFICATION, disableHostVerification);
    +    if (disableHostVerification != "true") {
    +        std::string hostPortStr = m_pEndpoint->getHost() + ":" + m_pEndpoint->getPort();
    +        ((SSLChannelContext_t *) pContext)->getSslContext().set_verify_callback(
    +                boost::asio::ssl::rfc2818_verification(hostPortStr.c_str()));
    +    }
    +
    +    std::string disableCertificateVerification;
    +    props->getProp(USERPROP_DISABLE_CERTVERIFICATION, disableCertificateVerification);
    +    if (disableCertificateVerification == "true") {
    +        ((SSLChannelContext_t *) pContext)->getSslContext().set_verify_mode(boost::asio::ssl::context::verify_none);
    +    }
    --- End diff --
    
    Removed it from Channel::init, since the verify mode is more correctly a property of the channel context.
    I'd prefer to keep it the way it is, rather than move things to the constructor. Typically, constructors are better off doing simple things that cannot throw errors/exceptions and so I prefer to do all initialization of the underlying implementations in a method which can potentially handle errors better.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142827417
  
    --- Diff: exec/java-exec/pom.xml ---
    @@ -589,6 +669,17 @@
     
       <build>
     
    +    <extensions>
    +      <!--
    +      Include the os-maven-plugin to get os.detected.classifier
    +      -->
    +      <extension>
    +        <groupId>kr.motd.maven</groupId>
    +        <artifactId>os-maven-plugin</artifactId>
    +        <version>1.4.0.Final</version>
    --- End diff --
    
    I can build easily on Centos. But I took your tip and updated it to 1.5.0.Final. Hopefully that fixes it for you as well.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140133705
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -100,6 +103,8 @@ protected void initChannel(SocketChannel ch) throws Exception {
                 ch.closeFuture().addListener(getCloseHandler(ch, connection));
     
                 final ChannelPipeline pipe = ch.pipeline();
    +            // Make sure that the SSL handler is the first handler in the pipeline so everything is encrypted
    +            setupSSL(pipe, sslHandshakeListener);
    --- End diff --
    
    this will be called all the time even when SSL is not enabled and then later we have a check inside setupSSL where we are doing all the setup inside that if condition. How about check that here instead and then calling setupSSL method based on that check ? That way we know setupSSL is to do some setup and will be called only when SSL is enabled.
    ```
    
    if (isSslEnabled()) {
       sslHandshakeListener = new ConnectionMultiListener.SSLHandshakeListener();
        setupSSL(pipe, sslHandshakeListener);
    }
    ```
    and then remove that check from inside the setupSSL method.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141215109
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,452 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(.*):([0-9]+)(?:/(.+))?");
    +    boost::cmatch matched;
    +
    +    if(boost::regex_match(m_connectString.c_str(), matched, connStrExpr)){
    +        m_protocol.assign(matched[1].first, matched[1].second);
    +        std::string host, port;
    +        host.assign(matched[2].first, matched[2].second);
    +        port.assign(matched[3].first, matched[3].second);
    +        if(isDirectConnection()){
    +            // if the connection is to a zookeeper, 
    +            // we will get the host and the port only after connecting to the Zookeeper
    +            m_host=host;
    +            m_port=port;
    +        }
    +        m_hostPortStr=host+std::string(":")+port;
    +        std::string pathToDrill;
    +        if(matched.size()==5){
    +            pathToDrill.assign(matched[4].first, matched[4].second);
    +            if(!pathToDrill.empty()){
    +                m_pathToDrill=std::string("/")+pathToDrill;
    +            }
    +        }
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) 
    +                << "Conn str: "<< m_connectString 
    +                << ";  protocol: " << m_protocol 
    +                << ";  host: " << host 
    +                << "; port: " << port 
    +                << ";  path to drill: " << m_pathToDrill 
    +                << std::endl;)
    +    } else {
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Invalid connect string. Regexp did not match" << std::endl;)
    +    }
    +
    +    return;
    +}
    +
    +bool ConnectionEndpoint::isDirectConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "local") || !strcmp(m_protocol.c_str(), "drillbit"));
    +}
    +
    +bool ConnectionEndpoint::isZookeeperConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "zk"));
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpointFromZk(){
    +    ZookeeperClient zook(m_pathToDrill);
    +    assert(!m_hostPortStr.empty());
    +    std::vector<std::string> drillbits;
    +    if(zook.getAllDrillbits(m_hostPortStr.c_str(), drillbits)!=0){
    +        return handleError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    +    }
    +    if (drillbits.empty()){
    +        return handleError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    +    }
    +    Utils::shuffle(drillbits);
    +    exec::DrillbitEndpoint endpoint;
    +    int err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    +    if(!err){
    +        m_host=boost::lexical_cast<std::string>(endpoint.address());
    +        m_port=boost::lexical_cast<std::string>(endpoint.user_port());
    +    }
    +    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    --- End diff --
    
    We should return proper error code in case of any error while fetching the Drillbit Endpoint. This was handled in older code as well.
    
    ```
    if(err){
          return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));		
    }
    ```


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140587424
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    +
    +  protected String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    --- End diff --
    
    Shouldn't we give preference to application specific configuration over Hadoop configuration ? Hadoop Config will be more general config for all the ecosystem and if any ecosystem wants to override it with its own config then that should be allowed ?


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/950
  
    Go for it. I won't do any more commits until you give the all-clear.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140603240
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -110,6 +110,18 @@
       String HASHAGG_FALLBACK_ENABLED_KEY = "drill.exec.hashagg.fallback.enabled";
       BooleanValidator HASHAGG_FALLBACK_ENABLED_VALIDATOR = new BooleanValidator(HASHAGG_FALLBACK_ENABLED_KEY);
     
    +  String SSL_PROVIDER = "drill.exec.ssl.provider"; // valid values are "JDK", "OPENSSL" // default JDK
    +  String SSL_PROTOCOL = "drill.exec.ssl.protocol"; // valid values are SSL, SSLV2, SSLV3, TLS, TLSV1, TLSv1.1, TLSv1.2(default)
    +  String SSL_KEYSTORE_TYPE = "drill.exec.ssl.keyStoreType";
    +  String SSL_KEYSTORE_PATH = "drill.exec.ssl.keyStorePath";     // path to keystore. default : $JRE_HOME/lib/security/keystore.jks
    +  String SSL_KEYSTORE_PASSWORD = "drill.exec.ssl.keyStorePassword"; // default: changeit
    +  String SSL_KEY_PASSWORD = "drill.exec.ssl.keyPassword"; //
    +  String SSL_TRUSTSTORE_TYPE = "drill.exec.ssl.trustStoreType"; // valid values are jks(default), jceks, pkcs12
    +  String SSL_TRUSTSTORE_PATH = "drill.exec.ssl.trustStorePath"; // path to keystore. default : $JRE_HOME/lib/security/cacerts.jks
    +  String SSL_TRUSTSTORE_PASSWORD = "drill.exec.ssl.trustStorePassword"; // default: changeit
    +  String SSL_USE_HADOOP_CONF = "drill.exec.ssl.useHadoopConfig"; // Initialize ssl params from hadoop if not provided by drill. default: true
    +  String SSL_HANDSHAKE_TIMEOUT = "drill.exec.security.user.encryption.ssl.handshakeTimeout"; // Default 10 seconds
    --- End diff --
    
    I'd like to leave these are they are. trustStoreType, trustStorePassword, etc are spelt the same way in JDK and hadoop. 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142717783
  
    --- Diff: contrib/native/client/CMakeLists.txt ---
    @@ -93,7 +94,7 @@ else()
         #    of boost. Arbirtarily, we choose the new namspace to be drill_boost.
         #    See the instructions in the readme for linux/macos and rebuild boost. Then
         #    uncomment the line below to build
    -        set(Boost_NAMESPACE drill_boost)
    +    #    set(Boost_NAMESPACE drill_boost)
    --- End diff --
    
    temporary change?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142687176
  
    --- Diff: distribution/pom.xml ---
    @@ -97,8 +97,41 @@
         <dependency>
           <groupId>org.apache.hbase</groupId>
           <artifactId>hbase-client</artifactId>
    +      <exclusions>
    +       <exclusion>
    +           <groupId>io.netty</groupId>
    +           <artifactId>netty</artifactId>
    +       </exclusion>
    +      </exclusions>
    +    </dependency>
    +    <dependency>
    --- End diff --
    
    Not sure why we need to netty dependency with `test` scope in distribution pom... 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142667318
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java ---
    @@ -132,38 +173,47 @@ public void submitQuery(UserResultsListener resultsListener, RunQuery query) {
       /**
        * Connects, and if required, authenticates. This method blocks until both operations are complete.
        *
    -   * @param endpoint endpoint to connect to
    -   * @param properties properties
    +   * @param endpoint    endpoint to connect to
    +   * @param properties  properties
        * @param credentials credentials
        * @throws RpcException if either connection or authentication fails
        */
       public void connect(final DrillbitEndpoint endpoint, final DrillProperties properties,
    -                      final UserCredentials credentials) throws RpcException {
    -    final UserToBitHandshake.Builder hsBuilder = UserToBitHandshake.newBuilder()
    -        .setRpcVersion(UserRpcConfig.RPC_VERSION)
    -        .setSupportListening(true)
    -        .setSupportComplexTypes(supportComplexTypes)
    -        .setSupportTimeout(true)
    -        .setCredentials(credentials)
    -        .setClientInfos(UserRpcUtils.getRpcEndpointInfos(clientName))
    -        .setSaslSupport(SaslSupport.SASL_PRIVACY)
    -        .setProperties(properties.serializeForServer());
    +      final UserCredentials credentials) throws RpcException {
    +    final UserToBitHandshake.Builder hsBuilder =
    +        UserToBitHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).setSupportListening(true)
    --- End diff --
    
    Having each parameter assignment on separate line was more readable...


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/950
  
    Squashed and rebased on master again.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142727129
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    --- End diff --
    
    maybe we should stop using C char* and use std::string more


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141218943
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            connectionStatus_t validateConnectionString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    +                    return boost::asio::ssl::context::sslv23;
    +                } else if (version == "tlsv1") {
    +                    return boost::asio::ssl::context::tlsv1;
    +                } else if (version == "sslv3") {
    +                    return boost::asio::ssl::context::sslv3;
    +                } else {
    +                    return boost::asio::ssl::context::tlsv12;
    +                }
    +            }
    +
    +        SSLChannelContext(DrillUserProperties *props, boost::asio::ssl::context::method tlsVersion, boost::asio::ssl::verify_mode verifyMode) :
    +                ChannelContext(props),
    +                m_SSLContext(tlsVersion) {
    +                m_SSLContext.set_default_verify_paths();
    +                m_SSLContext.set_options(
    +                        boost::asio::ssl::context::default_workarounds
    +                        | boost::asio::ssl::context::no_sslv2
    +                        | boost::asio::ssl::context::single_dh_use
    +                        );
    +                m_SSLContext.set_verify_mode(verifyMode);
    +            };
    +            ~SSLChannelContext(){};
    +            boost::asio::ssl::context& getSslContext(){ return m_SSLContext;}
    +        private:
    +            boost::asio::ssl::context m_SSLContext;
    +    };
    +
    +    typedef ChannelContext ChannelContext_t; 
    +    typedef SSLChannelContext SSLChannelContext_t; 
    +
    +    class ChannelContextFactory{
    +        public:
    +            static ChannelContext_t* getChannelContext(channelType_t t, DrillUserProperties* props);
    +    };
    +
    +    /***
    +     * The Channel class encapsulates a connection to a drillbit. Based on 
    +     * the connection string and the options, the connection will be either 
    +     * a simple socket or a socket using an ssl stream. The class also encapsulates
    +     * connecting to a drillbit directly or thru zookeeper.
    +     * The channel class owns the socket and the io_service that the applications
    +     * will use to communicate with the server.
    +     ***/
    +    class Channel{
    +        public: 
    +            Channel(const char* connStr);
    +            Channel(const char* host, const char* port);
    +            Channel(boost::asio::io_service& ioService, const char* connStr);
    +            Channel(boost::asio::io_service& ioService, const char* host, const char* port);
    +            virtual ~Channel();
    +            virtual connectionStatus_t init(ChannelContext_t* context)=0;
    --- End diff --
    
    Why are we using `connectionStatus_t` as return type for `init` of a channel. `init` is not dealing with creating any connection, it is just setting some state of Channel object and doing few validations. So return type can be changed to `bool type.`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142735461
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -65,108 +56,72 @@ struct ToRpcType: public std::unary_function<google::protobuf::int32, exec::user
     		return static_cast<exec::user::RpcType>(i);
     	}
     };
    -}
    -connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    -    std::string pathToDrill, protocol, hostPortStr;
    -    std::string host;
    -    std::string port;
    +} // anonymous
     
    +connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
         if (this->m_bIsConnected) {
    -        if(std::strcmp(connStr, m_connectStr.c_str())){ // trying to connect to a different address is not allowed if already connected
    +        if(!std::strcmp(connStr, m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
                 return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
             return CONN_SUCCESS;
         }
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
     
    -    m_connectStr=connStr;
    -    Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
    -    if(protocol == "zk"){
    -        ZookeeperClient zook(pathToDrill);
    -        std::vector<std::string> drillbits;
    -        int err = zook.getAllDrillbits(hostPortStr, drillbits);
    -        if(!err){
    -            if (drillbits.empty()){
    -                return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    -            }
    -            Utils::shuffle(drillbits);
    -            exec::DrillbitEndpoint endpoint;
    -            err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    -            if(!err){
    -                host=boost::lexical_cast<std::string>(endpoint.address());
    -                port=boost::lexical_cast<std::string>(endpoint.user_port());
    -            }
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    -
    -        }
    -        if(err){
    -            return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    -        }
    -        zook.close();
    -        m_bIsDirectConnection=true;
    -    }else if(protocol == "local"){
    -        boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
    -        char tempStr[MAX_CONNECT_STR+1];
    -        strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
    -        host=strtok(tempStr, ":");
    -        port=strtok(NULL, "");
    -        m_bIsDirectConnection=false;
    -    }else{
    -        return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
    -    }
    -    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
    -    std::string serviceHost;
    -    for (size_t i = 0; i < props->size(); i++) {
    -        if (props->keyAt(i) == USERPROP_SERVICE_HOST) {
    -            serviceHost = props->valueAt(i);
    -        }
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, connStr);
    +    ret=m_pChannel->init(m_pChannelContext);
    --- End diff --
    
    I guess it could be passed at construct time (or even better, the channel creates its own context?)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142732789
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    --- End diff --
    
    not necessary (since there's a catch-all)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142827719
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigServer.java ---
    @@ -0,0 +1,331 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslContextBuilder;
    +import io.netty.handler.ssl.SslProvider;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.text.MessageFormat;
    +
    +public class SSLConfigServer extends SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfigServer.class);
    +
    +  private final DrillConfig config;
    +  private final Configuration hadoopConfig;
    +  private final boolean userSslEnabled;
    +  private final boolean httpsEnabled;
    +  private final String keyStoreType;
    +  private final String keyStorePath;
    +  private final String keyStorePassword;
    +  private final String keyPassword;
    +  private final String trustStoreType;
    +  private final String trustStorePath;
    +  private final String trustStorePassword;
    +  private final String protocol;
    +  private final String provider;
    +
    +  public SSLConfigServer(DrillConfig config, Configuration hadoopConfig) throws DrillException {
    +    this.config = config;
    +    SSLFactory.Mode mode = SSLFactory.Mode.SERVER;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, getMode()));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +    userSslEnabled =
    +        config.hasPath(ExecConstants.USER_SSL_ENABLED) && config.getBoolean(ExecConstants.USER_SSL_ENABLED);
    +    trustStoreType = getConfigParam(ExecConstants.SSL_TRUSTSTORE_TYPE,
    +        resolveHadoopPropertyName(HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY, mode));
    +    trustStorePath = getConfigParam(ExecConstants.SSL_TRUSTSTORE_PATH,
    +        resolveHadoopPropertyName(HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY, mode));
    +    trustStorePassword = getConfigParam(ExecConstants.SSL_TRUSTSTORE_PASSWORD,
    +        resolveHadoopPropertyName(HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY, mode));
    +    keyStoreType = getConfigParam(ExecConstants.SSL_KEYSTORE_TYPE,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY, mode));
    +    keyStorePath = getConfigParam(ExecConstants.SSL_KEYSTORE_PATH,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY, mode));
    +    keyStorePassword = getConfigParam(ExecConstants.SSL_KEYSTORE_PASSWORD,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY, mode));
    +    // if no keypassword specified, use keystore password
    +    String keyPass = getConfigParam(ExecConstants.SSL_KEY_PASSWORD,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY, mode));
    +    keyPassword = keyPass.isEmpty() ? keyStorePassword : keyPass;
    +    protocol = getConfigParamWithDefault(ExecConstants.SSL_PROTOCOL, DEFAULT_SSL_PROTOCOL);
    +    provider = getConfigParamWithDefault(ExecConstants.SSL_PROVIDER, DEFAULT_SSL_PROVIDER);
    +  }
    +
    +  public void validateKeyStore() throws DrillException {
    +    //HTTPS validates the keystore is not empty. User Server SSL context initialization also validates keystore, but
    +    // much more strictly. User Client context initialization does not validate keystore.
    +    /*If keystorePath or keystorePassword is provided in the configuration file use that*/
    +    if ((isUserSslEnabled() || isHttpsEnabled())) {
    +      if (!keyStorePath.isEmpty() || !keyStorePassword.isEmpty()) {
    +        if (keyStorePath.isEmpty()) {
    +          throw new DrillException(
    +              " *.ssl.keyStorePath in the configuration file is empty, but *.ssl.keyStorePassword is set");
    +        } else if (keyStorePassword.isEmpty()) {
    +          throw new DrillException(
    +              " *.ssl.keyStorePassword in the configuration file is empty, but *.ssl.keyStorePath is set ");
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public SslContext initNettySslContext() throws DrillException {
    +    final SslContext sslCtx;
    +
    +    if (!userSslEnabled) {
    +      return null;
    +    }
    +
    +    KeyManagerFactory kmf;
    +    TrustManagerFactory tmf;
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      kmf = initializeKeyManagerFactory();
    +      tmf = initializeTrustManagerFactory();
    +      sslCtx = SslContextBuilder.forServer(kmf)
    +          .trustManager(tmf)
    +          .protocols(protocol)
    +          .sslProvider(getProvider())
    +          .build(); // Will throw an exception if the key password is not correct
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(new StringBuilder()
    +          .append("SSL is enabled but cannot be initialized - ")
    +          .append("[ ")
    +          .append(e.getMessage())
    +          .append("]. ")
    +          .toString());
    +    }
    +    this.nettySslContext = sslCtx;
    +    return sslCtx;
    +  }
    +
    +  @Override
    +  public SSLContext initJDKSSLContext() throws DrillException {
    +    final SSLContext sslCtx;
    +
    +    if (!userSslEnabled) {
    +      return null;
    +    }
    +
    +    KeyManagerFactory kmf;
    +    TrustManagerFactory tmf;
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      kmf = initializeKeyManagerFactory();
    +      tmf = initializeTrustManagerFactory();
    +      sslCtx = SSLContext.getInstance(protocol);
    +      sslCtx.init(kmf.getKeyManagers(), tmf.getTrustManagers(), null);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    +          new StringBuilder().append("SSL is enabled but cannot be initialized - ")
    +              .append("[ ")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    this.jdkSSlContext = sslCtx;
    +    return sslCtx;
    +  }
    +
    +  @Override
    +  public SSLEngine createSSLEngine(BufferAllocator allocator, String peerHost, int peerPort) {
    +    SSLEngine engine = super.createSSLEngine(allocator, peerHost, peerPort);
    +
    +    engine.setUseClientMode(false);
    +
    +    // No need for client side authentication (HTTPS like behaviour)
    +    engine.setNeedClientAuth(false);
    +
    +    try {
    +      engine.setEnableSessionCreation(true);
    +    } catch (Exception e) {
    +      // Openssl implementation may throw this.
    +      logger.debug("Session creation not enabled. Exception: {}", e.getMessage());
    +    }
    +
    +    return engine;
    +  }
    +
    +  private String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  private String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    --- End diff --
    
    OK


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142729078
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,448 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                DRILL_LOG(LOG_INFO) << "Failed to get endpoint from zk" << std::endl;
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(((.*):([0-9]+),?)+)(/.+)?");
    +    boost::cmatch matched;
    +
    +    if(boost::regex_match(m_connectString.c_str(), matched, connStrExpr)){
    --- End diff --
    
    I'm surprised a conversion to a C char* is required by boost...


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140610830
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    +
    +  protected String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    +    value = hadoopConfig.get(name, "");
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getConfigParamWithDefault(String name, String defaultValue) {
    +    String value = "";
    +    if (config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    if (value.isEmpty()) {
    +      value = defaultValue;
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String resolveHadoopPropertyName(String nameTemplate, SSLFactory.Mode mode) {
    +    return MessageFormat.format(nameTemplate, mode.toString().toLowerCase());
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  public abstract SslContext initSslContext() throws DrillException;
    +
    +  public abstract SSLContext initSSLContext() throws DrillException;
    +
    +  public abstract boolean isUserSslEnabled();
    +
    +  public abstract boolean isHttpsEnabled();
    +
    +  public abstract String getKeyStoreType();
    +
    +  public abstract String getKeyStorePath();
    +
    +  public abstract String getKeyStorePassword();
    +
    +  public abstract String getKeyPassword();
    +
    +  public abstract String getTrustStoreType();
    +
    +  public abstract boolean hasTrustStorePath();
    +
    +  public abstract String getTrustStorePath();
    +
    +  public abstract boolean hasTrustStorePassword();
    +
    +  public abstract String getTrustStorePassword();
    +
    +  public abstract String getProtocol();
    +
    +  public abstract SslProvider getProvider();
    +
    +  public abstract int getHandshakeTimeout();
    +
    +  public abstract SSLFactory.Mode getMode();
    +
    +  public abstract boolean disableHostVerification();
    +
    +  public abstract boolean disableCertificateVerification();
    +
    +  public abstract boolean useSystemTrustStore();
    +
    +  public abstract boolean isSslValid();
    +
    +  public SslContext getNettySslContext() {
    +    return nettySslContext;
    +  }
    +
    +  public TrustManagerFactory initializeTrustManagerFactory() throws DrillException {
    +    TrustManagerFactory tmf;
    +    KeyStore ts = null;
    +    //Support Windows/MacOs system trust store
    +    try {
    +      String trustStoreType = getTrustStoreType();
    +      if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +        // This is valid for MS-Windows and MacOs
    +        logger.debug("Initializing System truststore.");
    +        ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +        ts.load(null, null);
    +      } else if (!getTrustStorePath().isEmpty()) {
    +          // if truststore is not provided then we will use the default. Note that the default depends on
    +          // the TrustManagerFactory that in turn depends on the Security Provider.
    +          // Use null as the truststore which will result in the default truststore being picked up
    +          logger.debug("Initializing truststore {}.", getTrustStorePath());
    +          ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +          InputStream tsStream = new FileInputStream(getTrustStorePath());
    +          ts.load(tsStream, getTrustStorePassword().toCharArray());
    +      } else {
    +        logger.debug("Initializing default truststore.");
    +      }
    +      if (disableCertificateVerification()) {
    +        tmf = InsecureTrustManagerFactory.INSTANCE;
    +      } else {
    +        tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
    +      }
    +      tmf.init(ts);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the truststore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return tmf;
    +  }
    +
    +  public KeyManagerFactory initializeKeyManagerFactory() throws DrillException {
    +    KeyManagerFactory kmf;
    +    String keyStorePath = getKeyStorePath();
    +    String keyStorePassword = getKeyStorePassword();
    +    String keyStoreType = getKeyStoreType();
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      KeyStore ks =
    +          KeyStore.getInstance(!keyStoreType.isEmpty() ? keyStoreType : KeyStore.getDefaultType());
    +      //initialize the key manager factory
    +      // Will throw an exception if the file is not found/accessible.
    +      InputStream ksStream = new FileInputStream(keyStorePath);
    +      // A key password CANNOT be null or an empty string.
    +      if (keyStorePassword.isEmpty()) {
    +        throw new DrillException("The Keystore password cannot be empty.");
    +      }
    +      ks.load(ksStream, keyStorePassword.toCharArray());
    +      // Empty Keystore. (Remarkably, it is possible to do this).
    +      if (ks.size() == 0) {
    +        throw new DrillException("The Keystore has no entries.");
    +      }
    +      kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
    +      kmf.init(ks, getKeyPassword().toCharArray());
    +
    +    } catch (Exception e) {
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the keystore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return kmf;
    +  }
    +
    +  public void initContext() throws DrillException {
    +    if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +      initSSLContext();
    +      logger.debug("Initialized Windows SSL context using JDK.");
    +    } else {
    +      initSslContext();
    +      logger.debug("Initialized SSL context.");
    +    }
    +    return;
    +  }
    +
    +  public SSLEngine createSSLEngine(BufferAllocator allocator, String peerHost, int peerPort) {
    +    SSLEngine engine;
    +    if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +      if (peerHost != null) {
    +        engine = jdkSSlContext.createSSLEngine(peerHost, peerPort);
    +        logger.debug("Initializing Windows SSLEngine with hostname verification.");
    +      } else {
    +        engine = jdkSSlContext.createSSLEngine();
    +        logger.debug("Initializing Windows SSLEngine with no hostname verification.");
    +      }
    +    } else {
    +      if (peerHost != null) {
    +        engine = nettySslContext.newEngine(allocator.getAsByteBufAllocator(), peerHost, peerPort);
    +        logger.debug("Initializing SSLEngine with hostname verification.");
    +      } else {
    +        engine = nettySslContext.newEngine(allocator.getAsByteBufAllocator());
    +        logger.debug("Initializing SSLEngine with no hostname verification.");
    +      }
    +    }
    +    return engine;
    --- End diff --
    
    I'm going to leave this as it is. Seems easier to read to me. 
    I changed the logging message (you did notice it was for debugging, right?). Hostname verification is not enabled/disabled in this method but in the caller.  


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140399380
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java ---
    @@ -70,22 +78,80 @@
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
       private static final String SERVER_NAME = "Apache Drill Server";
     
    +  private final BootStrapContext bootStrapContext;
    +  private final BufferAllocator allocator;
       private final UserConnectionConfig config;
    +  private final SSLConfig sslConfig;
    +  private Channel sslChannel;
       private final UserWorker userWorker;
     
       public UserServer(BootStrapContext context, BufferAllocator allocator, EventLoopGroup eventLoopGroup,
                         UserWorker worker) throws DrillbitStartupException {
         super(UserRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
             allocator.getAsByteBufAllocator(),
             eventLoopGroup);
    +    this.bootStrapContext = context;
    +    this.allocator = allocator;
         this.config = new UserConnectionConfig(allocator, context, new UserServerRequestHandler(worker));
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder()
    +          .config(bootStrapContext.getConfig())
    +          .mode(SSLFactory.Mode.SERVER)
    +          .initializeSSLContext(true)
    +          .validateKeyStore(true)
    +          .build();
    +    } catch (DrillException e) {
    +      throw new DrillbitStartupException(e.getMessage(), e.getCause());
    +    }
         this.userWorker = worker;
     
         // Initialize Singleton instance of UserRpcMetrics.
         ((UserRpcMetrics)UserRpcMetrics.getInstance()).initialize(config.isEncryptionEnabled(), allocator);
       }
     
       @Override
    +  protected void setupSSL(ChannelPipeline pipe) {
    +    if (sslConfig.isUserSslEnabled()) {
    +
    +      SSLEngine sslEngine = sslConfig.createSSLEngine(allocator, null, 0);
    +      sslEngine.setUseClientMode(false);
    +
    +      // No need for client side authentication (HTTPS like behaviour)
    +      sslEngine.setNeedClientAuth(false);
    +
    +      try {
    +        sslEngine.setEnableSessionCreation(true);
    +      } catch (Exception e) {
    +        // Openssl implementation may throw this.
    +        logger.debug("Session creation not enabled. Exception: {}", e.getMessage());
    +      }
    --- End diff --
    
    All these setup of sslEngine can be moved to SSLConfigServer:createSSLEngine(..) and same thing for client side setupSSL which can be moved to SSLConfigClient::createSSLEngine(..)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140604469
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java ---
    @@ -70,22 +78,80 @@
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
       private static final String SERVER_NAME = "Apache Drill Server";
     
    +  private final BootStrapContext bootStrapContext;
    +  private final BufferAllocator allocator;
    --- End diff --
    
    Hmm, you're right. But don't you think the bootstrap context and allocator are more accurately properties of the Server object than a Connection object?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142718781
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -544,4 +573,4 @@ int main(int argc, char* argv[]) {
         }
     
         return 0;
    -}
    +}
    --- End diff --
    
    (style) I know some tools complain when removing the last newline


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141245539
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -65,108 +66,70 @@ struct ToRpcType: public std::unary_function<google::protobuf::int32, exec::user
     		return static_cast<exec::user::RpcType>(i);
     	}
     };
    -}
    -connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    -    std::string pathToDrill, protocol, hostPortStr;
    -    std::string host;
    -    std::string port;
    +} // anonymous
     
    -    if (this->m_bIsConnected) {
    -        if(std::strcmp(connStr, m_connectStr.c_str())){ // trying to connect to a different address is not allowed if already connected
    +connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    +        if(!std::strcmp(connStr, m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
                 return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
             return CONN_SUCCESS;
         }
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
     
    -    m_connectStr=connStr;
    -    Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
    -    if(protocol == "zk"){
    -        ZookeeperClient zook(pathToDrill);
    -        std::vector<std::string> drillbits;
    -        int err = zook.getAllDrillbits(hostPortStr, drillbits);
    -        if(!err){
    -            if (drillbits.empty()){
    -                return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    -            }
    -            Utils::shuffle(drillbits);
    -            exec::DrillbitEndpoint endpoint;
    -            err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    -            if(!err){
    -                host=boost::lexical_cast<std::string>(endpoint.address());
    -                port=boost::lexical_cast<std::string>(endpoint.user_port());
    -            }
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    -
    -        }
    -        if(err){
    -            return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    -        }
    -        zook.close();
    -        m_bIsDirectConnection=true;
    -    }else if(protocol == "local"){
    -        boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
    -        char tempStr[MAX_CONNECT_STR+1];
    -        strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
    -        host=strtok(tempStr, ":");
    -        port=strtok(NULL, "");
    -        m_bIsDirectConnection=false;
    -    }else{
    -        return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
    -    }
    -    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
    -    std::string serviceHost;
    -    for (size_t i = 0; i < props->size(); i++) {
    -        if (props->keyAt(i) == USERPROP_SERVICE_HOST) {
    -            serviceHost = props->valueAt(i);
    -        }
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, connStr);
    +    ret=m_pChannel->init(m_pChannelContext);
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    if (serviceHost.empty()) {
    -        props->setProperty(USERPROP_SERVICE_HOST, host);
    +    ret= m_pChannel->connect();
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    connectionStatus_t ret = this->connect(host.c_str(), port.c_str());
    +    props->setProperty(USERPROP_SERVICE_HOST, m_pChannel->getEndpoint()->getHost());
         return ret;
     }
     
    -connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
    -    using boost::asio::ip::tcp;
    -    tcp::endpoint endpoint;
    -    try{
    -        tcp::resolver resolver(m_io_service);
    -        tcp::resolver::query query(tcp::v4(), host, port);
    -        tcp::resolver::iterator iter = resolver.resolve(query);
    -        tcp::resolver::iterator end;
    -        while (iter != end){
    -            endpoint = *iter++;
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << endpoint << std::endl;)
    -        }
    -        boost::system::error_code ec;
    -        m_socket.connect(endpoint, ec);
    -        if(ec){
    -            return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_FAILURE, host, port, ec.message().c_str()));
    -        }
    -
    -    }catch(const std::exception & e){
    -        // Handle case when the hostname cannot be resolved. "resolve" is hard-coded in boost asio resolver.resolve
    -        if (!strcmp(e.what(), "resolve")) {
    -            return handleConnError(CONN_HOSTNAME_RESOLUTION_ERROR, getMessage(ERR_CONN_EXCEPT, e.what()));
    +connectionStatus_t DrillClientImpl::connect(const char* host, const char* port, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    +        std::string connStr = std::string(host)+":"+std::string(port);
    +        if(!std::strcmp(connStr.c_str(), m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
    +            return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
    -        return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_EXCEPT, e.what()));
    +        return CONN_SUCCESS;
         }
    -
    -    m_bIsConnected=true;
    -    // set socket keep alive
    -    boost::asio::socket_base::keep_alive keepAlive(true);
    -    m_socket.set_option(keepAlive);
    -    // set no_delay
    -    boost::asio::ip::tcp::no_delay noDelay(true);
    -    m_socket.set_option(noDelay);
    -
    -    std::ostringstream connectedHost;
    -    connectedHost << "id: " << m_socket.native_handle() << " address: " << host << ":" << port;
    -    m_connectedHost = connectedHost.str();
    -    DRILL_MT_LOG(DRILL_LOG(LOG_INFO) << "Connected to endpoint: " << m_connectedHost << std::endl;)
    -
    -    return CONN_SUCCESS;
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
    +
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, host, port);
    +    m_pChannel->init(m_pChannelContext);
    +    ret=m_pChannel->init(m_pChannelContext);
    --- End diff --
    
    duplicate call to `m_pChannel->init(m_pChannelContext);`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142824715
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    +                    return boost::asio::ssl::context::sslv23;
    +                } else if (version == "tlsv1") {
    +                    return boost::asio::ssl::context::tlsv1;
    +                } else if (version == "sslv3") {
    +                    return boost::asio::ssl::context::sslv3;
    +                } else {
    +                    return boost::asio::ssl::context::tlsv12;
    +                }
    +            }
    +
    +        SSLChannelContext(DrillUserProperties *props, boost::asio::ssl::context::method tlsVersion, boost::asio::ssl::verify_mode verifyMode) :
    +                ChannelContext(props),
    +                m_SSLContext(tlsVersion) {
    +                m_SSLContext.set_default_verify_paths();
    +                m_SSLContext.set_options(
    +                        boost::asio::ssl::context::default_workarounds
    +                        | boost::asio::ssl::context::no_sslv2
    +                        | boost::asio::ssl::context::single_dh_use
    +                        );
    +                m_SSLContext.set_verify_mode(verifyMode);
    +            };
    +            ~SSLChannelContext(){};
    +            boost::asio::ssl::context& getSslContext(){ return m_SSLContext;}
    +        private:
    +            boost::asio::ssl::context m_SSLContext;
    +    };
    +
    +    typedef ChannelContext ChannelContext_t; 
    +    typedef SSLChannelContext SSLChannelContext_t; 
    +
    +    class ChannelContextFactory{
    --- End diff --
    
    Done. That got rid of a bunch of code.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142732709
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    --- End diff --
    
    version should be a cref


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142683194
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSL.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc.user.security;
    +
    +import com.typesafe.config.ConfigValueFactory;
    +import io.netty.handler.ssl.util.SelfSignedCertificate;
    +import junit.framework.TestCase;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.exec.ExecConstants;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.net.InetAddress;
    +import java.security.KeyStore;
    +import java.util.Properties;
    +
    +import static junit.framework.TestCase.fail;
    +import static org.junit.Assert.assertEquals;
    +
    +public class TestUserBitSSL extends BaseTestQuery {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(TestUserBitSSL.class);
    +
    +  private static DrillConfig newConfig;
    +  private static Properties initProps; // initial client properties
    +  private static ClassLoader classLoader;
    +  private static String ksPath;
    +  private static String tsPath;
    +  private static String emptyTSPath;
    +  private static String unknownKsPath;
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +
    +    // Create a new DrillConfig
    +    classLoader = TestUserBitSSL.class.getClassLoader();
    +    ksPath = new File(classLoader.getResource("ssl/keystore.ks").getFile()).getAbsolutePath();
    +    unknownKsPath = new File(classLoader.getResource("ssl/unknownkeystore.ks").getFile()).getAbsolutePath();
    +    tsPath = new File(classLoader.getResource("ssl/truststore.ks").getFile()).getAbsolutePath();
    +    emptyTSPath = new File(classLoader.getResource("ssl/emptytruststore.ks").getFile()).getAbsolutePath();
    +    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
    +        .withValue(ExecConstants.SSL_USE_HADOOP_CONF,
    +            ConfigValueFactory.fromAnyRef(false))
    +        .withValue(ExecConstants.USER_SSL_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.SSL_KEYSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(ksPath))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(tsPath))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_PROTOCOL,
    +            ConfigValueFactory.fromAnyRef("TLSv1.2")),
    +      false);
    +
    +    initProps = new Properties();
    +    initProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    initProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +
    +    // Start an SSL enabled cluster
    +    updateTestCluster(1, newConfig, initProps);
    +  }
    +
    +  @AfterClass
    +  public static void cleanTest() throws Exception {
    +    DrillConfig restoreConfig =
    +        new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties()), false);
    +    updateTestCluster(1, restoreConfig);
    +  }
    +
    +  @Test
    +  public void testSSLConnection() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionWithKeystore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ksPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailBadTrustStore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ""); // NO truststore
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailBadPassword() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "bad_password");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailEmptyTrustStore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, emptyTSPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testSSLQuery() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +    test("SELECT * FROM cp.`region.json`");
    +  }
    +
    +  @Test
    +  public void testClientConfigHostnameVerification() {
    +    String password = "test_password";
    +    String trustStoreFileName = "drillTestTrustStore";
    +    String keyStoreFileName = "drillTestKeyStore";
    +    KeyStore ts, ks;
    +    File tempFile1, tempFile2;
    +    String trustStorePath;
    +    String keyStorePath;
    +
    +    try {
    +      String fqdn = InetAddress.getLocalHost().getHostName();
    +      SelfSignedCertificate certificate = new SelfSignedCertificate(fqdn);
    +
    +      tempFile1 = File.createTempFile(trustStoreFileName, ".ks");
    +      tempFile1.deleteOnExit();
    +      trustStorePath = tempFile1.getAbsolutePath();
    +      //generate a truststore.
    +      ts = KeyStore.getInstance(KeyStore.getDefaultType());
    +      ts.load(null, password.toCharArray());
    +      ts.setCertificateEntry("drillTest", certificate.cert());
    +      // Store away the truststore.
    +      FileOutputStream fos1 = new FileOutputStream(tempFile1);
    +      ts.store(fos1, password.toCharArray());
    +      fos1.close();
    --- End diff --
    
    It's better to use try with resources thus stream will be definitely closed in case of exception.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142821976
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -544,4 +573,4 @@ int main(int argc, char* argv[]) {
         }
     
         return 0;
    -}
    +}
    --- End diff --
    
    added newline


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141676885
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,91 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl) {
    --- End diff --
    
    Good idea. I'll make it a generic error message string. Store name is windows specific and later we might enhance this to support other native system certificate stores


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140970043
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,91 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl) {
    --- End diff --
    
    Can we update this method to take a second parameter like `string& store_name` which we can set in case of error while opening store. so the caller in case of error can also print the store name which caused the error.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140130050
  
    --- Diff: common/src/main/java/org/apache/drill/common/config/DrillConfig.java ---
    @@ -110,8 +110,8 @@ public static DrillConfig create() {
        *
        * @return {@link DrillConfig} instance
        */
    -  public static DrillConfig forClient() {
    -    return create(null, false);
    +  public static DrillConfig forClient(Properties props) {
    +    return create(null, props, false);
    --- End diff --
    
    This change is not required since this will duplicate the parameters passed by JDBC application to DrillClient. We are passing that information using _info (Properties type object) in DrillConnectionImpl_


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140395315
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java ---
    @@ -70,22 +78,80 @@
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
       private static final String SERVER_NAME = "Apache Drill Server";
     
    +  private final BootStrapContext bootStrapContext;
    +  private final BufferAllocator allocator;
       private final UserConnectionConfig config;
    +  private final SSLConfig sslConfig;
    +  private Channel sslChannel;
       private final UserWorker userWorker;
     
       public UserServer(BootStrapContext context, BufferAllocator allocator, EventLoopGroup eventLoopGroup,
                         UserWorker worker) throws DrillbitStartupException {
         super(UserRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
             allocator.getAsByteBufAllocator(),
             eventLoopGroup);
    +    this.bootStrapContext = context;
    +    this.allocator = allocator;
         this.config = new UserConnectionConfig(allocator, context, new UserServerRequestHandler(worker));
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder()
    +          .config(bootStrapContext.getConfig())
    --- End diff --
    
    instead `context.getConfig()`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140621612
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -100,6 +103,8 @@ protected void initChannel(SocketChannel ch) throws Exception {
                 ch.closeFuture().addListener(getCloseHandler(ch, connection));
     
                 final ChannelPipeline pipe = ch.pipeline();
    +            // Make sure that the SSL handler is the first handler in the pipeline so everything is encrypted
    +            setupSSL(pipe, sslHandshakeListener);
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140394583
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java ---
    @@ -102,19 +115,78 @@
       // these are used for authentication
       private volatile List<String> serverAuthMechanisms = null;
       private volatile boolean authComplete = true;
    +  private SSLConfig sslConfig;
    +  private Channel sslChannel;
    +  private DrillbitEndpoint endpoint;
     
       public UserClient(String clientName, DrillConfig config, boolean supportComplexTypes,
    -      BufferAllocator allocator, EventLoopGroup eventLoopGroup, Executor eventExecutor) {
    -    super(
    -        UserRpcConfig.getMapping(config, eventExecutor),
    -        allocator.getAsByteBufAllocator(),
    -        eventLoopGroup,
    -        RpcType.HANDSHAKE,
    -        BitToUserHandshake.class,
    -        BitToUserHandshake.PARSER);
    +      BufferAllocator allocator, EventLoopGroup eventLoopGroup, Executor eventExecutor,
    +      DrillbitEndpoint endpoint) throws NonTransientRpcException {
    +    super(UserRpcConfig.getMapping(config, eventExecutor), allocator.getAsByteBufAllocator(),
    +        eventLoopGroup, RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER);
    +    this.endpoint = endpoint; // save the endpoint; it might be needed by SSL init.
         this.clientName = clientName;
         this.allocator = allocator;
         this.supportComplexTypes = supportComplexTypes;
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder().config(config).mode(SSLFactory.Mode.CLIENT)
    +          .initializeSSLContext(true).validateKeyStore(false).build();
    +    } catch (DrillException e) {
    +      throw new NonTransientRpcException(e.getMessage());
    +    }
    +
    +  }
    +
    +  @Override protected void setupSSL(ChannelPipeline pipe,
    +      ConnectionMultiListener.SSLHandshakeListener sslHandshakeListener) {
    +    if (sslConfig.isUserSslEnabled()) {
    +
    +      String peerHost = endpoint.getAddress();
    +      int peerPort = endpoint.getUserPort();
    +      SSLEngine sslEngine = sslConfig.createSSLEngine(allocator, peerHost, peerPort);
    +
    +      if (!sslConfig.disableHostVerification()) {
    +        SSLParameters sslParameters = sslEngine.getSSLParameters();
    +        // only available since Java 7
    +        sslParameters.setEndpointIdentificationAlgorithm("HTTPS");
    +        sslEngine.setSSLParameters(sslParameters);
    +      }
    +
    +      sslEngine.setUseClientMode(true);
    +
    +      // set Security property jdk.certpath.disabledAlgorithms  to disable specific ssl algorithms
    +      sslEngine.setEnabledProtocols(sslEngine.getEnabledProtocols());
    +
    +      // set Security property jdk.tls.disabledAlgorithms to disable specific cipher suites
    +      sslEngine.setEnabledCipherSuites(sslEngine.getEnabledCipherSuites());
    +      sslEngine.setEnableSessionCreation(true);
    +
    +      // Add SSL handler into pipeline
    +      SslHandler sslHandler = new SslHandler(sslEngine);
    +      sslHandler.setHandshakeTimeoutMillis(sslConfig.getHandshakeTimeout());
    +
    +      // Add a listener for SSL Handshake complete. The Drill client handshake will be enabled only
    +      // after this is done.
    +      sslHandler.handshakeFuture().addListener(sslHandshakeListener);
    +      pipe.addFirst(RpcConstants.SSL_HANDLER, sslHandler);
    +    }
    +    logger.debug(sslConfig.toString());
    +  }
    +
    +  @Override protected boolean isSslEnabled() {
    +    return sslConfig.isUserSslEnabled();
    +  }
    +
    +  @Override public void setSslChannel(Channel c) {
    +    sslChannel = c;
    +    return;
    --- End diff --
    
    _return_ not required


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141742537
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -65,108 +66,70 @@ struct ToRpcType: public std::unary_function<google::protobuf::int32, exec::user
     		return static_cast<exec::user::RpcType>(i);
     	}
     };
    -}
    -connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    -    std::string pathToDrill, protocol, hostPortStr;
    -    std::string host;
    -    std::string port;
    +} // anonymous
     
    -    if (this->m_bIsConnected) {
    -        if(std::strcmp(connStr, m_connectStr.c_str())){ // trying to connect to a different address is not allowed if already connected
    +connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    +        if(!std::strcmp(connStr, m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
                 return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
             return CONN_SUCCESS;
         }
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
     
    -    m_connectStr=connStr;
    -    Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
    -    if(protocol == "zk"){
    -        ZookeeperClient zook(pathToDrill);
    -        std::vector<std::string> drillbits;
    -        int err = zook.getAllDrillbits(hostPortStr, drillbits);
    -        if(!err){
    -            if (drillbits.empty()){
    -                return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    -            }
    -            Utils::shuffle(drillbits);
    -            exec::DrillbitEndpoint endpoint;
    -            err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    -            if(!err){
    -                host=boost::lexical_cast<std::string>(endpoint.address());
    -                port=boost::lexical_cast<std::string>(endpoint.user_port());
    -            }
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    -
    -        }
    -        if(err){
    -            return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    -        }
    -        zook.close();
    -        m_bIsDirectConnection=true;
    -    }else if(protocol == "local"){
    -        boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
    -        char tempStr[MAX_CONNECT_STR+1];
    -        strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
    -        host=strtok(tempStr, ":");
    -        port=strtok(NULL, "");
    -        m_bIsDirectConnection=false;
    -    }else{
    -        return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
    -    }
    -    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
    -    std::string serviceHost;
    -    for (size_t i = 0; i < props->size(); i++) {
    -        if (props->keyAt(i) == USERPROP_SERVICE_HOST) {
    -            serviceHost = props->valueAt(i);
    -        }
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, connStr);
    +    ret=m_pChannel->init(m_pChannelContext);
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    if (serviceHost.empty()) {
    -        props->setProperty(USERPROP_SERVICE_HOST, host);
    +    ret= m_pChannel->connect();
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    connectionStatus_t ret = this->connect(host.c_str(), port.c_str());
    +    props->setProperty(USERPROP_SERVICE_HOST, m_pChannel->getEndpoint()->getHost());
         return ret;
     }
     
    -connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
    -    using boost::asio::ip::tcp;
    -    tcp::endpoint endpoint;
    -    try{
    -        tcp::resolver resolver(m_io_service);
    -        tcp::resolver::query query(tcp::v4(), host, port);
    -        tcp::resolver::iterator iter = resolver.resolve(query);
    -        tcp::resolver::iterator end;
    -        while (iter != end){
    -            endpoint = *iter++;
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << endpoint << std::endl;)
    -        }
    -        boost::system::error_code ec;
    -        m_socket.connect(endpoint, ec);
    -        if(ec){
    -            return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_FAILURE, host, port, ec.message().c_str()));
    -        }
    -
    -    }catch(const std::exception & e){
    -        // Handle case when the hostname cannot be resolved. "resolve" is hard-coded in boost asio resolver.resolve
    -        if (!strcmp(e.what(), "resolve")) {
    -            return handleConnError(CONN_HOSTNAME_RESOLUTION_ERROR, getMessage(ERR_CONN_EXCEPT, e.what()));
    +connectionStatus_t DrillClientImpl::connect(const char* host, const char* port, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    +        std::string connStr = std::string(host)+":"+std::string(port);
    +        if(!std::strcmp(connStr.c_str(), m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
    +            return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
    -        return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_EXCEPT, e.what()));
    +        return CONN_SUCCESS;
         }
    -
    -    m_bIsConnected=true;
    -    // set socket keep alive
    -    boost::asio::socket_base::keep_alive keepAlive(true);
    -    m_socket.set_option(keepAlive);
    -    // set no_delay
    -    boost::asio::ip::tcp::no_delay noDelay(true);
    -    m_socket.set_option(noDelay);
    -
    -    std::ostringstream connectedHost;
    -    connectedHost << "id: " << m_socket.native_handle() << " address: " << host << ":" << port;
    -    m_connectedHost = connectedHost.str();
    -    DRILL_MT_LOG(DRILL_LOG(LOG_INFO) << "Connected to endpoint: " << m_connectedHost << std::endl;)
    -
    -    return CONN_SUCCESS;
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
    +
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, host, port);
    +    m_pChannel->init(m_pChannelContext);
    +    ret=m_pChannel->init(m_pChannelContext);
    --- End diff --
    
    Yeah. I wonder why I did that.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140601290
  
    --- Diff: contrib/native/client/readme.linux ---
    @@ -84,6 +84,21 @@ OR
         ln -svf libboost_filesystem.a libboost_filesystem-mt.a
         ln -svf libboost_date_time.a libboost_date_time-mt.a
     
    +5) Install or  Cyrus SASL 
    --- End diff --
    
    yes


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140603400
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -126,10 +138,10 @@
       String HTTP_SESSION_MEMORY_RESERVATION = "drill.exec.http.session.memory.reservation";
       String HTTP_SESSION_MEMORY_MAXIMUM = "drill.exec.http.session.memory.maximum";
       String HTTP_SESSION_MAX_IDLE_SECS = "drill.exec.http.session_max_idle_secs";
    -  String HTTP_KEYSTORE_PATH = "drill.exec.ssl.keyStorePath";
    -  String HTTP_KEYSTORE_PASSWORD = "drill.exec.ssl.keyStorePassword";
    -  String HTTP_TRUSTSTORE_PATH = "drill.exec.ssl.trustStorePath";
    -  String HTTP_TRUSTSTORE_PASSWORD = "drill.exec.ssl.trustStorePassword";
    +  String HTTP_KEYSTORE_PATH = SSL_KEYSTORE_PATH;
    +  String HTTP_KEYSTORE_PASSWORD = SSL_KEYSTORE_PASSWORD;
    +  String HTTP_TRUSTSTORE_PATH = SSL_TRUSTSTORE_PATH;
    +  String HTTP_TRUSTSTORE_PASSWORD = SSL_TRUSTSTORE_PASSWORD;
    --- End diff --
    
    I didn't want to change HTTPs code any more than I needed to.


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on the issue:

    https://github.com/apache/drill/pull/950
  
    Thanks for the changes!
    +1 LGTM.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142822382
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    --- End diff --
    
    Well the tcp resolver uses a string and the parse method gets a string from the connect string, so why convert?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142679480
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigServer.java ---
    @@ -0,0 +1,331 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslContextBuilder;
    +import io.netty.handler.ssl.SslProvider;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.text.MessageFormat;
    +
    +public class SSLConfigServer extends SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfigServer.class);
    +
    +  private final DrillConfig config;
    +  private final Configuration hadoopConfig;
    +  private final boolean userSslEnabled;
    +  private final boolean httpsEnabled;
    +  private final String keyStoreType;
    +  private final String keyStorePath;
    +  private final String keyStorePassword;
    +  private final String keyPassword;
    +  private final String trustStoreType;
    +  private final String trustStorePath;
    +  private final String trustStorePassword;
    +  private final String protocol;
    +  private final String provider;
    +
    +  public SSLConfigServer(DrillConfig config, Configuration hadoopConfig) throws DrillException {
    +    this.config = config;
    +    SSLFactory.Mode mode = SSLFactory.Mode.SERVER;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, getMode()));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +    userSslEnabled =
    +        config.hasPath(ExecConstants.USER_SSL_ENABLED) && config.getBoolean(ExecConstants.USER_SSL_ENABLED);
    +    trustStoreType = getConfigParam(ExecConstants.SSL_TRUSTSTORE_TYPE,
    +        resolveHadoopPropertyName(HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY, mode));
    +    trustStorePath = getConfigParam(ExecConstants.SSL_TRUSTSTORE_PATH,
    +        resolveHadoopPropertyName(HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY, mode));
    +    trustStorePassword = getConfigParam(ExecConstants.SSL_TRUSTSTORE_PASSWORD,
    +        resolveHadoopPropertyName(HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY, mode));
    +    keyStoreType = getConfigParam(ExecConstants.SSL_KEYSTORE_TYPE,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY, mode));
    +    keyStorePath = getConfigParam(ExecConstants.SSL_KEYSTORE_PATH,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY, mode));
    +    keyStorePassword = getConfigParam(ExecConstants.SSL_KEYSTORE_PASSWORD,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY, mode));
    +    // if no keypassword specified, use keystore password
    +    String keyPass = getConfigParam(ExecConstants.SSL_KEY_PASSWORD,
    +        resolveHadoopPropertyName(HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY, mode));
    +    keyPassword = keyPass.isEmpty() ? keyStorePassword : keyPass;
    +    protocol = getConfigParamWithDefault(ExecConstants.SSL_PROTOCOL, DEFAULT_SSL_PROTOCOL);
    +    provider = getConfigParamWithDefault(ExecConstants.SSL_PROVIDER, DEFAULT_SSL_PROVIDER);
    +  }
    +
    +  public void validateKeyStore() throws DrillException {
    +    //HTTPS validates the keystore is not empty. User Server SSL context initialization also validates keystore, but
    +    // much more strictly. User Client context initialization does not validate keystore.
    +    /*If keystorePath or keystorePassword is provided in the configuration file use that*/
    +    if ((isUserSslEnabled() || isHttpsEnabled())) {
    +      if (!keyStorePath.isEmpty() || !keyStorePassword.isEmpty()) {
    +        if (keyStorePath.isEmpty()) {
    +          throw new DrillException(
    +              " *.ssl.keyStorePath in the configuration file is empty, but *.ssl.keyStorePassword is set");
    +        } else if (keyStorePassword.isEmpty()) {
    +          throw new DrillException(
    +              " *.ssl.keyStorePassword in the configuration file is empty, but *.ssl.keyStorePath is set ");
    +        }
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public SslContext initNettySslContext() throws DrillException {
    +    final SslContext sslCtx;
    +
    +    if (!userSslEnabled) {
    +      return null;
    +    }
    +
    +    KeyManagerFactory kmf;
    +    TrustManagerFactory tmf;
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      kmf = initializeKeyManagerFactory();
    +      tmf = initializeTrustManagerFactory();
    +      sslCtx = SslContextBuilder.forServer(kmf)
    +          .trustManager(tmf)
    +          .protocols(protocol)
    +          .sslProvider(getProvider())
    +          .build(); // Will throw an exception if the key password is not correct
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(new StringBuilder()
    +          .append("SSL is enabled but cannot be initialized - ")
    +          .append("[ ")
    +          .append(e.getMessage())
    +          .append("]. ")
    +          .toString());
    +    }
    +    this.nettySslContext = sslCtx;
    +    return sslCtx;
    +  }
    +
    +  @Override
    +  public SSLContext initJDKSSLContext() throws DrillException {
    +    final SSLContext sslCtx;
    +
    +    if (!userSslEnabled) {
    +      return null;
    +    }
    +
    +    KeyManagerFactory kmf;
    +    TrustManagerFactory tmf;
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      kmf = initializeKeyManagerFactory();
    +      tmf = initializeTrustManagerFactory();
    +      sslCtx = SSLContext.getInstance(protocol);
    +      sslCtx.init(kmf.getKeyManagers(), tmf.getTrustManagers(), null);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    +          new StringBuilder().append("SSL is enabled but cannot be initialized - ")
    +              .append("[ ")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    this.jdkSSlContext = sslCtx;
    +    return sslCtx;
    +  }
    +
    +  @Override
    +  public SSLEngine createSSLEngine(BufferAllocator allocator, String peerHost, int peerPort) {
    +    SSLEngine engine = super.createSSLEngine(allocator, peerHost, peerPort);
    +
    +    engine.setUseClientMode(false);
    +
    +    // No need for client side authentication (HTTPS like behaviour)
    +    engine.setNeedClientAuth(false);
    +
    +    try {
    +      engine.setEnableSessionCreation(true);
    +    } catch (Exception e) {
    +      // Openssl implementation may throw this.
    +      logger.debug("Session creation not enabled. Exception: {}", e.getMessage());
    +    }
    +
    +    return engine;
    +  }
    +
    +  private String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  private String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    --- End diff --
    
    Can be removed and initialized on the next line.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by superbstreak <gi...@git.apache.org>.
Github user superbstreak commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140129825
  
    --- Diff: contrib/native/client/src/include/drill/common.hpp ---
    @@ -163,9 +170,13 @@ typedef enum{
     #define USERPROP_USERNAME "userName"
     #define USERPROP_PASSWORD "password"
     #define USERPROP_SCHEMA   "schema"
    -#define USERPROP_USESSL   "useSSL"        // Not implemented yet
    -#define USERPROP_FILEPATH "pemLocation"   // Not implemented yet
    -#define USERPROP_FILENAME "pemFile"       // Not implemented yet
    +#define USERPROP_USESSL   "enableTLS"
    +#define USERPROP_TLSPROTOCOL "TLSProtocol" //TLS version
    +#define USERPROP_CERTFILEPATH "certFilePath" // pem file path and name
    +#define USERPROP_CERTPASSWORD "certPassword" // Password for certificate file
    --- End diff --
    
    I think we can remove this to avoid confusion :)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140621780
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -120,6 +125,25 @@ protected void initChannel(SocketChannel ch) throws Exception {
         // }
       }
     
    +  // Adds a SSL handler if enabled. Required only for client and server communications, so
    +  // a real implementation is only available for UserClient
    +  protected void setupSSL(ChannelPipeline pipe, ConnectionMultiListener.SSLHandshakeListener sslHandshakeListener) {
    +    // Do nothing
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141472481
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSL.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc.user.security;
    +
    +import com.typesafe.config.ConfigValueFactory;
    +import io.netty.handler.ssl.util.SelfSignedCertificate;
    +import junit.framework.TestCase;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.exec.ExecConstants;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.net.InetAddress;
    +import java.security.KeyStore;
    +import java.util.Properties;
    +
    +import static junit.framework.TestCase.fail;
    +import static org.junit.Assert.assertEquals;
    +
    +public class TestUserBitSSL extends BaseTestQuery {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(TestUserBitSSL.class);
    +
    +  private static DrillConfig newConfig;
    +  private static Properties initProps; // initial client properties
    +  private static ClassLoader classLoader;
    +  private static String ksPath;
    +  private static String tsPath;
    +  private static String emptyTSPath;
    +  private static String unknownKsPath;
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +
    +    // Create a new DrillConfig
    +    classLoader = TestUserBitSSL.class.getClassLoader();
    +    ksPath = new File(classLoader.getResource("ssl/keystore.ks").getFile()).getAbsolutePath();
    +    unknownKsPath = new File(classLoader.getResource("ssl/unknownkeystore.ks").getFile()).getAbsolutePath();
    +    tsPath = new File(classLoader.getResource("ssl/truststore.ks").getFile()).getAbsolutePath();
    +    emptyTSPath = new File(classLoader.getResource("ssl/emptytruststore.ks").getFile()).getAbsolutePath();
    +    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
    +        .withValue(ExecConstants.SSL_USE_HADOOP_CONF,
    +            ConfigValueFactory.fromAnyRef(false))
    +        .withValue(ExecConstants.USER_SSL_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.SSL_KEYSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(ksPath))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(tsPath))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_PROTOCOL,
    +            ConfigValueFactory.fromAnyRef("TLSv1.2")),
    +      false);
    +
    +    initProps = new Properties();
    +    initProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    initProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +
    +    // Start an SSL enabled cluster
    +    updateTestCluster(1, newConfig, initProps);
    +  }
    +
    +  @AfterClass
    +  public static void cleanTest() throws Exception {
    +    DrillConfig restoreConfig =
    +        new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties()), false);
    +    updateTestCluster(1, restoreConfig);
    +  }
    +
    +  @Test
    +  public void testSSLConnection() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionWithKeystore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ksPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailBadTrustStore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ""); // NO truststore
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    --- End diff --
    
    Yes. Initially this test was to check if the path to the truststore is bad. But this case fails because the default truststore does not have the server's certificate.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140602783
  
    --- Diff: distribution/src/resources/drill-override-example.conf ---
    @@ -222,7 +222,35 @@ drill.exec: {
       # Full workspace name should be indicated (including schema and workspace separated by dot).
       # Workspace MUST be file-based and writable. Workspace name is case-sensitive.
       default_temporary_workspace: "dfs.tmp"
    +
    +  # Enable and provide additional parameters for Client-Server communication over SSL
    +  # see also the javax.net.ssl parameters below
    +  security.user.encryption.ssl: {
    +    #Set this to true to enable all client server communication to occur over SSL.
    +    enabled: false,
    +    #key password is optional if it is the same as the keystore password
    +    keyPassword: "key_passwd",
    +    #Optional handshakeTimeout in milliseconds. Default is 10000 ms (10 seconds)
    +    handshakeTimeout: 10000,
    +    #protocol is optional. Drill will default to TLSv1.2
    --- End diff --
    
    sure


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141676935
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,91 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl) {
    +    HCERTSTORE hStore;
    +    PCCERT_CONTEXT pContext = NULL;
    +    X509 *x509;
    +	char* stores[] = {
    +	    "CA",
    +		"MY",
    +		"ROOT",
    +		"SPC"
    +	};
    +     
    +    SSL_CTX * ctx = SSL_get_SSL_CTX(ssl);
    +    X509_STORE *store = SSL_CTX_get_cert_store(ctx);
    +
    +	for(int i=0; i<4; i++){
    +    hStore = CertOpenSystemStore(NULL, stores[i]);
    +
    +    if (!hStore)
    +        return 1;
    --- End diff --
    
    agreed.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140395265
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java ---
    @@ -70,22 +78,80 @@
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
       private static final String SERVER_NAME = "Apache Drill Server";
     
    +  private final BootStrapContext bootStrapContext;
    +  private final BufferAllocator allocator;
    --- End diff --
    
    No need to make these member variables since they are already available via _UserConnectionConfig_ object.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142830326
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -250,7 +205,15 @@ void DrillClientImpl::doWriteToSocket(const char* dataPtr, size_t bytesToWrite,
         // Write all the bytes to socket. In case of error when all bytes are not successfully written
         // proper errorCode will be set.
         while(1) {
    -        size_t bytesWritten = m_socket.write_some(boost::asio::buffer(dataPtr, bytesToWrite), errorCode);
    +        size_t bytesWritten;
    +        {
    +            boost::lock_guard<boost::mutex> lock(m_channelMutex);
    --- End diff --
    
    Oh this was found by Rob Wu. The problem occurs when the heartbeat timer has gone off and we are in the handler which is about to send off a heartbeat. Before the heartbeat is sent if the caller deletes DrillClientImpl (via DrillClient) then the channel may be closed and the pointer to the channel may be set to null causing the heartbeat send to crash. This did not occur previously because the socket was not a pointer and/or boost was able to handle it quite nicely. 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141742427
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -65,108 +66,70 @@ struct ToRpcType: public std::unary_function<google::protobuf::int32, exec::user
     		return static_cast<exec::user::RpcType>(i);
     	}
     };
    -}
    -connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    -    std::string pathToDrill, protocol, hostPortStr;
    -    std::string host;
    -    std::string port;
    +} // anonymous
     
    -    if (this->m_bIsConnected) {
    -        if(std::strcmp(connStr, m_connectStr.c_str())){ // trying to connect to a different address is not allowed if already connected
    +connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    +        if(!std::strcmp(connStr, m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
                 return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
             return CONN_SUCCESS;
         }
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
     
    -    m_connectStr=connStr;
    -    Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
    -    if(protocol == "zk"){
    -        ZookeeperClient zook(pathToDrill);
    -        std::vector<std::string> drillbits;
    -        int err = zook.getAllDrillbits(hostPortStr, drillbits);
    -        if(!err){
    -            if (drillbits.empty()){
    -                return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    -            }
    -            Utils::shuffle(drillbits);
    -            exec::DrillbitEndpoint endpoint;
    -            err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    -            if(!err){
    -                host=boost::lexical_cast<std::string>(endpoint.address());
    -                port=boost::lexical_cast<std::string>(endpoint.user_port());
    -            }
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    -
    -        }
    -        if(err){
    -            return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    -        }
    -        zook.close();
    -        m_bIsDirectConnection=true;
    -    }else if(protocol == "local"){
    -        boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
    -        char tempStr[MAX_CONNECT_STR+1];
    -        strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
    -        host=strtok(tempStr, ":");
    -        port=strtok(NULL, "");
    -        m_bIsDirectConnection=false;
    -    }else{
    -        return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
    -    }
    -    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
    -    std::string serviceHost;
    -    for (size_t i = 0; i < props->size(); i++) {
    -        if (props->keyAt(i) == USERPROP_SERVICE_HOST) {
    -            serviceHost = props->valueAt(i);
    -        }
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, connStr);
    +    ret=m_pChannel->init(m_pChannelContext);
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    if (serviceHost.empty()) {
    -        props->setProperty(USERPROP_SERVICE_HOST, host);
    +    ret= m_pChannel->connect();
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    connectionStatus_t ret = this->connect(host.c_str(), port.c_str());
    +    props->setProperty(USERPROP_SERVICE_HOST, m_pChannel->getEndpoint()->getHost());
         return ret;
     }
     
    -connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
    -    using boost::asio::ip::tcp;
    -    tcp::endpoint endpoint;
    -    try{
    -        tcp::resolver resolver(m_io_service);
    -        tcp::resolver::query query(tcp::v4(), host, port);
    -        tcp::resolver::iterator iter = resolver.resolve(query);
    -        tcp::resolver::iterator end;
    -        while (iter != end){
    -            endpoint = *iter++;
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << endpoint << std::endl;)
    -        }
    -        boost::system::error_code ec;
    -        m_socket.connect(endpoint, ec);
    -        if(ec){
    -            return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_FAILURE, host, port, ec.message().c_str()));
    -        }
    -
    -    }catch(const std::exception & e){
    -        // Handle case when the hostname cannot be resolved. "resolve" is hard-coded in boost asio resolver.resolve
    -        if (!strcmp(e.what(), "resolve")) {
    -            return handleConnError(CONN_HOSTNAME_RESOLUTION_ERROR, getMessage(ERR_CONN_EXCEPT, e.what()));
    +connectionStatus_t DrillClientImpl::connect(const char* host, const char* port, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    --- End diff --
    
    I put in both conditions because I was lazy and didn't check if m_bIsConnected is set consistently or not. Anyway I've set it correctly now and removed the unnecessary condition. 
    Added a isConnected method to channel anyway, since it seems like it might be useful some day.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142728722
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,448 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                DRILL_LOG(LOG_INFO) << "Failed to get endpoint from zk" << std::endl;
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    --- End diff --
    
    style: not sure why `this` is used here but not in the previous test statement


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142827572
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.ssl;
    +
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig() {
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  // We need to use different SSLContext objects depending on what the user has chosen
    +  // For most uses we will use the Netty SslContext class. This allows us to use either
    +  // the JDK implementation or the OpenSSL implementation. However if the user wants to
    +  // use the system trust store, then the only way to access it is via the JDK's
    +  // SSLContext class. (See the createSSLEngine method below).
    +
    +  public abstract SslContext initNettySslContext() throws DrillException;
    +
    +  public abstract SSLContext initJDKSSLContext() throws DrillException;
    +
    +  public abstract boolean isUserSslEnabled();
    +
    +  public abstract boolean isHttpsEnabled();
    +
    +  public abstract String getKeyStoreType();
    +
    +  public abstract String getKeyStorePath();
    +
    +  public abstract String getKeyStorePassword();
    +
    +  public abstract String getKeyPassword();
    +
    +  public abstract String getTrustStoreType();
    +
    +  public abstract boolean hasTrustStorePath();
    +
    +  public abstract String getTrustStorePath();
    +
    +  public abstract boolean hasTrustStorePassword();
    +
    +  public abstract String getTrustStorePassword();
    +
    +  public abstract String getProtocol();
    +
    +  public abstract SslProvider getProvider();
    +
    +  public abstract int getHandshakeTimeout();
    +
    +  public abstract SSLFactory.Mode getMode();
    +
    +  public abstract boolean disableHostVerification();
    +
    +  public abstract boolean disableCertificateVerification();
    +
    +  public abstract boolean useSystemTrustStore();
    +
    +  public abstract boolean isSslValid();
    +
    +  public SslContext getNettySslContext() {
    +    return nettySslContext;
    +  }
    +
    +  public TrustManagerFactory initializeTrustManagerFactory() throws DrillException {
    +    TrustManagerFactory tmf;
    +    KeyStore ts = null;
    +    //Support Windows/MacOs system trust store
    +    try {
    +      String trustStoreType = getTrustStoreType();
    +      if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +        // This is valid for MS-Windows and MacOs
    +        logger.debug("Initializing System truststore.");
    +        ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +        ts.load(null, null);
    +      } else if (!getTrustStorePath().isEmpty()) {
    +          // if truststore is not provided then we will use the default. Note that the default depends on
    +          // the TrustManagerFactory that in turn depends on the Security Provider.
    +          // Use null as the truststore which will result in the default truststore being picked up
    +          logger.debug("Initializing truststore {}.", getTrustStorePath());
    +          ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +          InputStream tsStream = new FileInputStream(getTrustStorePath());
    +          ts.load(tsStream, getTrustStorePassword().toCharArray());
    +      } else {
    +        logger.debug("Initializing default truststore.");
    +      }
    +      if (disableCertificateVerification()) {
    +        tmf = InsecureTrustManagerFactory.INSTANCE;
    +      } else {
    +        tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
    +      }
    +      tmf.init(ts);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    --- End diff --
    
    Done. I personally prefer to have diagnostic messages instead of stack traces, but I included the original exception just the same.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142681811
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java ---
    @@ -49,12 +64,19 @@ public void testMissingKeystorePassword() throws Exception {
         ConfigBuilder config = new ConfigBuilder();
         config.put(ExecConstants.HTTP_KEYSTORE_PATH, "/root");
         config.put(ExecConstants.HTTP_KEYSTORE_PASSWORD, "");
    +    config.put(ExecConstants.SSL_USE_HADOOP_CONF, false);
    +    config.put(ExecConstants.USER_SSL_ENABLED, true);
         try {
    -      SSLConfig sslv = new SSLConfig(config.build());
    +      SSLConfig sslv = new SSLConfigBuilder()
    +          .config(config.build())
    +          .mode(SSLFactory.Mode.SERVER)
    +          .initializeSSLContext(false)
    +          .validateKeyStore(true)
    +          .build();
           fail();
           //Expected
         } catch (Exception e) {
    -      assertTrue(e instanceof DrillException);
    +
    --- End diff --
    
    So what is test is actually testing? Since we just fail and ignore the exception.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140590439
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSL.java ---
    @@ -0,0 +1,338 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc.user.security;
    +
    +import com.typesafe.config.ConfigValueFactory;
    +import io.netty.handler.ssl.util.SelfSignedCertificate;
    +import junit.framework.TestCase;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.exec.ExecConstants;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.net.InetAddress;
    +import java.security.KeyStore;
    +import java.util.Properties;
    +
    +import static junit.framework.TestCase.fail;
    +import static org.junit.Assert.assertEquals;
    +
    +public class TestUserBitSSL extends BaseTestQuery {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(TestUserBitSSL.class);
    +
    +  private static DrillConfig newConfig;
    +  private static Properties initProps; // initial client properties
    +  private static ClassLoader classLoader;
    +  private static String ksPath;
    +  private static String tsPath;
    +  private static String emptyTSPath;
    +  private static String unknownKsPath;
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +
    +    // Create a new DrillConfig
    +    classLoader = TestUserBitSSL.class.getClassLoader();
    +    ksPath = new File(classLoader.getResource("ssl/keystore.ks").getFile()).getAbsolutePath();
    +    unknownKsPath = new File(classLoader.getResource("ssl/unknownkeystore.ks").getFile()).getAbsolutePath();
    +    tsPath = new File(classLoader.getResource("ssl/truststore.ks").getFile()).getAbsolutePath();
    +    emptyTSPath = new File(classLoader.getResource("ssl/emptytruststore.ks").getFile()).getAbsolutePath();
    +    newConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
    +        .withValue(ExecConstants.SSL_USE_HADOOP_CONF,
    +            ConfigValueFactory.fromAnyRef(false))
    +        .withValue(ExecConstants.USER_SSL_ENABLED,
    +            ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.SSL_KEYSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(ksPath))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_TYPE,
    +            ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PATH,
    +            ConfigValueFactory.fromAnyRef(tsPath))
    +        .withValue(ExecConstants.SSL_TRUSTSTORE_PASSWORD,
    +            ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_PROTOCOL,
    +            ConfigValueFactory.fromAnyRef("TLSv1.2")),
    +      false);
    +
    +    initProps = new Properties();
    +    initProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    initProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +
    +    // Start an SSL enabled cluster
    +    updateTestCluster(1, newConfig, initProps);
    +  }
    +
    +  @AfterClass
    +  public static void cleanTest() throws Exception {
    +    DrillConfig restoreConfig =
    +        new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties()), false);
    +    updateTestCluster(1, restoreConfig);
    +  }
    +
    +  @Test
    +  public void testSSLConnection() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionWithKeystore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ksPath);
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      TestCase.fail( new StringBuilder()
    +          .append("SSL Connection failed with exception [" )
    +          .append( e.getMessage() )
    +          .append("]")
    +          .toString());
    +    }
    +  }
    +
    +  @Test
    +  public void testSSLConnectionFailBadTrustStore() throws Exception {
    +    final Properties connectionProps = new Properties();
    +    connectionProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PATH, ""); // NO truststore
    +    connectionProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    connectionProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +    boolean failureCaught = false;
    +    try {
    +      updateClient(connectionProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    --- End diff --
    
    Is this failure because the default truststore which will be used internally on client side will not be able to validate server side of test certificate ? Because providing TrustStore itself is optinal


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140611290
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigBuilder.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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.ssl;
    +
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillConfigurationException;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +
    +public class SSLConfigBuilder {
    +
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(org.apache.drill.exec.ssl.SSLConfigBuilder.class);
    +
    +  private DrillConfig config = null;
    +  private Configuration hadoopConfig = null;
    +  private SSLFactory.Mode mode = SSLFactory.Mode.SERVER;
    +  private boolean initializeSSLContext = false;
    +  private boolean validateKeyStore = false;
    +
    +  public SSLConfigBuilder() {
    +
    +  }
    +
    +  public SSLConfig build() throws DrillException {
    +    if (config == null) {
    +      throw new DrillConfigurationException(
    +          "Cannot create SSL configuration from null Drill configuration.");
    +    }
    +    SSLConfig sslConfig;
    +    if (mode == SSLFactory.Mode.SERVER) {
    +      sslConfig = new SSLConfigServer(config, hadoopConfig);
    +    } else {
    +      sslConfig = new SSLConfigClient(config, hadoopConfig);
    +    }
    +    if(initializeSSLContext){
    +      sslConfig.initContext();
    +    }
    +    if(validateKeyStore){
    +      sslConfig.validateKeyStore();
    +    }
    --- End diff --
    
    OK


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140608117
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    --- End diff --
    
    done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141676469
  
    --- Diff: contrib/native/client/src/clientlib/zkCluster.cpp ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +#include "drill/common.hpp"
    +#include <boost/thread.hpp>
    +#ifdef _WIN32
    +#include <zookeeper.h>
    +#else
    +#include <zookeeper/zookeeper.h>
    +#endif
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillClient.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "zkCluster.hpp"
    +
    +namespace Drill{
    +
    +char ZkCluster::s_drillRoot[]="/drill/";
    +char ZkCluster::s_defaultCluster[]="drillbits1";
    +
    +ZkCluster::ZkCluster(){
    +    m_pDrillbits=new String_vector;
    +    srand (time(NULL));
    +    m_bConnecting=true;
    +    memset(&m_id, 0, sizeof(m_id));
    +}
    +
    +ZkCluster::~ZkCluster(){
    +    delete m_pDrillbits;
    +}
    +
    +ZooLogLevel ZkCluster::getZkLogLevel(){
    +    //typedef enum {ZOO_LOG_LEVEL_ERROR=1,
    +    //    ZOO_LOG_LEVEL_WARN=2,
    +    //    ZOO_LOG_LEVEL_INFO=3,
    +    //    ZOO_LOG_LEVEL_DEBUG=4
    +    //} ZooLogLevel;
    +    switch(DrillClientConfig::getLogLevel()){
    +        case LOG_TRACE:
    +        case LOG_DEBUG:
    +            return ZOO_LOG_LEVEL_DEBUG;
    +        case LOG_INFO:
    +            return ZOO_LOG_LEVEL_INFO;
    +        case LOG_WARNING:
    +            return ZOO_LOG_LEVEL_WARN;
    +        case LOG_ERROR:
    +        case LOG_FATAL:
    +        default:
    +            return ZOO_LOG_LEVEL_ERROR;
    +    }
    +    return ZOO_LOG_LEVEL_ERROR;
    +}
    +
    +int ZkCluster::connectToZookeeper(const char* connectStr, const char* pathToDrill){
    +    uint32_t waitTime=30000; // 10 seconds
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142736318
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -250,7 +205,15 @@ void DrillClientImpl::doWriteToSocket(const char* dataPtr, size_t bytesToWrite,
         // Write all the bytes to socket. In case of error when all bytes are not successfully written
         // proper errorCode will be set.
         while(1) {
    -        size_t bytesWritten = m_socket.write_some(boost::asio::buffer(dataPtr, bytesToWrite), errorCode);
    +        size_t bytesWritten;
    +        {
    +            boost::lock_guard<boost::mutex> lock(m_channelMutex);
    --- End diff --
    
    it seems weird that a mutex is now required. Before, the socket was existing, why it would not be the case for the channel?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142678376
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigClient.java ---
    @@ -0,0 +1,273 @@
    +/*
    + * 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.ssl;
    +
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslContextBuilder;
    +import io.netty.handler.ssl.SslProvider;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.SSLParameters;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.util.Properties;
    +
    +public class SSLConfigClient extends SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfigClient.class);
    +
    +  Properties properties;
    --- End diff --
    
    private final?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141240324
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -65,108 +66,70 @@ struct ToRpcType: public std::unary_function<google::protobuf::int32, exec::user
     		return static_cast<exec::user::RpcType>(i);
     	}
     };
    -}
    -connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    -    std::string pathToDrill, protocol, hostPortStr;
    -    std::string host;
    -    std::string port;
    +} // anonymous
     
    -    if (this->m_bIsConnected) {
    -        if(std::strcmp(connStr, m_connectStr.c_str())){ // trying to connect to a different address is not allowed if already connected
    +connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    +        if(!std::strcmp(connStr, m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
                 return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
             return CONN_SUCCESS;
         }
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
     
    -    m_connectStr=connStr;
    -    Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
    -    if(protocol == "zk"){
    -        ZookeeperClient zook(pathToDrill);
    -        std::vector<std::string> drillbits;
    -        int err = zook.getAllDrillbits(hostPortStr, drillbits);
    -        if(!err){
    -            if (drillbits.empty()){
    -                return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    -            }
    -            Utils::shuffle(drillbits);
    -            exec::DrillbitEndpoint endpoint;
    -            err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    -            if(!err){
    -                host=boost::lexical_cast<std::string>(endpoint.address());
    -                port=boost::lexical_cast<std::string>(endpoint.user_port());
    -            }
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    -
    -        }
    -        if(err){
    -            return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    -        }
    -        zook.close();
    -        m_bIsDirectConnection=true;
    -    }else if(protocol == "local"){
    -        boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
    -        char tempStr[MAX_CONNECT_STR+1];
    -        strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
    -        host=strtok(tempStr, ":");
    -        port=strtok(NULL, "");
    -        m_bIsDirectConnection=false;
    -    }else{
    -        return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
    -    }
    -    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
    -    std::string serviceHost;
    -    for (size_t i = 0; i < props->size(); i++) {
    -        if (props->keyAt(i) == USERPROP_SERVICE_HOST) {
    -            serviceHost = props->valueAt(i);
    -        }
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, connStr);
    +    ret=m_pChannel->init(m_pChannelContext);
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    if (serviceHost.empty()) {
    -        props->setProperty(USERPROP_SERVICE_HOST, host);
    +    ret= m_pChannel->connect();
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    connectionStatus_t ret = this->connect(host.c_str(), port.c_str());
    +    props->setProperty(USERPROP_SERVICE_HOST, m_pChannel->getEndpoint()->getHost());
    --- End diff --
    
    we should set `this->m_bIsConnected = true` here once connection is successful.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142822299
  
    --- Diff: contrib/native/client/src/clientlib/CMakeLists.txt ---
    @@ -36,6 +40,7 @@ include_directories(${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/../i
     include_directories(${PROTOBUF_INCLUDE_DIR})
     include_directories(${Zookeeper_INCLUDE_DIRS})
     include_directories(${SASL_INCLUDE_DIRS})
    +include_directories("${OPENSSL_INCLUDE_DIR}")
    --- End diff --
    
    Yeah. I wonder why I did that ...


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141228991
  
    --- Diff: contrib/native/client/src/include/drill/userProperties.hpp ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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 USER_PROPERTIES_H
    +#define USER_PROPERTIES_H
    +
    +#include <map>
    +#include "drill/common.hpp"
    +
    +namespace Drill{
    +
    +class DECLSPEC_DRILL_CLIENT DrillUserProperties{
    +    public:
    +        static const std::map<std::string, uint32_t> USER_PROPERTIES;
    +
    +        DrillUserProperties(){};
    +
    +        void setProperty( const std::string& propName, const std::string& propValue){
    --- End diff --
    
    so for each `propName` we have defined if it's `string/boolean/etc`. How about checking here if `propName` is boolean then make sure to set the value in lower case? So when we retrieve those values and compare with string "true" or "false", we don't have to do the translation.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142822245
  
    --- Diff: contrib/native/client/readme.ssl ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.
    + */
    +
    +Installing OpenSSL - 
    +    On Mac: 
    +        brew install openssl
    +    On Linux :
    +
    +Set up the certificate
    +    Generate a private key
    +
    +        openssl genrsa -des3 -out drillTestServerKey.pem 1024
    +
    +    Generate Certificate signing request
    +
    +        openssl req -new -key drillTestServerKey.pem -out drillTestServer.csr
    +
    +    Sign certificate with private key
    +
    +        openssl x509 -req -days 3650 -in drillTestServer.csr -signkey drillTestServerKey.pem -out drillTestCert.pem
    +
    +    Remove password requirement (needed for example)
    +
    +        cp drillTestServerKey.pem drillTestServerKey.safe.pem
    +        openssl rsa -in drillTestServerKey.safe.pem -out drillTestServerKey.pem
    +
    +    Generate dhparam file
    +
    +        openssl dhparam -out dh512.pem 512
    --- End diff --
    
    The server side of the boost example uses this file to initialize diffie-helman key exchange. It is not needed.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142719411
  
    --- Diff: contrib/native/client/readme.ssl ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.
    + */
    +
    +Installing OpenSSL - 
    +    On Mac: 
    +        brew install openssl
    +    On Linux :
    +
    +Set up the certificate
    --- End diff --
    
    maybe should be mentionned for testing purposes only...


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140601254
  
    --- Diff: common/src/main/java/org/apache/drill/common/config/DrillConfig.java ---
    @@ -110,8 +110,8 @@ public static DrillConfig create() {
        *
        * @return {@link DrillConfig} instance
        */
    -  public static DrillConfig forClient() {
    -    return create(null, false);
    +  public static DrillConfig forClient(Properties props) {
    +    return create(null, props, false);
    --- End diff --
    
    Changed the way properties are passed in. Instead of merging into the config, now the properties will be passed in as a parameter to UserClient constructor 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142719553
  
    --- Diff: contrib/native/client/readme.ssl ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.
    + */
    +
    +Installing OpenSSL - 
    +    On Mac: 
    +        brew install openssl
    +    On Linux :
    +
    +Set up the certificate
    +    Generate a private key
    +
    +        openssl genrsa -des3 -out drillTestServerKey.pem 1024
    +
    +    Generate Certificate signing request
    +
    +        openssl req -new -key drillTestServerKey.pem -out drillTestServer.csr
    +
    +    Sign certificate with private key
    +
    +        openssl x509 -req -days 3650 -in drillTestServer.csr -signkey drillTestServerKey.pem -out drillTestCert.pem
    +
    +    Remove password requirement (needed for example)
    +
    +        cp drillTestServerKey.pem drillTestServerKey.safe.pem
    +        openssl rsa -in drillTestServerKey.safe.pem -out drillTestServerKey.pem
    +
    +    Generate dhparam file
    +
    +        openssl dhparam -out dh512.pem 512
    --- End diff --
    
    what this file is for?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141471056
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java ---
    @@ -91,10 +123,35 @@ public void testForTrustStore() throws Exception {
         ConfigBuilder config = new ConfigBuilder();
         config.put(ExecConstants.HTTP_TRUSTSTORE_PATH, "/root");
         config.put(ExecConstants.HTTP_TRUSTSTORE_PASSWORD, "root");
    -    SSLConfig sslv = new SSLConfig(config.build());
    +    config.put(ExecConstants.SSL_USE_HADOOP_CONF, false);
    +    SSLConfig sslv = new SSLConfigBuilder()
    +        .config(config.build())
    +        .mode(SSLFactory.Mode.SERVER)
    +        .initializeSSLContext(false)
    +        .validateKeyStore(true)
    +        .build();
         assertEquals(true, sslv.hasTrustStorePath());
         assertEquals(true,sslv.hasTrustStorePassword());
         assertEquals("/root",sslv.getTrustStorePath());
         assertEquals("root",sslv.getTrustStorePassword());
       }
    -}
    \ No newline at end of file
    +
    +  @Test
    +  public void testInvalidHadoopKeystore() throws Exception {
    +    Configuration hadoopConfig = new Configuration();
    +    String hadoopSSLFileProp = MessageFormat
    +        .format(HADOOP_SSL_CONF_TPL_KEY, SSLFactory.Mode.SERVER.toString().toLowerCase());
    +    hadoopConfig.set(hadoopSSLFileProp, "ssl-server-invalid.xml");
    +    ConfigBuilder config = new ConfigBuilder();
    +    config.put(ExecConstants.SSL_USE_HADOOP_CONF, true);
    +    SSLConfig sslv = new SSLConfigBuilder()
    +        .config(config.build())
    +        .mode(SSLFactory.Mode.SERVER)
    +        .initializeSSLContext(false)
    +        .validateKeyStore(true)
    +        .hadoopConfig(hadoopConfig)
    +        .build();
    +    assertEquals("FAIL", sslv.getKeyStorePassword());
    --- End diff --
    
    Yes. Buggy test. Fixed it. Previous test was passing because the SSL was not enabled in the config. This caused the validation of keystore to be skipped.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141725701
  
    --- Diff: contrib/native/client/src/clientlib/streamSocket.hpp ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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 STREAMSOCKET_HPP
    +#define STREAMSOCKET_HPP
    +
    +#include "drill/common.hpp"
    +#include "env.h"
    +#include "wincert.ipp"
    +
    +#include <boost/asio.hpp>
    +#include <boost/asio/ssl.hpp>
    +
    +namespace Drill {
    +
    +typedef boost::asio::ip::tcp::socket::lowest_layer_type streamSocket_t;
    +typedef boost::asio::ssl::stream<boost::asio::ip::tcp::socket> sslTCPSocket_t;
    +typedef boost::asio::ip::tcp::socket basicTCPSocket_t;
    +
    +
    +// Some helper typedefs to define the highly templatized boost::asio methods
    +typedef boost::asio::const_buffers_1 ConstBufferSequence; 
    +typedef boost::asio::mutable_buffers_1 MutableBufferSequence;
    +
    +// ReadHandlers have different possible signatures.
    +//
    +// As a standard C-type callback
    +//    typedef void (*ReadHandler)(const boost::system::error_code& ec, std::size_t bytes_transferred);
    +//
    +// Or as a C++ functor
    +//    struct ReadHandler {
    +//        virtual void operator()(
    +//                const boost::system::error_code& ec,
    +//                std::size_t bytes_transferred) = 0;
    +//};
    +//
    +// We need a different signature though, since we need to pass in a member function of a drill client 
    +// class (which is C++), as a functor generated by boost::bind as a ReadHandler
    +// 
    +typedef boost::function<void (const boost::system::error_code& ec, std::size_t bytes_transferred) > ReadHandler;
    +
    +class AsioStreamSocket{
    +    public:
    +        virtual ~AsioStreamSocket(){};
    +        virtual streamSocket_t& getInnerSocket() = 0;
    +
    +        virtual std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        virtual std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        //
    +        // boost::asio::async_read has the signature 
    +        // template<
    +        //     typename AsyncReadStream,
    +        //     typename MutableBufferSequence,
    +        //     typename ReadHandler>
    +        // void-or-deduced async_read(
    +        //     AsyncReadStream & s,
    +        //     const MutableBufferSequence & buffers,
    +        //     ReadHandler handler);
    +        //
    +        // For our use case, the derived class will have an instance of a concrete type for AsyncReadStream which 
    +        // will implement the requirements for the AsyncReadStream type. We need not pass that in as a parameter 
    +        // since the class already has the value
    +        // The method is templatized since the ReadHandler type is dependent on the class implementing the read 
    +        // handler (basically the class using the asio stream)
    +        //
    +        virtual void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler) = 0;
    +
    +        // call the underlying protocol's handshake method.
    +        // if the useSystemConfig flag is true, then use properties read
    +        // from the underlying operating system
    +        virtual void protocolHandshake(bool useSystemConfig) = 0;
    +        virtual void protocolClose() = 0;
    +};
    +
    +class Socket: 
    +    public AsioStreamSocket, 
    +    public basicTCPSocket_t{
    +
    +    public:
    +        Socket(boost::asio::io_service& ioService) : basicTCPSocket_t(ioService) {
    +            }
    +
    +        ~Socket(){
    +            boost::system::error_code ignorederr;
    +            this->shutdown(boost::asio::ip::tcp::socket::shutdown_both, ignorederr);
    +            this->close();
    +        };
    +
    +        basicTCPSocket_t& getSocketStream(){ return *this;}
    +
    +        streamSocket_t& getInnerSocket(){ return this->lowest_layer();}
    +
    +        std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->write_some(buffers, ec);
    +        }
    +
    +        std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->read_some(buffers, ec);
    +        }
    +
    +        void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler){
    +            return async_read(*this, buffers, handler);
    +        }
    +
    +        void protocolHandshake(bool useSystemConfig){}; //nothing to do
    +        void protocolClose(){ 
    +            // shuts down the socket!
    +            boost::system::error_code ignorederr;
    +            ((basicTCPSocket_t*)this)->shutdown(boost::asio::ip::tcp::socket::shutdown_both,
    +                ignorederr
    +                );         
    +        } 
    +};
    +
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +class SslSocket: 
    +    public AsioStreamSocket, 
    +    public sslTCPSocket_t{
    +
    +    public:
    +        SslSocket(boost::asio::io_service& ioService, boost::asio::ssl::context &sslContext) :
    +            sslTCPSocket_t(ioService, sslContext) {
    +            }
    +
    +        ~SslSocket(){};
    +
    +        sslTCPSocket_t& getSocketStream(){ return *this;}
    +
    +        streamSocket_t& getInnerSocket(){ return this->lowest_layer();}
    +
    +        std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->write_some(buffers, ec);
    +        }
    +
    +        std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->read_some(buffers, ec);
    +        }
    +
    +        void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler){
    +            return async_read(*this, buffers, handler);
    +        }
    +
    +        //
    +        // public method that can be invoked by callers to invoke the ssl handshake
    +        // throws: boost::system::system_error
    +        void protocolHandshake(bool useSystemConfig){
    +            if(useSystemConfig){
    +                if (loadSystemTrustStore(this->native_handle())) {
    +                    boost::system::error_code ec(EPROTO, boost::system::system_category());
    +                    boost::asio::detail::throw_error(ec, "Failed to load system trust store");
    +                }
    +            }
    +            this->handshake(boost::asio::ssl::stream<boost::asio::ip::tcp::socket>::client);
    +            return;
    +        };
    +        //
    +        // public method that can be invoked by callers to invoke a clean ssl shutdown
    +        // throws: boost::system::system_error
    +        void protocolClose(){
    +            try{
    +                this->shutdown();
    +            }catch(boost::system::system_error e){
    +                //swallow the exception. The channel is unusable anyway
    +            }
    +            // shuts down the socket!
    +            boost::system::error_code ignorederr;
    +            this->lowest_layer().shutdown(boost::asio::ip::tcp::socket::shutdown_both,
    +                ignorederr
    --- End diff --
    
    My mistake. Thanks for catching this.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142721373
  
    --- Diff: contrib/native/client/src/clientlib/CMakeLists.txt ---
    @@ -36,6 +40,7 @@ include_directories(${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/../i
     include_directories(${PROTOBUF_INCLUDE_DIR})
     include_directories(${Zookeeper_INCLUDE_DIRS})
     include_directories(${SASL_INCLUDE_DIRS})
    +include_directories("${OPENSSL_INCLUDE_DIR}")
    --- End diff --
    
    not sure why this variable is quoted (but others aren't)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140115903
  
    --- Diff: exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java ---
    @@ -69,20 +74,20 @@ private UnsafeDirectLittleEndian(AbstractByteBuf buf, boolean fake, AtomicLong b
         this.memoryAddress = buf.memoryAddress();
       }
     
    -  private long addr(int index) {
    -    return memoryAddress + index;
    -  }
    +    private long addr(int index) {
    +        return memoryAddress + index;
    --- End diff --
    
    Please fix indentation here and below.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140392979
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    +
    +  protected String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    +    value = hadoopConfig.get(name, "");
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getConfigParamWithDefault(String name, String defaultValue) {
    +    String value = "";
    +    if (config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    if (value.isEmpty()) {
    +      value = defaultValue;
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String resolveHadoopPropertyName(String nameTemplate, SSLFactory.Mode mode) {
    +    return MessageFormat.format(nameTemplate, mode.toString().toLowerCase());
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  public abstract SslContext initSslContext() throws DrillException;
    +
    +  public abstract SSLContext initSSLContext() throws DrillException;
    +
    +  public abstract boolean isUserSslEnabled();
    +
    +  public abstract boolean isHttpsEnabled();
    +
    +  public abstract String getKeyStoreType();
    +
    +  public abstract String getKeyStorePath();
    +
    +  public abstract String getKeyStorePassword();
    +
    +  public abstract String getKeyPassword();
    +
    +  public abstract String getTrustStoreType();
    +
    +  public abstract boolean hasTrustStorePath();
    +
    +  public abstract String getTrustStorePath();
    +
    +  public abstract boolean hasTrustStorePassword();
    +
    +  public abstract String getTrustStorePassword();
    +
    +  public abstract String getProtocol();
    +
    +  public abstract SslProvider getProvider();
    +
    +  public abstract int getHandshakeTimeout();
    +
    +  public abstract SSLFactory.Mode getMode();
    +
    +  public abstract boolean disableHostVerification();
    +
    +  public abstract boolean disableCertificateVerification();
    +
    +  public abstract boolean useSystemTrustStore();
    +
    +  public abstract boolean isSslValid();
    +
    +  public SslContext getNettySslContext() {
    +    return nettySslContext;
    +  }
    +
    +  public TrustManagerFactory initializeTrustManagerFactory() throws DrillException {
    +    TrustManagerFactory tmf;
    +    KeyStore ts = null;
    +    //Support Windows/MacOs system trust store
    +    try {
    +      String trustStoreType = getTrustStoreType();
    +      if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +        // This is valid for MS-Windows and MacOs
    +        logger.debug("Initializing System truststore.");
    +        ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +        ts.load(null, null);
    +      } else if (!getTrustStorePath().isEmpty()) {
    +          // if truststore is not provided then we will use the default. Note that the default depends on
    +          // the TrustManagerFactory that in turn depends on the Security Provider.
    +          // Use null as the truststore which will result in the default truststore being picked up
    +          logger.debug("Initializing truststore {}.", getTrustStorePath());
    +          ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +          InputStream tsStream = new FileInputStream(getTrustStorePath());
    +          ts.load(tsStream, getTrustStorePassword().toCharArray());
    +      } else {
    +        logger.debug("Initializing default truststore.");
    +      }
    +      if (disableCertificateVerification()) {
    +        tmf = InsecureTrustManagerFactory.INSTANCE;
    +      } else {
    +        tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
    +      }
    +      tmf.init(ts);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the truststore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return tmf;
    +  }
    +
    +  public KeyManagerFactory initializeKeyManagerFactory() throws DrillException {
    +    KeyManagerFactory kmf;
    +    String keyStorePath = getKeyStorePath();
    +    String keyStorePassword = getKeyStorePassword();
    +    String keyStoreType = getKeyStoreType();
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      KeyStore ks =
    +          KeyStore.getInstance(!keyStoreType.isEmpty() ? keyStoreType : KeyStore.getDefaultType());
    +      //initialize the key manager factory
    +      // Will throw an exception if the file is not found/accessible.
    +      InputStream ksStream = new FileInputStream(keyStorePath);
    +      // A key password CANNOT be null or an empty string.
    +      if (keyStorePassword.isEmpty()) {
    +        throw new DrillException("The Keystore password cannot be empty.");
    +      }
    +      ks.load(ksStream, keyStorePassword.toCharArray());
    +      // Empty Keystore. (Remarkably, it is possible to do this).
    +      if (ks.size() == 0) {
    +        throw new DrillException("The Keystore has no entries.");
    +      }
    +      kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
    +      kmf.init(ks, getKeyPassword().toCharArray());
    +
    +    } catch (Exception e) {
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the keystore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return kmf;
    +  }
    +
    +  public void initContext() throws DrillException {
    +    if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +      initSSLContext();
    +      logger.debug("Initialized Windows SSL context using JDK.");
    --- End diff --
    
    Please correct the logging message: This can be _Windows/Mac SSL_ Context based on the condition. ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140382333
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigBuilder.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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.ssl;
    +
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillConfigurationException;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +
    +public class SSLConfigBuilder {
    +
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(org.apache.drill.exec.ssl.SSLConfigBuilder.class);
    +
    +  private DrillConfig config = null;
    +  private Configuration hadoopConfig = null;
    +  private SSLFactory.Mode mode = SSLFactory.Mode.SERVER;
    +  private boolean initializeSSLContext = false;
    +  private boolean validateKeyStore = false;
    +
    +  public SSLConfigBuilder() {
    +
    +  }
    +
    +  public SSLConfig build() throws DrillException {
    +    if (config == null) {
    +      throw new DrillConfigurationException(
    +          "Cannot create SSL configuration from null Drill configuration.");
    +    }
    +    SSLConfig sslConfig;
    +    if (mode == SSLFactory.Mode.SERVER) {
    +      sslConfig = new SSLConfigServer(config, hadoopConfig);
    +    } else {
    +      sslConfig = new SSLConfigClient(config, hadoopConfig);
    +    }
    +    if(initializeSSLContext){
    +      sslConfig.initContext();
    +    }
    +    if(validateKeyStore){
    +      sslConfig.validateKeyStore();
    +    }
    --- End diff --
    
    _validateKeyStore_ should happen before _initContext_.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140621635
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClient.java ---
    @@ -179,6 +203,13 @@ void send(RpcOutcomeListener<RECEIVE> listener, T rpcType, SEND protobufBody,
         return super.send(connection, rpcType, protobufBody, clazz, dataBodies);
       }
     
    +  public <SEND extends MessageLite, RECEIVE extends MessageLite> void send(
    +      RpcOutcomeListener<RECEIVE> listener, SEND protobufBody, boolean allowInEventLoop,
    +      ByteBuf... dataBodies) {
    +    super.send(listener, connection, handshakeType, protobufBody, (Class<RECEIVE>) responseClass,
    +        allowInEventLoop, dataBodies);
    --- End diff --
    
    Seeing unchecked warning for this function. Please resolve this.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141230974
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,452 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(.*):([0-9]+)(?:/(.+))?");
    +    boost::cmatch matched;
    +
    +    if(boost::regex_match(m_connectString.c_str(), matched, connStrExpr)){
    +        m_protocol.assign(matched[1].first, matched[1].second);
    +        std::string host, port;
    +        host.assign(matched[2].first, matched[2].second);
    +        port.assign(matched[3].first, matched[3].second);
    +        if(isDirectConnection()){
    +            // if the connection is to a zookeeper, 
    +            // we will get the host and the port only after connecting to the Zookeeper
    +            m_host=host;
    +            m_port=port;
    +        }
    +        m_hostPortStr=host+std::string(":")+port;
    +        std::string pathToDrill;
    +        if(matched.size()==5){
    +            pathToDrill.assign(matched[4].first, matched[4].second);
    +            if(!pathToDrill.empty()){
    +                m_pathToDrill=std::string("/")+pathToDrill;
    +            }
    +        }
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) 
    +                << "Conn str: "<< m_connectString 
    +                << ";  protocol: " << m_protocol 
    +                << ";  host: " << host 
    +                << "; port: " << port 
    +                << ";  path to drill: " << m_pathToDrill 
    +                << std::endl;)
    +    } else {
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Invalid connect string. Regexp did not match" << std::endl;)
    +    }
    +
    +    return;
    +}
    +
    +bool ConnectionEndpoint::isDirectConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "local") || !strcmp(m_protocol.c_str(), "drillbit"));
    +}
    +
    +bool ConnectionEndpoint::isZookeeperConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "zk"));
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpointFromZk(){
    +    ZookeeperClient zook(m_pathToDrill);
    +    assert(!m_hostPortStr.empty());
    +    std::vector<std::string> drillbits;
    +    if(zook.getAllDrillbits(m_hostPortStr.c_str(), drillbits)!=0){
    +        return handleError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    +    }
    +    if (drillbits.empty()){
    +        return handleError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    +    }
    +    Utils::shuffle(drillbits);
    +    exec::DrillbitEndpoint endpoint;
    +    int err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    +    if(!err){
    +        m_host=boost::lexical_cast<std::string>(endpoint.address());
    +        m_port=boost::lexical_cast<std::string>(endpoint.user_port());
    +    }
    +    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    +        zook.close();
    +    return CONN_SUCCESS;
    +}
    +
    +connectionStatus_t ConnectionEndpoint::handleError(connectionStatus_t status, std::string msg){
    +    DrillClientError* pErr = new DrillClientError(status, DrillClientError::CONN_ERROR_START+status, msg);
    +    if(m_pError!=NULL){ delete m_pError; m_pError=NULL;}
    +    m_pError=pErr;
    +    return status;
    +}
    +
    +/****************************
    + * Channel Context Factory
    + ****************************/
    +ChannelContext* ChannelContextFactory::getChannelContext(channelType_t t, DrillUserProperties* props){
    +    ChannelContext* pChannelContext=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannelContext=new ChannelContext(props);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM: {
    +
    +            std::string protocol;
    +            props->getProp(USERPROP_TLSPROTOCOL, protocol);
    +            boost::asio::ssl::context::method tlsVersion = SSLChannelContext::getTlsVersion(protocol);
    +
    +            std::string noVerifyCert;
    +            props->getProp(USERPROP_DISABLE_CERTVERIFICATION, noVerifyCert);
    +            boost::asio::ssl::context::verify_mode verifyMode = boost::asio::ssl::context::verify_peer;
    +            if (noVerifyCert == "true") {
    +                verifyMode = boost::asio::ssl::context::verify_none;
    +            }
    +
    +            pChannelContext = new SSLChannelContext(props, tlsVersion, verifyMode);
    +        }
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannelContext;
    +} 
    +
    +/*******************
    + *  ChannelFactory
    + * *****************/
    +Channel* ChannelFactory::getChannel(channelType_t t, const char* connStr){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(connStr);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(connStr);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +Channel* ChannelFactory::getChannel(channelType_t t, const char* host, const char* port){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(host, port);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(host, port);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +Channel* ChannelFactory::getChannel(channelType_t t, boost::asio::io_service& ioService, const char* connStr){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(ioService, connStr);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(ioService, connStr);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +Channel* ChannelFactory::getChannel(channelType_t t, boost::asio::io_service& ioService, const char* host, const char* port){
    +    Channel* pChannel=NULL;
    +    switch(t){
    +        case CHANNEL_TYPE_SOCKET:
    +            pChannel=new SocketChannel(ioService, host, port);
    +            break;
    +#if defined(IS_SSL_ENABLED)
    +        case CHANNEL_TYPE_SSLSTREAM:
    +            pChannel=new SSLStreamChannel(ioService, host, port);
    +            break;
    +#endif
    +        default:
    +            DRILL_LOG(LOG_ERROR) << "Channel type " << t << " is not supported." << std::endl;
    +            break;
    +    }
    +    return pChannel;
    +}
    +
    +/*******************
    + *  Channel
    + * *****************/
    +
    +Channel::Channel(const char* connStr) : m_ioService(m_ioServiceFallback){
    +    m_pEndpoint=new ConnectionEndpoint(connStr);
    +    m_ownIoService = true;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::Channel(const char* host, const char* port) : m_ioService(m_ioServiceFallback){
    +    m_pEndpoint=new ConnectionEndpoint(host, port);
    +    m_ownIoService = true;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::Channel(boost::asio::io_service& ioService, const char* connStr):m_ioService(ioService){
    +    m_pEndpoint=new ConnectionEndpoint(connStr);
    +    m_ownIoService = false;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::Channel(boost::asio::io_service& ioService, const char* host, const char* port) : m_ioService(ioService){
    +    m_pEndpoint=new ConnectionEndpoint(host, port);
    +    m_ownIoService = true;
    +    m_pSocket=NULL;
    +    m_state=CHANNEL_UNINITIALIZED;
    +    m_pError=NULL;
    +}
    +
    +Channel::~Channel(){
    +    if(m_pEndpoint!=NULL){
    +        delete m_pEndpoint; m_pEndpoint=NULL;
    +    }
    +    if(m_pSocket!=NULL){
    +        delete m_pSocket; m_pSocket=NULL;
    +    }
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +template <typename SettableSocketOption> void Channel::setOption(SettableSocketOption& option){
    +    //May be useful some day. 
    +    //At the moment, we only need to set some well known options after we connect.
    +    assert(0); 
    +}
    +
    +connectionStatus_t Channel::init(ChannelContext_t* pContext){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    this->m_state=CHANNEL_INITIALIZED;
    +    this->m_pContext = pContext;
    +    return ret;
    +}
    +
    +connectionStatus_t Channel::connect(){
    +    connectionStatus_t ret=CONN_FAILURE;
    +    if(this->m_state==CHANNEL_INITIALIZED){
    +        ret=m_pEndpoint->getDrillbitEndpoint();
    +        if(ret==CONN_SUCCESS){
    +            DRILL_LOG(LOG_TRACE) << "Connecting to drillbit: " 
    +                << m_pEndpoint->getHost() 
    +                << ":" << m_pEndpoint->getPort() 
    +                << "." << std::endl;
    +            ret=this->connectInternal();
    +        }else{
    +            handleError(ret, m_pEndpoint->getError()->msg);
    +        }
    +    }
    +    this->m_state=(ret==CONN_SUCCESS)?CHANNEL_CONNECTED:this->m_state;
    +    return ret;
    +}
    +
    +connectionStatus_t Channel::handleError(connectionStatus_t status, std::string msg){
    +    DrillClientError* pErr = new DrillClientError(status, DrillClientError::CONN_ERROR_START+status, msg);
    +    if(m_pError!=NULL){ delete m_pError; m_pError=NULL;}
    +    m_pError=pErr;
    +    return status;
    +}
    +
    +connectionStatus_t Channel::connectInternal() {
    +    using boost::asio::ip::tcp;
    +    tcp::endpoint endpoint;
    +    const char *host = m_pEndpoint->getHost().c_str();
    +    const char *port = m_pEndpoint->getPort().c_str();
    +    try {
    +        tcp::resolver resolver(m_ioService);
    +        tcp::resolver::query query(tcp::v4(), host, port);
    +        tcp::resolver::iterator iter = resolver.resolve(query);
    +        tcp::resolver::iterator end;
    +        while (iter != end) {
    +            endpoint = *iter++;
    +            DRILL_LOG(LOG_TRACE) << endpoint << std::endl;
    +        }
    +        boost::system::error_code ec;
    +        m_pSocket->getInnerSocket().connect(endpoint, ec);
    +        if (ec) {
    +            return handleError(CONN_FAILURE, getMessage(ERR_CONN_FAILURE, host, port, ec.message().c_str()));
    +        }
    +    } catch (std::exception e) {
    +        // Handle case when the hostname cannot be resolved. "resolve" is hard-coded in boost asio resolver.resolve
    +        if (!strcmp(e.what(), "resolve")) {
    +            return handleError(CONN_HOSTNAME_RESOLUTION_ERROR, getMessage(ERR_CONN_EXCEPT, e.what()));
    +        }
    +        return handleError(CONN_FAILURE, getMessage(ERR_CONN_EXCEPT, e.what()));
    +    }
    +
    +    // set socket keep alive
    +    boost::asio::socket_base::keep_alive keepAlive(true);
    +    m_pSocket->getInnerSocket().set_option(keepAlive);
    +    // set no_delay
    +    boost::asio::ip::tcp::no_delay noDelay(true);
    +    m_pSocket->getInnerSocket().set_option(noDelay);
    +    // set reuse addr
    +    boost::asio::socket_base::reuse_address reuseAddr(true);
    +    m_pSocket->getInnerSocket().set_option(reuseAddr);
    +
    +    std::string useSystemTrustStore;
    +    m_pContext->getUserProperties()->getProp(USERPROP_USESYSTEMTRUSTSTORE, useSystemTrustStore);
    +
    +    return this->protocolHandshake(useSystemTrustStore=="true");
    +
    +}
    +
    +connectionStatus_t SocketChannel::init(ChannelContext_t* pContext){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    m_pSocket=new Socket(m_ioService);
    +    if(m_pSocket!=NULL){
    +        ret=Channel::init(pContext);
    +    }else{
    +        DRILL_LOG(LOG_ERROR) << "Channel initialization failure. " << std::endl;
    +        handleError(CONN_NOSOCKET, getMessage(ERR_CONN_NOSOCKET));
    +        ret=CONN_FAILURE;
    +    }
    +    return ret;
    +}
    +
    +#if defined(IS_SSL_ENABLED)
    +connectionStatus_t SSLStreamChannel::init(ChannelContext_t* pContext){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +
    +    const DrillUserProperties* props = pContext->getUserProperties();
    +	std::string useSystemTrustStore;
    +	props->getProp(USERPROP_USESYSTEMTRUSTSTORE, useSystemTrustStore);
    +	if (useSystemTrustStore != "true"){
    +		std::string certFile;
    +		props->getProp(USERPROP_CERTFILEPATH, certFile);
    +		try{
    +			((SSLChannelContext_t*)pContext)->getSslContext().load_verify_file(certFile);
    +		}
    +		catch (boost::system::system_error e){
    +			DRILL_LOG(LOG_ERROR) << "Channel initialization failure. Certificate file  "
    +				<< certFile
    +				<< " could not be loaded."
    +				<< std::endl;
    +			handleError(CONN_SSLERROR, getMessage(ERR_CONN_SSLCERTFAIL, certFile.c_str(), e.what()));
    +			ret = CONN_FAILURE;
    +		}
    +	}
    +
    +    std::string disableHostVerification;
    +    props->getProp(USERPROP_DISABLE_HOSTVERIFICATION, disableHostVerification);
    +    if (disableHostVerification != "true") {
    +        std::string hostPortStr = m_pEndpoint->getHost() + ":" + m_pEndpoint->getPort();
    +        ((SSLChannelContext_t *) pContext)->getSslContext().set_verify_callback(
    +                boost::asio::ssl::rfc2818_verification(hostPortStr.c_str()));
    +    }
    +
    +    std::string disableCertificateVerification;
    +    props->getProp(USERPROP_DISABLE_CERTVERIFICATION, disableCertificateVerification);
    +    if (disableCertificateVerification == "true") {
    +        ((SSLChannelContext_t *) pContext)->getSslContext().set_verify_mode(boost::asio::ssl::context::verify_none);
    +    }
    --- End diff --
    
    setting the `verifyMode` is duplicated inside `getChannelContext(..)` call where we create a `SSLChannelContext` object. How about moving all these setup of sslContext inside `SSLChannelContext` constructor ? We have access to props object there.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140593112
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSLServer.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc.user.security;
    +
    +import com.typesafe.config.ConfigValueFactory;
    +import junit.framework.TestCase;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.text.MessageFormat;
    +import java.util.Properties;
    +
    +import static org.apache.drill.exec.ssl.SSLConfig.HADOOP_SSL_CONF_TPL_KEY;
    +import static org.junit.Assert.assertEquals;
    +
    +public class TestUserBitSSLServer extends BaseTestQuery {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(TestUserBitSSLServer.class);
    +
    +  private static DrillConfig sslConfig;
    +  private static Properties initProps; // initial client properties
    +  private static ClassLoader classLoader;
    +  private static String ksPath;
    +  private static String tsPath;
    +  private static String emptyTSPath;
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +
    +    classLoader = TestUserBitSSLServer.class.getClassLoader();
    +    ksPath = new File(classLoader.getResource("ssl/keystore.ks").getFile()).getAbsolutePath();
    +    tsPath = new File(classLoader.getResource("ssl/truststore.ks").getFile()).getAbsolutePath();
    +    emptyTSPath = new File(classLoader.getResource("ssl/emptytruststore.ks").getFile()).getAbsolutePath();
    +    sslConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
    +        .withValue(ExecConstants.USER_SSL_ENABLED, ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.SSL_KEYSTORE_TYPE, ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH, ConfigValueFactory.fromAnyRef(ksPath))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD, ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD, ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_PROTOCOL, ConfigValueFactory.fromAnyRef("TLSv1.2")), false);
    +    initProps = new Properties();
    +    initProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    initProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +  }
    +
    +  @AfterClass
    +  public static void cleanTest() throws Exception {
    +    DrillConfig restoreConfig =
    +        new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties()), false);
    +    updateTestCluster(1, restoreConfig);
    +  }
    +
    +  @Test
    +  public void testInvalidKeystorePath() throws Exception {
    +    DrillConfig testConfig = new DrillConfig(DrillConfig.create(sslConfig)
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH, ConfigValueFactory.fromAnyRef("/bad/path")),
    +        false);
    +
    +    // Start an SSL enabled cluster
    +    boolean failureCaught = false;
    +    try {
    +      updateTestCluster(1, testConfig, initProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testInvalidKeystorePassword() throws Exception {
    +    DrillConfig testConfig = new DrillConfig(DrillConfig.create(sslConfig)
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD, ConfigValueFactory.fromAnyRef("badpassword")),
    +        false);
    +
    +    // Start an SSL enabled cluster
    +    boolean failureCaught = false;
    +    try {
    +      updateTestCluster(1, testConfig, initProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testInvalidKeyPassword() throws Exception {
    +    DrillConfig testConfig = new DrillConfig(DrillConfig.create(sslConfig)
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD, ConfigValueFactory.fromAnyRef("badpassword")),
    +        false);
    +
    +    // Start an SSL enabled cluster
    +    boolean failureCaught = false;
    +    try {
    +      updateTestCluster(1, testConfig, initProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    --- End diff --
    
    How about adding a test for default keyPasword being same as keyStorePassword ? Your configs are already generated with that :)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140401069
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java ---
    @@ -203,6 +226,9 @@ public void close() throws IOException {
           if (elapsed > 500) {
             logger.info("closed eventLoopGroup " + eventLoopGroup + " in " + elapsed + " ms");
           }
    +      if(isSslEnabled()) {
    --- End diff --
    
    `if(isSslEnabled())` check not required here and in `BasicClient` since `closeSSL` is taking care of it.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141245863
  
    --- Diff: contrib/native/client/src/clientlib/drillClientImpl.cpp ---
    @@ -65,108 +66,70 @@ struct ToRpcType: public std::unary_function<google::protobuf::int32, exec::user
     		return static_cast<exec::user::RpcType>(i);
     	}
     };
    -}
    -connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    -    std::string pathToDrill, protocol, hostPortStr;
    -    std::string host;
    -    std::string port;
    +} // anonymous
     
    -    if (this->m_bIsConnected) {
    -        if(std::strcmp(connStr, m_connectStr.c_str())){ // trying to connect to a different address is not allowed if already connected
    +connectionStatus_t DrillClientImpl::connect(const char* connStr, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    +        if(!std::strcmp(connStr, m_connectStr.c_str())){
    +            // trying to connect to a different address is not allowed if already connected
                 return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
             }
             return CONN_SUCCESS;
         }
    +    std::string val;
    +    channelType_t type = ( props->isPropSet(USERPROP_USESSL) &&
    +            props->getProp(USERPROP_USESSL, val) =="true") ?
    +        CHANNEL_TYPE_SSLSTREAM :
    +        CHANNEL_TYPE_SOCKET;
     
    -    m_connectStr=connStr;
    -    Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
    -    if(protocol == "zk"){
    -        ZookeeperClient zook(pathToDrill);
    -        std::vector<std::string> drillbits;
    -        int err = zook.getAllDrillbits(hostPortStr, drillbits);
    -        if(!err){
    -            if (drillbits.empty()){
    -                return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    -            }
    -            Utils::shuffle(drillbits);
    -            exec::DrillbitEndpoint endpoint;
    -            err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    -            if(!err){
    -                host=boost::lexical_cast<std::string>(endpoint.address());
    -                port=boost::lexical_cast<std::string>(endpoint.user_port());
    -            }
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    -
    -        }
    -        if(err){
    -            return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    -        }
    -        zook.close();
    -        m_bIsDirectConnection=true;
    -    }else if(protocol == "local"){
    -        boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
    -        char tempStr[MAX_CONNECT_STR+1];
    -        strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
    -        host=strtok(tempStr, ":");
    -        port=strtok(NULL, "");
    -        m_bIsDirectConnection=false;
    -    }else{
    -        return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
    -    }
    -    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
    -    std::string serviceHost;
    -    for (size_t i = 0; i < props->size(); i++) {
    -        if (props->keyAt(i) == USERPROP_SERVICE_HOST) {
    -            serviceHost = props->valueAt(i);
    -        }
    +    connectionStatus_t ret = CONN_SUCCESS;
    +    m_pChannelContext = ChannelContextFactory::getChannelContext(type, props);
    +    m_pChannel= ChannelFactory::getChannel(type, m_io_service, connStr);
    +    ret=m_pChannel->init(m_pChannelContext);
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    if (serviceHost.empty()) {
    -        props->setProperty(USERPROP_SERVICE_HOST, host);
    +    ret= m_pChannel->connect();
    +    if(ret!=CONN_SUCCESS){
    +        handleConnError(m_pChannel->getError());
    +        return ret;
         }
    -    connectionStatus_t ret = this->connect(host.c_str(), port.c_str());
    +    props->setProperty(USERPROP_SERVICE_HOST, m_pChannel->getEndpoint()->getHost());
         return ret;
     }
     
    -connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
    -    using boost::asio::ip::tcp;
    -    tcp::endpoint endpoint;
    -    try{
    -        tcp::resolver resolver(m_io_service);
    -        tcp::resolver::query query(tcp::v4(), host, port);
    -        tcp::resolver::iterator iter = resolver.resolve(query);
    -        tcp::resolver::iterator end;
    -        while (iter != end){
    -            endpoint = *iter++;
    -            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << endpoint << std::endl;)
    -        }
    -        boost::system::error_code ec;
    -        m_socket.connect(endpoint, ec);
    -        if(ec){
    -            return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_FAILURE, host, port, ec.message().c_str()));
    -        }
    -
    -    }catch(const std::exception & e){
    -        // Handle case when the hostname cannot be resolved. "resolve" is hard-coded in boost asio resolver.resolve
    -        if (!strcmp(e.what(), "resolve")) {
    -            return handleConnError(CONN_HOSTNAME_RESOLUTION_ERROR, getMessage(ERR_CONN_EXCEPT, e.what()));
    +connectionStatus_t DrillClientImpl::connect(const char* host, const char* port, DrillUserProperties* props){
    +    if (this->m_bIsConnected || (this->m_pChannelContext!=NULL && this->m_pChannel!=NULL)) {
    --- End diff --
    
    I think the second condition is not sufficient to say if there is already a valid connection. Instead there should be `isConnected()` method in channel which will `return m_state == CHANNEL_CONNECTED`
    
    Also why we need both `this->m_bIsConnected` and second condition ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141725904
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,452 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(.*):([0-9]+)(?:/(.+))?");
    --- End diff --
    
    Updated the regexp.



---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142822133
  
    --- Diff: contrib/native/client/readme.ssl ---
    @@ -0,0 +1,58 @@
    +/*
    + * 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.
    + */
    +
    +Installing OpenSSL - 
    +    On Mac: 
    +        brew install openssl
    +    On Linux :
    +
    +Set up the certificate
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140601194
  
    --- Diff: contrib/native/client/src/include/drill/common.hpp ---
    @@ -163,9 +170,13 @@ typedef enum{
     #define USERPROP_USERNAME "userName"
     #define USERPROP_PASSWORD "password"
     #define USERPROP_SCHEMA   "schema"
    -#define USERPROP_USESSL   "useSSL"        // Not implemented yet
    -#define USERPROP_FILEPATH "pemLocation"   // Not implemented yet
    -#define USERPROP_FILENAME "pemFile"       // Not implemented yet
    +#define USERPROP_USESSL   "enableTLS"
    +#define USERPROP_TLSPROTOCOL "TLSProtocol" //TLS version
    +#define USERPROP_CERTFILEPATH "certFilePath" // pem file path and name
    +#define USERPROP_CERTPASSWORD "certPassword" // Password for certificate file
    --- End diff --
    
    Commented it out. I'm leaving it there so it can be used when we implement support for trust stores which may require passwords. 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140400536
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java ---
    @@ -82,6 +83,9 @@ protected void initChannel(SocketChannel ch) throws Exception {
                 ch.closeFuture().addListener(getCloseHandler(ch, connection));
     
                 final ChannelPipeline pipe = ch.pipeline();
    +            // Make sure that the SSL handler is the first handler in the pipeline so everything is encrypted
    +            setupSSL(pipe);
    --- End diff --
    
    how about checking `isSslEnabled()` and then calling `setupSSL(..)` here and in `BasicClient` ? instead of checking it inside the `setupSSL` function. Since cases when SSL is disabled that will avoid the function call.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142824494
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    --- End diff --
    
    I made it a const reference (or were you suggesting std::cref?)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140356738
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    --- End diff --
    
    _httpsEnabled_ and _hadoopConfig_ both are Server side property and can be moved to SSLConfigServer. And all the logic here in SSLConfig constructor can also be moved to SSLConfigServer side.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142827342
  
    --- Diff: distribution/pom.xml ---
    @@ -97,8 +97,41 @@
         <dependency>
           <groupId>org.apache.hbase</groupId>
           <artifactId>hbase-client</artifactId>
    +      <exclusions>
    +       <exclusion>
    +           <groupId>io.netty</groupId>
    +           <artifactId>netty</artifactId>
    +       </exclusion>
    +      </exclusions>
    +    </dependency>
    +    <dependency>
    --- End diff --
    
    I did it this way because the assembly plugin includes jars with test scope and the parent pom needs to match what's in the assembly plugin. My maven isn't fluent enough to explain why that should be so, or why we ask the assembly plugin to package the jars in test scope. If you know how/why I can/should change this, I'll be happy to do so (probably as a separate JIRA).


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140612241
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigClient.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.ssl;
    +
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslContextBuilder;
    +import io.netty.handler.ssl.SslProvider;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.TrustManagerFactory;
    +
    +public class SSLConfigClient extends SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfigClient.class);
    +
    +  private final SSLFactory.Mode mode; // Let's reuse Hadoop's SSLFactory.Mode to distinguish client/server
    --- End diff --
    
    Removed it altogether. It's used only by the server. 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140973654
  
    --- Diff: contrib/native/client/src/clientlib/streamSocket.hpp ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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 STREAMSOCKET_HPP
    +#define STREAMSOCKET_HPP
    +
    +#include "drill/common.hpp"
    +#include "env.h"
    +#include "wincert.ipp"
    +
    +#include <boost/asio.hpp>
    +#include <boost/asio/ssl.hpp>
    +
    +namespace Drill {
    +
    +typedef boost::asio::ip::tcp::socket::lowest_layer_type streamSocket_t;
    +typedef boost::asio::ssl::stream<boost::asio::ip::tcp::socket> sslTCPSocket_t;
    +typedef boost::asio::ip::tcp::socket basicTCPSocket_t;
    +
    +
    +// Some helper typedefs to define the highly templatized boost::asio methods
    +typedef boost::asio::const_buffers_1 ConstBufferSequence; 
    +typedef boost::asio::mutable_buffers_1 MutableBufferSequence;
    +
    +// ReadHandlers have different possible signatures.
    +//
    +// As a standard C-type callback
    +//    typedef void (*ReadHandler)(const boost::system::error_code& ec, std::size_t bytes_transferred);
    +//
    +// Or as a C++ functor
    +//    struct ReadHandler {
    +//        virtual void operator()(
    +//                const boost::system::error_code& ec,
    +//                std::size_t bytes_transferred) = 0;
    +//};
    +//
    +// We need a different signature though, since we need to pass in a member function of a drill client 
    +// class (which is C++), as a functor generated by boost::bind as a ReadHandler
    +// 
    +typedef boost::function<void (const boost::system::error_code& ec, std::size_t bytes_transferred) > ReadHandler;
    +
    +class AsioStreamSocket{
    +    public:
    +        virtual ~AsioStreamSocket(){};
    +        virtual streamSocket_t& getInnerSocket() = 0;
    +
    +        virtual std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        virtual std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        //
    +        // boost::asio::async_read has the signature 
    +        // template<
    +        //     typename AsyncReadStream,
    +        //     typename MutableBufferSequence,
    +        //     typename ReadHandler>
    +        // void-or-deduced async_read(
    +        //     AsyncReadStream & s,
    +        //     const MutableBufferSequence & buffers,
    +        //     ReadHandler handler);
    +        //
    +        // For our use case, the derived class will have an instance of a concrete type for AsyncReadStream which 
    +        // will implement the requirements for the AsyncReadStream type. We need not pass that in as a parameter 
    +        // since the class already has the value
    +        // The method is templatized since the ReadHandler type is dependent on the class implementing the read 
    +        // handler (basically the class using the asio stream)
    +        //
    +        virtual void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler) = 0;
    +
    +        // call the underlying protocol's handshake method.
    +        // if the useSystemConfig flag is true, then use properties read
    +        // from the underlying operating system
    +        virtual void protocolHandshake(bool useSystemConfig) = 0;
    +        virtual void protocolClose() = 0;
    +};
    +
    +class Socket: 
    +    public AsioStreamSocket, 
    +    public basicTCPSocket_t{
    +
    +    public:
    +        Socket(boost::asio::io_service& ioService) : basicTCPSocket_t(ioService) {
    +            }
    +
    +        ~Socket(){
    +            boost::system::error_code ignorederr;
    +            this->shutdown(boost::asio::ip::tcp::socket::shutdown_both, ignorederr);
    +            this->close();
    +        };
    +
    +        basicTCPSocket_t& getSocketStream(){ return *this;}
    +
    +        streamSocket_t& getInnerSocket(){ return this->lowest_layer();}
    +
    +        std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->write_some(buffers, ec);
    +        }
    +
    +        std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->read_some(buffers, ec);
    +        }
    +
    +        void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler){
    +            return async_read(*this, buffers, handler);
    +        }
    +
    +        void protocolHandshake(bool useSystemConfig){}; //nothing to do
    +        void protocolClose(){ 
    +            // shuts down the socket!
    +            boost::system::error_code ignorederr;
    +            ((basicTCPSocket_t*)this)->shutdown(boost::asio::ip::tcp::socket::shutdown_both,
    +                ignorederr
    +                );         
    +        } 
    --- End diff --
    
    Why not close the socket here too instead of in destructor only ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142674202
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,265 @@
    +/*
    + * 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.ssl;
    +
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig() {
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  // We need to use different SSLContext objects depending on what the user has chosen
    +  // For most uses we will use the Netty SslContext class. This allows us to use either
    +  // the JDK implementation or the OpenSSL implementation. However if the user wants to
    +  // use the system trust store, then the only way to access it is via the JDK's
    +  // SSLContext class. (See the createSSLEngine method below).
    +
    +  public abstract SslContext initNettySslContext() throws DrillException;
    +
    +  public abstract SSLContext initJDKSSLContext() throws DrillException;
    +
    +  public abstract boolean isUserSslEnabled();
    +
    +  public abstract boolean isHttpsEnabled();
    +
    +  public abstract String getKeyStoreType();
    +
    +  public abstract String getKeyStorePath();
    +
    +  public abstract String getKeyStorePassword();
    +
    +  public abstract String getKeyPassword();
    +
    +  public abstract String getTrustStoreType();
    +
    +  public abstract boolean hasTrustStorePath();
    +
    +  public abstract String getTrustStorePath();
    +
    +  public abstract boolean hasTrustStorePassword();
    +
    +  public abstract String getTrustStorePassword();
    +
    +  public abstract String getProtocol();
    +
    +  public abstract SslProvider getProvider();
    +
    +  public abstract int getHandshakeTimeout();
    +
    +  public abstract SSLFactory.Mode getMode();
    +
    +  public abstract boolean disableHostVerification();
    +
    +  public abstract boolean disableCertificateVerification();
    +
    +  public abstract boolean useSystemTrustStore();
    +
    +  public abstract boolean isSslValid();
    +
    +  public SslContext getNettySslContext() {
    +    return nettySslContext;
    +  }
    +
    +  public TrustManagerFactory initializeTrustManagerFactory() throws DrillException {
    +    TrustManagerFactory tmf;
    +    KeyStore ts = null;
    +    //Support Windows/MacOs system trust store
    +    try {
    +      String trustStoreType = getTrustStoreType();
    +      if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +        // This is valid for MS-Windows and MacOs
    +        logger.debug("Initializing System truststore.");
    +        ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +        ts.load(null, null);
    +      } else if (!getTrustStorePath().isEmpty()) {
    +          // if truststore is not provided then we will use the default. Note that the default depends on
    +          // the TrustManagerFactory that in turn depends on the Security Provider.
    +          // Use null as the truststore which will result in the default truststore being picked up
    +          logger.debug("Initializing truststore {}.", getTrustStorePath());
    +          ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +          InputStream tsStream = new FileInputStream(getTrustStorePath());
    +          ts.load(tsStream, getTrustStorePassword().toCharArray());
    +      } else {
    +        logger.debug("Initializing default truststore.");
    +      }
    +      if (disableCertificateVerification()) {
    +        tmf = InsecureTrustManagerFactory.INSTANCE;
    +      } else {
    +        tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
    +      }
    +      tmf.init(ts);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    --- End diff --
    
    May be it is good to add original exception as well to have full stacktrace?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142739285
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,98 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl, std::string& msg) {
    --- End diff --
    
    maybe it should be a configurable callback where the drill client provides the certificate, and the callback returns if the certificate is valid or not? this way it would be easier to add support for system keystore support on mac or linux too...


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142733695
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    --- End diff --
    
    spec didn't mention support for ssl protocol. Shouldn't we disable support for those (since there are pretty insecure)? consensus is that tls1.2 should be used, so maybe it should be set as the mininum?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140621394
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/ConnectionMultiListener.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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 com.google.protobuf.MessageLite;
    +import io.netty.buffer.ByteBuf;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.util.concurrent.Future;
    +import io.netty.util.concurrent.GenericFutureListener;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.slf4j.Logger;
    +
    +import java.net.SocketAddress;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * @param <CC> Client Connection Listener
    + * @param <HS> Outbound handshake message type
    + * @param <HR> Inbound handshake message type
    + * @param <BC> BasicClient type
    + *             <p>
    + *             Implements a wrapper class that allows a client connection to associate different behaviours after
    + *             establishing a connection with the server. The client can choose to send an application handshake, or
    + *             in the case of SSL, wait for a SSL handshake completion and then send an application handshake.
    + */
    +
    +public class ConnectionMultiListener<CC extends ClientConnection, HS extends MessageLite, HR extends MessageLite, BC extends BasicClient> {
    +
    +  private static final Logger logger = org.slf4j.LoggerFactory.getLogger(ConnectionMultiListener.class);
    +
    +  private final RpcConnectionHandler<CC> connectionListener;
    +  private final HS handshakeValue;
    +  private final BC parent;
    +
    +  private ConnectionMultiListener(RpcConnectionHandler<CC> connectionListener, HS handshakeValue,
    +      BC basicClient) {
    +    assert connectionListener != null;
    +    assert handshakeValue != null;
    +
    +    this.connectionListener = connectionListener;
    +    this.handshakeValue = handshakeValue;
    +    this.parent = basicClient;
    +  }
    +
    +  @SuppressWarnings("unchecked")
    +  public static <CC extends ClientConnection, HS extends MessageLite, BC extends BasicClient> Builder
    +  newBuilder(RpcConnectionHandler<CC> connectionListener, HS handshakeValue,
    +      BC basicClient) {
    +    return new Builder(connectionListener, handshakeValue, basicClient);
    +  }
    +
    +  public ConnectionHandler connectionHandler = null;
    +  public HandshakeSendHandler handshakeSendHandler = null;
    +  public SSLConnectionHandler sslConnectionHandler = null;
    +
    +  /**
    +   * Manages connection establishment outcomes.
    +   */
    +  private class ConnectionHandler implements GenericFutureListener<ChannelFuture> {
    +
    +    @Override public void operationComplete(ChannelFuture future) throws Exception {
    +      boolean isInterrupted = false;
    +
    +      // We want to wait for at least 120 secs when interrupts occur. Establishing a connection fails/succeeds quickly,
    +      // So there is no point propagating the interruption as failure immediately.
    +      long remainingWaitTimeMills = 120000;
    +      long startTime = System.currentTimeMillis();
    +      // logger.debug("Connection operation finished.  Success: {}", future.isSuccess());
    +      while (true) {
    +        try {
    +          future.get(remainingWaitTimeMills, TimeUnit.MILLISECONDS);
    +          if (future.isSuccess()) {
    +            SocketAddress remote = future.channel().remoteAddress();
    +            SocketAddress local = future.channel().localAddress();
    +            parent.setAddresses(remote, local);
    +            // if SSL is enabled send the handshake after the ssl handshake is completed, otherwise send it
    +            // now
    +            if(!parent.isSslEnabled()) {
    +              // send a handshake on the current thread. This is the only time we will send from within the event thread.
    +              // We can do this because the connection will not be backed up.
    +              parent.send(handshakeSendHandler, handshakeValue, true);
    +            }
    +          } else {
    +            connectionListener.connectionFailed(RpcConnectionHandler.FailureType.CONNECTION,
    +                new RpcException("General connection failure."));
    +          }
    +          // logger.debug("Handshake queued for send.");
    +          break;
    +        } catch (final InterruptedException interruptEx) {
    +          remainingWaitTimeMills -= (System.currentTimeMillis() - startTime);
    +          startTime = System.currentTimeMillis();
    +          isInterrupted = true;
    +          if (remainingWaitTimeMills < 1) {
    +            connectionListener.connectionFailed(RpcConnectionHandler.FailureType.CONNECTION, interruptEx);
    +            break;
    +          }
    +          // Ignore the interrupt and continue to wait until we elapse remainingWaitTimeMills.
    +        } catch (final Exception ex) {
    +          logger.error("Failed to establish connection", ex);
    +          connectionListener.connectionFailed(RpcConnectionHandler.FailureType.CONNECTION, ex);
    +          break;
    +        }
    +      }
    +
    +      if (isInterrupted) {
    +        // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
    +        // interruption and respond to it if it wants to.
    +        Thread.currentThread().interrupt();
    +      }
    +    }
    +  }
    +
    +  private class SSLConnectionHandler implements GenericFutureListener<Future<Channel>> {
    +    @Override public void operationComplete(Future<Channel> future) throws Exception {
    +      // send the handshake
    +      parent.send(handshakeSendHandler, handshakeValue, true);
    +    }
    +  }
    +
    +  /**
    +   * manages handshake outcomes.
    +   */
    +  private class HandshakeSendHandler implements RpcOutcomeListener<HR> {
    +
    +    @Override public void failed(RpcException ex) {
    +      logger.debug("Failure while initiating handshake", ex);
    +      connectionListener.connectionFailed(RpcConnectionHandler.FailureType.HANDSHAKE_COMMUNICATION, ex);
    +    }
    +
    +    @Override public void success(HR value, ByteBuf buffer) {
    +      // logger.debug("Handshake received. {}", value);
    +      try {
    +        parent.validateHandshake(value);
    +        parent.finalizeConnection(value, parent.connection);
    +        connectionListener.connectionSucceeded((CC) parent.connection);
    +        // logger.debug("Handshake completed succesfully.");
    +      } catch (Exception ex) {
    +        logger.debug("Failure while validating handshake", ex);
    +        connectionListener.connectionFailed(RpcConnectionHandler.FailureType.HANDSHAKE_VALIDATION, ex);
    +      }
    +    }
    +
    +    @Override public void interrupted(final InterruptedException ex) {
    +      logger.warn("Interrupted while waiting for handshake response", ex);
    +      connectionListener.connectionFailed(RpcConnectionHandler.FailureType.HANDSHAKE_COMMUNICATION, ex);
    +    }
    +  }
    +
    +  /*
    +    The SSL Handshake listener is special in that it is needed at the time of initializing an SSL
    +    enabled pipeline and so is instantiated before the instance of the outer class may be needed.
    +    We create an instance and set a reference back to the outer class instance when it is created
    +    at the time of connection.
    +   */
    +  public static class SSLHandshakeListener implements GenericFutureListener<Future<Channel>> {
    +    ConnectionMultiListener parent;
    +    public SSLHandshakeListener() {
    +    }
    +
    +    public void setParent(ConnectionMultiListener cml){
    +      this.parent = cml;
    +    }
    +
    +    @Override public void operationComplete(Future<Channel> future) throws Exception {
    +      if(parent != null){
    +        if(future.isSuccess()) {
    +          Channel c = future.get();
    +          parent.sslConnectionHandler.operationComplete(future);
    +          parent.parent.setSslChannel(c);
    +        } else {
    +          throw new DrillException("SSL handshake failed.", future.cause());
    +        }
    +      } else {
    +        throw new RpcException("RPC Setup error. SSL handshake complete handler is not set up.");
    +      }
    +      return;
    +    }
    +  }
    +
    +
    +  public static class Builder<CC extends ClientConnection, HS extends MessageLite, HR extends MessageLite, BC extends BasicClient> {
    +
    +    private RpcConnectionHandler<CC> connectionListener;
    +    private HS handshakeValue;
    +    private BC basicClient;
    +    private ConnectionMultiListener cml;
    +
    +    private Builder(RpcConnectionHandler<CC> connectionListener, HS handshakeValue, BC basicClient) {
    +      this.connectionListener = connectionListener;
    +      this.handshakeValue = handshakeValue;
    +      this.basicClient = basicClient;
    +      this.cml = new ConnectionMultiListener(connectionListener, handshakeValue, basicClient);
    +    }
    +
    +    public Builder enableSSL() {
    +      cml.connectionHandler = cml.new ConnectionHandler();
    +      cml.sslConnectionHandler = cml.new SSLConnectionHandler();
    +      return this;
    +    }
    +
    +    public Builder enablePlain() {
    +      cml.connectionHandler = cml.new ConnectionHandler();
    +      return this;
    +    }
    +
    +    public Builder enableHandshake() {
    +      cml.handshakeSendHandler = cml.new HandshakeSendHandler();
    +      return this;
    +    }
    +
    +    public ConnectionMultiListener build() {
    +      //always enable handshake
    +      if (cml.handshakeSendHandler == null) {
    +        enableHandshake();
    +      }
    +      if (cml.connectionHandler == null && cml.sslConnectionHandler == null) {
    +        enablePlain();
    +      }
    +      return cml;
    --- End diff --
    
    Couple of things here: 1) This builder is responsible for creating ConnectionMultiListener which is not being created in build method rather in the constructor itself. 2) Since HandshakeSendHandler and ConnectionHandler are needed both for SSL and Plain case why not leave their instantiation as it was earlier which is in the ConnectionMultiListener class by default ?
    
    So we can do something like below:
    1) Accept ConnectionListener, HandshakeValue and BasicClient in the Builder constructor and assign that to member's of builder.
    2) Add new member for builder to have SSLConnectionHandler too.
    3)  Then just have enableSSL() or configurSSLHandler() method in which create a new SSLConnectionhandler and assign that to the member variable.
    4) Inside build() method then create an instance of ConnectionMultiListener with all the 4 members and return that.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142862510
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java ---
    @@ -49,12 +64,19 @@ public void testMissingKeystorePassword() throws Exception {
         ConfigBuilder config = new ConfigBuilder();
         config.put(ExecConstants.HTTP_KEYSTORE_PATH, "/root");
         config.put(ExecConstants.HTTP_KEYSTORE_PASSWORD, "");
    +    config.put(ExecConstants.SSL_USE_HADOOP_CONF, false);
    +    config.put(ExecConstants.USER_SSL_ENABLED, true);
         try {
    -      SSLConfig sslv = new SSLConfig(config.build());
    +      SSLConfig sslv = new SSLConfigBuilder()
    +          .config(config.build())
    +          .mode(SSLFactory.Mode.SERVER)
    +          .initializeSSLContext(false)
    +          .validateKeyStore(true)
    +          .build();
           fail();
           //Expected
         } catch (Exception e) {
    -      assertTrue(e instanceof DrillException);
    +
    --- End diff --
    
    But the assert `assertTrue(e instanceof DrillException);` was removed and catch block is empty, test will never fail...


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r143151998
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSL.java ---
    @@ -201,6 +202,7 @@ public void testSSLQuery() throws Exception {
         test("SELECT * FROM cp.`region.json`");
       }
     
    +  @Ignore
    --- End diff --
    
    Parth, can you please add explanation (`@Ignore("some reason")`) why test is ignored (it would be really helpful in future).


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142731548
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,448 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                DRILL_LOG(LOG_INFO) << "Failed to get endpoint from zk" << std::endl;
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(((.*):([0-9]+),?)+)(/.+)?");
    +    boost::cmatch matched;
    +
    +    if(boost::regex_match(m_connectString.c_str(), matched, connStrExpr)){
    +        m_protocol.assign(matched[1].first, matched[1].second);
    +        if(isDirectConnection()){
    +            m_host.assign(matched[4].first, matched[4].second);
    +            m_port.assign(matched[5].first, matched[5].second);
    +        }else {
    +            // if the connection is to a zookeeper,
    +            // we will get the host and the port only after connecting to the Zookeeper
    +            m_host = "";
    +            m_port = "";
    +        }
    +        m_hostPortStr.assign(matched[2].first, matched[2].second);
    +        if(matched[6].matched) {
    +            m_pathToDrill.assign(matched[6].first, matched[6].second);
    +        }
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG)
    +                             << "Conn str: "<< m_connectString
    +                             << ";  protocol: " << m_protocol
    +                             << ";  host: " << m_host
    +                             << ";  port: " << m_port
    +                             << ";  path to drill: " << m_pathToDrill
    +                             << std::endl;)
    +    } else {
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Invalid connect string. Regexp did not match" << std::endl;)
    +    }
    +    return;
    +}
    +
    +bool ConnectionEndpoint::isDirectConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "local") || !strcmp(m_protocol.c_str(), "drillbit"));
    --- End diff --
    
    let's not use strcmp if we can avoid it. `mprotocol != "local"` should work fine.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142739836
  
    --- Diff: contrib/native/client/src/include/drill/drillConfig.hpp ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +#ifndef DRILL_CONFIG_H
    +#define DRILL_CONFIG_H
    +
    +#include "drill/common.hpp"
    +#include <boost/thread.hpp>
    +
    +
    +
    +#if defined _WIN32 || defined __CYGWIN__
    --- End diff --
    
    maybe should be in a common header since used in multiple places?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142825441
  
    --- Diff: contrib/native/client/src/include/drill/drillConfig.hpp ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +#ifndef DRILL_CONFIG_H
    +#define DRILL_CONFIG_H
    +
    +#include "drill/common.hpp"
    +#include <boost/thread.hpp>
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141726875
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,452 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(.*):([0-9]+)(?:/(.+))?");
    +    boost::cmatch matched;
    +
    +    if(boost::regex_match(m_connectString.c_str(), matched, connStrExpr)){
    +        m_protocol.assign(matched[1].first, matched[1].second);
    +        std::string host, port;
    +        host.assign(matched[2].first, matched[2].second);
    +        port.assign(matched[3].first, matched[3].second);
    +        if(isDirectConnection()){
    +            // if the connection is to a zookeeper, 
    +            // we will get the host and the port only after connecting to the Zookeeper
    +            m_host=host;
    +            m_port=port;
    +        }
    +        m_hostPortStr=host+std::string(":")+port;
    +        std::string pathToDrill;
    +        if(matched.size()==5){
    +            pathToDrill.assign(matched[4].first, matched[4].second);
    +            if(!pathToDrill.empty()){
    +                m_pathToDrill=std::string("/")+pathToDrill;
    +            }
    +        }
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) 
    +                << "Conn str: "<< m_connectString 
    +                << ";  protocol: " << m_protocol 
    +                << ";  host: " << host 
    +                << "; port: " << port 
    +                << ";  path to drill: " << m_pathToDrill 
    +                << std::endl;)
    +    } else {
    +        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Invalid connect string. Regexp did not match" << std::endl;)
    +    }
    +
    +    return;
    +}
    +
    +bool ConnectionEndpoint::isDirectConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "local") || !strcmp(m_protocol.c_str(), "drillbit"));
    +}
    +
    +bool ConnectionEndpoint::isZookeeperConnection(){
    +    assert(!m_protocol.empty());
    +    return (!strcmp(m_protocol.c_str(), "zk"));
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpointFromZk(){
    +    ZookeeperClient zook(m_pathToDrill);
    +    assert(!m_hostPortStr.empty());
    +    std::vector<std::string> drillbits;
    +    if(zook.getAllDrillbits(m_hostPortStr.c_str(), drillbits)!=0){
    +        return handleError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
    +    }
    +    if (drillbits.empty()){
    +        return handleError(CONN_FAILURE, getMessage(ERR_CONN_ZKNODBIT));
    +    }
    +    Utils::shuffle(drillbits);
    +    exec::DrillbitEndpoint endpoint;
    +    int err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
    +    if(!err){
    +        m_host=boost::lexical_cast<std::string>(endpoint.address());
    +        m_port=boost::lexical_cast<std::string>(endpoint.user_port());
    +    }
    +    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
    --- End diff --
    
    Done. Good catch! 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142993025
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,98 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl, std::string& msg) {
    --- End diff --
    
    it looks like boost::asio support both loading a file and/or a verify callback:
    - http://www.boost.org/doc/libs/1_47_0/doc/html/boost_asio/reference/ssl__context/set_verify_callback/overload1.html
    - http://www.boost.org/doc/libs/1_47_0/doc/html/boost_asio/reference/ssl__context/load_verify_file.html
    
    It seems you wouldn't even need to access the native handle when using these functions


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/950
  
    @laurentgo are you good with this? I'll open a new JIRA for the enhancements to the C++ client. 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142727549
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    --- End diff --
    
    should either return a new string, or a const ref to the internal string. In both cases, this method should probably be const too (same for other methods)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142718173
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -390,11 +404,26 @@ int main(int argc, char* argv[]) {
             if(auth.length()>0){
                 props.setProperty(USERPROP_AUTH_MECHANISM, auth);
             }
    +        if(enableSSL.length()>0){
    --- End diff --
    
    (style) indentation issue
    you can also use `!enableSSL.empty()` instead of checking the length


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140122388
  
    --- Diff: distribution/src/resources/drill-override-example.conf ---
    @@ -222,7 +222,35 @@ drill.exec: {
       # Full workspace name should be indicated (including schema and workspace separated by dot).
       # Workspace MUST be file-based and writable. Workspace name is case-sensitive.
       default_temporary_workspace: "dfs.tmp"
    +
    +  # Enable and provide additional parameters for Client-Server communication over SSL
    +  # see also the javax.net.ssl parameters below
    +  security.user.encryption.ssl: {
    +    #Set this to true to enable all client server communication to occur over SSL.
    +    enabled: false,
    +    #key password is optional if it is the same as the keystore password
    +    keyPassword: "key_passwd",
    +    #Optional handshakeTimeout in milliseconds. Default is 10000 ms (10 seconds)
    +    handshakeTimeout: 10000,
    +    #protocol is optional. Drill will default to TLSv1.2
    --- End diff --
    
    Please add list of supported protocol's value. 
    
    Also add the example and list of values supported for: 
    _provider: "JDK"_


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141503730
  
    --- Diff: contrib/native/client/src/include/drill/userProperties.hpp ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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 USER_PROPERTIES_H
    +#define USER_PROPERTIES_H
    +
    +#include <map>
    +#include "drill/common.hpp"
    +
    +namespace Drill{
    +
    +class DECLSPEC_DRILL_CLIENT DrillUserProperties{
    +    public:
    +        static const std::map<std::string, uint32_t> USER_PROPERTIES;
    +
    +        DrillUserProperties(){};
    +
    +        void setProperty( const std::string& propName, const std::string& propValue){
    +            std::pair< std::string, std::string> in = make_pair(propName, propValue);
    +            m_properties.insert(in);
    +        }
    +
    +        size_t size() const { return m_properties.size(); }
    +
    +        const bool  isPropSet(const std::string& key) const{
    +            bool isSet=true;
    +            std::map<std::string, std::string>::const_iterator f=m_properties.find(key);
    +            if(f==m_properties.end()){
    +                isSet=false;
    +            }
    +            return isSet;
    +        }
    +
    +        const std::string&  getProp(const std::string& key, std::string& value) const{
    --- End diff --
    
    This is a common pattern.  E.g. strcpy -
    `char * strcpy ( char * destination, const char * source );`



---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140612371
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java ---
    @@ -140,12 +140,12 @@ protected DrillConnectionImpl(DriverImpl driver, AvaticaFactory factory,
     
             this.client = new DrillClient(dConfig, set.getCoordinator());
           } else if(config.isDirect()) {
    -        final DrillConfig dConfig = DrillConfig.forClient();
    +        final DrillConfig dConfig = DrillConfig.forClient(info);
    --- End diff --
    
    Removed


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140396437
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    +
    +  protected String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    +    value = hadoopConfig.get(name, "");
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getConfigParamWithDefault(String name, String defaultValue) {
    +    String value = "";
    +    if (config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    if (value.isEmpty()) {
    +      value = defaultValue;
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String resolveHadoopPropertyName(String nameTemplate, SSLFactory.Mode mode) {
    +    return MessageFormat.format(nameTemplate, mode.toString().toLowerCase());
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  public abstract SslContext initSslContext() throws DrillException;
    +
    +  public abstract SSLContext initSSLContext() throws DrillException;
    +
    +  public abstract boolean isUserSslEnabled();
    +
    +  public abstract boolean isHttpsEnabled();
    +
    +  public abstract String getKeyStoreType();
    +
    +  public abstract String getKeyStorePath();
    +
    +  public abstract String getKeyStorePassword();
    +
    +  public abstract String getKeyPassword();
    +
    +  public abstract String getTrustStoreType();
    +
    +  public abstract boolean hasTrustStorePath();
    +
    +  public abstract String getTrustStorePath();
    +
    +  public abstract boolean hasTrustStorePassword();
    +
    +  public abstract String getTrustStorePassword();
    +
    +  public abstract String getProtocol();
    +
    +  public abstract SslProvider getProvider();
    +
    +  public abstract int getHandshakeTimeout();
    +
    +  public abstract SSLFactory.Mode getMode();
    +
    +  public abstract boolean disableHostVerification();
    +
    +  public abstract boolean disableCertificateVerification();
    +
    +  public abstract boolean useSystemTrustStore();
    +
    +  public abstract boolean isSslValid();
    +
    +  public SslContext getNettySslContext() {
    +    return nettySslContext;
    +  }
    +
    +  public TrustManagerFactory initializeTrustManagerFactory() throws DrillException {
    +    TrustManagerFactory tmf;
    +    KeyStore ts = null;
    +    //Support Windows/MacOs system trust store
    +    try {
    +      String trustStoreType = getTrustStoreType();
    +      if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +        // This is valid for MS-Windows and MacOs
    +        logger.debug("Initializing System truststore.");
    +        ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +        ts.load(null, null);
    +      } else if (!getTrustStorePath().isEmpty()) {
    +          // if truststore is not provided then we will use the default. Note that the default depends on
    +          // the TrustManagerFactory that in turn depends on the Security Provider.
    +          // Use null as the truststore which will result in the default truststore being picked up
    +          logger.debug("Initializing truststore {}.", getTrustStorePath());
    +          ts = KeyStore.getInstance(!trustStoreType.isEmpty() ? trustStoreType : KeyStore.getDefaultType());
    +          InputStream tsStream = new FileInputStream(getTrustStorePath());
    +          ts.load(tsStream, getTrustStorePassword().toCharArray());
    +      } else {
    +        logger.debug("Initializing default truststore.");
    +      }
    +      if (disableCertificateVerification()) {
    +        tmf = InsecureTrustManagerFactory.INSTANCE;
    +      } else {
    +        tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
    +      }
    +      tmf.init(ts);
    +    } catch (Exception e) {
    +      // Catch any SSL initialization Exceptions here and abort.
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the truststore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return tmf;
    +  }
    +
    +  public KeyManagerFactory initializeKeyManagerFactory() throws DrillException {
    +    KeyManagerFactory kmf;
    +    String keyStorePath = getKeyStorePath();
    +    String keyStorePassword = getKeyStorePassword();
    +    String keyStoreType = getKeyStoreType();
    +    try {
    +      if (keyStorePath.isEmpty()) {
    +        throw new DrillException("No Keystore provided.");
    +      }
    +      KeyStore ks =
    +          KeyStore.getInstance(!keyStoreType.isEmpty() ? keyStoreType : KeyStore.getDefaultType());
    +      //initialize the key manager factory
    +      // Will throw an exception if the file is not found/accessible.
    +      InputStream ksStream = new FileInputStream(keyStorePath);
    +      // A key password CANNOT be null or an empty string.
    +      if (keyStorePassword.isEmpty()) {
    +        throw new DrillException("The Keystore password cannot be empty.");
    +      }
    +      ks.load(ksStream, keyStorePassword.toCharArray());
    +      // Empty Keystore. (Remarkably, it is possible to do this).
    +      if (ks.size() == 0) {
    +        throw new DrillException("The Keystore has no entries.");
    +      }
    +      kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
    +      kmf.init(ks, getKeyPassword().toCharArray());
    +
    +    } catch (Exception e) {
    +      throw new DrillException(
    +          new StringBuilder()
    +              .append("Exception while initializing the keystore: [")
    +              .append(e.getMessage())
    +              .append("]. ")
    +              .toString());
    +    }
    +    return kmf;
    +  }
    +
    +  public void initContext() throws DrillException {
    +    if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +      initSSLContext();
    +      logger.debug("Initialized Windows SSL context using JDK.");
    +    } else {
    +      initSslContext();
    +      logger.debug("Initialized SSL context.");
    +    }
    +    return;
    +  }
    +
    +  public SSLEngine createSSLEngine(BufferAllocator allocator, String peerHost, int peerPort) {
    +    SSLEngine engine;
    +    if ((isWindows || isMacOs) && useSystemTrustStore()) {
    +      if (peerHost != null) {
    +        engine = jdkSSlContext.createSSLEngine(peerHost, peerPort);
    +        logger.debug("Initializing Windows SSLEngine with hostname verification.");
    +      } else {
    +        engine = jdkSSlContext.createSSLEngine();
    +        logger.debug("Initializing Windows SSLEngine with no hostname verification.");
    +      }
    +    } else {
    +      if (peerHost != null) {
    +        engine = nettySslContext.newEngine(allocator.getAsByteBufAllocator(), peerHost, peerPort);
    +        logger.debug("Initializing SSLEngine with hostname verification.");
    +      } else {
    +        engine = nettySslContext.newEngine(allocator.getAsByteBufAllocator());
    +        logger.debug("Initializing SSLEngine with no hostname verification.");
    +      }
    +    }
    +    return engine;
    --- End diff --
    
    Since in initContext based on (isWindows || isMacOs && useSystemTrustStore) condition , either jdkSSlContext or nettySSLContext will be initialized. So we can simplify this method something like below:
    
    ```
    if(peerHost != null) {
        engine = (jdkSSlContext) ? jdkSSlContext.createSSLEngine(peerHost, peerPort) : nettySslContext.newEngine(allocator.getAsByteBufAllocator(), peerHost, peerPort);
    } else {
        engine = (jdkSSlContext) ? jdkSSlContext.createSSLEngine() : nettySslContext.newEngine(allocator.getAsByteBufAllocator());
    }
    
    return engine;
    ```
    
    Also the log message doesn't look to be true since hostname verification is disabled or not is not based on if peerHost is provided while initializing SSLEngine rather on the config _disableHostVerification_ ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142825478
  
    --- Diff: contrib/native/client/src/include/drill/drillConfig.hpp ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +#ifndef DRILL_CONFIG_H
    +#define DRILL_CONFIG_H
    +
    +#include "drill/common.hpp"
    +#include <boost/thread.hpp>
    +
    +
    +
    +#if defined _WIN32 || defined __CYGWIN__
    --- End diff --
    
    Done. It was already in common.hpp


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140122588
  
    --- Diff: distribution/src/resources/drill-override-example.conf ---
    @@ -222,7 +222,35 @@ drill.exec: {
       # Full workspace name should be indicated (including schema and workspace separated by dot).
       # Workspace MUST be file-based and writable. Workspace name is case-sensitive.
       default_temporary_workspace: "dfs.tmp"
    +
    +  # Enable and provide additional parameters for Client-Server communication over SSL
    +  # see also the javax.net.ssl parameters below
    +  security.user.encryption.ssl: {
    +    #Set this to true to enable all client server communication to occur over SSL.
    +    enabled: false,
    +    #key password is optional if it is the same as the keystore password
    +    keyPassword: "key_passwd",
    +    #Optional handshakeTimeout in milliseconds. Default is 10000 ms (10 seconds)
    +    handshakeTimeout: 10000,
    +    #protocol is optional. Drill will default to TLSv1.2
    +    protocol: "TLSv1.2"
    +  }
    +}
    +
    +# The SSL parameters below need to be set for custom transport layer settings. These are used by
    +# both the WebServer (for HTTPS) and for Client-Server communication over SSL.
    +javax.net.ssl {
    --- End diff --
    
    These are java system property and recommendation should not be to set these in configuration file instead it should be passed as command line arguments. Later when config resolution will happen it will take care of merging system property and the drill's configuration property value. Same for below options.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140621499
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/ConnectionMultiListener.java ---
    @@ -0,0 +1,240 @@
    +/*
    + * 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 com.google.protobuf.MessageLite;
    +import io.netty.buffer.ByteBuf;
    +import io.netty.channel.Channel;
    +import io.netty.channel.ChannelFuture;
    +import io.netty.util.concurrent.Future;
    +import io.netty.util.concurrent.GenericFutureListener;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.slf4j.Logger;
    +
    +import java.net.SocketAddress;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * @param <CC> Client Connection Listener
    + * @param <HS> Outbound handshake message type
    + * @param <HR> Inbound handshake message type
    + * @param <BC> BasicClient type
    + *             <p>
    + *             Implements a wrapper class that allows a client connection to associate different behaviours after
    + *             establishing a connection with the server. The client can choose to send an application handshake, or
    + *             in the case of SSL, wait for a SSL handshake completion and then send an application handshake.
    + */
    +
    +public class ConnectionMultiListener<CC extends ClientConnection, HS extends MessageLite, HR extends MessageLite, BC extends BasicClient> {
    --- End diff --
    
    I am seeing lots of unchecked warning for this file. Please fix those.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142824514
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142822406
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    --- End diff --
    
    Done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140130105
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillConnectionImpl.java ---
    @@ -140,12 +140,12 @@ protected DrillConnectionImpl(DriverImpl driver, AvaticaFactory factory,
     
             this.client = new DrillClient(dConfig, set.getCoordinator());
           } else if(config.isDirect()) {
    -        final DrillConfig dConfig = DrillConfig.forClient();
    +        final DrillConfig dConfig = DrillConfig.forClient(info);
    --- End diff --
    
    in the connect call this _info_ is available.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141670440
  
    --- Diff: contrib/native/client/src/include/drill/userProperties.hpp ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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 USER_PROPERTIES_H
    +#define USER_PROPERTIES_H
    +
    +#include <map>
    +#include "drill/common.hpp"
    +
    +namespace Drill{
    +
    +class DECLSPEC_DRILL_CLIENT DrillUserProperties{
    +    public:
    +        static const std::map<std::string, uint32_t> USER_PROPERTIES;
    +
    +        DrillUserProperties(){};
    +
    +        void setProperty( const std::string& propName, const std::string& propValue){
    --- End diff --
    
    Not a bad idea, but we might have to stay with this for the moment. The implementation currently does not own the string that's passed in, leaving it to the caller to free the memory when appropriate. To convert always to lowercase during the set method, will require me to change the behavior, make a copy and then remember to free the memory in case it is overwritten. If we're doing that, then we might as well enhance the implementation to get/set type specific options (setBoolean, setInt, etc) and that would become so much work for not much gain. 


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140394414
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java ---
    @@ -176,31 +256,28 @@ public void connect(final DrillbitEndpoint endpoint, final DrillProperties prope
       }
     
       private CheckedFuture<Void, RpcException> connect(final UserToBitHandshake handshake,
    -                                                    final DrillbitEndpoint endpoint) {
    +      final DrillbitEndpoint endpoint) {
         final SettableFuture<Void> connectionSettable = SettableFuture.create();
         final CheckedFuture<Void, RpcException> connectionFuture =
             new AbstractCheckedFuture<Void, RpcException>(connectionSettable) {
    -          @Override
    -          protected RpcException mapException(Exception e) {
    +          @Override protected RpcException mapException(Exception e) {
    --- End diff --
    
    Please undo these formatting changes above and below


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140398781
  
    --- Diff: contrib/native/client/readme.macos ---
    @@ -44,12 +44,23 @@ Install Prerequisites
     2.2) Install zookeeper
       $> brew install zookeeper
     
    -2.3) Install boost
    +2.3) Install or  Cyrus SASL 
    --- End diff --
    
    Same as readme.linux


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140971420
  
    --- Diff: contrib/native/client/src/clientlib/streamSocket.hpp ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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 STREAMSOCKET_HPP
    +#define STREAMSOCKET_HPP
    +
    +#include "drill/common.hpp"
    +#include "env.h"
    +#include "wincert.ipp"
    +
    +#include <boost/asio.hpp>
    +#include <boost/asio/ssl.hpp>
    +
    +namespace Drill {
    +
    +typedef boost::asio::ip::tcp::socket::lowest_layer_type streamSocket_t;
    +typedef boost::asio::ssl::stream<boost::asio::ip::tcp::socket> sslTCPSocket_t;
    +typedef boost::asio::ip::tcp::socket basicTCPSocket_t;
    +
    +
    +// Some helper typedefs to define the highly templatized boost::asio methods
    +typedef boost::asio::const_buffers_1 ConstBufferSequence; 
    +typedef boost::asio::mutable_buffers_1 MutableBufferSequence;
    +
    +// ReadHandlers have different possible signatures.
    +//
    +// As a standard C-type callback
    +//    typedef void (*ReadHandler)(const boost::system::error_code& ec, std::size_t bytes_transferred);
    +//
    +// Or as a C++ functor
    +//    struct ReadHandler {
    +//        virtual void operator()(
    +//                const boost::system::error_code& ec,
    +//                std::size_t bytes_transferred) = 0;
    +//};
    +//
    +// We need a different signature though, since we need to pass in a member function of a drill client 
    +// class (which is C++), as a functor generated by boost::bind as a ReadHandler
    +// 
    +typedef boost::function<void (const boost::system::error_code& ec, std::size_t bytes_transferred) > ReadHandler;
    +
    +class AsioStreamSocket{
    +    public:
    +        virtual ~AsioStreamSocket(){};
    +        virtual streamSocket_t& getInnerSocket() = 0;
    +
    +        virtual std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        virtual std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec) = 0;
    +
    +        //
    +        // boost::asio::async_read has the signature 
    +        // template<
    +        //     typename AsyncReadStream,
    +        //     typename MutableBufferSequence,
    +        //     typename ReadHandler>
    +        // void-or-deduced async_read(
    +        //     AsyncReadStream & s,
    +        //     const MutableBufferSequence & buffers,
    +        //     ReadHandler handler);
    +        //
    +        // For our use case, the derived class will have an instance of a concrete type for AsyncReadStream which 
    +        // will implement the requirements for the AsyncReadStream type. We need not pass that in as a parameter 
    +        // since the class already has the value
    +        // The method is templatized since the ReadHandler type is dependent on the class implementing the read 
    +        // handler (basically the class using the asio stream)
    +        //
    +        virtual void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler) = 0;
    +
    +        // call the underlying protocol's handshake method.
    +        // if the useSystemConfig flag is true, then use properties read
    +        // from the underlying operating system
    +        virtual void protocolHandshake(bool useSystemConfig) = 0;
    +        virtual void protocolClose() = 0;
    +};
    +
    +class Socket: 
    +    public AsioStreamSocket, 
    +    public basicTCPSocket_t{
    +
    +    public:
    +        Socket(boost::asio::io_service& ioService) : basicTCPSocket_t(ioService) {
    +            }
    +
    +        ~Socket(){
    +            boost::system::error_code ignorederr;
    +            this->shutdown(boost::asio::ip::tcp::socket::shutdown_both, ignorederr);
    +            this->close();
    +        };
    +
    +        basicTCPSocket_t& getSocketStream(){ return *this;}
    +
    +        streamSocket_t& getInnerSocket(){ return this->lowest_layer();}
    +
    +        std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->write_some(buffers, ec);
    +        }
    +
    +        std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->read_some(buffers, ec);
    +        }
    +
    +        void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler){
    +            return async_read(*this, buffers, handler);
    +        }
    +
    +        void protocolHandshake(bool useSystemConfig){}; //nothing to do
    +        void protocolClose(){ 
    +            // shuts down the socket!
    +            boost::system::error_code ignorederr;
    +            ((basicTCPSocket_t*)this)->shutdown(boost::asio::ip::tcp::socket::shutdown_both,
    +                ignorederr
    +                );         
    +        } 
    +};
    +
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +class SslSocket: 
    +    public AsioStreamSocket, 
    +    public sslTCPSocket_t{
    +
    +    public:
    +        SslSocket(boost::asio::io_service& ioService, boost::asio::ssl::context &sslContext) :
    +            sslTCPSocket_t(ioService, sslContext) {
    +            }
    +
    +        ~SslSocket(){};
    +
    +        sslTCPSocket_t& getSocketStream(){ return *this;}
    +
    +        streamSocket_t& getInnerSocket(){ return this->lowest_layer();}
    +
    +        std::size_t writeSome(
    +                const ConstBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->write_some(buffers, ec);
    +        }
    +
    +        std::size_t readSome(
    +                const MutableBufferSequence& buffers,
    +                boost::system::error_code & ec){
    +            return this->read_some(buffers, ec);
    +        }
    +
    +        void asyncRead( const MutableBufferSequence & buffers, ReadHandler handler){
    +            return async_read(*this, buffers, handler);
    +        }
    +
    +        //
    +        // public method that can be invoked by callers to invoke the ssl handshake
    +        // throws: boost::system::system_error
    +        void protocolHandshake(bool useSystemConfig){
    +            if(useSystemConfig){
    +                if (loadSystemTrustStore(this->native_handle())) {
    +                    boost::system::error_code ec(EPROTO, boost::system::system_category());
    +                    boost::asio::detail::throw_error(ec, "Failed to load system trust store");
    +                }
    +            }
    +            this->handshake(boost::asio::ssl::stream<boost::asio::ip::tcp::socket>::client);
    +            return;
    +        };
    +        //
    +        // public method that can be invoked by callers to invoke a clean ssl shutdown
    +        // throws: boost::system::system_error
    +        void protocolClose(){
    +            try{
    +                this->shutdown();
    +            }catch(boost::system::system_error e){
    +                //swallow the exception. The channel is unusable anyway
    +            }
    +            // shuts down the socket!
    +            boost::system::error_code ignorederr;
    +            this->lowest_layer().shutdown(boost::asio::ip::tcp::socket::shutdown_both,
    +                ignorederr
    --- End diff --
    
    I don't see` lowest_layer()` socket being closed here.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140588134
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java ---
    @@ -91,10 +123,35 @@ public void testForTrustStore() throws Exception {
         ConfigBuilder config = new ConfigBuilder();
         config.put(ExecConstants.HTTP_TRUSTSTORE_PATH, "/root");
         config.put(ExecConstants.HTTP_TRUSTSTORE_PASSWORD, "root");
    -    SSLConfig sslv = new SSLConfig(config.build());
    +    config.put(ExecConstants.SSL_USE_HADOOP_CONF, false);
    +    SSLConfig sslv = new SSLConfigBuilder()
    +        .config(config.build())
    +        .mode(SSLFactory.Mode.SERVER)
    +        .initializeSSLContext(false)
    +        .validateKeyStore(true)
    +        .build();
         assertEquals(true, sslv.hasTrustStorePath());
         assertEquals(true,sslv.hasTrustStorePassword());
         assertEquals("/root",sslv.getTrustStorePath());
         assertEquals("root",sslv.getTrustStorePassword());
       }
    -}
    \ No newline at end of file
    +
    +  @Test
    +  public void testInvalidHadoopKeystore() throws Exception {
    +    Configuration hadoopConfig = new Configuration();
    +    String hadoopSSLFileProp = MessageFormat
    +        .format(HADOOP_SSL_CONF_TPL_KEY, SSLFactory.Mode.SERVER.toString().toLowerCase());
    +    hadoopConfig.set(hadoopSSLFileProp, "ssl-server-invalid.xml");
    +    ConfigBuilder config = new ConfigBuilder();
    +    config.put(ExecConstants.SSL_USE_HADOOP_CONF, true);
    +    SSLConfig sslv = new SSLConfigBuilder()
    +        .config(config.build())
    +        .mode(SSLFactory.Mode.SERVER)
    +        .initializeSSLContext(false)
    +        .validateKeyStore(true)
    +        .hadoopConfig(hadoopConfig)
    +        .build();
    +    assertEquals("FAIL", sslv.getKeyStorePassword());
    --- End diff --
    
    Shouldn't this test fail while doing `validateKeyStore` since there is no keystore path but only password in the hadoop config file ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140893808
  
    --- Diff: contrib/native/client/src/include/drill/userProperties.hpp ---
    @@ -0,0 +1,73 @@
    +/*
    + * 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 USER_PROPERTIES_H
    +#define USER_PROPERTIES_H
    +
    +#include <map>
    +#include "drill/common.hpp"
    +
    +namespace Drill{
    +
    +class DECLSPEC_DRILL_CLIENT DrillUserProperties{
    +    public:
    +        static const std::map<std::string, uint32_t> USER_PROPERTIES;
    +
    +        DrillUserProperties(){};
    +
    +        void setProperty( const std::string& propName, const std::string& propValue){
    +            std::pair< std::string, std::string> in = make_pair(propName, propValue);
    +            m_properties.insert(in);
    +        }
    +
    +        size_t size() const { return m_properties.size(); }
    +
    +        const bool  isPropSet(const std::string& key) const{
    +            bool isSet=true;
    +            std::map<std::string, std::string>::const_iterator f=m_properties.find(key);
    +            if(f==m_properties.end()){
    +                isSet=false;
    +            }
    +            return isSet;
    +        }
    +
    +        const std::string&  getProp(const std::string& key, std::string& value) const{
    --- End diff --
    
    this method is little confusing since it's returning value both in input parameter and as a return value. I think we should choose either of it NOT both.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140914589
  
    --- Diff: contrib/native/client/src/clientlib/zkCluster.cpp ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +#include "drill/common.hpp"
    +#include <boost/thread.hpp>
    +#ifdef _WIN32
    +#include <zookeeper.h>
    +#else
    +#include <zookeeper/zookeeper.h>
    +#endif
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillClient.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "zkCluster.hpp"
    +
    +namespace Drill{
    +
    +char ZkCluster::s_drillRoot[]="/drill/";
    +char ZkCluster::s_defaultCluster[]="drillbits1";
    +
    +ZkCluster::ZkCluster(){
    +    m_pDrillbits=new String_vector;
    +    srand (time(NULL));
    +    m_bConnecting=true;
    +    memset(&m_id, 0, sizeof(m_id));
    +}
    +
    +ZkCluster::~ZkCluster(){
    +    delete m_pDrillbits;
    +}
    +
    +ZooLogLevel ZkCluster::getZkLogLevel(){
    +    //typedef enum {ZOO_LOG_LEVEL_ERROR=1,
    +    //    ZOO_LOG_LEVEL_WARN=2,
    +    //    ZOO_LOG_LEVEL_INFO=3,
    +    //    ZOO_LOG_LEVEL_DEBUG=4
    +    //} ZooLogLevel;
    +    switch(DrillClientConfig::getLogLevel()){
    +        case LOG_TRACE:
    +        case LOG_DEBUG:
    +            return ZOO_LOG_LEVEL_DEBUG;
    +        case LOG_INFO:
    +            return ZOO_LOG_LEVEL_INFO;
    +        case LOG_WARNING:
    +            return ZOO_LOG_LEVEL_WARN;
    +        case LOG_ERROR:
    +        case LOG_FATAL:
    +        default:
    +            return ZOO_LOG_LEVEL_ERROR;
    +    }
    +    return ZOO_LOG_LEVEL_ERROR;
    +}
    +
    +int ZkCluster::connectToZookeeper(const char* connectStr, const char* pathToDrill){
    +    uint32_t waitTime=30000; // 10 seconds
    --- End diff --
    
    10 seconds ?
    As discussed in person please remove these files as per your plan: `zkCluster.cpp and zkCluster.hpp`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142717986
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -50,7 +49,14 @@ struct Option{
         {"service_host", "Service host for Kerberos", false},
         {"service_name", "Service name for Kerberos", false},
         {"auth", "Authentication mechanism to use", false},
    -    {"sasl_encrypt", "Negotiate for encrypted connection", false}
    +    {"sasl_encrypt", "Negotiate for encrypted connection", false},
    +    {"enableSSL", "Enable SSL", false},
    +    {"TLSProtocol", "TLS protocol version", false},
    +    {"certFilePath", "Path to SSL certificate file", false},
    +    {"disableHostnameVerification", "disable host name verification", false},
    +    {"disableCertVerification", "disable certificate verification", false},
    +	{"useSystemTrustStore", "[Windows only]. Use the system truststore.", false }
    --- End diff --
    
    (style) indentation issue


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142740210
  
    --- Diff: contrib/native/client/src/include/drill/drillConfig.hpp ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +#ifndef DRILL_CONFIG_H
    +#define DRILL_CONFIG_H
    +
    +#include "drill/common.hpp"
    +#include <boost/thread.hpp>
    --- End diff --
    
    only using mutex I believe (defined in `<boost/thread/mutex.hpp>`)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140605580
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java ---
    @@ -70,22 +78,80 @@
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
       private static final String SERVER_NAME = "Apache Drill Server";
     
    +  private final BootStrapContext bootStrapContext;
    +  private final BufferAllocator allocator;
       private final UserConnectionConfig config;
    +  private final SSLConfig sslConfig;
    +  private Channel sslChannel;
       private final UserWorker userWorker;
     
       public UserServer(BootStrapContext context, BufferAllocator allocator, EventLoopGroup eventLoopGroup,
                         UserWorker worker) throws DrillbitStartupException {
         super(UserRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
             allocator.getAsByteBufAllocator(),
             eventLoopGroup);
    +    this.bootStrapContext = context;
    +    this.allocator = allocator;
         this.config = new UserConnectionConfig(allocator, context, new UserServerRequestHandler(worker));
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder()
    +          .config(bootStrapContext.getConfig())
    +          .mode(SSLFactory.Mode.SERVER)
    +          .initializeSSLContext(true)
    +          .validateKeyStore(true)
    +          .build();
    +    } catch (DrillException e) {
    +      throw new DrillbitStartupException(e.getMessage(), e.getCause());
    +    }
         this.userWorker = worker;
     
         // Initialize Singleton instance of UserRpcMetrics.
         ((UserRpcMetrics)UserRpcMetrics.getInstance()).initialize(config.isEncryptionEnabled(), allocator);
       }
     
       @Override
    +  protected void setupSSL(ChannelPipeline pipe) {
    +    if (sslConfig.isUserSslEnabled()) {
    +
    +      SSLEngine sslEngine = sslConfig.createSSLEngine(allocator, null, 0);
    +      sslEngine.setUseClientMode(false);
    +
    +      // No need for client side authentication (HTTPS like behaviour)
    +      sslEngine.setNeedClientAuth(false);
    +
    +      try {
    +        sslEngine.setEnableSessionCreation(true);
    +      } catch (Exception e) {
    +        // Openssl implementation may throw this.
    +        logger.debug("Session creation not enabled. Exception: {}", e.getMessage());
    +      }
    --- End diff --
    
    Fail to see why that would be better, but sure 


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/950
  
    Squashed the review changes and rebased on master.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140398730
  
    --- Diff: contrib/native/client/readme.linux ---
    @@ -84,6 +84,21 @@ OR
         ln -svf libboost_filesystem.a libboost_filesystem-mt.a
         ln -svf libboost_date_time.a libboost_date_time-mt.a
     
    +5) Install or  Cyrus SASL 
    --- End diff --
    
    Install or **build** ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140397110
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserConnectionConfig.java ---
    @@ -75,10 +76,20 @@
         } else {
           authEnabled = false;
         }
    -
         impersonationManager = !config.getBoolean(ExecConstants.IMPERSONATION_ENABLED)
             ? null
             : new InboundImpersonationManager();
    +
    +    if (config.getBoolean(ExecConstants.USER_SSL_ENABLED)) {
    +      sslEnabled = true;
    +    } else {
    +      sslEnabled = false;
    +    }
    --- End diff --
    
    `sslEnabled = config.getBoolean(ExecConstants.USER_SSL_ENABLED);`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142718970
  
    --- Diff: contrib/native/client/readme.linux ---
    @@ -84,6 +84,21 @@ OR
         ln -svf libboost_filesystem.a libboost_filesystem-mt.a
         ln -svf libboost_date_time.a libboost_date_time-mt.a
     
    +5) Install or  build Cyrus SASL 
    +   To Install 
    +       yum install cyrus-sasl-devel
    --- End diff --
    
    you can install multiple packages using one command: `yum install cyrus-sasl-devel cyrus-sasl-gssapi`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r143152156
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/TestSSLConfig.java ---
    @@ -76,7 +76,7 @@ public void testMissingKeystorePassword() throws Exception {
           fail();
           //Expected
         } catch (Exception e) {
    -
    +      assertTrue(e instanceof DrillException);
    --- End diff --
    
    Thanks, for adding `assertTrue`.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r143042574
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,98 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl, std::string& msg) {
    --- End diff --
    
    I'm actually already using these methods (see `SSLStreamChannel::init()`). The verification callback implements validating the certificate. In our case we are using the boost provided rfc2818 verification method. The load verify file should point to the truststore containing the certificates in pem format. OpenSSL will read this file and load the certificate into its in-memory X509 certificate store.
    The `loadSystemTrustStore` method reads the certificates from the Windows store (probably the registry) converts from the native store format into X509 and then loads it into the in-memory store. After that OpenSSL takes over and does the verification. 
    For Keychain, we will have to do something similar. 
    Writing our own certificate verification is going to be error prone, especially if you want to do rfc2818 verification. Not sure I'm up to it :(.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140366914
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    +
    +    this.config = config;
    +    httpsEnabled =
    +        config.hasPath(ExecConstants.HTTP_ENABLE_SSL) && config.getBoolean(ExecConstants.HTTP_ENABLE_SSL);
    +    // For testing we will mock up a hadoop configuration, however for regular use, we find the actual hadoop config.
    +    boolean enableHadoopConfig = config.getBoolean(ExecConstants.SSL_USE_HADOOP_CONF);
    +    if (enableHadoopConfig && this instanceof SSLConfigServer) {
    +      if (hadoopConfig == null) {
    +        this.hadoopConfig = new Configuration(); // get hadoop configuration
    +      } else {
    +        this.hadoopConfig = hadoopConfig;
    +      }
    +      String hadoopSSLConfigFile =
    +          this.hadoopConfig.get(resolveHadoopPropertyName(HADOOP_SSL_CONF_TPL_KEY, mode));
    +      logger.debug("Using Hadoop configuration for SSL");
    +      logger.debug("Hadoop SSL configuration file: {}", hadoopSSLConfigFile);
    +      this.hadoopConfig.addResource(hadoopSSLConfigFile);
    +    } else {
    +      this.hadoopConfig = null;
    +    }
    +  }
    +
    +  protected String getConfigParam(String name, String hadoopName) {
    +    String value = "";
    +    if (hadoopConfig != null) {
    +      value = getHadoopConfigParam(hadoopName);
    +    }
    +    if (value.isEmpty() && config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getHadoopConfigParam(String name) {
    +    Preconditions.checkArgument(this.hadoopConfig != null);
    +    String value = "";
    +    value = hadoopConfig.get(name, "");
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String getConfigParamWithDefault(String name, String defaultValue) {
    +    String value = "";
    +    if (config.hasPath(name)) {
    +      value = config.getString(name);
    +    }
    +    if (value.isEmpty()) {
    +      value = defaultValue;
    +    }
    +    value = value.trim();
    +    return value;
    +  }
    +
    +  protected String resolveHadoopPropertyName(String nameTemplate, SSLFactory.Mode mode) {
    +    return MessageFormat.format(nameTemplate, mode.toString().toLowerCase());
    +  }
    +
    +  public abstract void validateKeyStore() throws DrillException;
    +
    +  public abstract SslContext initSslContext() throws DrillException;
    +
    +  public abstract SSLContext initSSLContext() throws DrillException;
    --- End diff --
    
    May be change these functions name to _initializeNettySslContex_ and _initializeJDKSslContext_ ? Also it would be great to add a comment explaining why we have 2 separate methods for SSLContext creation. 


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/950
  
    Rebased again. Also updated protobuf files. 
    If there are no further comments, I'll be merging this PR in.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142691920
  
    --- Diff: exec/java-exec/pom.xml ---
    @@ -589,6 +669,17 @@
     
       <build>
     
    +    <extensions>
    +      <!--
    +      Include the os-maven-plugin to get os.detected.classifier
    +      -->
    +      <extension>
    +        <groupId>kr.motd.maven</groupId>
    +        <artifactId>os-maven-plugin</artifactId>
    +        <version>1.4.0.Final</version>
    --- End diff --
    
    Could not build Drill project on centos. Failed with exception:
    `[ERROR] Failed to execute goal org.apache.maven.plugins:maven-shade-plugin:2.4.1:shade (default) on project drill-jdbc-all: Error creating shaded jar: The name "os.detected.release.like."centos"" is not legal for JDOM/XML elements: XML names cannot contain the character """. -> [Help 1]`
    
    It seems we need to use `1.5.0.Final` version (example of similar problem - https://github.com/apache/beam/pull/2391).



---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141215252
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,452 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::~ConnectionEndpoint(){
    +    if(m_pError!=NULL){
    +        delete m_pError; m_pError=NULL;
    +    }
    +}
    +
    +connectionStatus_t ConnectionEndpoint::getDrillbitEndpoint(){
    +    connectionStatus_t ret=CONN_SUCCESS;
    +    if(!m_connectString.empty()){
    +        parseConnectString();
    +        if(m_protocol.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, "<invalid_string>"));
    +        }
    +        if(isZookeeperConnection()){
    +            if((ret=getDrillbitEndpointFromZk())!=CONN_SUCCESS){
    +                return ret;
    +            }
    +        }else if(!this->isDirectConnection()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, this->getProtocol().c_str()));
    +        }
    +    }else{
    +        if(m_host.empty() || m_port.empty()){
    +            return handleError(CONN_INVALID_INPUT, getMessage(ERR_CONN_NOCONNSTR));
    +        }
    +    }
    +    return ret;
    +}
    +
    +void ConnectionEndpoint::parseConnectString(){
    +    boost::regex connStrExpr("(.*)=(.*):([0-9]+)(?:/(.+))?");
    --- End diff --
    
    Haven't reviewed this change based on regex change discussed in person.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141474850
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/TestUserBitSSLServer.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.rpc.user.security;
    +
    +import com.typesafe.config.ConfigValueFactory;
    +import junit.framework.TestCase;
    +import org.apache.drill.BaseTestQuery;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +import org.junit.AfterClass;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.text.MessageFormat;
    +import java.util.Properties;
    +
    +import static org.apache.drill.exec.ssl.SSLConfig.HADOOP_SSL_CONF_TPL_KEY;
    +import static org.junit.Assert.assertEquals;
    +
    +public class TestUserBitSSLServer extends BaseTestQuery {
    +  private static final org.slf4j.Logger logger =
    +      org.slf4j.LoggerFactory.getLogger(TestUserBitSSLServer.class);
    +
    +  private static DrillConfig sslConfig;
    +  private static Properties initProps; // initial client properties
    +  private static ClassLoader classLoader;
    +  private static String ksPath;
    +  private static String tsPath;
    +  private static String emptyTSPath;
    +
    +  @BeforeClass
    +  public static void setupTest() throws Exception {
    +
    +    classLoader = TestUserBitSSLServer.class.getClassLoader();
    +    ksPath = new File(classLoader.getResource("ssl/keystore.ks").getFile()).getAbsolutePath();
    +    tsPath = new File(classLoader.getResource("ssl/truststore.ks").getFile()).getAbsolutePath();
    +    emptyTSPath = new File(classLoader.getResource("ssl/emptytruststore.ks").getFile()).getAbsolutePath();
    +    sslConfig = new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties())
    +        .withValue(ExecConstants.USER_SSL_ENABLED, ConfigValueFactory.fromAnyRef(true))
    +        .withValue(ExecConstants.SSL_KEYSTORE_TYPE, ConfigValueFactory.fromAnyRef("JKS"))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH, ConfigValueFactory.fromAnyRef(ksPath))
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD, ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD, ConfigValueFactory.fromAnyRef("drill123"))
    +        .withValue(ExecConstants.SSL_PROTOCOL, ConfigValueFactory.fromAnyRef("TLSv1.2")), false);
    +    initProps = new Properties();
    +    initProps.setProperty(DrillProperties.ENABLE_TLS, "true");
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PATH, tsPath);
    +    initProps.setProperty(DrillProperties.TRUSTSTORE_PASSWORD, "drill123");
    +    initProps.setProperty(DrillProperties.DISABLE_HOST_VERIFICATION, "true");
    +  }
    +
    +  @AfterClass
    +  public static void cleanTest() throws Exception {
    +    DrillConfig restoreConfig =
    +        new DrillConfig(DrillConfig.create(cloneDefaultTestConfigProperties()), false);
    +    updateTestCluster(1, restoreConfig);
    +  }
    +
    +  @Test
    +  public void testInvalidKeystorePath() throws Exception {
    +    DrillConfig testConfig = new DrillConfig(DrillConfig.create(sslConfig)
    +        .withValue(ExecConstants.SSL_KEYSTORE_PATH, ConfigValueFactory.fromAnyRef("/bad/path")),
    +        false);
    +
    +    // Start an SSL enabled cluster
    +    boolean failureCaught = false;
    +    try {
    +      updateTestCluster(1, testConfig, initProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testInvalidKeystorePassword() throws Exception {
    +    DrillConfig testConfig = new DrillConfig(DrillConfig.create(sslConfig)
    +        .withValue(ExecConstants.SSL_KEYSTORE_PASSWORD, ConfigValueFactory.fromAnyRef("badpassword")),
    +        false);
    +
    +    // Start an SSL enabled cluster
    +    boolean failureCaught = false;
    +    try {
    +      updateTestCluster(1, testConfig, initProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    +  @Test
    +  public void testInvalidKeyPassword() throws Exception {
    +    DrillConfig testConfig = new DrillConfig(DrillConfig.create(sslConfig)
    +        .withValue(ExecConstants.SSL_KEY_PASSWORD, ConfigValueFactory.fromAnyRef("badpassword")),
    +        false);
    +
    +    // Start an SSL enabled cluster
    +    boolean failureCaught = false;
    +    try {
    +      updateTestCluster(1, testConfig, initProps);
    +    } catch (Exception e) {
    +      failureCaught = true;
    +    }
    +    assertEquals(failureCaught, true);
    +  }
    +
    --- End diff --
    
    Sure


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140395374
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java ---
    @@ -70,22 +78,80 @@
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
       private static final String SERVER_NAME = "Apache Drill Server";
     
    +  private final BootStrapContext bootStrapContext;
    +  private final BufferAllocator allocator;
       private final UserConnectionConfig config;
    +  private final SSLConfig sslConfig;
    +  private Channel sslChannel;
       private final UserWorker userWorker;
     
       public UserServer(BootStrapContext context, BufferAllocator allocator, EventLoopGroup eventLoopGroup,
                         UserWorker worker) throws DrillbitStartupException {
         super(UserRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
             allocator.getAsByteBufAllocator(),
             eventLoopGroup);
    +    this.bootStrapContext = context;
    +    this.allocator = allocator;
         this.config = new UserConnectionConfig(allocator, context, new UserServerRequestHandler(worker));
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder()
    +          .config(bootStrapContext.getConfig())
    +          .mode(SSLFactory.Mode.SERVER)
    +          .initializeSSLContext(true)
    +          .validateKeyStore(true)
    +          .build();
    +    } catch (DrillException e) {
    +      throw new DrillbitStartupException(e.getMessage(), e.getCause());
    +    }
         this.userWorker = worker;
     
         // Initialize Singleton instance of UserRpcMetrics.
         ((UserRpcMetrics)UserRpcMetrics.getInstance()).initialize(config.isEncryptionEnabled(), allocator);
       }
     
       @Override
    +  protected void setupSSL(ChannelPipeline pipe) {
    +    if (sslConfig.isUserSslEnabled()) {
    +
    +      SSLEngine sslEngine = sslConfig.createSSLEngine(allocator, null, 0);
    --- End diff --
    
    replace allocator with `config.getAllocator()`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142822091
  
    --- Diff: contrib/native/client/readme.linux ---
    @@ -84,6 +84,21 @@ OR
         ln -svf libboost_filesystem.a libboost_filesystem-mt.a
         ln -svf libboost_date_time.a libboost_date_time-mt.a
     
    +5) Install or  build Cyrus SASL 
    +   To Install 
    +       yum install cyrus-sasl-devel
    +       yum install cyrus-sasl-gssapi
    +   libs are installed in /usr/lib64/sasl2
    +   includes are installed in /usr/include
    +
    +   To build your own 
    +   See readme.sasl for instructions
    +
    +6) Install OpenSSL
    +   yum install openssl
    --- End diff --
    
    Yes, probably not needed. But I left it in there, just in case. Yum can easily figure it out :)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142821994
  
    --- Diff: contrib/native/client/readme.linux ---
    @@ -84,6 +84,21 @@ OR
         ln -svf libboost_filesystem.a libboost_filesystem-mt.a
         ln -svf libboost_date_time.a libboost_date_time-mt.a
     
    +5) Install or  build Cyrus SASL 
    +   To Install 
    +       yum install cyrus-sasl-devel
    --- End diff --
    
    done


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142728092
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,448 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    --- End diff --
    
    we should probably copy it for safety, no? but maybe better would be to have factory method to do the parsing and fill all the fields instead of leaving some unassigned (with random values since there's no automatic initialization in C++?)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140602937
  
    --- Diff: distribution/src/resources/drill-override-example.conf ---
    @@ -222,7 +222,35 @@ drill.exec: {
       # Full workspace name should be indicated (including schema and workspace separated by dot).
       # Workspace MUST be file-based and writable. Workspace name is case-sensitive.
       default_temporary_workspace: "dfs.tmp"
    +
    +  # Enable and provide additional parameters for Client-Server communication over SSL
    +  # see also the javax.net.ssl parameters below
    +  security.user.encryption.ssl: {
    +    #Set this to true to enable all client server communication to occur over SSL.
    +    enabled: false,
    +    #key password is optional if it is the same as the keystore password
    +    keyPassword: "key_passwd",
    +    #Optional handshakeTimeout in milliseconds. Default is 10000 ms (10 seconds)
    +    handshakeTimeout: 10000,
    +    #protocol is optional. Drill will default to TLSv1.2
    +    protocol: "TLSv1.2"
    +  }
    +}
    +
    +# The SSL parameters below need to be set for custom transport layer settings. These are used by
    +# both the WebServer (for HTTPS) and for Client-Server communication over SSL.
    +javax.net.ssl {
    --- End diff --
    
    Removed these altogether. I included these because initially these were also in drill-module.conf


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142821955
  
    --- Diff: contrib/native/client/example/querySubmitter.cpp ---
    @@ -390,11 +404,26 @@ int main(int argc, char* argv[]) {
             if(auth.length()>0){
                 props.setProperty(USERPROP_AUTH_MECHANISM, auth);
             }
    +        if(enableSSL.length()>0){
    --- End diff --
    
    Fixed indent. I left the check as before so that it is consistent with the previous code.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140132632
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java ---
    @@ -102,19 +115,78 @@
       // these are used for authentication
       private volatile List<String> serverAuthMechanisms = null;
       private volatile boolean authComplete = true;
    +  private SSLConfig sslConfig;
    +  private Channel sslChannel;
    +  private DrillbitEndpoint endpoint;
     
       public UserClient(String clientName, DrillConfig config, boolean supportComplexTypes,
    -      BufferAllocator allocator, EventLoopGroup eventLoopGroup, Executor eventExecutor) {
    -    super(
    -        UserRpcConfig.getMapping(config, eventExecutor),
    -        allocator.getAsByteBufAllocator(),
    -        eventLoopGroup,
    -        RpcType.HANDSHAKE,
    -        BitToUserHandshake.class,
    -        BitToUserHandshake.PARSER);
    +      BufferAllocator allocator, EventLoopGroup eventLoopGroup, Executor eventExecutor,
    +      DrillbitEndpoint endpoint) throws NonTransientRpcException {
    +    super(UserRpcConfig.getMapping(config, eventExecutor), allocator.getAsByteBufAllocator(),
    +        eventLoopGroup, RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER);
    +    this.endpoint = endpoint; // save the endpoint; it might be needed by SSL init.
         this.clientName = clientName;
         this.allocator = allocator;
         this.supportComplexTypes = supportComplexTypes;
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder().config(config).mode(SSLFactory.Mode.CLIENT)
    +          .initializeSSLContext(true).validateKeyStore(false).build();
    +    } catch (DrillException e) {
    --- End diff --
    
    So based on comment in previous commit if we don't pass the info object which contains the Connection URL parameters inside DrillConfig to keep both separate then that will work well here as well. We can do the following:
    
    1) Add a method in SSLConfigBuilder to accept Properties type config as well not just DrillConfig.
    2) For SSLFactory.Mode.CLIENT we will always pass an instance of Properties type config whereas for SSLFactory.Mode.SERVER we will always pass an instance of DrillConfig. This check can be enforced inside the builder.build()
    3) Inside build() method when mode is client we are referencing all the DrillProperties inside DrillConfig which actually is part of the instance of Properties object passed to connect call. But with above change it will be consistent.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140400292
  
    --- Diff: exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicServer.java ---
    @@ -105,6 +109,25 @@ protected void initChannel(SocketChannel ch) throws Exception {
     //     }
       }
     
    +  // Adds a SSL handler if enabled. Required only for client and server communications, so
    +  // a real implementation is only available for UserServer
    +  protected void setupSSL(ChannelPipeline pipe) {
    +    // Do nothing
    +  }
    +
    +  protected boolean isSslEnabled() {
    +    return false;
    +  }
    +
    +  // Save the SslChannel after the SSL handshake so it can be closed later
    +  public void setSslChannel(Channel c) {
    +    return;
    +  }
    +
    +  protected void closeSSL() {
    +    return;
    +  }
    --- End diff --
    
    redundant `return` statement in both `closeSSL` and `setSSLChannel`.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141732868
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            connectionStatus_t validateConnectionString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    +                    return boost::asio::ssl::context::sslv23;
    +                } else if (version == "tlsv1") {
    +                    return boost::asio::ssl::context::tlsv1;
    +                } else if (version == "sslv3") {
    +                    return boost::asio::ssl::context::sslv3;
    +                } else {
    +                    return boost::asio::ssl::context::tlsv12;
    +                }
    +            }
    +
    +        SSLChannelContext(DrillUserProperties *props, boost::asio::ssl::context::method tlsVersion, boost::asio::ssl::verify_mode verifyMode) :
    +                ChannelContext(props),
    +                m_SSLContext(tlsVersion) {
    +                m_SSLContext.set_default_verify_paths();
    +                m_SSLContext.set_options(
    +                        boost::asio::ssl::context::default_workarounds
    +                        | boost::asio::ssl::context::no_sslv2
    +                        | boost::asio::ssl::context::single_dh_use
    +                        );
    +                m_SSLContext.set_verify_mode(verifyMode);
    +            };
    +            ~SSLChannelContext(){};
    +            boost::asio::ssl::context& getSslContext(){ return m_SSLContext;}
    +        private:
    +            boost::asio::ssl::context m_SSLContext;
    +    };
    +
    +    typedef ChannelContext ChannelContext_t; 
    +    typedef SSLChannelContext SSLChannelContext_t; 
    +
    +    class ChannelContextFactory{
    +        public:
    +            static ChannelContext_t* getChannelContext(channelType_t t, DrillUserProperties* props);
    +    };
    +
    +    /***
    +     * The Channel class encapsulates a connection to a drillbit. Based on 
    +     * the connection string and the options, the connection will be either 
    +     * a simple socket or a socket using an ssl stream. The class also encapsulates
    +     * connecting to a drillbit directly or thru zookeeper.
    +     * The channel class owns the socket and the io_service that the applications
    +     * will use to communicate with the server.
    +     ***/
    +    class Channel{
    +        public: 
    +            Channel(const char* connStr);
    +            Channel(const char* host, const char* port);
    +            Channel(boost::asio::io_service& ioService, const char* connStr);
    +            Channel(boost::asio::io_service& ioService, const char* host, const char* port);
    +            virtual ~Channel();
    +            virtual connectionStatus_t init(ChannelContext_t* context)=0;
    --- End diff --
    
    In general I prefer to return an enum that is more descriptive. I could return a channel specific enum, but a channel really encapsulates connection capability and so connection status seemed like the right enum.
    Also it minimized the change to the connect call in DrillClientImpl where this is used.
    I started to change it, but the validation checks in the init method also set a connection error, so I finally left it alone.  


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140124213
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
    @@ -126,10 +138,10 @@
       String HTTP_SESSION_MEMORY_RESERVATION = "drill.exec.http.session.memory.reservation";
       String HTTP_SESSION_MEMORY_MAXIMUM = "drill.exec.http.session.memory.maximum";
       String HTTP_SESSION_MAX_IDLE_SECS = "drill.exec.http.session_max_idle_secs";
    -  String HTTP_KEYSTORE_PATH = "drill.exec.ssl.keyStorePath";
    -  String HTTP_KEYSTORE_PASSWORD = "drill.exec.ssl.keyStorePassword";
    -  String HTTP_TRUSTSTORE_PATH = "drill.exec.ssl.trustStorePath";
    -  String HTTP_TRUSTSTORE_PASSWORD = "drill.exec.ssl.trustStorePassword";
    +  String HTTP_KEYSTORE_PATH = SSL_KEYSTORE_PATH;
    +  String HTTP_KEYSTORE_PASSWORD = SSL_KEYSTORE_PASSWORD;
    +  String HTTP_TRUSTSTORE_PATH = SSL_TRUSTSTORE_PATH;
    +  String HTTP_TRUSTSTORE_PASSWORD = SSL_TRUSTSTORE_PASSWORD;
    --- End diff --
    
    Why not just keep _SSL_KEYSTORE_ and _SSL_TRUSTSTORE_ constants and remove the HTTP ones ?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140623048
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java ---
    @@ -102,19 +115,78 @@
       // these are used for authentication
       private volatile List<String> serverAuthMechanisms = null;
       private volatile boolean authComplete = true;
    +  private SSLConfig sslConfig;
    +  private Channel sslChannel;
    --- End diff --
    
    I don't think you have to store the sslChannel reference explicitly here to make sure it's closed. The connection wrapper like AbstractRemoteConnection will already have reference to channel object and will take care of closing it.
    Also that path is taking care of channel close both in graceful (explicitly close being called on client) and failure scenario (in which case Netty channelClosedHandler will be invoked).
    
    Same for server side.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140397986
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java ---
    @@ -70,22 +78,80 @@
       private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
       private static final String SERVER_NAME = "Apache Drill Server";
     
    +  private final BootStrapContext bootStrapContext;
    +  private final BufferAllocator allocator;
       private final UserConnectionConfig config;
    +  private final SSLConfig sslConfig;
    +  private Channel sslChannel;
       private final UserWorker userWorker;
     
       public UserServer(BootStrapContext context, BufferAllocator allocator, EventLoopGroup eventLoopGroup,
                         UserWorker worker) throws DrillbitStartupException {
         super(UserRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
             allocator.getAsByteBufAllocator(),
             eventLoopGroup);
    +    this.bootStrapContext = context;
    +    this.allocator = allocator;
         this.config = new UserConnectionConfig(allocator, context, new UserServerRequestHandler(worker));
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder()
    +          .config(bootStrapContext.getConfig())
    +          .mode(SSLFactory.Mode.SERVER)
    +          .initializeSSLContext(true)
    +          .validateKeyStore(true)
    +          .build();
    +    } catch (DrillException e) {
    +      throw new DrillbitStartupException(e.getMessage(), e.getCause());
    +    }
         this.userWorker = worker;
     
         // Initialize Singleton instance of UserRpcMetrics.
         ((UserRpcMetrics)UserRpcMetrics.getInstance()).initialize(config.isEncryptionEnabled(), allocator);
       }
     
       @Override
    +  protected void setupSSL(ChannelPipeline pipe) {
    +    if (sslConfig.isUserSslEnabled()) {
    +
    +      SSLEngine sslEngine = sslConfig.createSSLEngine(allocator, null, 0);
    +      sslEngine.setUseClientMode(false);
    +
    +      // No need for client side authentication (HTTPS like behaviour)
    +      sslEngine.setNeedClientAuth(false);
    +
    +      // set Security property jdk.certpath.disabledAlgorithms  to disable specific ssl algorithms
    +      sslEngine.setEnabledProtocols(sslEngine.getEnabledProtocols());
    +
    +      // set Security property jdk.tls.disabledAlgorithms to disable specific cipher suites
    +      sslEngine.setEnabledCipherSuites(sslEngine.getEnabledCipherSuites());
    +      sslEngine.setEnableSessionCreation(true);
    +
    --- End diff --
    
    All these setup of sslEngine can be moved to `SSLConfigServer:createSSLEngine(..)` and same thing for client side setupSSL which can be moved to `SSLConfigClient::createSSLEngine(..)`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140333837
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java ---
    @@ -102,19 +115,78 @@
       // these are used for authentication
       private volatile List<String> serverAuthMechanisms = null;
       private volatile boolean authComplete = true;
    +  private SSLConfig sslConfig;
    +  private Channel sslChannel;
    +  private DrillbitEndpoint endpoint;
     
       public UserClient(String clientName, DrillConfig config, boolean supportComplexTypes,
    -      BufferAllocator allocator, EventLoopGroup eventLoopGroup, Executor eventExecutor) {
    -    super(
    -        UserRpcConfig.getMapping(config, eventExecutor),
    -        allocator.getAsByteBufAllocator(),
    -        eventLoopGroup,
    -        RpcType.HANDSHAKE,
    -        BitToUserHandshake.class,
    -        BitToUserHandshake.PARSER);
    +      BufferAllocator allocator, EventLoopGroup eventLoopGroup, Executor eventExecutor,
    +      DrillbitEndpoint endpoint) throws NonTransientRpcException {
    +    super(UserRpcConfig.getMapping(config, eventExecutor), allocator.getAsByteBufAllocator(),
    +        eventLoopGroup, RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER);
    +    this.endpoint = endpoint; // save the endpoint; it might be needed by SSL init.
         this.clientName = clientName;
         this.allocator = allocator;
         this.supportComplexTypes = supportComplexTypes;
    +    this.sslChannel = null;
    +    try {
    +      this.sslConfig = new SSLConfigBuilder().config(config).mode(SSLFactory.Mode.CLIENT)
    +          .initializeSSLContext(true).validateKeyStore(false).build();
    +    } catch (DrillException e) {
    +      throw new NonTransientRpcException(e.getMessage());
    --- End diff --
    
    The exception will be thrown if there is any issue with passed parameters for SSL so it would be better if we change this exception to InvalidConnectionInfoException.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142827694
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigClient.java ---
    @@ -0,0 +1,273 @@
    +/*
    + * 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.ssl;
    +
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslContextBuilder;
    +import io.netty.handler.ssl.SslProvider;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.config.DrillProperties;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.SSLParameters;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.util.Properties;
    +
    +public class SSLConfigClient extends SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfigClient.class);
    +
    +  Properties properties;
    --- End diff --
    
    OK


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140396852
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigBuilder.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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.ssl;
    +
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillConfigurationException;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +
    +public class SSLConfigBuilder {
    +
    +  private static final org.slf4j.Logger logger =
    --- End diff --
    
    Please remove this. Not used anywhere.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140611274
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfigBuilder.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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.ssl;
    +
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillConfigurationException;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +
    +public class SSLConfigBuilder {
    +
    +  private static final org.slf4j.Logger logger =
    --- End diff --
    
    I like to include loggers in classes so that they are always available for future use. You can never have too many logger, I always say. But, since you asked, I will remove it.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140357506
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ssl/SSLConfig.java ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.ssl;
    +
    +import com.google.common.base.Preconditions;
    +import io.netty.handler.ssl.SslContext;
    +import io.netty.handler.ssl.SslProvider;
    +import io.netty.handler.ssl.util.InsecureTrustManagerFactory;
    +import org.apache.drill.common.config.DrillConfig;
    +import org.apache.drill.common.exceptions.DrillException;
    +import org.apache.drill.exec.ExecConstants;
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.security.ssl.SSLFactory;
    +
    +import javax.net.ssl.KeyManagerFactory;
    +import javax.net.ssl.SSLContext;
    +import javax.net.ssl.SSLEngine;
    +import javax.net.ssl.TrustManagerFactory;
    +import java.io.FileInputStream;
    +import java.io.InputStream;
    +import java.security.KeyStore;
    +import java.text.MessageFormat;
    +
    +public abstract class SSLConfig {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SSLConfig.class);
    +
    +  public static final String DEFAULT_SSL_PROVIDER = "JDK"; // JDK or OPENSSL
    +  public static final String DEFAULT_SSL_PROTOCOL = "TLSv1.2";
    +  public static final int DEFAULT_SSL_HANDSHAKE_TIMEOUT_MS = 10 * 1000; // 10 seconds
    +
    +  protected final boolean httpsEnabled;
    +  protected final DrillConfig config;
    +  protected final Configuration hadoopConfig;
    +
    +  // Either the Netty SSL context or the JDK SSL context will be initialized
    +  // The JDK SSL context is use iff the useSystemTrustStore setting is enabled.
    +  protected SslContext nettySslContext;
    +  protected SSLContext jdkSSlContext;
    +
    +  private static final boolean isWindows = System.getProperty("os.name").toLowerCase().indexOf("win") >= 0;
    +  private static final boolean isMacOs = System.getProperty("os.name").toLowerCase().indexOf("mac") >= 0;
    +
    +  public static final String HADOOP_SSL_CONF_TPL_KEY = "hadoop.ssl.{0}.conf";
    +  public static final String HADOOP_SSL_KEYSTORE_LOCATION_TPL_KEY = "ssl.{0}.keystore.location";
    +  public static final String HADOOP_SSL_KEYSTORE_PASSWORD_TPL_KEY = "ssl.{0}.keystore.password";
    +  public static final String HADOOP_SSL_KEYSTORE_TYPE_TPL_KEY = "ssl.{0}.keystore.type";
    +  public static final String HADOOP_SSL_KEYSTORE_KEYPASSWORD_TPL_KEY =
    +      "ssl.{0}.keystore.keypassword";
    +  public static final String HADOOP_SSL_TRUSTSTORE_LOCATION_TPL_KEY = "ssl.{0}.truststore.location";
    +  public static final String HADOOP_SSL_TRUSTSTORE_PASSWORD_TPL_KEY = "ssl.{0}.truststore.password";
    +  public static final String HADOOP_SSL_TRUSTSTORE_TYPE_TPL_KEY = "ssl.{0}.truststore.type";
    +
    +  public SSLConfig(DrillConfig config, Configuration hadoopConfig, SSLFactory.Mode mode)
    +      throws DrillException {
    --- End diff --
    
    doesn't throws exception in any case so we can remove this from signature.


---

[GitHub] drill issue #950: DRILL-5431: SSL Support

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/950
  
    @parthchandra could you please resolve the conflicts?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141215569
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            connectionStatus_t validateConnectionString();
    --- End diff --
    
    Not seeing any implementation of this function: `validateConnectionString`


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r141740253
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,237 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            connectionStatus_t validateConnectionString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    +                if(version.empty()){
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv12") {
    +                    return boost::asio::ssl::context::tlsv12;
    +                } else if (version == "tlsv11") {
    +                    return boost::asio::ssl::context::tlsv11;
    +                } else if (version == "sslv23") {
    +                    return boost::asio::ssl::context::sslv23;
    +                } else if (version == "tlsv1") {
    +                    return boost::asio::ssl::context::tlsv1;
    +                } else if (version == "sslv3") {
    +                    return boost::asio::ssl::context::sslv3;
    +                } else {
    +                    return boost::asio::ssl::context::tlsv12;
    +                }
    +            }
    +
    +        SSLChannelContext(DrillUserProperties *props, boost::asio::ssl::context::method tlsVersion, boost::asio::ssl::verify_mode verifyMode) :
    +                ChannelContext(props),
    +                m_SSLContext(tlsVersion) {
    +                m_SSLContext.set_default_verify_paths();
    +                m_SSLContext.set_options(
    +                        boost::asio::ssl::context::default_workarounds
    +                        | boost::asio::ssl::context::no_sslv2
    +                        | boost::asio::ssl::context::single_dh_use
    +                        );
    +                m_SSLContext.set_verify_mode(verifyMode);
    +            };
    +            ~SSLChannelContext(){};
    +            boost::asio::ssl::context& getSslContext(){ return m_SSLContext;}
    +        private:
    +            boost::asio::ssl::context m_SSLContext;
    +    };
    +
    +    typedef ChannelContext ChannelContext_t; 
    +    typedef SSLChannelContext SSLChannelContext_t; 
    +
    +    class ChannelContextFactory{
    +        public:
    +            static ChannelContext_t* getChannelContext(channelType_t t, DrillUserProperties* props);
    +    };
    +
    +    /***
    +     * The Channel class encapsulates a connection to a drillbit. Based on 
    +     * the connection string and the options, the connection will be either 
    +     * a simple socket or a socket using an ssl stream. The class also encapsulates
    +     * connecting to a drillbit directly or thru zookeeper.
    +     * The channel class owns the socket and the io_service that the applications
    +     * will use to communicate with the server.
    +     ***/
    +    class Channel{
    +        public: 
    +            Channel(const char* connStr);
    +            Channel(const char* host, const char* port);
    +            Channel(boost::asio::io_service& ioService, const char* connStr);
    +            Channel(boost::asio::io_service& ioService, const char* host, const char* port);
    +            virtual ~Channel();
    +            virtual connectionStatus_t init(ChannelContext_t* context)=0;
    +            connectionStatus_t connect();
    +            connectionStatus_t protocolClose();
    +            template <typename SettableSocketOption> void setOption(SettableSocketOption& option);
    +            DrillClientError* getError(){ return m_pError;}
    +            void close(){ 
    +                if(m_state==CHANNEL_INITIALIZED||m_state==CHANNEL_CONNECTED){
    +                    m_pSocket->protocolClose();
    +                    m_state=CHANNEL_CLOSED;
    +                }
    +            } // Not OK to use the channel after this call. 
    +
    +            boost::asio::io_service& getIOService(){
    +                return m_ioService;
    +            }
    +
    +            // returns a reference to the underlying socket 
    +            // This access should really be removed and encapsulated in calls that 
    +            // manage async_send and async_recv 
    +            // Until then we will let DrillClientImpl have direct access
    +            streamSocket_t& getInnerSocket(){
    +                return m_pSocket->getInnerSocket();
    +            }
    +            
    +            AsioStreamSocket& getSocketStream(){
    +                return *m_pSocket;
    +            }
    +
    +            ConnectionEndpoint* getEndpoint(){return m_pEndpoint;}
    +
    +        protected:
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            boost::asio::io_service& m_ioService;
    +            boost::asio::io_service m_ioServiceFallback; // used if m_ioService is not provided
    +            AsioStreamSocket* m_pSocket;
    +            ConnectionEndpoint *m_pEndpoint;
    +            ChannelContext_t *m_pContext;
    +
    +        private:
    +            typedef enum channelState{ 
    +                CHANNEL_UNINITIALIZED=1, 
    +                CHANNEL_INITIALIZED, 
    +                CHANNEL_CONNECTED, 
    +                CHANNEL_CLOSED       
    +            } channelState_t;
    +            
    +            connectionStatus_t connectInternal();
    +            connectionStatus_t protocolHandshake(bool useSystemConfig){
    +                connectionStatus_t status = CONN_SUCCESS;
    +                try{
    +                    m_pSocket->protocolHandshake(useSystemConfig);
    +                } catch (boost::system::system_error e) {
    +                    status = handleError(CONN_HANDSHAKE_FAILED, e.what());
    +                }
    +                return status;
    +            }
    +
    +            channelState_t m_state;
    +            DrillClientError* m_pError;
    +            bool m_ownIoService;
    --- End diff --
    
    Artifact of an earlier version where the caller could pass in an io_service or choose to let the Channel provide a default one.  That led to all sorts of clean up issues and so I simplified it. The flag is not needed any more.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142729342
  
    --- Diff: contrib/native/client/src/clientlib/channel.cpp ---
    @@ -0,0 +1,448 @@
    +/*
    + * 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 <boost/lexical_cast.hpp>
    +#include <boost/regex.hpp>
    +#include "drill/drillConfig.hpp"
    +#include "drill/drillError.hpp"
    +#include "drill/userProperties.hpp"
    +#include "channel.hpp"
    +#include "errmsgs.hpp"
    +#include "logger.hpp"
    +#include "utils.hpp"
    +#include "zookeeperClient.hpp"
    +
    +#include "GeneralRPC.pb.h"
    +
    +namespace Drill{
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* connStr){
    +    m_connectString=connStr;
    +    m_pError=NULL;
    +}
    +
    +ConnectionEndpoint::ConnectionEndpoint(const char* host, const char* port){
    +    m_host=host;
    +    m_port=port;
    +    m_protocol="drillbit"; // direct connection
    --- End diff --
    
    maybe there should be some constants for the protocol types?


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r140603757
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserConnectionConfig.java ---
    @@ -75,10 +76,20 @@
         } else {
           authEnabled = false;
         }
    -
         impersonationManager = !config.getBoolean(ExecConstants.IMPERSONATION_ENABLED)
             ? null
             : new InboundImpersonationManager();
    +
    +    if (config.getBoolean(ExecConstants.USER_SSL_ENABLED)) {
    +      sslEnabled = true;
    +    } else {
    +      sslEnabled = false;
    +    }
    --- End diff --
    
    Sure. (Don't even remember writing this code)


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142825337
  
    --- Diff: contrib/native/client/src/clientlib/wincert.ipp ---
    @@ -0,0 +1,98 @@
    +/*
    + * 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.
    + */
    +
    +#if defined(IS_SSL_ENABLED)
    +
    +#include <openssl/x509.h>
    +#include <openssl/ssl.h>
    +
    +#if defined _WIN32  || defined _WIN64
    +
    +#include <stdio.h>
    +#include <windows.h>
    +#include <wincrypt.h>
    +#include <cryptuiapi.h>
    +#include <iostream>
    +#include <tchar.h>
    +
    +
    +#pragma comment (lib, "crypt32.lib")
    +#pragma comment (lib, "cryptui.lib")
    +
    +#define MY_ENCODING_TYPE  (PKCS_7_ASN_ENCODING | X509_ASN_ENCODING)
    +
    +inline
    +int loadSystemTrustStore(const SSL *ssl, std::string& msg) {
    --- End diff --
    
    And that is how boost does it too. But then checking for the certificate is complicated stuff, so it is easier to just load the certificates from the system store into openssl and let openssl deal with it.
    Anyway, I think that this counts as an enhancement. I can take care of this when I add support for Mac Keychain.


---

[GitHub] drill pull request #950: DRILL-5431: SSL Support

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/950#discussion_r142994068
  
    --- Diff: contrib/native/client/src/clientlib/channel.hpp ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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 CHANNEL_HPP
    +#define CHANNEL_HPP
    +
    +#include "drill/common.hpp"
    +#include "drill/drillClient.hpp"
    +#include "streamSocket.hpp"
    +
    +namespace Drill {
    +
    +class UserProperties;
    +
    +    class ConnectionEndpoint{
    +        public:
    +            ConnectionEndpoint(const char* connStr);
    +            ConnectionEndpoint(const char* host, const char* port);
    +            ~ConnectionEndpoint();
    +
    +            //parse the connection string and set up the host and port to connect to
    +            connectionStatus_t getDrillbitEndpoint();
    +
    +            std::string& getProtocol(){return m_protocol;}
    +            std::string& getHost(){return m_host;}
    +            std::string& getPort(){return m_port;}
    +            DrillClientError* getError(){ return m_pError;};
    +
    +        private:
    +            void parseConnectString();
    +            bool isDirectConnection();
    +            bool isZookeeperConnection();
    +            connectionStatus_t getDrillbitEndpointFromZk();
    +            connectionStatus_t handleError(connectionStatus_t status, std::string msg);
    +
    +            std::string m_connectString;
    +            std::string m_pathToDrill;
    +            std::string m_protocol; 
    +            std::string m_hostPortStr;
    +            std::string m_host;
    +            std::string m_port;
    +
    +            DrillClientError* m_pError;
    +
    +    };
    +
    +    class ChannelContext{
    +        public:
    +            ChannelContext(DrillUserProperties* props):m_properties(props){};
    +            virtual ~ChannelContext(){};
    +            const DrillUserProperties* getUserProperties() const { return m_properties;}
    +        protected:
    +            DrillUserProperties* m_properties;
    +    };
    +
    +    class SSLChannelContext: public ChannelContext{
    +        public:
    +            static boost::asio::ssl::context::method getTlsVersion(std::string version){
    --- End diff --
    
    sorry, it's abbreviation for const ref (I believe std::cref was introduced in C++11, which we are not using sadly, although there's a boost equivalent of course).


---