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 2017/02/15 15:49:36 UTC

[1/2] incubator-trafodion git commit: TRAFODION-2441 user has only select privilege on a table can do ... TRAFODION-2409 support privilege control(column privileges) for hive tables TRAFODION-2423 any user can perform 'initialize trafodion, drop' TRAFODI

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master f6f4402b5 -> cc13c6cc5


TRAFODION-2441 user has only select privilege on a table can do ...
TRAFODION-2409 support privilege control(column privileges) for hive tables
TRAFODION-2423 any user can perform 'initialize trafodion, drop'
TRAFODION-2435 Any user can perform TRUNCATE on native Hive tables.
TRAFODION-2463 Hive: Any user can do update statistics for hive tables

Fixed issues found while testing privileges with native Hive.

TRAFODION-2441:
  changed code that initializes owner privileges for views.

TRAFODION-2409:
  returning error message 1328 during attempt to grant unsupported column level
  privilege on hive table.

TRAFODION 2423:
  added privilege checks for all initialize commands, error 1017 is returned if
  not DB__ROOT

TRAFODION-2435:
  Returning error 1051 if TRUNCATE is attempted on a hive table where the
  current user has no privilege

TRAFODION-2463:
  Privilege checks added for Hive table during update statistics


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

Branch: refs/heads/master
Commit: db14e3922cb2d0722d0885f5c248cac2af2b904d
Parents: 60c0c42
Author: Roberta Marton <rm...@edev07.esgyn.local>
Authored: Mon Feb 13 23:20:54 2017 +0000
Committer: Roberta Marton <rm...@edev07.esgyn.local>
Committed: Mon Feb 13 23:20:54 2017 +0000

----------------------------------------------------------------------
 core/sql/bin/SqlciErrors.txt              |   4 +-
 core/sql/common/ComUser.cpp               |  27 +++++++
 core/sql/common/ComUser.h                 |   2 +
 core/sql/optimizer/BindRelExpr.cpp        |  12 +--
 core/sql/optimizer/NATable.cpp            |  20 +++--
 core/sql/optimizer/RelExeUtil.cpp         |  30 +++++++
 core/sql/regress/privs1/EXPECTED141       | Bin 100853 -> 102260 bytes
 core/sql/regress/privs1/TEST123           |   2 +-
 core/sql/regress/privs1/TEST141           |  10 ++-
 core/sql/regress/privs2/EXPECTED144       | Bin 59409 -> 59453 bytes
 core/sql/sqlcomp/CmpDescribe.cpp          |   9 ++-
 core/sql/sqlcomp/CmpSeabaseDDLauth.cpp    | 103 ++++++++++++++++++++++--
 core/sql/sqlcomp/CmpSeabaseDDLauth.h      |   1 +
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp  |  55 ++++++++++++-
 core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp |  18 +++++
 core/sql/sqlcomp/CmpSeabaseDDLschema.cpp  |  33 +++++---
 core/sql/sqlcomp/CmpSeabaseDDLtable.cpp   |  74 +++++++++++++++++
 core/sql/sqlcomp/CmpSeabaseDDLupgrade.cpp |   7 ++
 core/sql/sqlcomp/PrivMgr.cpp              |  10 +--
 core/sql/sqlcomp/PrivMgr.h                |   5 +-
 core/sql/sqlcomp/PrivMgrCommands.cpp      |  65 +++++++++++++--
 core/sql/sqlcomp/PrivMgrCommands.h        |   6 ++
 core/sql/sqlcomp/PrivMgrPrivileges.cpp    |  26 ++++--
 core/sql/sqlcomp/PrivMgrPrivileges.h      |   4 +-
 core/sql/sqlcomp/PrivMgrRoles.cpp         | 106 +++++++++++++------------
 core/sql/ustat/hs_globals.cpp             |   2 +-
 26 files changed, 517 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/bin/SqlciErrors.txt
----------------------------------------------------------------------
diff --git a/core/sql/bin/SqlciErrors.txt b/core/sql/bin/SqlciErrors.txt
index 8afa052..fe99787 100644
--- a/core/sql/bin/SqlciErrors.txt
+++ b/core/sql/bin/SqlciErrors.txt
@@ -222,8 +222,8 @@
 1224 ZZZZZ 99999 BEGINNER MAJOR DBADMIN An invalid data type was specified for routine parameter $0~String0.
 1225 ZZZZZ 99999 BEGINNER MAJOR DBADMIN Mixing EXECUTE privilege with other privileges is not allowed.
 1226 ZZZZZ 99999 BEGINNER MAJOR DBADMIN No valid combination of privileges was specified.
-1227 ZZZZZ 99999 BEGINNER MAJOR DBADMIN Cannot unregister user.  User has been granted privileges.
-1228 ZZZZZ 99999 BEGINNER MAJOR DBADMIN Cannot drop role.  Role has been granted privileges.
+1227 ZZZZZ 99999 BEGINNER MAJOR DBADMIN Cannot unregister user.  User $0~String0 has been granted privileges on $1~String1.
+1228 ZZZZZ 99999 BEGINNER MAJOR DBADMIN Cannot drop role.  Role $0~String0 has been granted privileges on $1~String1.
 1229 ZZZZZ 99999 BEGINNER MAJOR DBADMIN The $0~string0 option is not supported.
 1230 ZZZZZ 99999 BEGINNER MAJOR DBADMIN Object owner must be the schema owner in private schemas.
 1231 ZZZZZ 99999 BEGINNER MAJOR DBADMIN User-defined routine $0~String0 could not be created.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/common/ComUser.cpp
----------------------------------------------------------------------
diff --git a/core/sql/common/ComUser.cpp b/core/sql/common/ComUser.cpp
index 7e7d7ca..925c5eb 100644
--- a/core/sql/common/ComUser.cpp
+++ b/core/sql/common/ComUser.cpp
@@ -348,6 +348,33 @@ Int16 ComUser::getAuthNameFromAuthID(Int32   authID,
   return FEOK;
 }
 
+
+// ----------------------------------------------------------------------------
+// method: currentUserHasRole
+//
+// Searches the list of roles stored for the user to see if passed in role ID
+// is found
+//
+//  Returns:  true - role found
+//            false - role not found
+// ----------------------------------------------------------------------------
+bool ComUser::currentUserHasRole(Int32 roleID)
+{
+  Int32 numRoles = 0;
+  Int32 *roleIDs = 0;
+  if (SQL_EXEC_GetRoleList(numRoles, roleIDs) < 0)
+    return false;
+
+  for (Int32 i = 0; i < numRoles; i++)
+  {
+    Int32 userRole = roleIDs[i];
+    if (userRole == roleID)
+      return true;
+  }
+  return false;
+}
+
+
 // ----------------------------------------------------------------------------
 // method: getRoleList
 //

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/common/ComUser.h
----------------------------------------------------------------------
diff --git a/core/sql/common/ComUser.h b/core/sql/common/ComUser.h
index ea00c88..645b983 100644
--- a/core/sql/common/ComUser.h
+++ b/core/sql/common/ComUser.h
@@ -90,6 +90,8 @@ class ComUser
      static Int16 getAuthIDFromAuthName (const char  * authName,
                                          Int32 & authID);
 
