You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2015/03/24 20:41:09 UTC

[3/5] drill git commit: DRILL-2442: Initial implementation of C++ client support for impersonation.

DRILL-2442: Initial implementation of C++ client support for impersonation.


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/2f2338f3
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/2f2338f3
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/2f2338f3

Branch: refs/heads/master
Commit: 2f2338f3f8b66921d6db223064bb23ff41894486
Parents: e796b91
Author: Parth Chandra <pc...@maprtech.com>
Authored: Mon Mar 16 13:38:46 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Tue Mar 24 12:12:19 2015 -0700

----------------------------------------------------------------------
 .../native/client/example/querySubmitter.cpp    | 24 +++++++++--
 .../native/client/src/clientlib/drillClient.cpp | 42 ++++++++++++++++--
 .../client/src/clientlib/drillClientImpl.cpp    | 33 +++++++++++---
 .../client/src/clientlib/drillClientImpl.hpp    |  2 +-
 .../native/client/src/include/drill/common.hpp  | 32 ++++++++++++++
 .../client/src/include/drill/drillClient.hpp    | 45 +++++++++++++++++++-
 6 files changed, 163 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/2f2338f3/contrib/native/client/example/querySubmitter.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/example/querySubmitter.cpp b/contrib/native/client/example/querySubmitter.cpp
index 2b0f000..bef64bf 100644
--- a/contrib/native/client/example/querySubmitter.cpp
+++ b/contrib/native/client/example/querySubmitter.cpp
@@ -22,7 +22,7 @@
 #include <stdlib.h>
 #include "drill/drillc.hpp"
 
