You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by rm...@apache.org on 2016/11/01 00:18:47 UTC

[1/2] incubator-trafodion git commit: TRAFODION-2327 Reduce I/O when loading objects into caches

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master 48f20cc52 -> 6ce6a8ea9


TRAFODION-2327 Reduce I/O when loading objects into caches

For each authorization ID (user, role, or PUBLIC), a bitmap containing the
accumulated privileges (across all grantors) is stored with the object desc.
When the object desc is loaded into cache, the privilege bitmaps associated
with the current user, PUBLIC, and the current users' roles are extracted and
unioned together to calculate the final set of privileges.  This unioned list
is used during privilege checking.

Today, an I/O is performed to retrieve the list of roles granted to the current
user for each object loaded into NATable and NARoutine cache.  Since this list
does not change unless the current user changes (a new session with a different
user) or a grant/revoke role for the current user is performed, these extra
I/O's are not needed.

To remove the extra I/O's for each object, the list of roles will be stored in
the ContextCli.  Therefore, this in-memory role list can be used instead of
rereading metadata.

This checkin creates two new CLI requests:
- GetRoleList - returns the list of roles associated with the user
    If the list exists in ContextCli, it returns the stored values
    If the list does not exist, it retrieves them from Metadata, stores
      them and returns the values
- ResetRoleList - removes the list of roles from ContextCli

The first time GetRoleList is called in a session, the users' roles are
stored in ContextCli.  They remain in memory until the session ends and
restarts as a different user, or another process grants or revokes a role
from the current user.

If another process revokes a role from the current user, a query invalidation
key is created.  When the revoke role query invalidation key for the current
user is detected, ResetRoleList is called.  The next time GetRoleList is called
an updated role list is retrieved from metadata and stored in ContextCli.

If another process grants a role to the current user, there could be two
outcomes.  If the current user already has the privilege from another source
then nothing happens.  If the current user does not have the privilege, then
one recompilation is attempted.  Prior to performing the retry, code was
added to ResetRoleList.  The recompilation then gets the latest role list and
either succeeds or fails depending on the granted privileges.


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/commit/1c8f25b6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/1c8f25b6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/1c8f25b6

Branch: refs/heads/master
Commit: 1c8f25b6e61c47383f43633a4ffa7d0bda766cd9
Parents: 2c3c7c5
Author: Roberta Marton <rm...@edev07.esgyn.local>
Authored: Mon Oct 31 14:28:54 2016 +0000
Committer: Roberta Marton <rm...@edev07.esgyn.local>
Committed: Mon Oct 31 14:28:54 2016 +0000

----------------------------------------------------------------------
 core/sql/cli/Cli.cpp                   |  44 ++++++-
 core/sql/cli/Cli.h                     |   9 +-
 core/sql/cli/CliExtern.cpp             |  81 +++++++++++-
 core/sql/cli/Context.cpp               | 194 ++++++++++++++++++++--------
 core/sql/cli/Context.h                 |  31 +++--
 core/sql/cli/SQLCLIdev.h               |   7 +
 core/sql/common/ComSecurityKey.cpp     |  82 ++++++++++++
 core/sql/common/ComSecurityKey.h       |   5 +
 core/sql/optimizer/BindRelExpr.cpp     |   6 +-
 core/sql/optimizer/NARoutine.cpp       |  26 ++--
 core/sql/optimizer/NATable.cpp         |  67 +++++-----
 core/sql/optimizer/NATable.h           |   2 +-
 core/sql/regress/privs1/TEST120        |   4 +
 core/sql/sqlcomp/CmpMain.cpp           |  39 ++++--
 core/sql/sqlcomp/CmpMain.h             |   1 -
 core/sql/sqlcomp/CmpSeabaseDDLauth.cpp |  18 +++
 core/sql/sqlcomp/CmpSeabaseDDLauth.h   |   6 +-
 17 files changed, 497 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/cli/Cli.cpp
----------------------------------------------------------------------
diff --git a/core/sql/cli/Cli.cpp b/core/sql/cli/Cli.cpp
index 9d79d90..e953c75 100644
--- a/core/sql/cli/Cli.cpp
+++ b/core/sql/cli/Cli.cpp
@@ -6616,7 +6616,6 @@ ComDiagsArea &diags = currContext.diags();
 
 }
 