+     static bool currentUserHasRole(Int32 roleID);
+
      static Int32 getRoleList (char *roleList,
                                Int32 &actualLen,
                                const Int32 maxLen,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/optimizer/BindRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindRelExpr.cpp b/core/sql/optimizer/BindRelExpr.cpp
index e1eefe2..15f5e35 100644
--- a/core/sql/optimizer/BindRelExpr.cpp
+++ b/core/sql/optimizer/BindRelExpr.cpp
@@ -6881,9 +6881,7 @@ NABoolean RelRoot::checkPrivileges(BindWA* bindWA)
           *CmpCommon::diags() << DgSqlCode( -4400 );
         return FALSE;
       }
-      retcode = privInterface.getPrivileges( tab->objectUid().get_value(),
-                                             tab->getObjectType(), thisUserID,
-                                             privInfo);
+      retcode = privInterface.getPrivileges( tab, thisUserID, privInfo);
       cmpSBD.switchBackCompiler();
 
       if (retcode != STATUS_GOOD)
@@ -7003,9 +7001,7 @@ NABoolean RelRoot::checkPrivileges(BindWA* bindWA)
           *CmpCommon::diags() << DgSqlCode( -4400 );
         return FALSE;
       }
-      retcode = privInterface.getPrivileges( tab->objectUid().get_value(), 
-                                             tab->getObjectType(), thisUserID, 
-                                             privInfo);
+      retcode = privInterface.getPrivileges( tab, thisUserID, privInfo);
       cmpSBD.switchBackCompiler();
 
       if (retcode != STATUS_GOOD)
@@ -7062,9 +7058,7 @@ NABoolean RelRoot::checkPrivileges(BindWA* bindWA)
           *CmpCommon::diags() << DgSqlCode( -4400 );
         return FALSE;
       }