-int nOptions=11;
+int nOptions=13;
 
 struct Option{
     char name[32];
@@ -39,7 +39,9 @@ struct Option{
     {"testCancel", "Cancel the query afterthe first record batch.", false},
     {"syncSend", "Send query only after previous result is received", false},
     {"hshakeTimeout", "Handshake timeout (second).", false},
-    {"queryTimeout", "Query timeout (second).", false}
+    {"queryTimeout", "Query timeout (second).", false},
+    {"user", "Username", false},
+    {"password", "Password", false}
 };
 
 std::map<std::string, std::string> qsOptionValues;
@@ -273,6 +275,8 @@ int main(int argc, char* argv[]) {
         std::string syncSend=qsOptionValues["syncSend"];
         std::string hshakeTimeout=qsOptionValues["hshakeTimeout"];
         std::string queryTimeout=qsOptionValues["queryTimeout"];
+        std::string user=qsOptionValues["user"];
+        std::string password=qsOptionValues["password"];
 
         Drill::QueryType type;
 
@@ -324,7 +328,21 @@ int main(int argc, char* argv[]) {
         if (!queryTimeout.empty()){
             Drill::DrillClientConfig::setQueryTimeout(atoi(queryTimeout.c_str()));
         }
-        if(client.connect(connectStr.c_str(), schema.c_str())!=Drill::CONN_SUCCESS){
+
+        Drill::DrillUserProperties props;
+        if(schema.length()>0){
+            props.setProperty(USERPROP_SCHEMA, schema);
+        }
+        if(user.length()>0){
+            props.setProperty(USERPROP_USERNAME, user);
+        }
+        if(password.length()>0){
+            props.setProperty(USERPROP_PASSWORD, password);
+        }
+
+        props.setProperty("someRandomProperty", "someRandomValue");
+
+        if(client.connect(connectStr.c_str(), &props)!=Drill::CONN_SUCCESS){
             std::cerr<< "Failed to connect with error: "<< client.getError() << " (Using:"<<connectStr<<")"<<std::endl;
             return -1;
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/2f2338f3/contrib/native/client/src/clientlib/drillClient.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClient.cpp b/contrib/native/client/src/clientlib/drillClient.cpp
index 71907e0..90aa555 100644
--- a/contrib/native/client/src/clientlib/drillClient.cpp
+++ b/contrib/native/client/src/clientlib/drillClient.cpp
@@ -17,6 +17,7 @@
  */
 
 
+#include <boost/assign.hpp>
 #include "drill/common.hpp"
 #include "drill/drillClient.hpp"
 #include "drill/recordBatch.hpp"
@@ -119,6 +120,22 @@ logLevel_t DrillClientConfig::getLogLevel(){
     return s_logLevel;
 }
 
+//Using boost assign to initialize maps. 
+const std::map<std::string, uint32_t>  DrillUserProperties::USER_PROPERTIES=boost::assign::map_list_of
+    ( USERPROP_USERNAME,    USERPROP_FLAGS_SERVERPROP|USERPROP_FLAGS_STRING )
+    ( USERPROP_PASSWORD,    USERPROP_FLAGS_SERVERPROP|USERPROP_FLAGS_PASSWORD)
+    ( USERPROP_SCHEMA,      USERPROP_FLAGS_SERVERPROP|USERPROP_FLAGS_STRING)
+    ( USERPROP_USESSL,      USERPROP_FLAGS_BOOLEAN|USERPROP_FLAGS_SSLPROP)
+    ( USERPROP_FILEPATH,    USERPROP_FLAGS_STRING|USERPROP_FLAGS_SSLPROP|USERPROP_FLAGS_FILEPATH)
+    ( USERPROP_FILENAME,    USERPROP_FLAGS_STRING|USERPROP_FLAGS_SSLPROP|USERPROP_FLAGS_FILENAME)
+;
+
+bool DrillUserProperties::validate(std::string& err){
+    bool ret=true;
+    //We can add additional validation for any params here
+    return ret;
+}
+
 RecordIterator::~RecordIterator(){
     if(m_pColDefs!=NULL){
         for(std::vector<Drill::FieldMetadata*>::iterator it=m_pColDefs->begin();
@@ -288,11 +305,30 @@ DrillClient::~DrillClient(){
 connectionStatus_t DrillClient::connect(const char* connectStr, const char* defaultSchema){
     connectionStatus_t ret=CONN_SUCCESS;
     ret=this->m_pImpl->connect(connectStr);
-
-    if(ret==CONN_SUCCESS)
-        ret=this->m_pImpl->validateHandShake(defaultSchema);
+    DrillUserProperties props;
+    std::string schema(defaultSchema);
+    props.setProperty(USERPROP_SCHEMA,  schema);
+    if(ret==CONN_SUCCESS){
+        if(defaultSchema!=NULL){
+            ret=this->m_pImpl->validateHandshake(&props);
+        }else{
+            ret=this->m_pImpl->validateHandshake(NULL);
+        }
+    }
     return ret;
+}
 
+connectionStatus_t DrillClient::connect(const char* connectStr, DrillUserProperties* properties){
+    connectionStatus_t ret=CONN_SUCCESS;
+    ret=this->m_pImpl->connect(connectStr);
+    if(ret==CONN_SUCCESS){
+        if(properties!=NULL){
+            ret=this->m_pImpl->validateHandshake(properties);
+        }else{
+            ret=this->m_pImpl->validateHandshake(NULL);
+        }
+    }
+    return ret;
 }
 
 bool DrillClient::isActive(){

http://git-wip-us.apache.org/repos/asf/drill/blob/2f2338f3/contrib/native/client/src/clientlib/drillClientImpl.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.cpp b/contrib/native/client/src/clientlib/drillClientImpl.cpp
index ea3a7ee..e6f8009 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.cpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.cpp
@@ -270,7 +270,7 @@ void DrillClientImpl::handleHShakeReadTimeout(const boost::system::error_code &
     return;
 }
 
-connectionStatus_t DrillClientImpl::validateHandShake(const char* defaultSchema){
+connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* properties){
 
     DRILL_LOG(LOG_TRACE) << "validateHandShake\n";
 
@@ -279,13 +279,34 @@ connectionStatus_t DrillClientImpl::validateHandShake(const char* defaultSchema)
     u2b.set_rpc_version(DRILL_RPC_VERSION);
     u2b.set_support_listening(true);
 
-    if ( defaultSchema != NULL ){
-        DRILL_LOG(LOG_TRACE) << "defaultSchema = " << defaultSchema << "\n";
+    if(properties != NULL && properties->size()>0){
+        std::string err;
+        if(!properties->validate(err)){
+            DRILL_LOG(LOG_INFO) << "Invalid user input:" << err << std::endl;
+        }
         exec::user::UserProperties* userProperties = u2b.mutable_properties();
-        exec::user::Property* connSchema = userProperties->add_properties();
-        connSchema->set_key("schema");
-        connSchema->set_value(defaultSchema);
+          
+        std::map<char,int>::iterator it;
+        for(size_t i=0; i<properties->size(); i++){
+            std::map<std::string,uint32_t>::const_iterator it=DrillUserProperties::USER_PROPERTIES.find(properties->keyAt(i));
+            if(it==DrillUserProperties::USER_PROPERTIES.end()){
+                DRILL_LOG(LOG_WARNING) << "Connection property ("<< properties->keyAt(i) 
+                    << ") is unknown and is being skipped" << std::endl;
+                continue;
+            }
+            if(IS_BITSET((*it).second,USERPROP_FLAGS_SERVERPROP)){
+                exec::user::Property* connProp = userProperties->add_properties();
+                connProp->set_key(properties->keyAt(i));
+                connProp->set_value(properties->valueAt(i));
+                if(IS_BITSET((*it).second,USERPROP_FLAGS_PASSWORD)){
+                    DRILL_LOG(LOG_INFO) <<  properties->keyAt(i) << ": ********** " << std::endl;
+                }else{
+                    DRILL_LOG(LOG_INFO) << properties->keyAt(i) << ":" << properties->valueAt(i) << std::endl;
+                }
+            }// Server properties
+        }
     }
+
     {
         boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
         uint64_t coordId = this->getNextCoordinationId();

http://git-wip-us.apache.org/repos/asf/drill/blob/2f2338f3/contrib/native/client/src/clientlib/drillClientImpl.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.hpp b/contrib/native/client/src/clientlib/drillClientImpl.hpp
index d287bfc..a6de40b 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.hpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.hpp
@@ -241,7 +241,7 @@ class DrillClientImpl{
         DrillClientError* getError(){ return m_pError;}
         DrillClientQueryResult* SubmitQuery(::exec::shared::QueryType t, const std::string& plan, pfnQueryResultsListener listener, void* listenerCtx);
         void waitForResults();
-        connectionStatus_t validateHandShake(const char* defaultSchema);
+        connectionStatus_t validateHandshake(DrillUserProperties* props);
 
     private:
         friend class DrillClientQueryResult;

http://git-wip-us.apache.org/repos/asf/drill/blob/2f2338f3/contrib/native/client/src/include/drill/common.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/common.hpp b/contrib/native/client/src/include/drill/common.hpp
index e149ed1..dbfa6fe 100644
--- a/contrib/native/client/src/include/drill/common.hpp
+++ b/contrib/native/client/src/include/drill/common.hpp
@@ -58,6 +58,16 @@
 #endif // _WIN32 && !_WIN64
 
 
+//DEPRECATED MACRO
+#if defined(__GNUC__) || defined(__llvm__)
+#define DEPRECATED __attribute__((deprecated))
+#elif defined(_MSC_VER)
+#define DEPRECATED __declspec(deprecated)
+#else
+#pragma message("WARNING: DEPRECATED not available for this compiler")
+#define DEPRECATED
+#endif
+
 namespace Drill {
 
 typedef std::vector<uint8_t> DataBuf;
@@ -118,6 +128,28 @@ typedef enum{
     RET_FAILURE=1
 } ret_t;
 
+
+// User Property Names
+#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
+
+// Bitflags to describe user properties
+// Used in DrillUserProperties::USER_PROPERTIES
+#define USERPROP_FLAGS_SERVERPROP 0x00000001
+#define USERPROP_FLAGS_SSLPROP    0x00000002
+#define USERPROP_FLAGS_PASSWORD   0x00000004
+#define USERPROP_FLAGS_FILENAME   0x00000008
+#define USERPROP_FLAGS_FILEPATH   0x00000010
+#define USERPROP_FLAGS_STRING     0x00000020
+#define USERPROP_FLAGS_BOOLEAN    0x00000040
+
+#define IS_BITSET(val, bit) \
+    ((val&bit)==bit)
+
 } // namespace Drill
 
 #endif

http://git-wip-us.apache.org/repos/asf/drill/blob/2f2338f3/contrib/native/client/src/include/drill/drillClient.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/drillClient.hpp b/contrib/native/client/src/include/drill/drillClient.hpp
index 71a5c80..9289df3 100644
--- a/contrib/native/client/src/include/drill/drillClient.hpp
+++ b/contrib/native/client/src/include/drill/drillClient.hpp
@@ -135,6 +135,29 @@ class DECLSPEC_DRILL_CLIENT DrillClientConfig{
 };
 
 
+class DECLSPEC_DRILL_CLIENT DrillUserProperties{
+    public:
+        static const std::map<std::string, uint32_t> USER_PROPERTIES;
+
+        DrillUserProperties(){};
+
+        void setProperty( std::string propName, std::string propValue){
+            std::pair< std::string, std::string> in = make_pair(propName, propValue);
+            m_properties.push_back(in);
+        }
+
+        size_t size() const { return m_properties.size(); }
+
+        const std::string& keyAt(size_t i) const { return m_properties.at(i).first; }
+
+        const std::string& valueAt(size_t i) const { return m_properties.at(i).second; }
+
+        bool validate(std::string& err);
+
+    private:
+        std::vector< std::pair< std::string, std::string> > m_properties;
+};
+
 /*
  * Handle to the Query submitted for execution.
  * */
@@ -233,6 +256,15 @@ class DECLSPEC_DRILL_CLIENT DrillClient{
         /**
          * Connect the client to a Drillbit using connection string and default schema.
          *
+         * @param[in] connectStr: connection string
+         * @param[in] defaultSchema: default schema (set to NULL and ignore it
+         * if not specified)
+         * @return    connection status
+         */
+        DEPRECATED connectionStatus_t connect(const char* connectStr, const char* defaultSchema=NULL);
+
+        /*  
+         * Connect the client to a Drillbit using connection string and a set of user properties.
          * The connection string format can be found in comments of
          * [DRILL-780](https://issues.apache.org/jira/browse/DRILL-780)
          *
@@ -253,12 +285,21 @@ class DECLSPEC_DRILL_CLIENT DrillClient{
          * local=127.0.0.1:31010
          * ```
          *
+         * User properties is a set of name value pairs. The following properties are recognized:
+         *     schema
+         *     userName
+         *     password
+         *     useSSL [true|false]
+         *     pemLocation
+         *     pemFile
+         *     (see drill/common.hpp for friendly defines and the latest list of supported proeprties)
+         *
          * @param[in] connectStr: connection string
-         * @param[in] defaultSchema: default schema (set to NULL and ignore it
+         * @param[in] properties
          * if not specified)
          * @return    connection status
          */
-        connectionStatus_t connect(const char* connectStr, const char* defaultSchema=NULL);
+        connectionStatus_t connect(const char* connectStr, DrillUserProperties* properties);
 
         /* test whether the client is active */
         bool isActive();