-
 Lng32 SQLCLI_GetAuthName (
     /*IN*/            CliGlobals *cliGlobals,
     /*IN*/            Lng32       auth_id,
@@ -6716,6 +6715,49 @@ Int32 SQLCLI_GetAuthState (
   return CliEpilogue(cliGlobals, NULL, retcode);
 }
 
+Lng32 SQLCLI_GetRoleList(
+   CliGlobals * cliGlobals,
+   Int32 &numRoles,
+   Int32 *&roleIDs)
+
+{
+   Lng32 retcode = 0;
+
+   // create initial context, if first call, and add module, if any.
+   retcode = CliPrologue(cliGlobals, NULL);
+   if (isERROR(retcode))
+      return retcode;
+
+   ContextCli &currContext = *(cliGlobals->currContext());
+   ComDiagsArea &diags = currContext.diags();
+
+   retcode = currContext.getRoleList(numRoles,roleIDs);
+
+   return CliEpilogue(cliGlobals, NULL, retcode);
+
+}
+
+Lng32 SQLCLI_ResetRoleList(
+   CliGlobals * cliGlobals)
+
+{
+   Lng32 retcode = 0;
+
+   // create initial context, if first call, and add module, if any.
+   retcode = CliPrologue(cliGlobals, NULL);
+   if (isERROR(retcode))
+      return retcode;
+
+   ContextCli &currContext = *(cliGlobals->currContext());
+   ComDiagsArea &diags = currContext.diags();
+
+   retcode = currContext.resetRoleList();
+
+   return CliEpilogue(cliGlobals, NULL, retcode);
+
+}
+
+
 Lng32 SQLCLI_SetSessionAttr(/*IN*/ CliGlobals *cliGlobals,
 			    /*IN SESSIONATTR_TYPE*/ Lng32 attrName,
 			    /*IN OPTIONAL*/         Lng32 numeric_value,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/cli/Cli.h
----------------------------------------------------------------------
diff --git a/core/sql/cli/Cli.h b/core/sql/cli/Cli.h
index 86ddaca..594df3d 100644
--- a/core/sql/cli/Cli.h
+++ b/core/sql/cli/Cli.h
@@ -524,7 +524,6 @@ Lng32 SQLCLI_GetAuthID(
    const char * authName,
    Lng32 & authID);
                             
-                         
 Lng32 SQLCLI_GetAuthName (
     /*IN*/            CliGlobals *cliGlobals,
     /*IN*/            Lng32       auth_id,
@@ -549,6 +548,14 @@ Int32 SQLCLI_GetAuthState (
     /*OUT*/           bool      &authorizationReady,
     /*OUT*/           bool      &auditingEnabled);
 
+Lng32 SQLCLI_GetRoleList(
+   CliGlobals *cliGlobals,
+   Int32 &numRoles,
+   Int32 *&roleIDs);
+                         
+Lng32 SQLCLI_ResetRoleList (
+    /*IN*/            CliGlobals *cliGlobals);
+
 Lng32 SQLCLI_GetUniqueQueryIdAttrs( /*IN*/ CliGlobals *cliGlobals,
 				   /*IN*/    char * queryId,
 				   /*IN*/    Lng32 queryIdLen,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/cli/CliExtern.cpp
----------------------------------------------------------------------
diff --git a/core/sql/cli/CliExtern.cpp b/core/sql/cli/CliExtern.cpp
index 2f179ef..0c7f423 100644
--- a/core/sql/cli/CliExtern.cpp
+++ b/core/sql/cli/CliExtern.cpp
@@ -4490,7 +4490,7 @@ Lng32 retcode;
    return retcode;
 
 }
-
+ 
 
 SQLCLI_LIB_FUNC
 Lng32 SQL_EXEC_GetAuthName_Internal(
@@ -4705,6 +4705,85 @@ Lng32 SQL_EXEC_SetSessionAttr_Internal(
    return retcode;
 }
 
+
+Lng32 SQL_EXEC_GetRoleList(
+   Int32 &numRoles,
+   Int32 *&roleIDs)
+
+{
+
+   Lng32 retcode;
+   CLISemaphore *tmpSemaphore;
+   ContextCli   *threadContext;
+
+   CLI_NONPRIV_PROLOGUE(retcode);
+   try
+   {
+      tmpSemaphore = getCliSemaphore(threadContext);
+      tmpSemaphore->get();
+      threadContext->incrNumOfCliCalls();
+      retcode =
+      SQLCLI_GetRoleList(GetCliGlobals(),
+                         numRoles,
+                         roleIDs);
+   }
+   catch(...)
+   {
+     retcode = -CLI_INTERNAL_ERROR;
+#if defined(_THROW_EXCEPTIONS)
+     if (cliWillThrow())
+       {
+         threadContext->decrNumOfCliCalls();
+         tmpSemaphore->release();
+         throw;
+       }
+#endif
+   }
+
+   threadContext->decrNumOfCliCalls();
+   tmpSemaphore->release();
+   return retcode;
+
+}
+
+ 
+SQLCLI_LIB_FUNC
+Lng32 SQL_EXEC_ResetRoleList_Internal()
+{
+   Lng32 retcode;
+   CLISemaphore *tmpSemaphore;
+   ContextCli   *threadContext;
+
+   CLI_NONPRIV_PROLOGUE(retcode);
+
+   try
+   {
+      tmpSemaphore = getCliSemaphore(threadContext);
+      tmpSemaphore->get();
+      threadContext->incrNumOfCliCalls();
+      retcode =
+      SQLCLI_ResetRoleList(GetCliGlobals());
+   }
+   catch(...)
+   {
+     retcode = -CLI_INTERNAL_ERROR;
+#if defined(_THROW_EXCEPTIONS)
+     if (cliWillThrow())
+       {
+         threadContext->decrNumOfCliCalls();
+         tmpSemaphore->release();
+         throw;
+       }
+#endif
+   }
+
+   threadContext->decrNumOfCliCalls();
+   tmpSemaphore->release();
+   return retcode;
+
+}
+
+ 
 SQLCLI_LIB_FUNC
 Lng32 SQL_EXEC_GetUniqueQueryIdAttrs(
                 /*IN*/    char * uniqueQueryId,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/cli/Context.cpp
----------------------------------------------------------------------
diff --git a/core/sql/cli/Context.cpp b/core/sql/cli/Context.cpp
index ef65ac5..b454a3a 100644
--- a/core/sql/cli/Context.cpp
+++ b/core/sql/cli/Context.cpp
@@ -187,7 +187,9 @@ ContextCli::ContextCli(CliGlobals *cliGlobals)
     cmpContextInfo_(&exHeap_),
     cmpContextInUse_(&exHeap_),
     arkcmpInitFailed_(&exHeap_),
-    trustedRoutines_(&exHeap_)
+    trustedRoutines_(&exHeap_),
+    roleIDs_(NULL),
+    numRoles_(0)
 {
   exHeap_.setJmpBuf(cliGlobals->getJmpBuf());
   cliSemaphore_ = new (&exHeap_) CLISemaphore();
@@ -5073,9 +5075,12 @@ void ContextCli::initializeUserInfoFromOS()
 // *    is the maximum number of characters that can be written to             *
 // *  authNameFromTable.                                                       *
 // *                                                                           *
-// *  <authIDFromTable>              char *                           Out      *
+// *  <authIDFromTable>              int                              Out      *
 // *    passes back the numeric ID of the authorization entry.                 *
 // *                                                                           *
+// *  <roleIDs>                      std::vector<int32_t> &           Out      *
+// *    passes back the list of number ID's for the user                       *
+// *                                                                           *
 // *****************************************************************************
 // *                                                                           *
 // *  Returns: RETCODE                                                         *
@@ -5086,33 +5091,34 @@ void ContextCli::initializeUserInfoFromOS()
 // *****************************************************************************
 
 RETCODE ContextCli::authQuery(
-   AuthQueryType queryType,         
-   const char  * authName,          
-   Int32         authID,            
-   char        * authNameFromTable, 
-   Int32         authNameMaxLen,
-   Int32       & authIDFromTable)   
+   AuthQueryType          queryType,         
+   const char           * authName,          
+   Int32                  authID,            
+   char                 * authNameFromTable, 
+   Int32                  authNameMaxLen,
+   Int32                & authIDFromTable,
+   std::vector<int32_t> & roleIDs)   
    
 {
 
-// We may need to perform a transactional lookup into metadata.
-// The following steps will be taken to manage the
-// transaction. The same steps are used in the Statement class when
-// we read the authorization information
-//
-//  1. Disable autocommit
-//  2. Note whether a transaction is already in progress
-//  3. Do the work
-//  4. Commit the transaction if a new one was started
-//  5. Enable autocommit if it was disabled
-
-// If we hit errors and need to inject the user name in error
-// messages, but the caller passed in a user ID not a name, we will
-// use the string form of the user ID.
-
-const char *nameForDiags = authName;
-char localNameBuf[32];
-char isValidFromUsersTable[3];
+   // We may need to perform a transactional lookup into metadata.
+   // The following steps will be taken to manage the
+   // transaction. The same steps are used in the Statement class when
+   // we read the authorization information
+   //
+   //  1. Disable autocommit
+   //  2. Note whether a transaction is already in progress
+   //  3. Do the work
+   //  4. Commit the transaction if a new one was started
+   //  5. Enable autocommit if it was disabled
+
+   // If we hit errors and need to inject the user name in error
+   // messages, but the caller passed in a user ID not a name, we will
+   // use the string form of the user ID.
+
+   const char *nameForDiags = authName;
+   char localNameBuf[32];
+   char isValidFromUsersTable[3];
 
    if (queryType == USERS_QUERY_BY_USER_ID)
    {
@@ -5120,8 +5126,8 @@ char isValidFromUsersTable[3];
       nameForDiags = localNameBuf;
    }
 
-//  1. Disable autocommit 
-NABoolean autoCommitDisabled = FALSE;
+   //  1. Disable autocommit 
+   NABoolean autoCommitDisabled = FALSE;
 
    if (transaction_->autoCommit())
    {
@@ -5129,16 +5135,16 @@ NABoolean autoCommitDisabled = FALSE;
       transaction_->disableAutoCommit();
    }
 
-//  2. Note whether a transaction is already in progress
-NABoolean txWasInProgress = transaction_->xnInProgress();
+   //  2. Note whether a transaction is already in progress
+   NABoolean txWasInProgress = transaction_->xnInProgress();
 
-//  3. Do the work
-CmpSeabaseDDLauth::AuthStatus authStatus = CmpSeabaseDDLauth::STATUS_GOOD;
-RETCODE result = SUCCESS;
-CmpSeabaseDDLuser userInfo;
-CmpSeabaseDDLauth authInfo;
-CmpSeabaseDDLrole roleInfo;
-CmpSeabaseDDLauth *authInfoPtr = NULL;
+   //  3. Do the work
+   CmpSeabaseDDLauth::AuthStatus authStatus = CmpSeabaseDDLauth::STATUS_GOOD;
+   RETCODE result = SUCCESS;
+   CmpSeabaseDDLuser userInfo;
+   CmpSeabaseDDLauth authInfo;
+   CmpSeabaseDDLrole roleInfo;
+   CmpSeabaseDDLauth *authInfoPtr = NULL;
 
    switch (queryType)
    {
@@ -5176,6 +5182,14 @@ CmpSeabaseDDLauth *authInfoPtr = NULL;
          authStatus = roleInfo.getAuthDetails(authID);
       }
       break;
+
+      case ROLES_QUERY_BY_AUTH_ID:
+      {
+         authInfoPtr = &authInfo;
+         authStatus = authInfo.getRoleIDs(authID, roleIDs);
+      }   
+      break;
+
       default:
       {
          ex_assert(0, "Invalid query type");
@@ -5286,6 +5300,7 @@ RETCODE ContextCli::setDatabaseUserByID(Int32 userID)
 {
   char username[MAX_USERNAME_LEN +1];
   Int32 userIDFromMetadata;
+  std::vector<int32_t> roleIDs;
 
   // See if the USERS row exists
   RETCODE result = authQuery(USERS_QUERY_BY_USER_ID,
@@ -5293,7 +5308,8 @@ RETCODE ContextCli::setDatabaseUserByID(Int32 userID)
                              userID,      // IN user ID
                              username, //OUT
                              sizeof(username),
-                             userIDFromMetadata);    //OUT
+                             userIDFromMetadata,
+                             roleIDs);    //OUT
 
   // Update the instance if the USERS lookup was successful
   if (result != ERROR)
@@ -5310,13 +5326,15 @@ RETCODE ContextCli::setDatabaseUserByName(const char *userName)
 {
   char usersNameFromUsersTable[MAX_USERNAME_LEN +1];
   Int32 userIDFromUsersTable;
+  std::vector<int32_t> roleIDs;
 
   RETCODE result = authQuery(USERS_QUERY_BY_USER_NAME,
                              userName,    // IN user name
                              0,           // IN user ID (ignored)
                              usersNameFromUsersTable, //OUT
                              sizeof(usersNameFromUsersTable),
-                             userIDFromUsersTable);  // OUT
+                             userIDFromUsersTable,
+                             roleIDs);  // OUT
 
   // Update the instance if the lookup was successful
   if (result != ERROR)
@@ -5326,6 +5344,72 @@ RETCODE ContextCli::setDatabaseUserByName(const char *userName)
 }
 
 
+// ****************************************************************************
+// *
+// * Function: ContextCli::getRoleList
+// *
+// * Return the role IDs granted to the current user 
+// *   If the list of roles is already stored, just return this list.
+// *   If the list of roles does not exist extract the roles granted to the
+// *     current user from the Metadata and store in roleIDs_.
+// *
+// ****************************************************************************
+RETCODE ContextCli::getRoleList(
+  Int32 &numRoles,
+  Int32 *&roleIDs)
+{
+  // If role list has not been created, go read metadata
+  if (roleIDs_ == NULL)
+  {
+    // Get roles for userID
+    char usersNameFromUsersTable[MAX_USERNAME_LEN +1];
+    Int32 userIDFromUsersTable;
+    std::vector<int32_t> myRoles;
+    RETCODE result = authQuery (ROLES_QUERY_BY_AUTH_ID,
+                                NULL,  // user name
+                                databaseUserID_,
+                                usersNameFromUsersTable, //OUT
+                                sizeof(usersNameFromUsersTable),
+                                userIDFromUsersTable,
+                                myRoles);  // OUT
+    if (result != SUCCESS)
+      return result;
+
+    // Include the public user
+    myRoles.push_back(PUBLIC_USER);
+
+    // Add role info to ContextCli
+    numRoles_ = myRoles.size();
+    roleIDs_ = new (&exHeap_) Int32[numRoles_];
+    for (size_t i = 0; i < numRoles_; i++)
+      roleIDs_[i] = myRoles[i];
+  }
+
+  numRoles = numRoles_;
+  roleIDs = roleIDs_;
+
+  return SUCCESS;
+}
+  
+// ****************************************************************************
+// *
+// * Function: ContextCli::resetRoleList
+// *
+// * Deletes the current role list, a subsequent call to getRoleList reads the 
+// * metadata to get the list of roles associated with the current user and, 
+// * for the current user, stores the list in roleIDs_ and numRoles_ members
+// *
+// ****************************************************************************
+RETCODE ContextCli::resetRoleList()
+{
+  if (roleIDs_)
+    NADELETEBASIC(roleIDs_, &exHeap_);
+  roleIDs_ = NULL;
+  numRoles_ = 0;
+
+  return SUCCESS;
+}
+
 // *****************************************************************************
 // *                                                                           *
 // * Function: ContextCli::getAuthIDFromName                                   *
@@ -5390,8 +5474,9 @@ char authNameFromTable[MAX_USERNAME_LEN + 1];
   
 //TODO: If list of roles granted to user is cached in context, search there first.
 
+   std::vector<int32_t> roleIDs;
    return authQuery(AUTH_QUERY_BY_NAME,authName,0,authNameFromTable,
-                    sizeof(authNameFromTable),authID); 
+                    sizeof(authNameFromTable),authID, roleIDs); 
                        
 }
 //******************* End of ContextCli::getAuthIDFromName *********************
@@ -5443,9 +5528,10 @@ RETCODE ContextCli::getAuthNameFromID(
    
 {
 
-RETCODE result = SUCCESS;
-char authNameFromTable[MAX_USERNAME_LEN + 1];
-Int32 authIDFromTable;
+   RETCODE result = SUCCESS;
+   char authNameFromTable[MAX_USERNAME_LEN + 1];
+   Int32 authIDFromTable;
+   std::vector<int32_t> roleIDs;
 
    requiredLen = 0;
   
@@ -5474,7 +5560,8 @@ Int32 authIDFromTable;
                             authID,      // IN user ID
                             authNameFromTable, //OUT
                             sizeof(authNameFromTable),
-                            authIDFromTable);  // OUT
+                            authIDFromTable,
+                            roleIDs);  // OUT
          if (result == SUCCESS)
             return storeName(authNameFromTable,authName,maxBufLen,requiredLen);
       	 break;
@@ -5485,7 +5572,8 @@ Int32 authIDFromTable;
                             authID,      // IN user ID
                             authNameFromTable, //OUT
                             sizeof(authNameFromTable),
-                            authIDFromTable);  // OUT
+                            authIDFromTable,
+                            roleIDs);  // OUT
          if (result == SUCCESS)
             return storeName(authNameFromTable,authName,maxBufLen,requiredLen);
       	 break;
@@ -5519,6 +5607,7 @@ RETCODE ContextCli::getDBUserNameFromID(Int32 userID,         // IN
   RETCODE result = SUCCESS;
   char usersNameFromUsersTable[MAX_USERNAME_LEN + 1];
   Int32 userIDFromUsersTable;
+  std::vector<int32_t> roleIDs;
   if (requiredLen)
     *requiredLen = 0;
   
@@ -5544,7 +5633,8 @@ RETCODE ContextCli::getDBUserNameFromID(Int32 userID,         // IN
                        userID,      // IN user ID
                        usersNameFromUsersTable, //OUT
                        sizeof(usersNameFromUsersTable),
-                       userIDFromUsersTable);  // OUT
+                       userIDFromUsersTable,
+                       roleIDs);  // OUT
     if (result != ERROR)
       currentUserName = usersNameFromUsersTable;
   }
@@ -5604,18 +5694,20 @@ RETCODE ContextCli::getDBUserIDFromName(const char *userName, // IN
       return SUCCESS;
    }
   
-// See if the AUTHS row exists
+   // See if the AUTHS row exists
 
-RETCODE result = SUCCESS;
-char usersNameFromUsersTable[MAX_USERNAME_LEN + 1];
-Int32 userIDFromUsersTable;
+   RETCODE result = SUCCESS;
+   char usersNameFromUsersTable[MAX_USERNAME_LEN + 1];
+   Int32 userIDFromUsersTable;
+   std::vector<int32_t> roleIDs;
 
    result = authQuery(USERS_QUERY_BY_USER_NAME,
                       userName,    // IN user name
                       0,           // IN user ID (ignored)
                       usersNameFromUsersTable, //OUT
                       sizeof(usersNameFromUsersTable),
-                      userIDFromUsersTable);  // OUT
+                      userIDFromUsersTable,
+                      roleIDs);  // OUT
    if (result == SUCCESS && userID)
       *userID = userIDFromUsersTable;
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/cli/Context.h
----------------------------------------------------------------------
diff --git a/core/sql/cli/Context.h b/core/sql/cli/Context.h
index f860820..0489acf 100644
--- a/core/sql/cli/Context.h
+++ b/core/sql/cli/Context.h
@@ -106,7 +106,7 @@ public:
   const char *getDatabaseUserName() { return databaseUserName_;}
   Int32 *getDatabaseUserID() { return &databaseUserID_; }
   Int32 *getSessionUserID() { return &sessionUserID_; }