-      retcode = privInterface.getPrivileges(tab->objectUid().get_value(), 
-                                            COM_SEQUENCE_GENERATOR_OBJECT, 
-                                            thisUserID, privInfo);
+      retcode = privInterface.getPrivileges(tab, thisUserID, privInfo);
       cmpSBD.switchBackCompiler();
       if (retcode != STATUS_GOOD)
       {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/optimizer/NATable.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NATable.cpp b/core/sql/optimizer/NATable.cpp
index ea15cc4..d734e2e 100644
--- a/core/sql/optimizer/NATable.cpp
+++ b/core/sql/optimizer/NATable.cpp
@@ -5687,8 +5687,7 @@ NATable::NATable(BindWA *bindWA,
 		schemaOwner_ = SUPER_USER;
 	}
 
-	if (hasExternalTable())
-		getPrivileges(NULL); 
+        getPrivileges(NULL); 
 
 	// TBD - if authorization is enabled and there is no external table to store
 	// privileges, go get privilege information from HIVE metadata ...
@@ -6654,11 +6653,20 @@ void NATable::getPrivileges(TrafDesc * priv_desc)
 
   // If current user is root, object owner, or this is a volatile table
   // automatically have owner default privileges.
-  if ((!isSeabaseTable() && !isHiveTable()) ||
+ if ((!isSeabaseTable() && !isHiveTable()) ||
        !CmpCommon::context()->isAuthorizationEnabled() ||
        isVolatileTable() ||
-       ComUser::isRootUserID()||
-       ComUser::getCurrentUser() == owner_)
+       (ComUser::isRootUserID() && !isHiveTable()) )
+  {
+    privInfo_ = new(heap_) PrivMgrUserPrivs;
+    privInfo_->setOwnerDefaultPrivs();
+    return;
+  }
+
+  // Generally, if the current user is the object owner, then the automatically
+  // have all privs.  However, if this is a shared schema then views can be
+  // owned by the current user but not have all privs
+  if (ComUser::getCurrentUser() == owner_ && objectType_ != COM_VIEW_OBJECT)
   {
     privInfo_ = new(heap_) PrivMgrUserPrivs;
     privInfo_->setOwnerDefaultPrivs();
@@ -6741,7 +6749,7 @@ void NATable::readPrivileges ()
 	std::vector <ComSecurityKey *> secKeyVec;
 
 	if (testError || (STATUS_GOOD !=
-				privInterface.getPrivileges(objectUid().get_value(), objectType_,
+				privInterface.getPrivileges((NATable *)this,
 					ComUser::getCurrentUser(), *privInfo_, &secKeyVec)))
 	{
 		if (testError)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/optimizer/RelExeUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExeUtil.cpp b/core/sql/optimizer/RelExeUtil.cpp
index 4dedcbc..a7ced3c 100644
--- a/core/sql/optimizer/RelExeUtil.cpp
+++ b/core/sql/optimizer/RelExeUtil.cpp
@@ -5164,6 +5164,36 @@ RelExpr * ExeUtilHiveTruncate::bindNode(BindWA *bindWA)
       return NULL;
     }
 
+  // If the current user has been granted the Trafodion Hive/DB root role or
+  // is DB__ROOT, allow the operation. 
+  // If the current user has select and delete privileges, allow the operation
+  if (bindWA->currentCmpContext()->isAuthorizationEnabled())
+  {
+    NABoolean found = FALSE;
+    if (ComUser::isRootUserID() ||
+        ComUser::currentUserHasRole(HIVE_ROLE_ID) ||
+        ComUser::currentUserHasRole(ROOT_ROLE_ID))
+      found = TRUE;
+
+    if (!found)
+    {
+      PrivMgrUserPrivs *pPrivInfo = naTable->getPrivInfo();
+      if (pPrivInfo &&
+          pPrivInfo->hasPriv(SELECT_PRIV) &&
+          pPrivInfo->hasPriv(DELETE_PRIV))
+        found = TRUE;
+
+      if (!found)
+      {
+        *CmpCommon::diags()
+           << DgSqlCode( -1051 )
+           << DgTableName(naTable->getTableName().getQualifiedNameAsAnsiString());
+        bindWA->setErrStatus();
+        return NULL;
+      }
+    }
+  }
+
   const HHDFSTableStats* hTabStats = 
     naTable->getClusteringIndex()->getHHDFSTableStats();
   

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/regress/privs1/EXPECTED141
----------------------------------------------------------------------
diff --git a/core/sql/regress/privs1/EXPECTED141 b/core/sql/regress/privs1/EXPECTED141
index b846af7..7735750 100644
Binary files a/core/sql/regress/privs1/EXPECTED141 and b/core/sql/regress/privs1/EXPECTED141 differ

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/regress/privs1/TEST123
----------------------------------------------------------------------
diff --git a/core/sql/regress/privs1/TEST123 b/core/sql/regress/privs1/TEST123
index 366200d..f32cf61 100644
--- a/core/sql/regress/privs1/TEST123
+++ b/core/sql/regress/privs1/TEST123
@@ -1,5 +1,5 @@
 -- ============================================================================
--- TEST123 - tests revoke query invalidation
+-- TEST123 - tests get statements
 --
 -- @@@ START COPYRIGHT @@@
 --

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/regress/privs1/TEST141
----------------------------------------------------------------------
diff --git a/core/sql/regress/privs1/TEST141 b/core/sql/regress/privs1/TEST141
index a605f64..273bfeb 100755
--- a/core/sql/regress/privs1/TEST141
+++ b/core/sql/regress/privs1/TEST141
@@ -202,7 +202,7 @@ sh sqlci -i "TEST141(user3_objects)" -u sql_user3;
 ?section test_shared_user
 -- ============================================================================
 -- verify someone with CREATE permission can create objects in someone elses
--- shared schema.  Make sure the schema owner owns the object and the creator
+-- shared schema.  Make sure the current user owns the object and the creator
 -- has appropriate privileges.
 
 values (user);
@@ -243,7 +243,7 @@ execute get_obj_privs;
 sh sqlci -i "TEST141(user1_objects)" -u sql_user1;
 
 -- grant privileges to sql_user1 and retry
-grant select on t141_user1.u1t1 to sql_user1;
+grant select, insert, delete on t141_user1.u1t1 to sql_user1;
 grant select on t141_user1.u2t1 to sql_user1; 
 sh sqlci -i "TEST141(user1_objects)" -u sql_user1;
 execute get_obj_privs;
@@ -286,7 +286,7 @@ execute get_obj_privs;
 ?section test_shared_role
 -- ============================================================================
 -- verify someone with CREATE permission can create objects in someone elses
--- shared schema.  Make sure the schema owner owns the object and the creator
+-- shared schema.  Make sure the current user owns the object and the creator
 -- has appropriate privileges.
 
 values (user);
@@ -324,7 +324,11 @@ values (user);
 
 set schema t141_user1;
 create view u1v1 as select * from u1t1;
+insert into u1v1 values (6,6,6);
+delete from u1v1 where c1 = 6;
 create view u1v2 as select * from u2t1;
+insert into u1v2 values (6,6,6);
+delete from u1v2 where c1 = 6;
 create view u1v3(c1, c2) as select u1.c1, u2.c1 from u1t1 u1, u2t1 u2;
 
 -- ============================================================================

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/regress/privs2/EXPECTED144
----------------------------------------------------------------------
diff --git a/core/sql/regress/privs2/EXPECTED144 b/core/sql/regress/privs2/EXPECTED144
index d89bd39..70d088f 100644
Binary files a/core/sql/regress/privs2/EXPECTED144 and b/core/sql/regress/privs2/EXPECTED144 differ

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpDescribe.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpDescribe.cpp b/core/sql/sqlcomp/CmpDescribe.cpp
index 32fdd27..c9e98e3 100644
--- a/core/sql/sqlcomp/CmpDescribe.cpp
+++ b/core/sql/sqlcomp/CmpDescribe.cpp
@@ -2256,6 +2256,12 @@ short CmpDescribeHiveTable (
   // emit an initial newline
   outputShortLine(space, " ");
 
+  if (!CmpDescribeIsAuthorized(SQLOperation::UNKNOWN,
+                               naTable->getPrivInfo(),
+                               COM_BASE_TABLE_OBJECT))
+    return -1;
+
+
   if (type == 1)
     {
       sprintf(buf,  "-- Definition of hive table %s\n"
@@ -2817,8 +2823,7 @@ short CmpDescribeSeabaseTable (
 
   NABoolean displayPrivilegeGrants = TRUE;
   if (((CmpCommon::getDefault(SHOWDDL_DISPLAY_PRIVILEGE_GRANTS) == DF_SYSTEM) && sqlmxRegr) ||
-      (CmpCommon::getDefault(SHOWDDL_DISPLAY_PRIVILEGE_GRANTS) == DF_OFF) ||
-      (isExternalTable))
+      (CmpCommon::getDefault(SHOWDDL_DISPLAY_PRIVILEGE_GRANTS) == DF_OFF)) 
     displayPrivilegeGrants = FALSE;
  
   // display syscols for invoke if not running regrs

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp b/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
index ad3c3dd..ea2d3f2 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
@@ -265,6 +265,80 @@ CmpSeabaseDDLauth::AuthStatus CmpSeabaseDDLauth::getRoleIDs(
 }
 
 // ----------------------------------------------------------------------------
+// public method:  getObjectName
+//
+// Returns the first object name from the list of passed in objectUIDs.
+//
+// Input:
+//    objectUIDs - list of objectUIDs
+//
+//  Output:
+//    returns the fully qualified object name
+//    returns NULL string if no objects were found
+// ----------------------------------------------------------------------------
+NAString CmpSeabaseDDLauth::getObjectName (const std::vector <int64_t> objectUIDs)
+{
+  char longBuf [sizeof(int64_t)*8+1];
+  bool isFirst = true;
+  NAString objectList;
+  NAString objectName;
+
+  if (objectUIDs.size() == 0)
+    return objectName;
+
+  // convert objectUIDs into an "in" clause list
+  for (int i = 0; i < objectUIDs.size(); i++)
+  {
+    if (isFirst)
+      objectList = "(";
+    else
+      objectList += ", ";
+    isFirst = false;
+    sprintf (longBuf, "%ld", objectUIDs[i]);
+    objectList += longBuf;
+  }
+  objectList += ")";
+
+  NAString sysCat = CmpSeabaseDDL::getSystemCatalogStatic();
+  char buf[1000 + objectList.length()];
+  str_sprintf(buf, "select [first 1] trim(catalog_name) || '.' || "
+                   "trim(schema_name) || '.' || trim(object_name) "
+                   " from %s.\"%s\".%s where object_uid in %s",
+                   sysCat.data(), SEABASE_MD_SCHEMA, SEABASE_OBJECTS, objectList.data());
+
+  ExeCliInterface cliInterface(STMTHEAP);
+  Int32 cliRC = cliInterface.fetchRowsPrologue(buf, true/*no exec*/);
+  if (cliRC != 0)
+  {
+    cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+    UserException excp (NULL, 0);
+    throw excp;
+  }
+
+  cliRC = cliInterface.clearExecFetchClose(NULL, 0);
+  if (cliRC < 0)
+  {
+    cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+    UserException excp (NULL, 0);
+    throw excp;
+  }
+
+  // return an empty string
+  if (cliRC == 100)
+    return objectName;
+
+  // get the objectname
+  char * ptr = NULL;
+  Lng32 len = 0;
+
+  // object name returned
+  cliInterface.getPtrAndLen(1,ptr,len);
+  NAString returnedName(ptr,len);
+  return returnedName;
+}
+
+
+// ----------------------------------------------------------------------------
 // method:  getUniqueID
 //
 // This method is not valid for the base class
@@ -985,10 +1059,18 @@ void CmpSeabaseDDLuser::unregisterUser(StmtDDLRegisterUser * pNode)
     
     privClasses.push_back(PrivClass::ALL);
     
-    if (privMgr.isAuthIDGrantedPrivs(getAuthID(),privClasses))
+    std::vector<int64_t> objectUIDs;
+    if (privMgr.isAuthIDGrantedPrivs(getAuthID(),privClasses, objectUIDs))
     {
-       *CmpCommon::diags() << DgSqlCode(-CAT_NO_UNREG_USER_HAS_PRIVS);
-       return;
+       NAString objectName = getObjectName(objectUIDs);
+       if (objectName.length() > 0)
+       {
+          *CmpCommon::diags() << DgSqlCode(-CAT_NO_UNREG_USER_HAS_PRIVS)
+                              << DgString0(dbUserName.data())
+                              << DgString1(objectName.data());
+
+          return;
+       }
     }
     
     // delete the row
@@ -1727,11 +1809,18 @@ void CmpSeabaseDDLrole::dropRole(StmtDDLCreateRole * pNode)
          std::vector<PrivClass> privClasses;
          
          privClasses.push_back(PrivClass::ALL);
-         
-         if (privMgr.isAuthIDGrantedPrivs(getAuthID(),privClasses))
+         std::vector<int64_t> objectUIDs;
+
+         if (privMgr.isAuthIDGrantedPrivs(getAuthID(),privClasses, objectUIDs))
          {
-            *CmpCommon::diags() << DgSqlCode(-CAT_ROLE_HAS_PRIVS_NO_DROP);
-            return;
+            NAString objectName = getObjectName(objectUIDs);
+            if (objectName.length() > 0)
+            {
+              *CmpCommon::diags() << DgSqlCode(-CAT_ROLE_HAS_PRIVS_NO_DROP)
+                                  << DgString0(roleName.data())
+                                  << DgString1(objectName.data());
+              return;
+            }
          }
 
          // Role has not been granted and no privileges have been granted to

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpSeabaseDDLauth.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLauth.h b/core/sql/sqlcomp/CmpSeabaseDDLauth.h
index 5ccda8a..7ecf96d 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLauth.h
+++ b/core/sql/sqlcomp/CmpSeabaseDDLauth.h
@@ -77,6 +77,7 @@ class CmpSeabaseDDLauth
                                     NAString &authText);
      AuthStatus   getRoleIDs     (const Int32 authID,
                                     std::vector<int32_t> &roleIDs);
+     NAString     getObjectName  (const std::vector <int64_t> objectUIDs);
 
      // accessors
      Int32          getAuthCreator() const    { return authCreator_; }

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
index f862963..6e69a95 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
@@ -7005,6 +7005,14 @@ short CmpSeabaseDDL::updateSeabaseAuths(
 void CmpSeabaseDDL::initSeabaseMD(NABoolean ddlXns, NABoolean minimal)
 {
   int breadCrumb = -1;  // useful for debugging
+
+  // verify user is authorized
+  if (!ComUser::isRootUserID())
+    {
+       *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+       return;
+    }
+
   Lng32 retcode = 0;
   Lng32 cliRC = 0;
   NABoolean xnWasStartedHere = FALSE;
@@ -7289,6 +7297,12 @@ void CmpSeabaseDDL::initSeabaseMD(NABoolean ddlXns, NABoolean minimal)
 
 void CmpSeabaseDDL::createSeabaseMDviews()
 {
+  if (!ComUser::isRootUserID())
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      return;
+    }
+
   Lng32 retcode = 0;
   Lng32 cliRC = 0;
 
@@ -7311,6 +7325,13 @@ void CmpSeabaseDDL::createSeabaseMDviews()
 
 void CmpSeabaseDDL::dropSeabaseMDviews()
 {
+  // verify user is authorized
+  if (!ComUser::isRootUserID())
+    {
+       *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+       return;
+    }
+
   Lng32 retcode = 0;
   Lng32 cliRC = 0;
 
@@ -7333,6 +7354,12 @@ void CmpSeabaseDDL::dropSeabaseMDviews()
 
 void CmpSeabaseDDL::createSeabaseSchemaObjects()
 {
+  if (!ComUser::isRootUserID())
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+       return;
+    }
+
   Lng32 retcode = 0;
   Lng32 cliRC = 0;
 
@@ -7994,6 +8021,13 @@ short CmpSeabaseDDL::dropSeabaseObjectsFromHbase(const char * pattern,
 
 void CmpSeabaseDDL::dropSeabaseMD(NABoolean ddlXns)
 {
+  // verify user is authorized
+  if (!ComUser::isRootUserID())
+    {
+       *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+       return;
+    }
+
   Lng32 cliRC;
   Lng32 retcode = 0;
   NABoolean xnWasStartedHere = FALSE;
@@ -8377,6 +8411,12 @@ NABoolean CmpSeabaseDDL::deletePrivMgrInfo(const NAString &objectName,
 
 short CmpSeabaseDDL::dropMDTable(ExpHbaseInterface *ehi, const char * tab)
 {
+  if (!ComUser::isRootUserID())
+  {
+     *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+     return -1;
+  }
+
   Lng32 retcode = 0;
 
   HbaseStr hbaseObjStr;
@@ -8400,6 +8440,12 @@ short CmpSeabaseDDL::dropMDTable(ExpHbaseInterface *ehi, const char * tab)
 
 void CmpSeabaseDDL::updateVersion()
 {
+  if (!ComUser::isRootUserID())
+    {
+       *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+       return;
+    }
+
   Lng32 retcode = 0;
   Lng32 cliRC = 0;
 
@@ -8932,9 +8978,12 @@ short CmpSeabaseDDL::executeSeabaseDDL(DDLExpr * ddlExpr, ExprNode * ddlNode,
            (ddlExpr->dropRepos()) ||
            (ddlExpr->upgradeRepos()))
     {
-      processRepository(ddlExpr->createRepos(), 
-                        ddlExpr->dropRepos(), 
-                        ddlExpr->upgradeRepos());
+      if (!ComUser::isRootUserID())
+          *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      else
+        processRepository(ddlExpr->createRepos(), 
+                          ddlExpr->dropRepos(), 
+                          ddlExpr->upgradeRepos());
     }
   else
     {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp b/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
index d444832..67148e1 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
@@ -1590,6 +1590,12 @@ short CmpSeabaseDDL::validateRoutine(ExeCliInterface *cliInterface,
 
 short CmpSeabaseDDL::createSeabaseLibmgr(ExeCliInterface * cliInterface)
 {
+  if (!ComUser::isRootUserID())
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      return -1;
+    }
+
   Lng32 cliRC = 0;
   
   if ((CmpCommon::context()->isUninitializedSeabase()) &&
@@ -1707,6 +1713,12 @@ short CmpSeabaseDDL::grantLibmgrPrivs(ExeCliInterface *cliInterface)
 
 short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
 {
+  if (!ComUser::isRootUserID())
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      return -1;
+    }
+
   Lng32 cliRC = 0;
 
   cliRC = existsInSeabaseMDTable(cliInterface,
@@ -1730,6 +1742,12 @@ short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
 
 short CmpSeabaseDDL::dropSeabaseLibmgr(ExeCliInterface *cliInterface)
 {
+    if (!ComUser::isRootUserID())
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      return -1;
+    }
+
   Lng32 cliRC = 0;
 
   char queryBuf[strlen(getSystemCatalog()) + strlen(SEABASE_LIBMGR_SCHEMA) + 100];

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp b/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
index f850449..046ece1 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
@@ -249,17 +249,30 @@ void CmpSeabaseDDL::createSeabaseSchema(
    Int32 objectOwner = NA_UserIdDefault;
    Int32 schemaOwner = NA_UserIdDefault;
 
-   int32_t retCode = verifyDDLCreateOperationAuthorized(&cliInterface,
-                                                        SQLOperation::CREATE_SCHEMA,
-                                                        catName,
-                                                        schName,
-                                                        schemaClass,
-                                                        objectOwner,
-                                                        schemaOwner);
-   if (retCode != 0)
+
+   // If creating the hive statistics schema, make owners
+   // the HIVE_ROLE_ID and skip authorization check.
+   // Schema is being created as part of an update statistics cmd
+   if (schName == HIVE_STATS_SCHEMA_NO_QUOTES &&
+       Get_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL))
    {
-      handleDDLCreateAuthorizationError(retCode,catName,schName);
-      return;
+      objectOwner = HIVE_ROLE_ID;
+      schemaOwner = HIVE_ROLE_ID;
+   }
+   else
+   {
+      int32_t retCode = verifyDDLCreateOperationAuthorized(&cliInterface,
+                                                           SQLOperation::CREATE_SCHEMA,
+                                                           catName,
+                                                           schName,
+                                                           schemaClass,
+                                                           objectOwner,
+                                                           schemaOwner);
+      if (retCode != 0)
+      {
+         handleDDLCreateAuthorizationError(retCode,catName,schName);
+         return;
+      }
    }
    
    Int32 schemaOwnerID = NA_UserIdDefault; 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp b/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
index 93f16be..4a7d2d9 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
@@ -9541,6 +9541,25 @@ void CmpSeabaseDDL::seabaseGrantRevoke(
         return;
       }
 
+  // If column privs specified for non SELECT ops for Hive native tables, 
+  // return an error
+  if (cn.isHive() && (colPrivs.size() > 0))
+    {
+      if (hasValue(colPrivs, INSERT_PRIV) ||
+          hasValue(colPrivs, UPDATE_PRIV) ||
+          hasValue(colPrivs, REFERENCES_PRIV))
+      {
+         NAString text1("INSERT, UPDATE, REFERENCES");
+         NAString text2("Hive columns on");
+         *CmpCommon::diags() << DgSqlCode(-CAT_INVALID_PRIV_FOR_OBJECT)
+                             << DgString0(text1.data())
+                             << DgString1(text2.data())
+                             << DgTableName(extTableName);
+         processReturn();
+         return;
+      }
+    }
+
  // Prepare to call privilege manager
   NAString MDLoc;
   CONCAT_CATSCH(MDLoc, getSystemCatalog(), SEABASE_MD_SCHEMA);
@@ -9577,6 +9596,61 @@ void CmpSeabaseDDL::seabaseGrantRevoke(
       return;
     }
 
+  // for Hive tables, the table must have an external table defined
+  if (objectUID == 0 && naTable &&
+      (naTable->isHiveTable() &&
+      !naTable->hasExternalTable()))
+    {
+      // For native hive tables, grantor must be DB__ROOT or belong
+      // to one of the admin roles:  DB__ROOTROLE, DB__HIVEROLE
+      // In hive, you must be an admin, DB__ROOTROLE and DB__HIVEROLE
+      // is the equivalent of an admin.
+      if (!ComUser::isRootUserID() &&
+          !ComUser::currentUserHasRole(ROOT_ROLE_ID) &&
+          !ComUser::currentUserHasRole(HIVE_ROLE_ID)) 
+        {
+          *CmpCommon::diags() << DgSqlCode (-CAT_NOT_AUTHORIZED);
+          processReturn();
+          return;
+        }
+
+      // create an external table for this hive table.
+      // if the trafodion schema containing hive table descriptions does
+      // not exists, the "create external" table command creates it.
+      char query[(ComMAX_ANSI_IDENTIFIER_EXTERNAL_LEN*4) + 100];
+      snprintf(query, sizeof(query),
+               "create external table \"%s\" for %s.\"%s\".\"%s\"",
+               objectNamePart.data(),
+               catalogNamePart.data(),
+               schemaNamePart.data(),
+               objectNamePart.data());
+      Lng32 retcode = cliInterface.executeImmediate(query);
+      if (retcode < 0)
+        {
+          cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+          return;
+        }
+
+      // remove NATable and reload it to include external table defn.
+      ActiveSchemaDB()->getNATableDB()->removeNATable
+        (cn,
+         ComQiScope::REMOVE_MINE_ONLY, COM_BASE_TABLE_OBJECT,
+         FALSE, FALSE);
+
+      naTable = bindWA.getNATable(cn);
+      if (naTable == NULL)
+        {
+          SEABASEDDL_INTERNAL_ERROR("Bad NATable pointer in seabaseGrantRevoke");
+          return;
+        }
+
+      objectUID = (int64_t)naTable->objectUid().get_value();
+      objectOwnerID = (int32_t)naTable->getOwner();
+      schemaOwnerID = naTable->getSchemaOwner();
+      objectType = naTable->getObjectType();
+    }
+
+
   // for metadata tables, the objectUID is not initialized in the NATable
   // structure
   if (objectUID == 0)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/CmpSeabaseDDLupgrade.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLupgrade.cpp b/core/sql/sqlcomp/CmpSeabaseDDLupgrade.cpp
index 7cdc9aa..5703769 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLupgrade.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLupgrade.cpp
@@ -355,6 +355,13 @@ short CmpSeabaseMDupgrade::executeSeabaseMDupgrade(CmpDDLwithStatusInfo *mdui,
 	    mdui->setSubstep(0);
 	    mdui->setEndStep(TRUE);
 	    
+            if (!ComUser::isRootUserID())
+              {
+                 mdui->setMsg("Metadata Upgrade: not authorized");
+                 mdui->setStep(UPGRADE_FAILED);
+                 mdui->setSubstep(0);
+              }
+
 	    if (sendAllControlsAndFlags())
 	      {
 		mdui->setStep(UPGRADE_FAILED);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/PrivMgr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgr.cpp b/core/sql/sqlcomp/PrivMgr.cpp
index b75f4b2..817c796 100644
--- a/core/sql/sqlcomp/PrivMgr.cpp
+++ b/core/sql/sqlcomp/PrivMgr.cpp
@@ -606,9 +606,9 @@ const char * PrivMgr::getSQLOperationDescription(SQLOperation operation)
 // *                                                                           *
 // *****************************************************************************
 bool PrivMgr::isAuthIDGrantedPrivs(
-   int32_t authID,
-   std::vector<PrivClass> privClasses) 
-
+   const int32_t authID,
+   std::vector<PrivClass> privClasses,
+   std::vector<int64_t> &objectUIDs) 
 {
 
 // Check for empty vector.
@@ -626,7 +626,7 @@ bool PrivMgr::isAuthIDGrantedPrivs(
    {
       PrivMgrPrivileges objectPrivileges(metadataLocation_,pDiags_); 
       
-      if (objectPrivileges.isAuthIDGrantedPrivs(authID))
+      if (objectPrivileges.isAuthIDGrantedPrivs(authID, objectUIDs))
          return true;
       
       PrivMgrComponentPrivileges componentPrivileges(metadataLocation_,pDiags_); 
@@ -646,7 +646,7 @@ bool PrivMgr::isAuthIDGrantedPrivs(
          {
             PrivMgrPrivileges objectPrivileges(metadataLocation_,pDiags_); 
             
-            if (objectPrivileges.isAuthIDGrantedPrivs(authID))
+            if (objectPrivileges.isAuthIDGrantedPrivs(authID, objectUIDs))
                return true;
              
             break;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/PrivMgr.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgr.h b/core/sql/sqlcomp/PrivMgr.h
index e7f412e..925a9af 100644
--- a/core/sql/sqlcomp/PrivMgr.h
+++ b/core/sql/sqlcomp/PrivMgr.h
@@ -168,8 +168,9 @@ class PrivMgr
     bool isAuthorizationEnabled(void); 
     void setAuthorizationEnabled(PrivMDStatus authStatus) {authorizationEnabled_ = authStatus;}
     bool isAuthIDGrantedPrivs(
-       int32_t authID,
-       std::vector<PrivClass> privClasses);
+       const int32_t authID,
+       std::vector<PrivClass> privClasses,
+       std::vector<int64_t> &objectUIDs);
     void resetFlags();
     void setFlags();
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/PrivMgrCommands.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrCommands.cpp b/core/sql/sqlcomp/PrivMgrCommands.cpp
index 30b6eea..38a6539 100644
--- a/core/sql/sqlcomp/PrivMgrCommands.cpp
+++ b/core/sql/sqlcomp/PrivMgrCommands.cpp
@@ -536,6 +536,62 @@ PrivStatus PrivMgrCommands::getGrantorDetailsForObject(
 // ----------------------------------------------------------------------------
 // method: getPrivileges
 //
+// returns the list of privileges and security keys for the userID given the  
+// specified objectUID
+//
+// Input:
+//     naTable - a pointer to the object
+//     userID - user to gather privs
+//     userPrivileges - returns available privileges
+//     secKeySet - the security keys for the object/user
+//
+// returns true if results were found, false otherwise
+// The Trafodion diags area contains any errors that were encountered
+// ----------------------------------------------------------------------------
+PrivStatus PrivMgrCommands::getPrivileges(
+  NATable *naTable,
+  const int32_t userID,
+  PrivMgrUserPrivs &userPrivs,
+  std::vector <ComSecurityKey *>* secKeySet)
+{
+  PrivMgrDesc privsOfTheUser;
+
+  // authorization is not enabled, return bitmaps with all bits set
+  // With all bits set, privilege checks will always succeed
+  if (!authorizationEnabled())
+  {
+    privsOfTheUser.setAllTableGrantPrivileges(true);
+    userPrivs.initUserPrivs(privsOfTheUser);
+    return STATUS_GOOD;
+  }
+
+  // if a hive table and does not have an external table, assume no privs
+  if (naTable->isHiveTable() && !naTable->hasExternalTable())
+  {
+    PrivMgrDesc emptyDesc;
+    userPrivs.initUserPrivs(emptyDesc);
+  }
+  else
+  {
+    PrivMgrPrivileges objectPrivs (metadataLocation_, pDiags_);
+    PrivStatus retcode = objectPrivs.getPrivsOnObjectForUser((int64_t)naTable->objectUid().get_value(),
+                                                             naTable->getObjectType(),
+                                                             userID,
+                                                             privsOfTheUser,
+                                                             secKeySet);
+    if (retcode != STATUS_GOOD)
+      return retcode;
+
+    userPrivs.initUserPrivs(privsOfTheUser);
+  }
+
+  return STATUS_GOOD;
+}
+
+
+// ----------------------------------------------------------------------------
+// method: getPrivileges
+//
 // Creates a set of priv descriptors for all user grantees on an object
 // Used by Trafodion compiler to store as part of the table descriptor.
 //                                                       
@@ -595,9 +651,9 @@ PrivStatus PrivMgrCommands::getPrivileges(
   {
     PrivMgrPrivileges objectPrivs (metadataLocation_, pDiags_);
     PrivStatus retcode = objectPrivs.getPrivsOnObjectForUser(objectUID,
-                                                             objectType, 
-                                                             userID, 
-                                                             privsOfTheUser, 
+                                                             objectType,
+                                                             userID,
+                                                             privsOfTheUser,
                                                              secKeySet);
     if (retcode != STATUS_GOOD)
       return retcode;
@@ -609,11 +665,10 @@ PrivStatus PrivMgrCommands::getPrivileges(
     privsOfTheUser.getTablePrivs().setAllPrivAndWgo(true);
 
   userPrivs.initUserPrivs(privsOfTheUser);
-    
+
   return STATUS_GOOD;
 }
 
-
 // ----------------------------------------------------------------------------
 // method: getRoles
 //

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/PrivMgrCommands.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrCommands.h b/core/sql/sqlcomp/PrivMgrCommands.h
index 74849c3..3ca8510 100644
--- a/core/sql/sqlcomp/PrivMgrCommands.h
+++ b/core/sql/sqlcomp/PrivMgrCommands.h
@@ -474,6 +474,12 @@ public:
       std::string &effectiveGrantorName);
 
    PrivStatus getPrivileges(
+      NATable *naTable,
+      const int32_t granteeUID,
+      PrivMgrUserPrivs &userPrivileges,
+      std::vector <ComSecurityKey *>* secKeySet = NULL);
+
+   PrivStatus getPrivileges(
       const int64_t objectUID,
       ComObjectType objectType,
       std::vector<PrivMgrDesc> &userPrivileges);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/PrivMgrPrivileges.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrPrivileges.cpp b/core/sql/sqlcomp/PrivMgrPrivileges.cpp
index ba4b82a..016c42b 100644
--- a/core/sql/sqlcomp/PrivMgrPrivileges.cpp
+++ b/core/sql/sqlcomp/PrivMgrPrivileges.cpp
@@ -1681,7 +1681,8 @@ PrivStatus privStatus = objectPrivsTable.insert(row);
 // no need to grant privileges.
 // 
 // This creator grant may be controlled by a CQD in the future.
-   if (ownerID == creatorID || creatorID == ComUser::getRootUserID())
+   if (ownerID == creatorID || creatorID == ComUser::getRootUserID() ||
+       ownerID == HIVE_ROLE_ID || ownerID == HBASE_ROLE_ID)
       return STATUS_GOOD;
  
 // Add a grant from the private schema owner to the creator.     
@@ -4785,11 +4786,13 @@ PrivStatus PrivMgrPrivileges::updateDependentObjects(
 // * false: Authorization ID has not been granted any object privileges.     
 // *                                                               
 // *****************************************************************************
-bool PrivMgrPrivileges::isAuthIDGrantedPrivs(const int32_t authID)
-
+bool PrivMgrPrivileges::isAuthIDGrantedPrivs(
+  const int32_t authID, 
+  std::vector<int64_t> &objectUIDs)
 {
 
    std::string whereClause(" WHERE GRANTEE_ID = ");   
+   std::string orderByClause (" ORDER BY object_name ");
 
    char authIDString[20];
 
@@ -4800,10 +4803,23 @@ bool PrivMgrPrivileges::isAuthIDGrantedPrivs(const int32_t authID)
    // set pointer in diags area
    int32_t diagsMark = pDiags_->mark();
 
-   int64_t rowCount = 0;   
+   std::vector<PrivMgrMDRow *> rowList;
    ObjectPrivsMDTable myTable(objectTableName_,pDiags_);
 
-   PrivStatus privStatus = myTable.selectCountWhere(whereClause,rowCount);
+   PrivStatus privStatus = myTable.selectWhere(whereClause, orderByClause ,rowList);
+   if (privStatus != STATUS_GOOD)
+   {
+      deleteRowList(rowList);
+      return privStatus;
+   }
+
+   int32_t rowCount = rowList.size();
+   for (size_t i = 0; i < rowCount; i++)
+   {
+      ObjectPrivsMDRow &row = static_cast<ObjectPrivsMDRow &> (*rowList[i]);
+      objectUIDs.push_back(row.objectUID_);
+   }
+   deleteRowList(rowList);
 
    if ((privStatus == STATUS_GOOD || privStatus == STATUS_WARNING) &&
         rowCount > 0)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/PrivMgrPrivileges.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrPrivileges.h b/core/sql/sqlcomp/PrivMgrPrivileges.h
index 285f6e1..ede37c1 100644
--- a/core/sql/sqlcomp/PrivMgrPrivileges.h
+++ b/core/sql/sqlcomp/PrivMgrPrivileges.h
@@ -223,7 +223,9 @@ public:
   // -------------------------------------------------------------------
   // helpers
   // -------------------------------------------------------------------
-  bool isAuthIDGrantedPrivs(const int32_t authID);
+  bool isAuthIDGrantedPrivs(
+    const int32_t authID,
+    std::vector<int64_t> &objectUIDs);
 
 protected:
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/sqlcomp/PrivMgrRoles.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrRoles.cpp b/core/sql/sqlcomp/PrivMgrRoles.cpp
index 4419c85..8bf9023 100644
--- a/core/sql/sqlcomp/PrivMgrRoles.cpp
+++ b/core/sql/sqlcomp/PrivMgrRoles.cpp
@@ -283,30 +283,32 @@ bool PrivMgrRoles::dependentObjectsExist(
 
 {
 
-// First, determine if the role has been granted any object privileges.
-// Only REFERENCES, SELECT and USAGE are relevant, but for now check all DML.
-// If no granted privileges, then there are no objects created by this user
-// depending on privileges granted to this role.
+   // First, determine if the role has been granted any object privileges.
+   // Only REFERENCES, SELECT and USAGE are relevant, but for now check all DML.
+   // If no granted privileges, then there are no objects created by this user
+   // depending on privileges granted to this role.
 
-std::vector<PrivClass> privClass;
+   std::vector<PrivClass> privClass;
 
    privClass.push_back(PrivClass::OBJECT);
 
-   if (!isAuthIDGrantedPrivs(roleID,privClass))
+  
+   std::vector<int64_t> objectUIDs;
+   if (!isAuthIDGrantedPrivs(roleID,privClass, objectUIDs))
       return false;
       
-// Second, see if the user owns any objects that are dependent, i.e., views,
-// referential integrity (RI) constraints, or UDFs/SPJs.
+   // Second, see if the user owns any objects that are dependent, i.e., views,
+   // referential integrity (RI) constraints, or UDFs/SPJs.
 
-std::string whereClause(" WHERE OBJECT_TYPE IN ('RC','SP','UR','VI') AND OBJECT_OWNER = ");
+   std::string whereClause(" WHERE OBJECT_TYPE IN ('RC','SP','UR','VI') AND OBJECT_OWNER = ");
 
    whereClause += authIDToString(userID);
    
-PrivMgrObjects objects(trafMetadataLocation_,pDiags_);
+   PrivMgrObjects objects(trafMetadataLocation_,pDiags_);
 
-std::vector<UIDAndType> ownedUIDandTypes;
+   std::vector<UIDAndType> ownedUIDandTypes;
 
-PrivStatus privStatus = objects.fetchUIDandTypes(whereClause,ownedUIDandTypes);
+   PrivStatus privStatus = objects.fetchUIDandTypes(whereClause,ownedUIDandTypes);
 
    if (privStatus == STATUS_ERROR)
    {
@@ -314,30 +316,30 @@ PrivStatus privStatus = objects.fetchUIDandTypes(whereClause,ownedUIDandTypes);
       return true;
    }
    
-// If the user does not own any views, user defined functions, or referential
-// integrity constraints, they do not have any dependent objects.
+   // If the user does not own any views, user defined functions, or referential
+   // integrity constraints, they do not have any dependent objects.
 
    if (privStatus == STATUS_NOTFOUND || ownedUIDandTypes.size() == 0)
       return false;   
       
-//TODO: User owns dependent objects and role has been granted privileges.
-//      But do they correlate?  For instance, if a role is granted INSERT,
-//      losing that privilege does not affect any views the user may own.
-//      Could get list of privilege types granted the role, and then
-//      compare with types of dependent objects.  If no correlation,
-//      short-circuit this check.  See file private function 
-//      isDependentObjectPrivPair().
+   //TODO: User owns dependent objects and role has been granted privileges.
+   //      But do they correlate?  For instance, if a role is granted INSERT,
+   //      losing that privilege does not affect any views the user may own.
+   //      Could get list of privilege types granted the role, and then
+   //      compare with types of dependent objects.  If no correlation,
+   //      short-circuit this check.  See file private function 
+   //      isDependentObjectPrivPair().
    
-// OK, user owns dependent objects and role has been granted privileges.  
-// One of more of those objects could depend on privileges granted this role.
-// Get the name of the role to use in error messages.   
-int32_t length;
+   // OK, user owns dependent objects and role has been granted privileges.  
+   // One of more of those objects could depend on privileges granted this role.
+   // Get the name of the role to use in error messages.   
+   int32_t length;
 
-char roleName[MAX_DBUSERNAME_LEN + 1];
+   char roleName[MAX_DBUSERNAME_LEN + 1];
          
-Int16 retCode = ComUser::getAuthNameFromAuthID(roleID,roleName,
+   Int16 retCode = ComUser::getAuthNameFromAuthID(roleID,roleName,
                                                sizeof(roleName),length);
-// Should not fail, role ID was derived from name provided by user.
+   // Should not fail, role ID was derived from name provided by user.
    if (retCode != 0)
    {
       std::string errorText("Unable to look up role name for role ID ");
@@ -347,25 +349,25 @@ Int16 retCode = ComUser::getAuthNameFromAuthID(roleID,roleName,
       return true;
    }
 
-// For each owned object, get the objects that the owned objects reference.
-// Determine if the user has the requisite privilege on each of the 
-// referenced objects if the privileges granted to the role are not included.
-// If the user has the privilege (either directly or through another role),
-// move on to the next referenced object, and then on to the next owned object.
-// If any dependency on a privilege granted to this role is found, it is an
-// error if the behavior is restrict.  For cascade (not yet supported), the
-// dependent object is automatically dropped.
-
-// For each referenced object, get privileges the user has on that object 
-// either directly or by another role granted to the user, but exclude 
-// any privileges from the role being revoked.  The first part of the query
-// can be built once, and the object UID for each referenced object added
-// within the loop.
-//
-// WHERE (GRANTEE_ID = -1 OR GRANTEE_ID = userID OR
-//        GRANTEE_ID IN (SELECT ROLE_ID FROM ROLE_USAGE WHERE GRANTEE_ID = userID)) AND 
-//        GRANTEE_ID <> roleID AND OBJECT_UID = objectUID; 
-std::string whereClauseHeader(" WHERE (GRANTEE_ID = -1 OR GRANTEE_ID = ");
+   // For each owned object, get the objects that the owned objects reference.
+   // Determine if the user has the requisite privilege on each of the 
+   // referenced objects if the privileges granted to the role are not included.
+   // If the user has the privilege (either directly or through another role),
+   // move on to the next referenced object, and then on to the next owned object.
+   // If any dependency on a privilege granted to this role is found, it is an
+   // error if the behavior is restrict.  For cascade (not yet supported), the
+   // dependent object is automatically dropped.
+
+   // For each referenced object, get privileges the user has on that object 
+   // either directly or by another role granted to the user, but exclude 
+   // any privileges from the role being revoked.  The first part of the query
+   // can be built once, and the object UID for each referenced object added
+   // within the loop.
+   //
+   // WHERE (GRANTEE_ID = -1 OR GRANTEE_ID = userID OR
+   //        GRANTEE_ID IN (SELECT ROLE_ID FROM ROLE_USAGE WHERE GRANTEE_ID = userID)) AND 
+   //        GRANTEE_ID <> roleID AND OBJECT_UID = objectUID; 
+   std::string whereClauseHeader(" WHERE (GRANTEE_ID = -1 OR GRANTEE_ID = ");
    
    whereClauseHeader += authIDToString(userID);
    whereClauseHeader += " OR GRANTEE_ID IN (SELECT RU.ROLE_ID FROM ";
@@ -376,12 +378,12 @@ std::string whereClauseHeader(" WHERE (GRANTEE_ID = -1 OR GRANTEE_ID = ");
    whereClauseHeader += authIDToString(roleID);
    whereClauseHeader += " AND OBJECT_UID = ";
 
-//TODO: When support is added for schema and column privileges, will need to
-//      check those privileges as well. 
+   //TODO: When support is added for schema and column privileges, will need to
+   //      check those privileges as well. 
 
-// Assume no dependencies exist.
-bool dependencyFound = false; 
-PrivMgrMDAdmin admin(trafMetadataLocation_,metadataLocation_,pDiags_);
+   // Assume no dependencies exist.
+   bool dependencyFound = false; 
+   PrivMgrMDAdmin admin(trafMetadataLocation_,metadataLocation_,pDiags_);
 
    for (size_t u3 = 0; u3 < ownedUIDandTypes.size(); u3++)
    {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/db14e392/core/sql/ustat/hs_globals.cpp
----------------------------------------------------------------------
diff --git a/core/sql/ustat/hs_globals.cpp b/core/sql/ustat/hs_globals.cpp
index df0594c..91c7bbf 100644
--- a/core/sql/ustat/hs_globals.cpp
+++ b/core/sql/ustat/hs_globals.cpp
@@ -3521,7 +3521,7 @@ NABoolean HSGlobalsClass::isAuthorized(NABoolean isShowStats)
 
    // no privilege support available for hbase and hive tables
    HS_ASSERT (objDef->getNATable());
-   if (CmpSeabaseDDL::isHbase(objDef->getCatName()) || isHiveCat(objDef->getCatName()))
+   if (CmpSeabaseDDL::isHbase(objDef->getCatName()))
      return TRUE;
 
   // Let keep track of how long authorization takes


[2/2] incubator-trafodion git commit: Merge [TRAFODION-2441] and others pr-957 various fixes for native Hive tables and privileges

Posted by rm...@apache.org.
Merge [TRAFODION-2441] and others pr-957 various fixes for native Hive tables and 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/cc13c6cc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/cc13c6cc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/cc13c6cc

Branch: refs/heads/master
Commit: cc13c6cc56d1f0725fdf90803808b2aeb1f4f5f2
Parents: f6f4402 db14e39
Author: Roberta Marton <rm...@edev07.esgyn.local>
Authored: Wed Feb 15 15:49:14 2017 +0000
Committer: Roberta Marton <rm...@edev07.esgyn.local>
Committed: Wed Feb 15 15:49:14 2017 +0000

----------------------------------------------------------------------
 core/sql/bin/SqlciErrors.txt              |   4 +-
 core/sql/common/ComUser.cpp               |  27 +++++++
 core/sql/common/ComUser.h                 |   2 +
 core/sql/optimizer/BindRelExpr.cpp        |  12 +--
 core/sql/optimizer/NATable.cpp            |  20 +++--
 core/sql/optimizer/RelExeUtil.cpp         |  30 +++++++
 core/sql/regress/privs1/EXPECTED141       | Bin 100853 -> 102260 bytes
 core/sql/regress/privs1/TEST123           |   2 +-
 core/sql/regress/privs1/TEST141           |  10 ++-
 core/sql/regress/privs2/EXPECTED144       | Bin 59409 -> 59453 bytes
 core/sql/sqlcomp/CmpDescribe.cpp          |   9 ++-
 core/sql/sqlcomp/CmpSeabaseDDLauth.cpp    | 103 ++++++++++++++++++++++--
 core/sql/sqlcomp/CmpSeabaseDDLauth.h      |   1 +
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp  |  55 ++++++++++++-
 core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp |  18 +++++
 core/sql/sqlcomp/CmpSeabaseDDLschema.cpp  |  33 +++++---
 core/sql/sqlcomp/CmpSeabaseDDLtable.cpp   |  74 +++++++++++++++++
 core/sql/sqlcomp/CmpSeabaseDDLupgrade.cpp |   7 ++
 core/sql/sqlcomp/PrivMgr.cpp              |  10 +--
 core/sql/sqlcomp/PrivMgr.h                |   5 +-
 core/sql/sqlcomp/PrivMgrCommands.cpp      |  65 +++++++++++++--
 core/sql/sqlcomp/PrivMgrCommands.h        |   6 ++
 core/sql/sqlcomp/PrivMgrPrivileges.cpp    |  26 ++++--
 core/sql/sqlcomp/PrivMgrPrivileges.h      |   4 +-
 core/sql/sqlcomp/PrivMgrRoles.cpp         | 106 +++++++++++++------------
 core/sql/ustat/hs_globals.cpp             |   2 +-
 26 files changed, 517 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/cc13c6cc/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
----------------------------------------------------------------------