-
+  
   // Functions to switch user identity. The specified user must exist
   // as a valid user in the USERS metadata table. Otherwise an error
   // code is returned and error conditions will be written to the
@@ -118,8 +118,7 @@ public:
   void setDatabaseUser(const Int32 &uid, // IN
                        const char *uname);   // IN
 
-  // Functions to map between Trafodion authentication IDs and names. The 
-  // mapping operation is not supported on other platforms.
+  // Functions to map between Trafodion authentication IDs and names. 
   RETCODE getAuthIDFromName(
      const char *authName,  
      Int32 & authID);   
@@ -159,6 +158,12 @@ public:
                      bool &auditingEnabled);
 
 
+  // functions to get and set roles for the current user
+  RETCODE getRoleList(Int32  &numRoles,
+                      Int32  *&roleIDs);
+
+  RETCODE resetRoleList();
+
   SequenceValueGenerator* &seqGen() { return seqGen_; }
 
   HashQueue * trafSElist() { return trafSElist_; }
@@ -239,6 +244,10 @@ private:
   // column in the USERS table. Max 128 characters.
   char *databaseUserName_;
 
+  // List of active roles for the databaseUser
+  Int32  *roleIDs_;
+  Int32   numRoles_;
+
   NABoolean userNameChanged_;
 
   // queues(lists) to keep track of loaded modules, statements
@@ -527,7 +536,8 @@ private:
       USERS_QUERY_BY_USER_ID,
       USERS_QUERY_BY_EXTERNAL_NAME,
       ROLE_QUERY_BY_ROLE_ID,
-      AUTH_QUERY_BY_NAME
+      AUTH_QUERY_BY_NAME,
+      ROLES_QUERY_BY_AUTH_ID
     };
 
   // Private method to perform single-row lookups into the AUTHS
@@ -542,12 +552,13 @@ private:
   // USERS_QUERY_BY_EXTERNAL_NAME, the authName argument must be
   // provided.
   RETCODE authQuery(
-     AuthQueryType queryType,         
-     const char  * authName,          
-     Int32         authID,            
-     char        * authNameFromTable,
-     Int32         authNameMaxLen, 
-     Int32       & authIDFromTable);  
+     AuthQueryType          queryType,         
+     const char           * authName,          
+     Int32                  authID,            
+     char                 * authNameFromTable,
+     Int32                  authNameMaxLen, 
+     Int32                & authIDFromTable,
+     std::vector<int32_t> & roleIDs);  
       
   RETCODE storeName(
      const char *src,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/cli/SQLCLIdev.h
----------------------------------------------------------------------
diff --git a/core/sql/cli/SQLCLIdev.h b/core/sql/cli/SQLCLIdev.h
index ad580c5..9ef68b8 100644
--- a/core/sql/cli/SQLCLIdev.h
+++ b/core/sql/cli/SQLCLIdev.h
@@ -936,6 +936,13 @@ SQLCLI_LIB_FUNC Lng32 SQL_EXEC_SetErrorCodeInRTS(
                 /*IN*/ SQLSTMT_ID * statement_id,
 	        /*IN*/ Lng32     sqlErrorCode);  
 
+SQLCLI_LIB_FUNC Lng32 SQL_EXEC_GetRoleList(
+   Int32 &numRoles,
+   Int32 *&roleIDs);
+
+SQLCLI_LIB_FUNC Lng32 SQL_EXEC_ResetRoleList_Internal ();
+
+
 /*
 Statistics info collected for Replicate Operator
 ComTdb::ex_REPLICATE in the replicator processes

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/common/ComSecurityKey.cpp
----------------------------------------------------------------------
diff --git a/core/sql/common/ComSecurityKey.cpp b/core/sql/common/ComSecurityKey.cpp
index ae43920..f3f52e1 100644
--- a/core/sql/common/ComSecurityKey.cpp
+++ b/core/sql/common/ComSecurityKey.cpp
@@ -168,6 +168,88 @@ bool buildSecurityKeys( const int32_t granteeID,
   return true;
 }
 
+// ****************************************************************************
+// Function that returns the types of invalidation to perform
+//   For DDL invalidation keys, always need to update caches
+//   For security invalidation keys
+//      update caches if key is for an object revoke from the current user
+//      reset list of roles if key is a revoke role from the current user
+//
+// returns:
+//    resetRoleList -- need to reset the list of roles for the current user
+//    updateCaches -- need to update cache entries related for the keys
+// ****************************************************************************
+void qiInvalidationType (const Int32 numInvalidationKeys,
+                         const SQL_QIKEY* invalidationKeys,
+                         const Int32 userID,
+                         bool &resetRoleList,
+                         bool &updateCaches)
+{
+  resetRoleList = false;
+  updateCaches = false;
+  ComQIActionType invalidationKeyType = COM_QI_INVALID_ACTIONTYPE;
+
+  // Have the ComSecurityKey constructor compute the hash value for the the User's ID.
+  // Note: The following code doesn't care about the object's hash value or the resulting 
+  // ComSecurityKey's ActionType....we just need the hash value for the User's ID.
+  // Perhaps a new constructor would be good (also done in RelRoot::checkPrivileges)
+  int64_t objectUID = 12345;
+  ComSecurityKey userKey( userID, objectUID
+                         , SELECT_PRIV
+                         , ComSecurityKey::OBJECT_IS_OBJECT
+                        );
+  uint32_t userHashValue = userKey.getSubjectHashValue();
+
+  for ( Int32 i = 0; i < numInvalidationKeys && !resetRoleList && !updateCaches; i++ )
+  {
+    invalidationKeyType = ComQIActionTypeLiteralToEnum( invalidationKeys[i].operation );
+    switch (invalidationKeyType)
+    {
+      // Object changed, need to update caches
+      case COM_QI_OBJECT_REDEF:
+      case COM_QI_STATS_UPDATED:
+        updateCaches = true;
+        break;
+
+      // Privilege changed on an object, need to update caches if
+      // any QI keys are associated with the current user
+      case COM_QI_OBJECT_SELECT:
+      case COM_QI_OBJECT_INSERT:
+      case COM_QI_OBJECT_DELETE:
+      case COM_QI_OBJECT_UPDATE:
+      case COM_QI_OBJECT_USAGE:
+      case COM_QI_OBJECT_REFERENCES:
+      case COM_QI_OBJECT_EXECUTE:
+        if (invalidationKeys[i].revokeKey.subject == userHashValue)
+          updateCaches = true;
+        break;
+
+      // For public user (SPECIAL_ROLE), the subject is a special hash
+      case COM_QI_USER_GRANT_SPECIAL_ROLE:
+        if (invalidationKeys[i].revokeKey.subject == ComSecurityKey::SPECIAL_SUBJECT_HASH)
+          updateCaches = true;
+        break;
+
+      // A revoke role from a user was performed.  Need to reset role list
+      // if QI key associated with the current user and remove any plans
+      // that include the role key
+      case COM_QI_USER_GRANT_ROLE:
+        if (invalidationKeys[i].revokeKey.subject == userHashValue)
+        {
+          resetRoleList = true;
+          updateCaches = true;
+        }
+        break;
+
+      // unknown key type, search and update cache (should not happen)
+      default:
+        resetRoleList = true;
+        updateCaches = true;
+        break;
+      }
+  }
+}
+
 
 // *****************************************************************************
 //    ComSecurityKey methods

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/common/ComSecurityKey.h
----------------------------------------------------------------------
diff --git a/core/sql/common/ComSecurityKey.h b/core/sql/common/ComSecurityKey.h
index 42a5d01..ff03ec1 100644
--- a/core/sql/common/ComSecurityKey.h
+++ b/core/sql/common/ComSecurityKey.h
@@ -48,6 +48,11 @@ bool buildSecurityKeys( const int32_t granteeID,
                         const PrivMgrCoreDesc &privs,
                         ComSecurityKeySet &secKeySet);
 
+void qiInvalidationType (const Int32 numInvalidationKeys,
+                         const SQL_QIKEY* invalidationKeys,
+                         const Int32 userID,
+                         bool &resetRoleList,
+                         bool &updateCaches);
 
 // ****************************************************************************
 // Class:  ComSecurityKey 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/optimizer/BindRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindRelExpr.cpp b/core/sql/optimizer/BindRelExpr.cpp
index ee65f9e..61bb96f 100644
--- a/core/sql/optimizer/BindRelExpr.cpp
+++ b/core/sql/optimizer/BindRelExpr.cpp
@@ -6774,7 +6774,7 @@ NABoolean RelRoot::checkPrivileges(BindWA* bindWA)
       if (retcode != STATUS_GOOD)
       {
         bindWA->setFailedForPrivileges( TRUE );
-        RemoveNATableEntryFromCache = TRUE;
+        tab->setRemoveFromCacheBNC(TRUE); // To be removed by CmpMain before Compilation retry
         *CmpCommon::diags() << DgSqlCode( -1034 );
         return FALSE;
       }
@@ -6832,7 +6832,7 @@ NABoolean RelRoot::checkPrivileges(BindWA* bindWA)
       if (retcode != STATUS_GOOD)
       {
         bindWA->setFailedForPrivileges(TRUE);
-        RemoveNATableEntryFromCache = TRUE;
+        tab->setRemoveFromCacheBNC(TRUE); // Not used until sequences stored in table cache
         *CmpCommon::diags() << DgSqlCode( -1034 );
         return FALSE;
       }
@@ -6854,7 +6854,7 @@ NABoolean RelRoot::checkPrivileges(BindWA* bindWA)
     else
     {
       bindWA->setFailedForPrivileges( TRUE );
-      RemoveNATableEntryFromCache = TRUE;
+      tab->setRemoveFromCacheBNC(TRUE); // To be removed by CmpMain before Compilation retry
       *CmpCommon::diags()
         << DgSqlCode( -4491 )
         << DgString0( "USAGE" )

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/optimizer/NARoutine.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NARoutine.cpp b/core/sql/optimizer/NARoutine.cpp
index 096e2bc..edf703c 100644
--- a/core/sql/optimizer/NARoutine.cpp
+++ b/core/sql/optimizer/NARoutine.cpp
@@ -705,23 +705,25 @@ void NARoutine::getPrivileges(TrafDesc *priv_desc)
   }
   else
   {
-    // getActive roles
-    std::vector<int32_t> roleIDs;
-    CmpSeabaseDDL cmpSBD(STMTHEAP);
-    if (cmpSBD.switchCompiler(CmpContextInfo::CMPCONTEXT_TYPE_META))
-    { 
-      if (CmpCommon::diags()->getNumber(DgSqlCode::ERROR_) == 0)
-        *CmpCommon::diags() << DgSqlCode( -4400 );
+    // get roles granted to current user 
+    // SQL_EXEC_GetRoleList returns the list of roles from the CliContext
+    std::vector<int32_t> myRoles;
+    Int32 numRoles = 0;
+    Int32 *roleIDs = NULL;
+    if (SQL_EXEC_GetRoleList(numRoles, roleIDs) < 0)
+    {
+      *CmpCommon::diags() << DgSqlCode(-1034);
       return;
     }
 
-    PrivStatus retcode = privInterface.getRoles( ComUser::getCurrentUser(), roleIDs);
-    cmpSBD.switchBackCompiler();
-    if (retcode == STATUS_ERROR)
-      return;
+    // At this time we should have at least one entry in roleIDs (PUBLIC_USER)
+    CMPASSERT (roleIDs && numRoles > 0);
+
+    for (Int32 i = 0; i < numRoles; i++)
+      myRoles.push_back(roleIDs[i]);
 
     privInfo_ = new (heap_) PrivMgrUserPrivs;
-    privInfo_->initUserPrivs(roleIDs, priv_desc, ComUser::getCurrentUser(),objectUID_, routineSecKeySet_);
+    privInfo_->initUserPrivs(myRoles, priv_desc, ComUser::getCurrentUser(),objectUID_, routineSecKeySet_);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/optimizer/NATable.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NATable.cpp b/core/sql/optimizer/NATable.cpp
index c03b4ae..bd545fb 100644
--- a/core/sql/optimizer/NATable.cpp
+++ b/core/sql/optimizer/NATable.cpp
@@ -6846,43 +6846,40 @@ void NATable::getPrivileges(TrafDesc * priv_desc)
 
   ComSecurityKeySet secKeyVec(heap_);
   if (priv_desc == NULL)
+  {
+    if (isHiveTable())
+      readPrivileges();
+    else
+      privInfo_ = NULL;
+    return;
+  }
+  else
+  {
+    // get roles granted to current user 
+    // SQL_EXEC_GetRoleList returns the list of roles from the CliContext
+    std::vector<int32_t> myRoles;
+    Int32 numRoles = 0;
+    Int32 *roleIDs = NULL;
+    if (SQL_EXEC_GetRoleList(numRoles, roleIDs) < 0)
     {
-      if (isHiveTable())
-        readPrivileges();
-      else
-        privInfo_ = NULL;
+      *CmpCommon::diags() << DgSqlCode(-1034);
       return;
     }
-  else
-    {
-      // Build privInfo_ based on the priv_desc
-      std::vector<int32_t> roleIDs;
 
-      CmpSeabaseDDL cmpSBD(STMTHEAP);
-      if (cmpSBD.switchCompiler(CmpContextInfo::CMPCONTEXT_TYPE_META))
-      {
-        if (CmpCommon::diags()->getNumber(DgSqlCode::ERROR_) == 0)
-          *CmpCommon::diags() << DgSqlCode( -4400 );
-        return;
-      }
+    // At this time we should have at least one entry in roleIDs (PUBLIC_USER)
+    CMPASSERT (roleIDs && numRoles > 0);
 
-      NAString privMDLoc = CmpSeabaseDDL::getSystemCatalogStatic();
-      privMDLoc += ".\"";
-      privMDLoc += SEABASE_PRIVMGR_SCHEMA;
-      privMDLoc += "\"";
-
-      PrivMgrCommands privInterface(privMDLoc.data(), CmpCommon::diags(),PrivMgr::PRIV_INITIALIZED);
-      PrivStatus retcode = privInterface.getRoles( ComUser::getCurrentUser(), roleIDs);
-      cmpSBD.switchBackCompiler();
-      if (retcode == STATUS_ERROR)
-        return;
+    for (Int32 i = 0; i < numRoles; i++)
+      myRoles.push_back(roleIDs[i]);
 
-      privInfo_ = new(heap_) PrivMgrUserPrivs;
-      privInfo_->initUserPrivs(roleIDs, priv_desc, 
-                               ComUser::getCurrentUser(), 
-                               objectUID_.get_value(), secKeySet_);
+    // Build privInfo_ based on the priv_desc
+    privInfo_ = new(heap_) PrivMgrUserPrivs;
+    privInfo_->initUserPrivs(myRoles, priv_desc, 
+                             ComUser::getCurrentUser(), 
+                             objectUID_.get_value(), secKeySet_);
   }
 
+
   if (privInfo_ == NULL)
     {
       *CmpCommon::diags() << DgSqlCode(-1034);
@@ -9097,6 +9094,7 @@ NATableDB::RemoveFromNATableCache( NATable * NATablep , UInt32 currIndx )
 //
 // Remove ALL entries from the NATable Cache that have been
 // marked for removal before the next compilation.
+// Remove nonCacheable entries also.
 //
 void
 NATableDB::remove_entries_marked_for_removal()
@@ -9120,6 +9118,17 @@ NATableDB::remove_entries_marked_for_removal()
       }
       else currIndx++ ; //Note: No increment if the entry was removed !
    }
+
+   //remove the nonCacheableTableList and delete the name, 
+   //this is needed to remove objects such as sequence generators which 
+   //are not stored in the cached list
+   for(CollIndex i=0; i < nonCacheableTableList_.entries(); i++){
+     remove(nonCacheableTableList_[i]);
+     delete nonCacheableTableList_[i]; // delete the name only
+   }
+
+   //clear the list of special tables
+   nonCacheableTableList_.clear();
 }
 
 //

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/optimizer/NATable.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NATable.h b/core/sql/optimizer/NATable.h
index 5f8991b..c2b3730 100644
--- a/core/sql/optimizer/NATable.h
+++ b/core/sql/optimizer/NATable.h
@@ -1279,7 +1279,7 @@ public:
                      NABoolean ddlXns, NABoolean atCommit);
    
   void RemoveFromNATableCache( NATable * NATablep , UInt32 currIndx );
-  static void remove_entries_marked_for_removal();
+  void remove_entries_marked_for_removal();
   static void unmark_entries_marked_for_removal();
 
   void free_entries_with_QI_key( Int32 numSiKeys, SQL_QIKEY* qiKeyArray );

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/regress/privs1/TEST120
----------------------------------------------------------------------
diff --git a/core/sql/regress/privs1/TEST120 b/core/sql/regress/privs1/TEST120
index 45021eb..c24225a 100755
--- a/core/sql/regress/privs1/TEST120
+++ b/core/sql/regress/privs1/TEST120
@@ -43,6 +43,10 @@ revoke role t120role3 from sql_user9 by sql_user3;
 revoke role t120role2 from sql_user6 by sql_user3;
 revoke role t120role3 from sql_user6 by sql_user3;
 revoke role t120role4 from sql_user6 by sql_user3;
+revoke role t120role3 from sql_user9;
+revoke role t120role2 from sql_user6;
+revoke role t120role3 from sql_user6;
+revoke role t120role4 from sql_user6;
 revoke role t120role1 from sql_user3; 
 revoke role t120role2 from sql_user3; 
 revoke role t120role3 from sql_user3; 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/sqlcomp/CmpMain.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpMain.cpp b/core/sql/sqlcomp/CmpMain.cpp
index cf1d8fc..8023853 100644
--- a/core/sql/sqlcomp/CmpMain.cpp
+++ b/core/sql/sqlcomp/CmpMain.cpp
@@ -860,6 +860,12 @@ CmpMain::ReturnStatus CmpMain::sqlcomp(QueryText& input,            //IN
              UnmarkMarkedNATableCacheEntries();
           return rs;
        }
+
+       // Update the list of roles incase a grant came in since last attempt
+       // Ignore errors from Reset.  A check is made later that returns
+       // an error if unable to reset the list at this time
+       SQL_EXEC_ResetRoleList_Internal();
+
        RemoveMarkedNATableCacheEntries();
        Retried_for_priv_failure = TRUE;
     }
@@ -1165,12 +1171,25 @@ void CmpMain:: getAndProcessAnySiKeys(TimeVal begTime)
      }
      else if ( returnedNumSiKeys > 0 )
      {
-       CheckForSpecialRoleRevoke(returnedNumSiKeys, sikKeyArray);
-       CURRENTQCACHE->free_entries_with_QI_keys(returnedNumSiKeys, 
-         sikKeyArray);
-       InvalidateNATableCacheEntries(returnedNumSiKeys, sikKeyArray);
-       InvalidateNARoutineCacheEntries(returnedNumSiKeys, sikKeyArray);
-       InvalidateHistogramCacheEntries(returnedNumSiKeys, sikKeyArray);
+       bool resetRoleList = false;
+       bool updateCaches = false;
+       qiInvalidationType(returnedNumSiKeys, sikKeyArray, 
+                          ComUser::getCurrentUser(), 
+                          resetRoleList, updateCaches); 
+
+       if (updateCaches)
+       {
+         CURRENTQCACHE->free_entries_with_QI_keys(returnedNumSiKeys, 
+           sikKeyArray);
+         InvalidateNATableCacheEntries(returnedNumSiKeys, sikKeyArray);
+         InvalidateNARoutineCacheEntries(returnedNumSiKeys, sikKeyArray);
+         InvalidateHistogramCacheEntries(returnedNumSiKeys, sikKeyArray);
+       }
+      
+       // Ignore errors from ResetRoleList.  A check is made later that returns
+       // an error if unable to reset the list at this time
+       if (resetRoleList)
+         SQL_EXEC_ResetRoleList_Internal();
      }
    }
   // Always update previous QI time 
@@ -1191,9 +1210,6 @@ Int32 CmpMain::getAnySiKeys(TimeVal      begTime,
 
   // SQL_EXEC_GetSecInvalidKeys() is coded to work in JulianTimeStamp values
   // so add number of microseconds between 4713 B.C., Jan 1 and the Epoc (Jan 1, 1970).
-  //
-  // NOTE: Conversion is not needed for NT because SQL_EXEC_GetSecInvalidKeys(...)
-  //       never returns any SI_KEYs and that's because QVP doesn't run on NT.
   prev_QI_Time += HS_EPOCH_TIMESTAMP ;
   Int64 Max_QI_Time = prev_QI_Time ;
 
@@ -1335,11 +1351,6 @@ Int32 CmpMain::getAnySiKeys(TimeVal      begTime,
   return sqlcode;
 } 
 
-void CmpMain::CheckForSpecialRoleRevoke( Int32 NumSiKeys, SQL_QIKEY * SiKeyArray )
-{
-
-}
-
 void CmpMain::InvalidateNATableCacheEntries(Int32 returnedNumQiKeys,
                                             SQL_QIKEY * qiKeyArray)
 {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/sqlcomp/CmpMain.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpMain.h b/core/sql/sqlcomp/CmpMain.h
index dcf2d94..771d080 100644
--- a/core/sql/sqlcomp/CmpMain.h
+++ b/core/sql/sqlcomp/CmpMain.h
@@ -295,7 +295,6 @@ private:
                                         SQL_QIKEY *qiKeyArray);
    void InvalidateHistogramCacheEntries(Int32 returnedNumSiKeys, 
                                         SQL_QIKEY *qiKeyArray);
-   void CheckForSpecialRoleRevoke( Int32 NumSiKeys, SQL_QIKEY * pSiKeyArray );
    void RemoveMarkedNATableCacheEntries();
    void UnmarkMarkedNATableCacheEntries();
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp b/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
index 58118cf..ad3c3dd 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
@@ -246,6 +246,24 @@ CmpSeabaseDDLauth::AuthStatus CmpSeabaseDDLauth::getAuthDetails(Int32 authID)
   }
 }
 
+CmpSeabaseDDLauth::AuthStatus CmpSeabaseDDLauth::getRoleIDs(
+  const Int32 authID,
+  std::vector<int32_t> &roleIDs)
+{
+  NAString privMgrMDLoc;
+  CONCAT_CATSCH(privMgrMDLoc,systemCatalog_.data(),SEABASE_PRIVMGR_SCHEMA);
+
+  PrivMgrRoles role(std::string(MDSchema_.data()),
+                    std::string(privMgrMDLoc.data()),
+                    CmpCommon::diags());
+  std::vector<std::string> roleNames;
+  std::vector<int32_t> roleDepths;
+
+  if (role.fetchRolesForUser(authID,roleNames,roleIDs,roleDepths) == PrivStatus::STATUS_ERROR)
+    return STATUS_ERROR;
+  return STATUS_GOOD; 
+}
+
 // ----------------------------------------------------------------------------
 // method:  getUniqueID
 //

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/1c8f25b6/core/sql/sqlcomp/CmpSeabaseDDLauth.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLauth.h b/core/sql/sqlcomp/CmpSeabaseDDLauth.h
index 17d49ad..5ccda8a 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLauth.h
+++ b/core/sql/sqlcomp/CmpSeabaseDDLauth.h
@@ -37,6 +37,8 @@
 
 #include "ComSmallDefs.h"
 #include "NAUserId.h"
+#include <vector>
+
 
 class StmtDDLRegisterUser;
 class StmtDDLAlterUser;
@@ -73,6 +75,8 @@ class CmpSeabaseDDLauth
                                     bool isExternal = false);
      virtual bool describe       (const NAString &authName, 
                                     NAString &authText);
+     AuthStatus   getRoleIDs     (const Int32 authID,
+                                    std::vector<int32_t> &roleIDs);
 
      // accessors
      Int32          getAuthCreator() const    { return authCreator_; }
@@ -162,7 +166,7 @@ class CmpSeabaseDDLuser : public CmpSeabaseDDLauth
      void unregisterUser(StmtDDLRegisterUser * pNode);
      
      CmpSeabaseDDLauth::AuthStatus getUserDetails(const char *pUserName, 
-                                                    bool isExternal = false);
+                                                  bool isExternal = false);
      CmpSeabaseDDLauth::AuthStatus getUserDetails(Int32 userID);
 
      bool describe (const NAString &authName, NAString &authText);



[2/2] incubator-trafodion git commit: Merge TRAFODION-2327 pull 803 Reduce I/O's when loading objects into caches

Posted by rm...@apache.org.
Merge TRAFODION-2327 pull 803 Reduce I/O's when loading objects into caches


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/commit/6ce6a8ea
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/6ce6a8ea
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/6ce6a8ea

Branch: refs/heads/master
Commit: 6ce6a8ea93d186b599fc3811d6be73b1ce34d485
Parents: 48f20cc 1c8f25b
Author: Roberta Marton <ro...@apache.org>
Authored: Tue Nov 1 00:18:24 2016 +0000
Committer: Roberta Marton <ro...@apache.org>
Committed: Tue Nov 1 00:18:24 2016 +0000

----------------------------------------------------------------------
 core/sql/cli/Cli.cpp                   |  44 ++++++-
 core/sql/cli/Cli.h                     |   9 +-
 core/sql/cli/CliExtern.cpp             |  81 +++++++++++-
 core/sql/cli/Context.cpp               | 194 ++++++++++++++++++++--------
 core/sql/cli/Context.h                 |  31 +++--
 core/sql/cli/SQLCLIdev.h               |   7 +
 core/sql/common/ComSecurityKey.cpp     |  82 ++++++++++++
 core/sql/common/ComSecurityKey.h       |   5 +
 core/sql/optimizer/BindRelExpr.cpp     |   6 +-
 core/sql/optimizer/NARoutine.cpp       |  26 ++--
 core/sql/optimizer/NATable.cpp         |  67 +++++-----
 core/sql/optimizer/NATable.h           |   2 +-
 core/sql/regress/privs1/TEST120        |   4 +
 core/sql/sqlcomp/CmpMain.cpp           |  39 ++++--
 core/sql/sqlcomp/CmpMain.h             |   1 -
 core/sql/sqlcomp/CmpSeabaseDDLauth.cpp |  18 +++
 core/sql/sqlcomp/CmpSeabaseDDLauth.h   |   6 +-
 17 files changed, 497 insertions(+), 125 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/6ce6a8ea/core/sql/optimizer/BindRelExpr.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/6ce6a8ea/core/sql/optimizer/NATable.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/6ce6a8ea/core/sql/sqlcomp/CmpMain.cpp
----------------------------------------------------------------------