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/03/10 17:17:00 UTC

[1/3] incubator-trafodion git commit: TRAFODION [1879] - Integrate Library Management into Trafodion Metadata

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master 5a6549b23 -> 41e575942


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp b/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
index 34a8cf0..67fd24f 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
@@ -553,7 +553,7 @@ void CmpSeabaseDDL::dropSeabaseSchema(StmtDDLDropSchema * dropSchemaNode)
      }
    }
 
-   // Drop libraries, procedures (SPJs), UDFs (functions), and views 
+   // Drop procedures (SPJs), UDFs (functions), and views 
     objectsQueue->position();
     for (int idx = 0; idx < objectsQueue->numEntries(); idx++)
     {
@@ -576,14 +576,9 @@ void CmpSeabaseDDL::dropSeabaseSchema(StmtDDLDropSchema * dropSchemaNode)
           case COM_REFERENTIAL_CONSTRAINT_OBJECT:
           case COM_SEQUENCE_GENERATOR_OBJECT:
           case COM_UNIQUE_CONSTRAINT_OBJECT:
-          {
-             continue;
-          }
           case COM_LIBRARY_OBJECT:
           {
-             objectTypeString = "LIBRARY";
-             cascade = "CASCADE";
-             break;
+             continue;
           }
 
           // If the library where procedures and functions reside is dropped
@@ -634,6 +629,29 @@ void CmpSeabaseDDL::dropSeabaseSchema(StmtDDLDropSchema * dropSchemaNode)
           someObjectsCouldNotBeDropped = true;
    } 
 
+   // Drop libraries in the schema
+   objectsQueue->position();
+   for (int idx = 0; idx < objectsQueue->numEntries(); idx++)
+   {
+      OutputInfo * vi = (OutputInfo*)objectsQueue->getNext();
+
+      char * objName = vi->get(0);
+      NAString objType = vi->get(1);
+
+      if (objType == COM_LIBRARY_OBJECT_LIT)
+      {
+         char buf [1000];
+
+         dirtiedMetadata = TRUE;
+         str_sprintf(buf, "DROP LIBRARY \"%s\".\"%s\".\"%s\" CASCADE",
+                     (char*)catName.data(), (char*)schName.data(), objName);
+         cliRC = cliInterface.executeImmediate(buf);
+
+         if (cliRC < 0 && cliRC != -CAT_OBJECT_DOES_NOT_EXIST_IN_TRAFODION)
+            someObjectsCouldNotBeDropped = true;
+      }
+   }
+
    // Drop all tables in the schema.  This will also drop any associated constraints. 
 
    objectsQueue->position();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrComponentOperations.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrComponentOperations.cpp b/core/sql/sqlcomp/PrivMgrComponentOperations.cpp
index f6b68fd..b58d400 100644
--- a/core/sql/sqlcomp/PrivMgrComponentOperations.cpp
+++ b/core/sql/sqlcomp/PrivMgrComponentOperations.cpp
@@ -468,11 +468,18 @@ PrivStatus PrivMgrComponentOperations::createOperationInternal(
    const std::string & operationDescription,
    const int32_t granteeID,
    const std::string & granteeName,
-   const int32_t grantDepth)
+   const int32_t grantDepth,
+   const bool checkExistence)
   
 {
 
-MyRow row(fullTableName_);
+   PrivStatus privStatus = STATUS_GOOD;
+
+   // If operation already created, no need to create
+   if (checkExistence && nameExists(componentUID,operationName))
+      return STATUS_GOOD;
+
+   MyRow row(fullTableName_);
 
    row.componentUID_ = componentUID;
    row.operationCode_ = operationCode;
@@ -480,17 +487,17 @@ MyRow row(fullTableName_);
    row.isSystem_ = isSystemOperation;
    row.operationDescription_ = operationDescription;
    
-MyTable &myTable = static_cast<MyTable &>(myTable_);
+   MyTable &myTable = static_cast<MyTable &>(myTable_);
 
-PrivStatus privStatus = myTable.insert(row);
+   privStatus = myTable.insert(row);
    
    if (privStatus != STATUS_GOOD)
       return privStatus;
       
-// Grant authority to creator
-PrivMgrComponentPrivileges componentPrivileges(metadataLocation_,pDiags_);
+   // Grant authority to creator
+   PrivMgrComponentPrivileges componentPrivileges(metadataLocation_,pDiags_);
 
-std::vector<std::string> operationCodes;
+   std::vector<std::string> operationCodes;
 
    operationCodes.push_back(operationCode);                                                     
                                                      
@@ -499,7 +506,8 @@ std::vector<std::string> operationCodes;
                                                            SYSTEM_USER,
                                                            ComUser::getSystemUserName(),
                                                            granteeID,
-                                                           granteeName,grantDepth);
+                                                           granteeName,grantDepth,
+                                                           checkExistence);
                                                      
    return privStatus;
                                                      

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrComponentOperations.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrComponentOperations.h b/core/sql/sqlcomp/PrivMgrComponentOperations.h
index a506d2f..7d0069d 100644
--- a/core/sql/sqlcomp/PrivMgrComponentOperations.h
+++ b/core/sql/sqlcomp/PrivMgrComponentOperations.h
@@ -71,7 +71,8 @@ public:
       const std::string & operationDescription,
       const int32_t granteeID,
       const std::string & granteeName,
-      const int32_t grantDepth);
+      const int32_t grantDepth,
+      const bool checkExistence);
       
   PrivStatus describeComponentOperations(
       const std::string & componentUIDString,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp b/core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp
index fae5d0e..70dd089 100644
--- a/core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp
+++ b/core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp
@@ -854,12 +854,13 @@ PrivStatus PrivMgrComponentPrivileges::grantPrivilegeInternal(
    const std::string & grantorName,
    const int32_t granteeID,
    const std::string & granteeName,
-   const int32_t grantDepth)
+   const int32_t grantDepth,
+   const bool checkExistence)
   
 {
 
-MyTable &myTable = static_cast<MyTable &>(myTable_);
-MyRow row(fullTableName_);
+   MyTable &myTable = static_cast<MyTable &>(myTable_);
+   MyRow row(fullTableName_);
 
    row.componentUID_ = componentUID;
    row.grantDepth_ = grantDepth;
@@ -868,9 +869,17 @@ MyRow row(fullTableName_);
    row.grantorID_ = grantorID;
    row.grantorName_ = grantorName;
    
+   const std::string componentUIDString = to_string((long long int)componentUID);
+
    for (size_t oc = 0; oc < operationCodes.size(); oc++)
    {
       row.operationCode_ = operationCodes[oc];
+
+      if (checkExistence &&
+          grantExists(componentUIDString, row.operationCode_, row.grantorID_,
+                      row.granteeID_, row.grantDepth_))
+         continue;
+
       PrivStatus privStatus = myTable.insert(row);
       
       if (privStatus != STATUS_GOOD)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrComponentPrivileges.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrComponentPrivileges.h b/core/sql/sqlcomp/PrivMgrComponentPrivileges.h
index 0b798a5..566b051 100644
--- a/core/sql/sqlcomp/PrivMgrComponentPrivileges.h
+++ b/core/sql/sqlcomp/PrivMgrComponentPrivileges.h
@@ -102,7 +102,8 @@ public:
       const std::string & grantorName,
       const int32_t granteeID,
       const std::string & granteeName,
-      const int32_t grantDepth);
+      const int32_t grantDepth,
+      const bool checkExistence);
       
    PrivStatus grantPrivilegeToCreator(
       const int64_t componentUID,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrMD.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrMD.cpp b/core/sql/sqlcomp/PrivMgrMD.cpp
index 811a71b..df0ca1c 100644
--- a/core/sql/sqlcomp/PrivMgrMD.cpp
+++ b/core/sql/sqlcomp/PrivMgrMD.cpp
@@ -159,24 +159,20 @@ PrivStatus PrivMgrMDAdmin::initializeComponentPrivileges()
    log(__FILE__, "initializing component privileges", -1);
    PrivStatus privStatus = STATUS_GOOD;
 
-   // First, let's start with a clean slate.  Drop all components as well as 
-   // their respective operations and and any privileges granted.  This should be  
-   // a NOP unless PrivMgr metadata was damaged and reintialization is occurring.
-
-   PrivMgrComponents components(metadataLocation_,pDiags_);
-   components.dropAll();
-   
-   // Next, register the component.
-
-   privStatus = components.registerComponentInternal(SQL_OPERATION_NAME,
-                                                     SQL_OPERATIONS_COMPONENT_UID,
-                                                     true,"Component for SQL operations");
-                                             
-   if (privStatus != STATUS_GOOD)
-   {
+  // First register the component.
+  PrivMgrComponents components(metadataLocation_,pDiags_);
+  bool componentExists = (components.exists(SQL_OPERATION_NAME));
+  if (!componentExists)
+  {
+    privStatus = components.registerComponentInternal(SQL_OPERATION_NAME,
+                                                      SQL_OPERATIONS_COMPONENT_UID,
+                                                      true,"Component for SQL operations");
+    if (privStatus != STATUS_GOOD)
+    {
       log(__FILE__, "ERROR: unable to register SQL_OPERATIONS component", -1);
-      return STATUS_ERROR;  
-   }
+      return STATUS_ERROR;
+    }
+  }
       
 // Component is registered, now create all the operations associated with
 // the component.  A grant from the system to the grantee (DB__ROOT) will
@@ -197,7 +193,8 @@ std::string DB__ROOTName(ComUser::getRootUserName());
                                                                PrivMgr::getSQLOperationName(operation),
                                                                codePtr,true,
                                                                PrivMgr::getSQLOperationDescription(operation),
-                                                               DB__ROOTID,DB__ROOTName,-1);
+                                                               DB__ROOTID,DB__ROOTName,-1,
+                                                               componentExists);
                                                        
       if (privStatus == STATUS_GOOD)
          operationCodes.push_back(codePtr); 
@@ -221,7 +218,8 @@ PrivMgrComponentPrivileges componentPrivileges(metadataLocation_,pDiags_);
                                                            ComUser::getRootUserID(),
                                                            ComUser::getRootUserName(),
                                                            ROOT_ROLE_ID,
-                                                           DB__ROOTROLE,-1);
+                                                           DB__ROOTROLE,-1,
+                                                           componentExists);
                                                            
    if (privStatus != STATUS_GOOD)
    {
@@ -241,7 +239,8 @@ std::vector<std::string> CSOperationCodes;
                                                            ComUser::getRootUserID(),
                                                            ComUser::getRootUserName(),
                                                            PUBLIC_USER,
-                                                           PUBLIC_AUTH_NAME,0);
+                                                           PUBLIC_AUTH_NAME,0,
+                                                           componentExists);
                                       
    if (privStatus != STATUS_GOOD)
    {
@@ -342,7 +341,6 @@ PrivStatus PrivMgrMDAdmin::initializeMetadata (
   //     If doesn't need upgrading - done
   //     else - upgrade table 
   bool populateObjectPrivs = false;
-  bool populateRoleGrants = false;
 
   try
   {
@@ -385,8 +383,6 @@ PrivStatus PrivMgrMDAdmin::initializeMetadata (
 
         if (tableDefinition.tableName == PRIVMGR_OBJECT_PRIVILEGES)
           populateObjectPrivs = true;
-        if (tableDefinition.tableName == PRIVMGR_ROLE_USAGE)
-          populateRoleGrants = true;
       }
 
       // upgrade tables
@@ -406,7 +402,7 @@ PrivStatus PrivMgrMDAdmin::initializeMetadata (
     // populate metadata tables
     PrivStatus privStatus = updatePrivMgrMetadata
       (objectsLocation,authsLocation,
-       populateObjectPrivs,populateRoleGrants);
+       populateObjectPrivs);
 
     // if error occurs, drop tables already created
     if (privStatus == STATUS_ERROR)
@@ -560,13 +556,28 @@ PrivStatus PrivMgrMDAdmin::dropMetadata (
     cliInterface.retrieveSQLDiagnostics(pDiags_);
     retcode = STATUS_ERROR;
   }
+
   CmpSeabaseDDLrole role;
-    
-  role.dropStandardRole(DB__ROOTROLE);
-  role.dropStandardRole(DB__HIVEROLE);
-  role.dropStandardRole(DB__HBASEROLE);
-  log(__FILE__, "dropped roles DB__ROOTROLE, DB_HIVEROLE, DB_HBASEROLE", -1);
-   
+  std::vector<std::string> rolesCreated;
+  int32_t numberRoles = sizeof(systemRoles)/sizeof(SystemRolesStruct);
+  for (int32_t i = 0; i < numberRoles; i++)
+  {
+    const SystemRolesStruct &roleDefinition = systemRoles[i];
+
+    // special Auth includes roles that are not registered in the metadata
+    if (roleDefinition.isSpecialAuth)
+      continue;
+
+    role.dropStandardRole(roleDefinition.roleName);
+  }
+
+  int32_t actualSize = 0;
+  char buf[500];
+  ComUser::getRoleList(buf, actualSize, 500);
+  buf[actualSize] = 0;
+  traceMsg = "dropped roles: ";
+  traceMsg + buf;
+  log(__FILE__, traceMsg,  -1);
 
 //TODO: should notify QI
   log (__FILE__, "*** drop authorization completed ***", -1);
@@ -1220,7 +1231,6 @@ int32_t diagsMark = pDiags_->mark();
   return true;
 }
 
-
 // ----------------------------------------------------------------------------
 // method: compareTableDefs
 //
@@ -1471,8 +1481,7 @@ static int32_t renameTable (
 PrivStatus PrivMgrMDAdmin::updatePrivMgrMetadata(
    const std::string &objectsLocation,
    const std::string &authsLocation,
-   const bool shouldPopulateObjectPrivs,
-   const bool shouldPopulateRoleGrants)
+   const bool shouldPopulateObjectPrivs)
    
 {
    std::string traceMsg;
@@ -1486,27 +1495,57 @@ PrivStatus PrivMgrMDAdmin::updatePrivMgrMetadata(
          return STATUS_ERROR;
    }
    
-    
+   // Create any roles.  If this is an upgrade operation, some roles may
+   // already exist, just create any new roles. If this is an initialize
+   // operation, than all system roles are created.
    CmpSeabaseDDLrole role;
-    
-   role.createStandardRole(DB__ROOTROLE,ROOT_ROLE_ID);
-   role.createStandardRole(DB__HIVEROLE,HIVE_ROLE_ID);
-   role.createStandardRole(DB__HBASEROLE,HBASE_ROLE_ID);
-   log(__FILE__, "created roles DB__ROOTROLE, DB__HIVEROLE, and DB__HBASEROLE", -1);
+   std::vector<std::string> rolesCreated;
+   int32_t numberRoles = sizeof(systemRoles)/sizeof(SystemRolesStruct);
+   for (int32_t i = 0; i < numberRoles; i++)
+   {
+     const SystemRolesStruct &roleDefinition = systemRoles[i];
+
+     // special Auth includes roles that are not registered in the metadata
+     if (roleDefinition.isSpecialAuth)
+       continue;
+
+     // returns true is role was created, false if it already existed
+     if (role.createStandardRole(roleDefinition.roleName, roleDefinition.roleID))
+       rolesCreated.push_back(roleDefinition.roleName);
+   }
+
+   // Report the number roles created
+   traceMsg = "created roles ";
+   char buf[MAX_AUTHNAME_LEN + 5];
+   char sep = ' ';
+   for (size_t i = 0; i < rolesCreated.size(); i++)
+   {
+      sprintf(buf, "%c'%s' ", sep, rolesCreated[i].c_str());
+      traceMsg.append(buf);
+      sep = ',';
+   }
+   log(__FILE__, traceMsg, -1);
    
-   if (shouldPopulateRoleGrants)
+   if (rolesCreated.size() > 0)
    {
       PrivMgrRoles role(" ",metadataLocation_,pDiags_);
                         
-      privStatus = role.populateCreatorGrants(authsLocation);
+      privStatus = role.populateCreatorGrants(authsLocation, rolesCreated);
       if (privStatus != STATUS_GOOD)
          return STATUS_ERROR;
    }
+ 
+   // If someone initializes authorization, creates some roles, then drops 
+   // authorization, these roles exist in th system metadata (e.g. AUTHS table)
+   // but all usages are lost, including the initial creator grants.
+   // See if there are any roles that exist in AUTHS but do not have creator 
+   // grants - probably should add creator grants.
+   // TBD
     
-      privStatus = initializeComponentPrivileges();
-   
-      if (privStatus != STATUS_GOOD)
-         return STATUS_ERROR;
+   privStatus = initializeComponentPrivileges();
+  
+   if (privStatus != STATUS_GOOD)
+      return STATUS_ERROR;
       
    // When new components and component operations are added
    // add an upgrade procedure

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrMD.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrMD.h b/core/sql/sqlcomp/PrivMgrMD.h
index a4c3f5e..c52a635 100644
--- a/core/sql/sqlcomp/PrivMgrMD.h
+++ b/core/sql/sqlcomp/PrivMgrMD.h
@@ -301,8 +301,7 @@ class PrivMgrMDAdmin : public PrivMgr
     PrivStatus updatePrivMgrMetadata(
        const std::string &objectsLocation,
        const std::string &authsLocation,
-       const bool shouldPopulateObjectPrivs,
-       const bool shouldPopulateRoleGrants);
+       const bool shouldPopulateObjectPrivs);
 
     PrivStatus upgradeMetadata (
       const PrivMgrTableStruct &tableDefinition,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrRoles.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrRoles.cpp b/core/sql/sqlcomp/PrivMgrRoles.cpp
index 9618f95..08489a1 100644
--- a/core/sql/sqlcomp/PrivMgrRoles.cpp
+++ b/core/sql/sqlcomp/PrivMgrRoles.cpp
@@ -40,6 +40,7 @@
 // CmpCommon.h contains STMTHEAP declaration
 #include "CmpCommon.h"
 #include "CmpDDLCatErrorCodes.h"
+#include "NAUserId.h"
 #include "ComUser.h"
 #include "ComSecurityKey.h"
 
@@ -119,7 +120,9 @@ public:
    
    virtual PrivStatus insert(const PrivMgrMDRow &row);
    
-   PrivStatus insertSelect(const std::string & authsLocation); 
+   PrivStatus insertSelect(
+      const std::string & authsLocation,
+      const std::string & inClause); 
    
    PrivStatus selectAllWhere(
       const std::string & whereClause,
@@ -1296,7 +1299,9 @@ PrivStatus privStatus = myTable.selectCountWhere(whereClause,rowCount);
 // *              the diags area.                                              *
 // *                                                                           *
 // *****************************************************************************
-PrivStatus PrivMgrRoles::populateCreatorGrants(const std::string & authsLocation)
+PrivStatus PrivMgrRoles::populateCreatorGrants(
+  const std::string & authsLocation,
+  const std::vector<std::string> &rolesToAdd)
 
 {
 
@@ -1305,22 +1310,28 @@ PrivStatus PrivMgrRoles::populateCreatorGrants(const std::string & authsLocation
 
    MyTable &myTable = static_cast<MyTable &>(myTable_);
 
-   // See if the table is empty before inserting any rows
+   int32_t numberRoles = sizeof(systemRoles)/sizeof(SystemRolesStruct) - 
+                         NUMBER_SPECIAL_SYSTEM_ROLES;
+
+   // Calculate the number of roles that have already been created
+   int64_t expectedRows = numberRoles - rolesToAdd.size();
 
    std::string whereClause;
 
-   int64_t expectedRows = 0;
+   int64_t foundRows = 0;
 
-   PrivStatus privStatus = myTable.selectCountWhere(whereClause,expectedRows);
+   PrivStatus privStatus = myTable.selectCountWhere(whereClause,foundRows);
 
    if (privStatus == STATUS_ERROR)
       return privStatus;
 
-   if (expectedRows != 0)
+   if (foundRows != expectedRows)
    {
       std::string message ("Found ");
+      message += to_string((long long int)foundRows);
+      message += " rows in ROLE_USAGE table, expecting ";
       message += to_string((long long int)expectedRows);
-      message += " rows in ROLE_USAGE table, expecting 0 rows";
+      message += " rows";
       traceMsg = "ERROR: ";
       traceMsg += message;
       PrivMgr::log(__FILE__, message, -1);
@@ -1329,7 +1340,19 @@ PrivStatus PrivMgrRoles::populateCreatorGrants(const std::string & authsLocation
    }
 
    // insert the rows
-   privStatus = myTable.insertSelect(authsLocation);
+   std::string inClause (" auth_db_name in (");
+   std::string sep = "";
+   for (size_t i = 0; i < rolesToAdd.size(); i++)
+   {
+     inClause.append(sep);
+     inClause.append ("'");
+     inClause.append(rolesToAdd[i]);
+     inClause.append ("'");
+     sep = ",";
+   }
+   inClause += ")";
+
+   privStatus = myTable.insertSelect(authsLocation, inClause);
 
    if (privStatus == STATUS_ERROR)
    {
@@ -1340,7 +1363,7 @@ PrivStatus PrivMgrRoles::populateCreatorGrants(const std::string & authsLocation
       return privStatus;
    }
   
-   // make sure that the number rows inserted match the expected.
+   // make sure that the number rows inserted match the total.
    // get the number of rows inserted
    int64_t insertedRows;
 
@@ -1352,7 +1375,17 @@ PrivStatus PrivMgrRoles::populateCreatorGrants(const std::string & authsLocation
    // get number rows expected
    std::string selectStmt ("SELECT COUNT(*) FROM  ");
 
-   whereClause = " where AUTH_TYPE = 'R'";
+   int32_t actualSize = 0;
+   char buf[500];
+   if (ComUser::getRoleList(buf, actualSize, 500))
+   {
+      PRIVMGR_INTERNAL_ERROR("internal error getting role list");
+      return STATUS_ERROR;
+   }
+   whereClause = " where AUTH_TYPE = 'R' AND AUTH_DB_NAME IN (";
+   whereClause += buf;
+   whereClause += ")";
+
    selectStmt += authsLocation;
    selectStmt += " ";
    selectStmt += whereClause;
@@ -1371,9 +1404,9 @@ PrivStatus PrivMgrRoles::populateCreatorGrants(const std::string & authsLocation
    }
 
    // Check to see if rows inserted match expected rows
-   if (expectedRows != insertedRows)
+   if (numberRoles != insertedRows)
    {
-      std::string message ("Expected to insert ");
+      std::string message ("Expected to find ");
       message += to_string((long long int)expectedRows);
       message += " rows into ROLE_USAGE table, instead ";
       message += to_string((long long int)insertedRows);
@@ -1885,7 +1918,9 @@ char grantorAuthClass[3] = {0};
 // *           *: Insert failed. A CLI error is put into the diags area.       *
 // *                                                                           *
 // *****************************************************************************
-PrivStatus MyTable::insertSelect(const std::string & authsLocation)
+PrivStatus MyTable::insertSelect(
+  const std::string & authsLocation,
+  const std::string & inClause)
 
 {
 
@@ -1894,9 +1929,9 @@ char insertStatement[2000];
    sprintf(insertStatement, "INSERT INTO %s SELECT A1.AUTH_ID, A1.AUTH_DB_NAME, A1.AUTH_CREATOR,"
            "(SELECT AUTH_DB_NAME FROM %s A2 WHERE A2.auth_ID = A1.AUTH_CREATOR)," 
            "(SELECT AUTH_TYPE FROM %s A3 WHERE A3.auth_ID = A1.AUTH_CREATOR),"
-           "-2,'_SYSTEM','%c',-1 FROM %s A1 WHERE A1.AUTH_TYPE = 'R'",
+           "-2,'_SYSTEM','%c',-1 FROM %s A1 WHERE A1.AUTH_TYPE = 'R' AND %s",
            tableName_.c_str(),authsLocation.c_str(),authsLocation.c_str(), 
-           'U',authsLocation.c_str()); 
+           'U',authsLocation.c_str(), inClause.c_str()); 
                
    return CLIImmediate(insertStatement);
                   

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/PrivMgrRoles.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/PrivMgrRoles.h b/core/sql/sqlcomp/PrivMgrRoles.h
index eb8000b..052ce6e 100644
--- a/core/sql/sqlcomp/PrivMgrRoles.h
+++ b/core/sql/sqlcomp/PrivMgrRoles.h
@@ -93,7 +93,9 @@ public:
       
    bool isUserGrantedAnyRole(const int32_t authID);
    
-   PrivStatus populateCreatorGrants(const std::string &authsLocation);      
+   PrivStatus populateCreatorGrants(
+      const std::string &authsLocation,
+      const std::vector<std::string> &rolesToAdd);      
      
    PrivStatus revokeRole(
       const std::vector<int32_t> & roleIDs,


[3/3] incubator-trafodion git commit: Merge [TRAFODION-1879] PR-372 Integrate Library Management into Trafodion Metadata

Posted by rm...@apache.org.
Merge [TRAFODION-1879] PR-372 Integrate Library Management into Trafodion Metadata


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

Branch: refs/heads/master
Commit: 41e57594210ba95d4a76afda51b4a071bfe7b7f0
Parents: 5a6549b 9e3dea6
Author: Roberta Marton <ro...@apache.org>
Authored: Thu Mar 10 16:15:35 2016 +0000
Committer: Roberta Marton <ro...@apache.org>
Committed: Thu Mar 10 16:15:35 2016 +0000

----------------------------------------------------------------------
 core/sql/common/ComSmallDefs.h                  |   5 +
 core/sql/common/ComUser.cpp                     |  54 +++
 core/sql/common/ComUser.h                       |   6 +
 core/sql/common/NAUserId.h                      |  32 +-
 core/sql/optimizer/RelExeUtil.cpp               |   5 +
 core/sql/optimizer/RelExeUtil.h                 |  17 +-
 core/sql/parser/ParKeyWords.cpp                 |   1 +
 core/sql/parser/sqlparser.y                     |  53 +++
 core/sql/regress/privs2/EXPECTED135             |   2 +
 core/sql/regress/tools/runregr_udr.ksh          |   2 +-
 core/sql/regress/udr/EXPECTED102                | 421 +++++++++++++++++++
 core/sql/regress/udr/TEST102                    | 119 ++++++
 core/sql/sqlcomp/CmpSeabaseDDL.h                |   7 +
 core/sql/sqlcomp/CmpSeabaseDDLauth.cpp          |  24 +-
 core/sql/sqlcomp/CmpSeabaseDDLauth.h            |   2 +-
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp        |  45 +-
 core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp       | 167 ++++++++
 core/sql/sqlcomp/CmpSeabaseDDLroutine.h         | 280 ++++++++++++
 core/sql/sqlcomp/CmpSeabaseDDLschema.cpp        |  32 +-
 core/sql/sqlcomp/PrivMgrComponentOperations.cpp |  24 +-
 core/sql/sqlcomp/PrivMgrComponentOperations.h   |   3 +-
 core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp |  15 +-
 core/sql/sqlcomp/PrivMgrComponentPrivileges.h   |   3 +-
 core/sql/sqlcomp/PrivMgrMD.cpp                  | 129 ++++--
 core/sql/sqlcomp/PrivMgrMD.h                    |   3 +-
 core/sql/sqlcomp/PrivMgrRoles.cpp               |  65 ++-
 core/sql/sqlcomp/PrivMgrRoles.h                 |   4 +-
 27 files changed, 1416 insertions(+), 104 deletions(-)
----------------------------------------------------------------------



[2/3] incubator-trafodion git commit: TRAFODION [1879] - Integrate Library Management into Trafodion Metadata

Posted by rm...@apache.org.
TRAFODION [1879] - Integrate Library Management into Trafodion Metadata

The initial release of library management has been delivered to Trafodion. This
is follow-on work that integrates library management operations into the
existing Trafodion infrastructure. Currently, with the initial release of
library management, the consumer needs to run a special script to set up
everything. This delivery incorporates the steps into existing SQL commands and
removes the need for the script.

This delivery contains:

-> Support for three new INITIALIZE TRAFODION options
-> Support for a new role (plus infrastructure to make it easier to add roles)
-> Change initialize authorization to handle upgrade better
-> Fixed couple issues found while testing
-> New regression test (udr/TEST102)

*** Support for three new INITIALIZE TRAFODION options:

Three new INITIALIZE TRAFODION options have been added:
   CREATE LIBRARY MANAGEMENT - create and populates the libmgr schema
   DROP LIBRARY MANAGEMENT - drops the libmgr schema
   UPGRADE LIBRARY MANAGEMENT - adds new procedures to the libmgr library

Parser was changed to support these new options, a new keyword - MANAGEMENT
was added.
  (parser/ParKeyWords.cpp/sqlparser.y)

DDL compiler was changed to recognize the new options and call implementation
methods.
  (optimizer/RelExeUtil, sqlcomp/CmpSeabaseDDLcommon.cpp)

The following implementation methods were added:
  createSeabaseLibmgr (code for create library management)
  dropSeabaseLibmgr (code for drop library management)
  upgradeSeabaseLibmgr(code for upgrade library management)
  createLibmgrProcs – a helper method called by create and upgrade code to
                      create libmgr procedures
  grantLibmgrPrivs – a helper method called by createLibmgrProcs and
                     initialize authorization that add grants to procedures for
                     the new DB__LIBMGRROLE role.
   (CmpSeabaseDDL.h/CmpSeabaseDDLroutine.cpp)

Changes were made to define the new schema, library and procedures.
  (common/ComSmallDefs.h, CmpSeabaseDDLroutine.h)

All procedures are defined in a new file called sqlcomp/CmpSeabaseDDLroutine.h.
This file is based on similar support for system metadata tables
(CmpSeabaseDDLmd.h).  It contains the text of all the procedures.  During
"create library management" and "upgrade library management" operations, this
list is used to create/add procedures.

*** Support for a new role:

Initialize authorization code was changed to create the new DB__LIBMGRROLE role.
As part of this change, role infrastructure was modified to make it easier to
add system roles in the future. The initialize code checks to see what roles
have not been added, and adds them.
  (common/NAUserId.h, common/ComUser, sqlcomp/PrivMgrMD,
   sqlcomp/CmpSeabaseDDLauth, sqlcomp/PrivMgrRoles)

In addition, a check is made – if the SEABASE_LIBMGR_SCHEMA exists, then
grants are performed on the procedures to allow DB__LIBMGRROLE execute
privilege.
  (sqlcomp/CmpSeabaseDDLcommon.cpp)

*** Change initialize authorization to handle upgrade better:

Changes were made so initialize authorization performs an upgrade operation if
called and component privileges already exists.
  (sqlcomp/PrivMgrComponentOperations, sqlcomp/PrivMgrComponentPrivileges,
   sqlcomp/PrivMgrMD)

*** Fixed couple of issues found while testing:

Invalid role ID generation.  Role ID generation code was using the wrong range
to determine the next role ID.
  (CmpSeabaseDDLauth.cpp)

Drop schema issues with libraries.  Drop library cascade fails when the order
of libraries and functions is incorrect.  If you drop function, drop library it
fails with missing procedure.
  (CmpSeabaseDDLschema.cpp)


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

Branch: refs/heads/master
Commit: 9e3dea641f54995fd5adbdbf724dae9c10dbfe40
Parents: 97a7da4
Author: Roberta Marton <ro...@apache.org>
Authored: Tue Mar 8 18:00:03 2016 +0000
Committer: Roberta Marton <ro...@apache.org>
Committed: Tue Mar 8 18:00:03 2016 +0000

----------------------------------------------------------------------
 core/sql/common/ComSmallDefs.h                  |   5 +
 core/sql/common/ComUser.cpp                     |  54 +++
 core/sql/common/ComUser.h                       |   6 +
 core/sql/common/NAUserId.h                      |  32 +-
 core/sql/optimizer/RelExeUtil.cpp               |   5 +
 core/sql/optimizer/RelExeUtil.h                 |  17 +-
 core/sql/parser/ParKeyWords.cpp                 |   1 +
 core/sql/parser/sqlparser.y                     |  53 +++
 core/sql/regress/privs2/EXPECTED135             |   2 +
 core/sql/regress/tools/runregr_udr.ksh          |   2 +-
 core/sql/regress/udr/EXPECTED102                | 421 +++++++++++++++++++
 core/sql/regress/udr/TEST102                    | 119 ++++++
 core/sql/sqlcomp/CmpSeabaseDDL.h                |   7 +
 core/sql/sqlcomp/CmpSeabaseDDLauth.cpp          |  24 +-
 core/sql/sqlcomp/CmpSeabaseDDLauth.h            |   2 +-
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp        |  45 +-
 core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp       | 167 ++++++++
 core/sql/sqlcomp/CmpSeabaseDDLroutine.h         | 280 ++++++++++++
 core/sql/sqlcomp/CmpSeabaseDDLschema.cpp        |  32 +-
 core/sql/sqlcomp/PrivMgrComponentOperations.cpp |  24 +-
 core/sql/sqlcomp/PrivMgrComponentOperations.h   |   3 +-
 core/sql/sqlcomp/PrivMgrComponentPrivileges.cpp |  15 +-
 core/sql/sqlcomp/PrivMgrComponentPrivileges.h   |   3 +-
 core/sql/sqlcomp/PrivMgrMD.cpp                  | 129 ++++--
 core/sql/sqlcomp/PrivMgrMD.h                    |   3 +-
 core/sql/sqlcomp/PrivMgrRoles.cpp               |  65 ++-
 core/sql/sqlcomp/PrivMgrRoles.h                 |   4 +-
 27 files changed, 1416 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/common/ComSmallDefs.h
----------------------------------------------------------------------
diff --git a/core/sql/common/ComSmallDefs.h b/core/sql/common/ComSmallDefs.h
index 9a10fc8..e14f2bc 100644
--- a/core/sql/common/ComSmallDefs.h
+++ b/core/sql/common/ComSmallDefs.h
@@ -169,6 +169,11 @@ typedef NABoolean               ComBoolean;
 
 #define SEABASE_REGRESS_DEFAULT_SCHEMA "SCH"
 
+// Trafodion system library and procedures reserved schema
+// Procedures are defined in CmpSeabaseDDLroutine.h
+#define SEABASE_LIBMGR_SCHEMA "_LIBMGR_"
+#define SEABASE_LIBMGR_LIBRARY "DB__LIBMGRNAME"
+
 // length of explain_plan column in metric_query_table.
 // explain_plan greater than this length are chunked and store in multiple
 // rows in metric_text_table

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/common/ComUser.cpp
----------------------------------------------------------------------
diff --git a/core/sql/common/ComUser.cpp b/core/sql/common/ComUser.cpp
index 504ce06..7e7d7ca 100644
--- a/core/sql/common/ComUser.cpp
+++ b/core/sql/common/ComUser.cpp
@@ -347,3 +347,57 @@ Int16 ComUser::getAuthNameFromAuthID(Int32   authID,
 
   return FEOK;
 }
+
+// ----------------------------------------------------------------------------
+// method: getRoleList
+//
+// Returns the list of system roles
+// Params:
+//   (out) roleList - the list of roles, space is managed by the caller
+//   (out) actualLen - the length of the returned role list
+//   ( in) maxLen - the size of the roleList allocated by the caller
+//   ( in) delimited - delimiter to use (defaults to single quote)
+//   ( in) separator - specified what separator to use (defaults to comma)
+//   ( in) includeSpecialAuths - includes the special auths (PUBLIC and _SYSTEM) 
+//
+//  Returns:  FEOK -- found
+//            FEBUFTOOSMALL -- space allocated for role list is too small 
+// ----------------------------------------------------------------------------
+Int32 ComUser::getRoleList (char * roleList,
+                            Int32 &actualLen,
+                            const Int32 maxLen,
+                            const char delimiter,
+                            const char separator,
+                            const bool includeSpecialAuths)
+{
+  Int32 numberRoles = sizeof(systemRoles)/sizeof(SystemRolesStruct);
+  Int32 roleListLen = (MAX_AUTHNAME_LEN*numberRoles)+(numberRoles * 4); // 4 = 2 del + 2 sep
+  char generatedRoleList[roleListLen];
+  char *pRoles = generatedRoleList;
+  char roleName[MAX_AUTHNAME_LEN + 4];
+  char currentSeparator = ' ';
+  for (Int32 i = 0; i < numberRoles; i++)
+  {
+    const SystemRolesStruct &roleDefinition = systemRoles[i];
+    if (!includeSpecialAuths && roleDefinition.isSpecialAuth)
+      continue;
+
+    // str_sprintf does not support the %c format
+    sprintf(roleName, "%c%c%s%c",
+                currentSeparator, delimiter, roleDefinition.roleName, delimiter);
+    str_cpy_all(pRoles, roleName, sizeof(roleName)-1); // don't copy null terminator 
+    currentSeparator = separator;
+    pRoles = pRoles + strlen(roleName);
+  }
+
+  pRoles = '\0'; // null terminate string
+  pRoles = generatedRoleList;
+  actualLen = strlen(pRoles);
+  if (actualLen > maxLen) 
+    return FEBUFTOOSMALL;
+ 
+  str_cpy_all(roleList, pRoles, strlen(pRoles));
+  roleList[strlen(pRoles)] = 0; // null terminate string
+  return FEOK;
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/common/ComUser.h
----------------------------------------------------------------------
diff --git a/core/sql/common/ComUser.h b/core/sql/common/ComUser.h
index d5f5c2e..ea00c88 100644
--- a/core/sql/common/ComUser.h
+++ b/core/sql/common/ComUser.h
@@ -90,6 +90,12 @@ class ComUser
      static Int16 getAuthIDFromAuthName (const char  * authName,
                                          Int32 & authID);
 
+     static Int32 getRoleList (char *roleList,
+                               Int32 &actualLen,
+                               const Int32 maxLen,
+                               const char delimiter = '\'',
+                               const char separator =',',
+                               const bool includeSpecialAuths = false);
 private:
    // default constructor
    ComUser ();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/common/NAUserId.h
----------------------------------------------------------------------
diff --git a/core/sql/common/NAUserId.h b/core/sql/common/NAUserId.h
index 992c2c3..46de6bc 100644
--- a/core/sql/common/NAUserId.h
+++ b/core/sql/common/NAUserId.h
@@ -42,23 +42,28 @@
 #define MAX_USERNAME_LEN 128
 #define MAX_AUTHNAME_LEN 128
 #define MAX_AUTHID_AS_STRING_LEN 20
+
 #define MIN_USERID 33333
 #define MAX_USERID 999999
 #define MIN_ROLEID 1000000
-#define MAX_ROLEID 1490000
+#define MAX_ROLEID_RANGE1 1490000
+#define MAX_ROLEID        1500000
 #define NA_UserId Int32
 #define NA_AuthID Int32
 #define NA_UserIdDefault 0
 
-// Defines for special users
-#define SYSTEM_AUTH_NAME "_SYSTEM"
-#define DB__ROOT         "DB__ROOT"
-
 // Defines for special roles
+// For new system roles, add a define and include it in the
+// systemRoles constant
 #define PUBLIC_AUTH_NAME "PUBLIC"
 #define DB__HIVEROLE     "DB__HIVEROLE"
 #define DB__HBASEROLE    "DB__HBASEROLE"
 #define DB__ROOTROLE     "DB__ROOTROLE"
+#define DB__LIBMGRROLE   "DB__LIBMGRROLE"
+
+// Defines for special users
+#define SYSTEM_AUTH_NAME "_SYSTEM"
+#define DB__ROOT         "DB__ROOT"
 
 #define SUPER_USER_LIT "33333"
 
@@ -71,5 +76,22 @@
 #define HIVE_ROLE_ID     1490000 
 #define HBASE_ROLE_ID    1490001
 
+struct SystemRolesStruct
+{
+   const char *roleName;
+   bool       isSpecialAuth;
+   int32_t    roleID;
+};
+
+static const SystemRolesStruct systemRoles[] 
+{ { DB__HIVEROLE, false, HIVE_ROLE_ID },
+  { DB__HBASEROLE, false, HBASE_ROLE_ID },
+  { DB__ROOTROLE, false, ROOT_ROLE_ID },
+  { DB__LIBMGRROLE, false, NA_UserIdDefault },
+  { PUBLIC_AUTH_NAME, true, PUBLIC_USER },
+  { SYSTEM_AUTH_NAME, true, SYSTEM_USER } };
+
+#define NUMBER_SPECIAL_SYSTEM_ROLES 2;
+
 
 #endif  /*  NAUSERID_H*/

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/optimizer/RelExeUtil.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExeUtil.cpp b/core/sql/optimizer/RelExeUtil.cpp
index e3401b5..a5b210d 100644
--- a/core/sql/optimizer/RelExeUtil.cpp
+++ b/core/sql/optimizer/RelExeUtil.cpp
@@ -3819,6 +3819,11 @@ RelExpr * DDLExpr::bindNode(BindWA *bindWA)
     isHbase_ = TRUE;
     hbaseDDLNoUserXn_ = TRUE;
   }
+  else if (createLibmgr() || dropLibmgr() || upgradeLibmgr())
+    {
+      isHbase_ = TRUE;
+      hbaseDDLNoUserXn_ = TRUE;
+    }
   else if (createRepos() || dropRepos() || upgradeRepos())
     {
       isHbase_ = TRUE;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/optimizer/RelExeUtil.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExeUtil.h b/core/sql/optimizer/RelExeUtil.h
index 21a771e..a2ab73d 100644
--- a/core/sql/optimizer/RelExeUtil.h
+++ b/core/sql/optimizer/RelExeUtil.h
@@ -405,6 +405,18 @@ public:
   {(v ? flags_ |= GET_MD_VERSION : flags_ &= ~GET_MD_VERSION); }
   NABoolean getMDVersion() { return (flags_ & GET_MD_VERSION) != 0;}
 
+  void setCreateLibmgr(NABoolean v)
+  {(v ? flags_ |= CREATE_LIBMGR : flags_ &= ~CREATE_LIBMGR); }
+  NABoolean createLibmgr() { return (flags_ & CREATE_LIBMGR) != 0;}
+
+  void setDropLibmgr(NABoolean v)
+  {(v ? flags_ |= DROP_LIBMGR : flags_ &= ~DROP_LIBMGR); }
+  NABoolean dropLibmgr() { return (flags_ & DROP_LIBMGR) != 0;}
+
+  void setUpgradeLibmgr(NABoolean v)
+  {(v ? flags_ |= UPGRADE_LIBMGR : flags_ &= ~UPGRADE_LIBMGR); }
+  NABoolean upgradeLibmgr() { return (flags_ & UPGRADE_LIBMGR) != 0;}
+
   void setCreateRepos(NABoolean v)
   {(v ? flags_ |= CREATE_REPOS : flags_ &= ~CREATE_REPOS); }
   NABoolean createRepos() { return (flags_ & CREATE_REPOS) != 0;}
@@ -432,7 +444,10 @@ public:
     CREATE_REPOS            = 0x0008,
     DROP_REPOS              = 0x0010,
     UPGRADE_REPOS           = 0x0020,
-    CLEANUP_AUTH            = 0X0040  
+    CLEANUP_AUTH            = 0X0040,
+    CREATE_LIBMGR           = 0x0080,
+    DROP_LIBMGR             = 0x0100,
+    UPGRADE_LIBMGR          = 0x0200
   };
 
   // see method processSpecialDDL in sqlcomp/parser.cpp

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/parser/ParKeyWords.cpp
----------------------------------------------------------------------
diff --git a/core/sql/parser/ParKeyWords.cpp b/core/sql/parser/ParKeyWords.cpp
index b46660e..0c79874 100644
--- a/core/sql/parser/ParKeyWords.cpp
+++ b/core/sql/parser/ParKeyWords.cpp
@@ -620,6 +620,7 @@ ParKeyWord ParKeyWords::keyWords_[] = {
   ParKeyWord("LZO",                TOK_LZO,         NONRESTOKEN_),
   ParKeyWord("M",                  TOK_M,           NONRESTOKEN_),
   ParKeyWord("MAINTAIN",           TOK_MAINTAIN,    SECOND_|COMPAQ_|RESWORD_),
+  ParKeyWord("MANAGEMENT",         TOK_MANAGEMENT,  NONRESTOKEN_),
   ParKeyWord("MANUAL",             TOK_MANUAL,      FLAGSNONE_),
   ParKeyWord("MAP",                TOK_MAP,         COMPAQ_|RESWORD_),
   ParKeyWord("MASTER",             TOK_MASTER,      NONRESTOKEN_),

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/parser/sqlparser.y
----------------------------------------------------------------------
diff --git a/core/sql/parser/sqlparser.y b/core/sql/parser/sqlparser.y
index e565f14..cfe8833 100755
--- a/core/sql/parser/sqlparser.y
+++ b/core/sql/parser/sqlparser.y
@@ -818,6 +818,7 @@ static void enableMakeQuotedStringISO88591Mechanism()
 %token <tokval> TOK_LSDECIMAL
 %token <tokval> TOK_LTRIM               /*  ODBC extension  */
 %token <tokval> TOK_MAINTAIN
+%token <tokval> TOK_MANAGEMENT
 %token <tokval> TOK_MANUAL				// MV
 %token <tokval> TOK_MASTER
 %token <tokval> TOK_MATCH
@@ -16127,6 +16128,57 @@ exe_util_init_hbase : TOK_INITIALIZE TOK_TRAFODION
 
                }
 
+             | TOK_INITIALIZE TOK_TRAFODION ',' TOK_CREATE TOK_LIBRARY TOK_MANAGEMENT
+               {
+		 CharInfo::CharSet stmtCharSet = CharInfo::UnknownCharSet;
+		 NAString * stmt = getSqlStmtStr ( stmtCharSet  // out - CharInfo::CharSet &
+					         , PARSERHEAP() 
+	                                         );
+
+		 DDLExpr * de = new(PARSERHEAP()) DDLExpr(NULL,
+                                                          (char*)stmt->data(),
+                                                           stmtCharSet);
+
+                 de->setCreateLibmgr(TRUE);
+
+                 $$ = de;
+
+               }
+
+             | TOK_INITIALIZE TOK_TRAFODION ',' TOK_DROP TOK_LIBRARY TOK_MANAGEMENT
+               {
+                 CharInfo::CharSet stmtCharSet = CharInfo::UnknownCharSet;
+                 NAString * stmt = getSqlStmtStr ( stmtCharSet  // out - CharInfo::CharSet &
+                                                 , PARSERHEAP()
+                                                 );
+
+                 DDLExpr * de = new(PARSERHEAP()) DDLExpr(NULL,
+                                                          (char*)stmt->data(),
+                                                          stmtCharSet);
+
+                 de->setDropLibmgr(TRUE);
+
+                 $$ = de;
+
+               }
+
+             | TOK_INITIALIZE TOK_TRAFODION ',' TOK_UPGRADE TOK_LIBRARY TOK_MANAGEMENT
+               {
+                 CharInfo::CharSet stmtCharSet = CharInfo::UnknownCharSet;
+                 NAString * stmt = getSqlStmtStr ( stmtCharSet  // out - CharInfo::CharSet &
+                                                 , PARSERHEAP()
+                                                 );
+
+                 DDLExpr * de = new(PARSERHEAP()) DDLExpr(NULL,
+                                                          (char*)stmt->data(),
+                                                          stmtCharSet);
+
+                 de->setUpgradeLibmgr(TRUE);
+
+                 $$ = de;
+
+               }
+
             | TOK_INITIALIZE TOK_TRAFODION ',' TOK_CREATE TOK_REPOSITORY
                {
 		 CharInfo::CharSet stmtCharSet = CharInfo::UnknownCharSet;
@@ -33147,6 +33199,7 @@ nonreserved_word :      TOK_ABORT
                       | TOK_SEQUENCES
                       | TOK_SINCE
 //		      | TOK_MAINTAIN
+                      | TOK_MANAGEMENT
 		      | TOK_MANUAL
 		      | TOK_MIXED
 		      | TOK_MVS  // MV

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/regress/privs2/EXPECTED135
----------------------------------------------------------------------
diff --git a/core/sql/regress/privs2/EXPECTED135 b/core/sql/regress/privs2/EXPECTED135
index 9fbc688..8ce623f 100644
--- a/core/sql/regress/privs2/EXPECTED135
+++ b/core/sql/regress/privs2/EXPECTED135
@@ -670,6 +670,7 @@ Roles
 
 DB__HBASEROLE
 DB__HIVEROLE
+DB__LIBMGRROLE
 DB__ROOTROLE
 PUBLIC
 T135_ROLE1
@@ -1087,6 +1088,7 @@ Roles
 
 DB__HBASEROLE
 DB__HIVEROLE
+DB__LIBMGRROLE
 DB__ROOTROLE
 PUBLIC
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/regress/tools/runregr_udr.ksh
----------------------------------------------------------------------
diff --git a/core/sql/regress/tools/runregr_udr.ksh b/core/sql/regress/tools/runregr_udr.ksh
index 8cb3bcc..2906026 100755
--- a/core/sql/regress/tools/runregr_udr.ksh
+++ b/core/sql/regress/tools/runregr_udr.ksh
@@ -446,7 +446,7 @@ done
 seabase="$SEABASE_REGRESS"
 # sbtestfiles contains the list of tests to be run in seabase mode
 if [ "$seabase" -ne 0 ]; then
-  TESTFILES="TEST001 TEST002 TEST100 TEST101 TEST107 TEST108 TEST163"
+  TESTFILES="TEST001 TEST002 TEST100 TEST101 TEST102 TEST107 TEST108 TEST163"
   SBPFILES=
   for i in $PFILES; do
     for j in $TESTFILES; do

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/regress/udr/EXPECTED102
----------------------------------------------------------------------
diff --git a/core/sql/regress/udr/EXPECTED102 b/core/sql/regress/udr/EXPECTED102
new file mode 100644
index 0000000..04063b0
--- /dev/null
+++ b/core/sql/regress/udr/EXPECTED102
@@ -0,0 +1,421 @@
+>>cqd SHOWDDL_DISPLAY_PRIVILEGE_GRANTS 'ON';
+
+--- SQL operation complete.
+>>set schema "_LIBMGR_";
+
+--- SQL operation complete.
+>>
+>>prepare get_schemas from
++>select distinct substring (schema_name,1,16) as schema_name
++>from trafodion."_MD_".objects
++>where schema_name in ('SEABASE', '_LIBMGR_', '_MD_', '_PRIVMGR_MD_', '_REPOS_')
++>order by 1;
+
+--- SQL command prepared.
+>>
+>>
+>>-- ****************************************************************************
+>>--  Verify enabling library management followed by authorization is correct
+>>-- ****************************************************************************
+>>
+>>-- authorization and library management should be disabled
+>>execute get_schemas;
+
+SCHEMA_NAME                                                     
+----------------------------------------------------------------
+
+SEABASE                                                         
+_MD_                                                            
+_REPOS_                                                         
+
+--- 3 row(s) selected.
+>>
+>>-- create library management 
+>>initialize trafodion, create library management;
+
+--- SQL operation complete.
+>>execute get_schemas;
+
+SCHEMA_NAME                                                     
+----------------------------------------------------------------
+
+SEABASE                                                         
+_LIBMGR_                                                        
+_MD_                                                            
+_REPOS_                                                         
+
+--- 4 row(s) selected.
+>>get libraries;
+
+Libraries in Schema TRAFODION._LIBMGR_
+======================================
+
+DB__LIBMGRNAME
+
+--- SQL operation complete.
+>>get procedures;
+
+Procedures in Schema TRAFODION._LIBMGR_
+=======================================
+
+ADDLIB
+ALTERLIB
+DROPLIB
+GETFILE
+HELP
+LS
+LSALL
+PUT
+RM
+RMREX
+
+--- SQL operation complete.
+>>showddl procedure rm;
+
+CREATE PROCEDURE TRAFODION."_LIBMGR_".RM
+  (
+    IN FILENAME VARCHAR(256 CHARS) CHARACTER SET UTF8
+  )
+  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.rm (java.lang.String)'
+  LIBRARY TRAFODION."_LIBMGR_".DB__LIBMGRNAME
+  EXTERNAL SECURITY DEFINER
+  LANGUAGE JAVA
+  PARAMETER STYLE JAVA
+  READS SQL DATA
+  DYNAMIC RESULT SETS 0
+  NO TRANSACTION REQUIRED
+  ISOLATE
+  ;
+
+--- SQL operation complete.
+>>set param ?h 'PUT';
+>>call help (?h);
+
+COMMANDNAME
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+
+PUT - Upload a JAR. SHOWDDL PROCEDURE [SCHEMA NAME.]PUT for more info.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                               
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                       
+
+--- SQL operation complete.
+>>
+>>-- enable authorization
+>>initialize authorization;
+
+--- SQL operation complete.
+>>execute get_schemas;
+
+SCHEMA_NAME                                                     
+----------------------------------------------------------------
+
+SEABASE                                                         
+_LIBMGR_                                                        
+_MD_                                                            
+_PRIVMGR_MD_                                                    
+_REPOS_                                                         
+
+--- 5 row(s) selected.
+>>get roles;
+
+Roles
+=====
+
+DB__HBASEROLE
+DB__HIVEROLE
+DB__LIBMGRROLE
+DB__ROOTROLE
+PUBLIC
+
+--- SQL operation complete.
+>>showddl procedure rm;
+
+CREATE PROCEDURE TRAFODION."_LIBMGR_".RM
+  (
+    IN FILENAME VARCHAR(256 CHARS) CHARACTER SET UTF8
+  )
+  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.rm (java.lang.String)'
+  LIBRARY TRAFODION."_LIBMGR_".DB__LIBMGRNAME
+  EXTERNAL SECURITY DEFINER
+  LANGUAGE JAVA
+  PARAMETER STYLE JAVA
+  READS SQL DATA
+  DYNAMIC RESULT SETS 0
+  NO TRANSACTION REQUIRED
+  ISOLATE
+  ;
+
+-- GRANT EXECUTE ON FUNCTION TRAFODION."_LIBMGR_".RM TO DB__ROOT WITH GRANT OPTION;
+  GRANT EXECUTE ON FUNCTION TRAFODION."_LIBMGR_".RM TO DB__LIBMGRROLE WITH
+  GRANT OPTION;
+
+--- SQL operation complete.
+>>
+>>-- reset
+>>initialize authorization, drop;
+
+--- SQL operation complete.
+>>initialize trafodion, drop library management;
+
+--- SQL operation complete.
+>>execute get_schemas;
+
+SCHEMA_NAME                                                     
+----------------------------------------------------------------
+
+SEABASE                                                         
+_MD_                                                            
+_REPOS_                                                         
+
+--- 3 row(s) selected.
+>>get procedures;
+
+--- SQL operation complete.
+>>get roles;
+
+Roles
+=====
+
+PUBLIC
+
+--- SQL operation complete.
+>>
+>>-- ****************************************************************************
+>>--  Verify enabling authorization followed by library management is correct
+>>-- ****************************************************************************
+>>
+>>-- enable authorization
+>>initialize authorization;
+
+--- SQL operation complete.
+>>execute get_schemas;
+
+SCHEMA_NAME                                                     
+----------------------------------------------------------------
+
+SEABASE                                                         
+_MD_                                                            
+_PRIVMGR_MD_                                                    
+_REPOS_                                                         
+
+--- 4 row(s) selected.
+>>get roles;
+
+Roles
+=====
+
+DB__HBASEROLE
+DB__HIVEROLE
+DB__LIBMGRROLE
+DB__ROOTROLE
+PUBLIC
+
+--- SQL operation complete.
+>>
+>>-- enable library management
+>>initialize trafodion, create library management;
+
+--- SQL operation complete.
+>>execute get_schemas;
+
+SCHEMA_NAME                                                     
+----------------------------------------------------------------
+
+SEABASE                                                         
+_LIBMGR_                                                        
+_MD_                                                            
+_PRIVMGR_MD_                                                    
+_REPOS_                                                         
+
+--- 5 row(s) selected.
+>>get libraries;
+
+Libraries in Schema TRAFODION._LIBMGR_
+======================================
+
+DB__LIBMGRNAME
+
+--- SQL operation complete.
+>>get procedures;
+
+Procedures in Schema TRAFODION._LIBMGR_
+=======================================
+
+ADDLIB
+ALTERLIB
+DROPLIB
+GETFILE
+HELP
+LS
+LSALL
+PUT
+RM
+RMREX
+
+--- SQL operation complete.
+>>showddl procedure rm;
+
+CREATE PROCEDURE TRAFODION."_LIBMGR_".RM
+  (
+    IN FILENAME VARCHAR(256 CHARS) CHARACTER SET UTF8
+  )
+  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.rm (java.lang.String)'
+  LIBRARY TRAFODION."_LIBMGR_".DB__LIBMGRNAME
+  EXTERNAL SECURITY DEFINER
+  LANGUAGE JAVA
+  PARAMETER STYLE JAVA
+  READS SQL DATA
+  DYNAMIC RESULT SETS 0
+  NO TRANSACTION REQUIRED
+  ISOLATE
+  ;
+
+-- GRANT EXECUTE ON FUNCTION TRAFODION."_LIBMGR_".RM TO DB__ROOT WITH GRANT OPTION;
+  GRANT EXECUTE ON FUNCTION TRAFODION."_LIBMGR_".RM TO DB__LIBMGRROLE WITH
+  GRANT OPTION;
+
+--- SQL operation complete.
+>>set param ?h 'PUT';
+>>call help (?h);
+
+COMMANDNAME
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+
+PUT - Upload a JAR. SHOWDDL PROCEDURE [SCHEMA NAME.]PUT for more info.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                               
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                       
+
+--- SQL operation complete.
+>>
+>>-- ****************************************************************************
+>>--  Test upgrade
+>>-- ****************************************************************************
+>>
+>>-- upgrade, remove a couple of procedures and run upgrade
+>>set parserflags 131072;
+
+--- SQL operation complete.
+>>drop procedure PUT;
+
+--- SQL operation complete.
+>>drop procedure ls;
+
+--- SQL operation complete.
+>>reset parserflags 131072;
+
+--- SQL operation complete.
+>>get procedures;
+
+Procedures in Schema TRAFODION._LIBMGR_
+=======================================
+
+ADDLIB
+ALTERLIB
+DROPLIB
+GETFILE
+HELP
+LSALL
+RM
+RMREX
+
+--- SQL operation complete.
+>>call ls(?);
+
+*** ERROR[1389] Object LS does not exist in Trafodion.
+
+*** ERROR[8822] The statement was not prepared.
+
+>>
+>>initialize trafodion, upgrade library management;
+
+--- SQL operation complete.
+>>get procedures;
+
+Procedures in Schema TRAFODION._LIBMGR_
+=======================================
+
+ADDLIB
+ALTERLIB
+DROPLIB
+GETFILE
+HELP
+LS
+LSALL
+PUT
+RM
+RMREX
+
+--- SQL operation complete.
+>>showddl procedure put;
+
+CREATE PROCEDURE TRAFODION."_LIBMGR_".PUT
+  (
+    IN FILEDATA VARCHAR(102400) CHARACTER SET ISO88591
+  , IN FILENAME VARCHAR(256 CHARS) CHARACTER SET UTF8
+  , IN CREATEFLAG INTEGER
+  , IN FILEOVERWRITE INTEGER
+  )
+  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.put (java.lang.String,java.lang.String,int,int)'
+  LIBRARY TRAFODION."_LIBMGR_".DB__LIBMGRNAME
+  EXTERNAL SECURITY DEFINER
+  LANGUAGE JAVA
+  PARAMETER STYLE JAVA
+  READS SQL DATA
+  DYNAMIC RESULT SETS 0
+  NO TRANSACTION REQUIRED
+  ISOLATE
+  ;
+
+-- GRANT EXECUTE ON FUNCTION TRAFODION."_LIBMGR_".PUT TO DB__ROOT WITH GRANT OPTION;
+  GRANT EXECUTE ON FUNCTION TRAFODION."_LIBMGR_".PUT TO DB__LIBMGRROLE WITH
+  GRANT OPTION;
+
+--- SQL operation complete.
+>>call ls ('lib_mgmt.jar',?);
+
+FILENAMES
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
 ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
+
+<ls></ls>                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      
                                                                                                                                                                                                                                                                               
+
+--- SQL operation complete.
+>>
+>>-- should fail - library management already enabled
+>>initialize trafodion, create library management;
+
+*** ERROR[1390] Object TRAFODION."_LIBMGR_".DB__LIBMGRNAME already exists in Trafodion.
+
+--- SQL operation failed with errors.
+>>
+>>-- make sure everything is enabled
+>>execute get_schemas;
+
+SCHEMA_NAME                                                     
+----------------------------------------------------------------
+
+SEABASE                                                         
+_LIBMGR_                                                        
+_MD_                                                            
+_PRIVMGR_MD_                                                    
+_REPOS_                                                         
+
+--- 5 row(s) selected.
+>>get procedures;
+
+Procedures in Schema TRAFODION._LIBMGR_
+=======================================
+
+ADDLIB
+ALTERLIB
+DROPLIB
+GETFILE
+HELP
+LS
+LSALL
+PUT
+RM
+RMREX
+
+--- SQL operation complete.
+>>
+>>-- we are done
+>>exit;
+
+End of MXCI Session
+

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/regress/udr/TEST102
----------------------------------------------------------------------
diff --git a/core/sql/regress/udr/TEST102 b/core/sql/regress/udr/TEST102
new file mode 100644
index 0000000..ad99838
--- /dev/null
+++ b/core/sql/regress/udr/TEST102
@@ -0,0 +1,119 @@
+-- @@@ START COPYRIGHT @@@
+--
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--   http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing,
+-- software distributed under the License is distributed on an
+-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+-- KIND, either express or implied.  See the License for the
+-- specific language governing permissions and limitations
+-- under the License.
+--
+-- @@@ END COPYRIGHT @@@
+--
+-- This tests the following commands:
+--
+--   INITIALIZE TRAFODION, CREATE LIBRARY MANAGEMENT
+--   INITIALIZE TRAFODION, DROP LIBRARY MANAGEMENT
+--   INITIALIZE TRAFODION, UPGRADE LIBRARY MANAGEMENT
+
+-- start with both authorization and library management disabled
+initialize trafodion, drop library management;
+initialize authorization, drop;
+
+log LOG102 clear;
+cqd SHOWDDL_DISPLAY_PRIVILEGE_GRANTS 'ON';
+set schema "_LIBMGR_";
+
+prepare get_schemas from
+select distinct substring (schema_name,1,16) as schema_name
+from trafodion."_MD_".objects
+where schema_name in ('SEABASE', '_LIBMGR_', '_MD_', '_PRIVMGR_MD_', '_REPOS_')
+order by 1;
+
+
+-- ****************************************************************************
+--  Verify enabling library management followed by authorization is correct
+-- ****************************************************************************
+
+-- authorization and library management should be disabled
+execute get_schemas;
+
+-- create library management 
+initialize trafodion, create library management;
+execute get_schemas;
+get libraries;
+get procedures;
+showddl procedure rm;
+set param ?h 'PUT';
+call help (?h);
+
+-- enable authorization
+initialize authorization;
+execute get_schemas;
+get roles;
+showddl procedure rm;
+
+-- reset
+initialize authorization, drop;
+initialize trafodion, drop library management;
+execute get_schemas;
+get procedures;
+get roles;
+
+-- ****************************************************************************
+--  Verify enabling authorization followed by library management is correct
+-- ****************************************************************************
+
+-- enable authorization
+initialize authorization;
+execute get_schemas;
+get roles;
+
+-- enable library management
+initialize trafodion, create library management;
+execute get_schemas;
+get libraries;
+get procedures;
+showddl procedure rm;
+set param ?h 'PUT';
+call help (?h);
+
+-- ****************************************************************************
+--  Test upgrade
+-- ****************************************************************************
+
+-- upgrade, remove a couple of procedures and run upgrade
+set parserflags 131072;
+drop procedure PUT;
+drop procedure ls;
+reset parserflags 131072;
+get procedures;
+call ls(?);
+
+initialize trafodion, upgrade library management;
+get procedures;
+showddl procedure put;
+call ls ('lib_mgmt.jar',?);
+
+-- should fail - library management already enabled
+initialize trafodion, create library management;
+
+-- make sure everything is enabled
+execute get_schemas;
+get procedures;
+
+-- we are done
+exit;
+
+
+
+

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/CmpSeabaseDDL.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDL.h b/core/sql/sqlcomp/CmpSeabaseDDL.h
index e1eee25..243c95e 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDL.h
+++ b/core/sql/sqlcomp/CmpSeabaseDDL.h
@@ -1130,6 +1130,13 @@ class CmpSeabaseDDL
   void dropSeabaseRoutine(StmtDDLDropRoutine  * dropRoutineNode,
                           NAString &currCatName, NAString &currSchName);
 
+  short createSeabaseLibmgr(ExeCliInterface * cliInterface);
+  short upgradeSeabaseLibmgr(ExeCliInterface * inCliInterface);
+  short dropSeabaseLibmgr(ExeCliInterface *inCliInterface);
+  short createLibmgrProcs(ExeCliInterface * cliInterface);
+  short grantLibmgrPrivs(ExeCliInterface *cliInterface);
+
+
   void createSeabaseSequence(StmtDDLCreateSequence  * createSequenceNode,
 			     NAString &currCatName, NAString &currSchName);
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp b/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
index 7d7ac01..d143724 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLauth.cpp
@@ -1427,23 +1427,26 @@ void CmpSeabaseDDLrole::createRole(StmtDDLCreateRole * pNode)
 // Input:  
 //    role name
 //    role ID
+//
+// returns:  true - added role,
+//           false - did not add role
 // ----------------------------------------------------------------------------
-void CmpSeabaseDDLrole::createStandardRole(
+bool CmpSeabaseDDLrole::createStandardRole(
    const std::string roleName,
    const int32_t roleID)
 
 {
 
-// Verify name is a standard name
+   // Verify name is a standard name
 
-size_t prefixLength = strlen(RESERVED_AUTH_NAME_PREFIX);
+   size_t prefixLength = strlen(RESERVED_AUTH_NAME_PREFIX);
 
    if (roleName.size() <= prefixLength ||
        roleName.compare(0,prefixLength,RESERVED_AUTH_NAME_PREFIX) != 0)
    {
        *CmpCommon::diags() << DgSqlCode(-CAT_ROLE_NOT_EXIST)
                            << DgString0(roleName.data());
-       return;
+       return false;
    }
 
    setAuthDbName(roleName.c_str());
@@ -1457,13 +1460,16 @@ size_t prefixLength = strlen(RESERVED_AUTH_NAME_PREFIX);
 
    // Make sure role has not already been registered
    if (authExists(getAuthDbName(),false))
-      return;
-   
-   setAuthID(roleID);
+      return false;
+
+   Int32 newRoleID = (roleID == NA_UserIdDefault) ? getUniqueID() : roleID;
+   setAuthID(newRoleID);
    setAuthCreator(ComUser::getRootUserID());
 
 // Add the role to AUTHS table
    insertRow();
+  
+   return true;
 
 }
 
@@ -1838,7 +1844,7 @@ Int32 CmpSeabaseDDLrole::getUniqueID()
   sprintf(roleIDString,"%d",MIN_ROLEID);
   whereClause += roleIDString;
   whereClause += " and auth_id < ";
-  sprintf(roleIDString, "%d", MAX_ROLEID);
+  sprintf(roleIDString, "%d", MAX_ROLEID_RANGE1);
   whereClause += roleIDString;
 
   newRoleID = selectMaxAuthID(whereClause);
@@ -1850,7 +1856,7 @@ Int32 CmpSeabaseDDLrole::getUniqueID()
   // We have 490000 available ID's.  Don't expect to run out of ID's for awhile
   // but if/when we do, the algorithm needs to change.  Can reuse ID's for roles 
   // that were dropped.
-  if (newRoleID >= MAX_ROLEID)
+  if (newRoleID >= MAX_ROLEID_RANGE1)
     SEABASEDDL_INTERNAL_ERROR("CmpSeabaseDDLrole::getUniqueID failed, ran out of available IDs");
 
   return newRoleID;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/CmpSeabaseDDLauth.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLauth.h b/core/sql/sqlcomp/CmpSeabaseDDLauth.h
index 382c64d..17d49ad 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLauth.h
+++ b/core/sql/sqlcomp/CmpSeabaseDDLauth.h
@@ -193,7 +193,7 @@ class CmpSeabaseDDLrole : public CmpSeabaseDDLauth
 
      void createRole(StmtDDLCreateRole * pNode);
         
-     void createStandardRole(
+     bool createStandardRole(
         const std::string roleName,
         const int32_t roleID);
         

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
index 938d8f2..781ee1f 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
@@ -67,6 +67,7 @@
 #include "ComUser.h"
 #include "ComMisc.h"
 #include "CmpSeabaseDDLmd.h"
+#include "CmpSeabaseDDLroutine.h"
 #include "hdfs.h"
 void cleanupLOBDataDescFiles(const char*, int, const char *);
 
@@ -6829,6 +6830,11 @@ void CmpSeabaseDDL::initSeabaseMD(NABoolean ddlXns)
      goto label_error;
    }
 
+ if (createSeabaseLibmgr (&cliInterface))
+   {
+     goto label_error;
+   }
+
   cliRC = cliInterface.restoreCQD("traf_bootstrap_md_mode");
 
   return;
@@ -7648,10 +7654,28 @@ short CmpSeabaseDDL::initSeabaseAuthorization(
 
   if (retcode != STATUS_ERROR)
   {
-    // change authorization status in compiler context and kill arkcmps
-    GetCliGlobals()->currContext()->setAuthStateInCmpContexts(TRUE, TRUE);
-    for (short i = 0; i < GetCliGlobals()->currContext()->getNumArkcmps(); i++)
-      GetCliGlobals()->currContext()->getArkcmp(i)->endConnection();
+     // Commit the transaction so privmgr schema exists in other processes
+     endXnIfStartedHere(cliInterface, xnWasStartedHere, 0);
+     if (beginXnIfNotInProgress(cliInterface, xnWasStartedHere))
+     {
+       SEABASEDDL_INTERNAL_ERROR("initialize authorization");
+       return -1;
+     }
+
+     // change authorization status in compiler context and kill arkcmps
+     GetCliGlobals()->currContext()->setAuthStateInCmpContexts(TRUE, TRUE);
+     for (short i = 0; i < GetCliGlobals()->currContext()->getNumArkcmps(); i++)
+       GetCliGlobals()->currContext()->getArkcmp(i)->endConnection();
+
+     // If someone initializes trafodion with library management but does not 
+     // initialize authorization, then the role DB__LIBMGRROLE has not been 
+     // granted to LIBMGR procedures.  Do this now
+     cliRC = existsInSeabaseMDTable(cliInterface,
+                                    getSystemCatalog(), SEABASE_LIBMGR_SCHEMA, 
+                                    SEABASE_LIBMGR_LIBRARY,
+                                    COM_LIBRARY_OBJECT, TRUE, FALSE);
+     if (cliRC == 1) // library exists
+       cliRC = grantLibmgrPrivs(cliInterface);
   }
   else
   {
@@ -8292,6 +8316,7 @@ short CmpSeabaseDDL::executeSeabaseDDL(DDLExpr * ddlExpr, ExprNode * ddlNode,
        (ddlExpr->dropRepos()) ||
        (ddlExpr->upgradeRepos()) ||
        (ddlExpr->addSchemaObjects()) ||
+       (ddlExpr->createLibmgr()) ||
        (ddlExpr->updateVersion())))
     ignoreUninitTrafErr = TRUE;
 
@@ -8476,6 +8501,18 @@ short CmpSeabaseDDL::executeSeabaseDDL(DDLExpr * ddlExpr, ExprNode * ddlNode,
     {
       createSeabaseSchemaObjects();
     }
+  else if (ddlExpr->createLibmgr())
+    {
+      createSeabaseLibmgr(&cliInterface);
+    }
+  else if (ddlExpr->dropLibmgr())
+    {
+      dropSeabaseLibmgr(&cliInterface);
+    }
+  else if (ddlExpr->upgradeLibmgr())
+    {
+      upgradeSeabaseLibmgr(&cliInterface);
+    }
   else if (ddlExpr->updateVersion())
     {
       updateVersion();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp b/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
index 85a3f26..fdfc301 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
@@ -40,6 +40,7 @@
 
 #include "ComObjectName.h"
 #include "ComUser.h"
+#include "CmpSeabaseDDLroutine.h"
 
 #include "StmtDDLCreateRoutine.h"
 #include "StmtDDLDropRoutine.h"
@@ -1425,3 +1426,169 @@ short CmpSeabaseDDL::validateRoutine(ExeCliInterface *cliInterface,
   return -1;
 
 } // CmpSeabaseDDL::validateRoutine
+
+short CmpSeabaseDDL::createSeabaseLibmgr(ExeCliInterface * cliInterface)
+{
+  Lng32 cliRC = 0;
+  
+  if ((CmpCommon::context()->isUninitializedSeabase()) &&
+      (CmpCommon::context()->uninitializedSeabaseErrNum() == -1393))
+    {
+      *CmpCommon::diags() << DgSqlCode(-1393);
+      return -1;
+    }
+
+  NAString jarLocation(getenv("MY_SQROOT"));
+  jarLocation += "/export/lib/lib_mgmt.jar";
+  char queryBuf[strlen(getSystemCatalog()) + strlen(SEABASE_LIBMGR_SCHEMA) +
+                strlen(SEABASE_LIBMGR_LIBRARY) + strlen(DB__LIBMGRROLE) + 
+                jarLocation.length() + 100];
+
+  // Create the SEABASE_LIBMGR_SCHEMA schema
+  str_sprintf(queryBuf, "create schema if not exists %s.\"%s\" authorization %s ",
+              getSystemCatalog(),SEABASE_LIBMGR_SCHEMA, DB__ROOT);
+
+  cliRC = cliInterface->executeImmediate(queryBuf);
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+
+  // Create the SEABASE_LIBMGR_LIBRARY library
+  str_sprintf(queryBuf, "create library %s.\"%s\".%s file '%s'",
+                         getSystemCatalog(), SEABASE_LIBMGR_SCHEMA, SEABASE_LIBMGR_LIBRARY,
+                         jarLocation.data());
+
+  cliRC = cliInterface->executeImmediate(queryBuf);
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+
+  return (createLibmgrProcs(cliInterface));
+}
+
+short CmpSeabaseDDL::createLibmgrProcs(ExeCliInterface * cliInterface)
+{
+  Lng32 cliRC = 0;
+  char queryBuf[2000];
+
+ // Create the procedures if they don't already exist
+  for (Int32 i = 0; i < sizeof(allLibmgrRoutineInfo)/sizeof(LibmgrRoutineInfo); i++)
+    {
+      // Get the next procedure routine details
+      const LibmgrRoutineInfo &prd = allLibmgrRoutineInfo[i];
+
+      const QString * qs = NULL;
+      Int32 sizeOfqs = 0;
+
+      qs = prd.newDDL;
+      sizeOfqs = prd.sizeOfnewDDL;
+
+      Int32 qryArraySize = sizeOfqs / sizeof(QString);
+      char * gluedQuery;
+      Lng32 gluedQuerySize;
+      glueQueryFragments(qryArraySize,  qs,
+                         gluedQuery, gluedQuerySize);
+
+      param_[0] = getSystemCatalog();
+      param_[1] = SEABASE_LIBMGR_SCHEMA;
+      param_[2] = getSystemCatalog();
+      param_[3] = SEABASE_LIBMGR_SCHEMA;
+      param_[4] = SEABASE_LIBMGR_LIBRARY;
+
+      str_sprintf(queryBuf, gluedQuery, param_[0], param_[1], param_[2], param_[3], param_[4]);
+      NADELETEBASIC(gluedQuery, STMTHEAP);
+
+      cliRC = cliInterface->executeImmediate(queryBuf);
+      if (cliRC < 0)
+        {
+          cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+        return -1;
+        }
+    } // for
+
+  return (grantLibmgrPrivs(cliInterface));
+}
+
+// If authorization is enabled, grant privileges to DB__LIBMGRROLE
+short CmpSeabaseDDL::grantLibmgrPrivs(ExeCliInterface *cliInterface)
+{
+  if (!isAuthorizationEnabled())
+    return 0;
+
+  Lng32 cliRC = 0;
+  char queryBuf[strlen(getSystemCatalog()) + strlen(SEABASE_LIBMGR_SCHEMA) +
+                strlen(SEABASE_LIBMGR_LIBRARY) + strlen(DB__LIBMGRROLE) + 200];
+  for (Int32 i = 0; i < sizeof(allLibmgrRoutineInfo)/sizeof(LibmgrRoutineInfo); i++)
+    {
+      // Get the next procedure routine details
+      const LibmgrRoutineInfo &prd = allLibmgrRoutineInfo[i];
+
+      str_sprintf (queryBuf, "grant execute on procedure %s.\"%s\".%s to %s with grant option",
+                              getSystemCatalog(),SEABASE_LIBMGR_SCHEMA,prd.newName,
+                              DB__LIBMGRROLE);
+      cliRC = cliInterface->executeImmediate(queryBuf);
+      if (cliRC < 0)
+        {
+          cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+          return -1;
+        }
+    }
+  return 0;
+}
+
+short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
+{
+  Lng32 cliRC = 0;
+
+  cliRC = existsInSeabaseMDTable(cliInterface,
+                                 getSystemCatalog(), SEABASE_LIBMGR_SCHEMA,
+                                 SEABASE_LIBMGR_LIBRARY,
+                                 COM_LIBRARY_OBJECT, TRUE, FALSE);
+  if (cliRC < 0)
+    return -1;
+
+  if (cliRC == 0) // does not exist
+    {
+      NAString libraryName(getSystemCatalog());
+      libraryName + ".\"" + SEABASE_LIBMGR_SCHEMA + "\"" + SEABASE_LIBMGR_LIBRARY;
+      *CmpCommon::diags() << DgSqlCode(-1389)
+                          << DgString0(libraryName.data());
+      return -1;
+    }
+
+  return (createLibmgrProcs(cliInterface));
+}
+
+short CmpSeabaseDDL::dropSeabaseLibmgr(ExeCliInterface *cliInterface)
+{
+  Lng32 cliRC = 0;
+
+  char queryBuf[strlen(getSystemCatalog()) + strlen(SEABASE_LIBMGR_SCHEMA) + 100];
+
+  // save the current parserflags setting
+  ULng32 savedParserFlags = Get_SqlParser_Flags(0xFFFFFFFF);
+  Set_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL);
+
+  str_sprintf(queryBuf, "drop schema if exists %s.\"%s\" cascade ",
+              getSystemCatalog(),SEABASE_LIBMGR_SCHEMA, DB__ROOT);
+
+  // Drop the SEABASE_LIBMGR_SCHEMA schema
+  cliRC = cliInterface->executeImmediate(queryBuf);
+
+  // Restore parser flags settings to what they originally were
+  Assign_SqlParser_Flags(savedParserFlags);
+
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+  return 0;
+}
+
+
+  

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/9e3dea64/core/sql/sqlcomp/CmpSeabaseDDLroutine.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLroutine.h b/core/sql/sqlcomp/CmpSeabaseDDLroutine.h
new file mode 100644
index 0000000..18b6d7a
--- /dev/null
+++ b/core/sql/sqlcomp/CmpSeabaseDDLroutine.h
@@ -0,0 +1,280 @@
+// @@@ START COPYRIGHT @@@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// @@@ END COPYRIGHT @@@
+
+#ifndef _CMP_SEABASE_PROCEDURES_H_
+#define _CMP_SEABASE_PROCEDURES_H_
+
+#include "CmpSeabaseDDL.h"
+
+// To add a new procedure:
+//   update export/lib/lib_mgmt.jar to include code for the new procedure 
+//   add a define representing the procedures below
+//   add a static const QString representing the create procedure text
+//   add a new entry in allLibmgrRoutineInfo 
+//   perform initialize trafodion, upgrade library management
+// recommend that new procedures are added in alphabetic order
+
+// At this time there is no support to drop or change the signature of an
+// existing procedure.  Since customers may be using the procedures, it is
+// recommended that they not be dropped or changed - instead add new ones
+// to handle the required change.
+  
+// List of supported system procedures - in alphabetic order
+#define SYSTEM_PROC_ADDLIB       "ADDLIB"
+#define SYSTEM_PROC_ALTERLIB     "ALTERLIB"
+#define SYSTEM_PROC_DROPLIB      "DROPLIB"
+#define SYSTEM_PROC_GETFILE      "GETFILE"
+#define SYSTEM_PROC_HELP         "HELP"
+#define SYSTEM_PROC_LS           "LS"
+#define SYSTEM_PROC_LSALL        "LSALL"
+#define SYSTEM_PROC_PUT          "PUT"
+#define SYSTEM_PROC_RM           "RM"
+#define SYSTEM_PROC_RMREX        "RMREX"
+
+// Create procedure text for system procedures
+static const QString seabaseProcAddlibDDL[] =
+{
+  {"  CREATE PROCEDURE IF NOT EXISTS %s.\"%s\"."SYSTEM_PROC_ADDLIB" "},
+  {" ( "},
+  {"  IN LIBNAME VARCHAR(1024) CHARACTER SET UTF8, "},
+  {"  IN FILENAME VARCHAR(1024) CHARACTER SET UTF8, "},
+  {"  IN HOSTNAME VARCHAR(1024) CHARACTER SET UTF8, "},
+  {"  IN LOCALFILE VARCHAR(1024) CHARACTER SET UTF8) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.addLib (java.lang.String,java.lang.String,java.lang.String,java.lang.String)' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  CONTAINS SQL "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcAlterlibDDL[] = 
+{
+  {"  CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".ALTERLIB "},
+  {" ( "},
+  {"  IN LIBNAME VARCHAR(1024) CHARACTER SET UTF8,"},
+  {"  IN FILENAME VARCHAR(1024) CHARACTER SET UTF8,"},
+  {"  IN HOSTNAME VARCHAR(1024) CHARACTER SET UTF8,"},
+  {"  IN LOCALFILE VARCHAR(1024) CHARACTER SET UTF8) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.alterLib (java.lang.String,java.lang.String,java.lang.String,java.lang.String)' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  CONTAINS SQL "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcDroplibDDL[] = 
+{
+  {"  CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".DROPLIB "},
+  {" ( "},
+  {"  IN LIBNAME VARCHAR(1024) CHARACTER SET UTF8, "},
+  {"  IN MODETYPE VARCHAR(1024) CHARACTER SET ISO88591) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.dropLib (java.lang.String,java.lang.String)' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  CONTAINS SQL "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcGetfileDDL[] = 
+{
+  {"  CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".GETFILE  "},
+  {" ( "},
+  {"  IN FILENAME VARCHAR(256) CHARACTER SET UTF8,"},
+  {"  IN OFFSET INTEGER,"},
+  {"  OUT FILEDATA VARCHAR(12800) CHARACTER SET UTF8,"},
+  {"  OUT DATALENGTH LARGEINT)"},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.get (java.lang.String,int,java.lang.String[],long[])' "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  READS SQL DATA "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcHelpDDL[] =
+{
+  {" CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".HELP "},
+  {" ( "},
+  {" INOUT COMMANDNAME VARCHAR(2560) CHARACTER SET ISO88591) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.help (java.lang.String[])' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  READS SQL DATA "},
+  {" ; "}
+};
+
+static const QString seabaseProcLsDDL[] = 
+{
+  {" CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".LS  "},
+  {" ( "},
+  {"  IN FILENAME VARCHAR(256) CHARACTER SET UTF8, "},
+  {"  OUT FILENAMES VARCHAR(10240) CHARACTER SET ISO88591) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.ls(java.lang.String,java.lang.String[])' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  READS SQL DATA "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcLsallDDL[] = 
+{
+  {"  CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".LSALL "},
+  {" ( "},
+  {"  OUT FILENAMES VARCHAR(10240) CHARACTER SET ISO88591) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.lsAll(java.lang.String[])' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "}, 
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  READS SQL DATA "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcPutDDL[] =
+{
+  {" CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".PUT "},
+  {" ( "},
+  {"  IN FILEDATA VARCHAR(102400) CHARACTER SET ISO88591, "},
+  {"  IN FILENAME VARCHAR(256) CHARACTER SET UTF8, "},
+  {"  IN CREATEFLAG INTEGER, "},
+  {"  IN FILEOVERWRITE INTEGER) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.put(java.lang.String,java.lang.String,int,int)' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  READS SQL DATA "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcRmDDL[] = 
+{
+  {"  CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".RM "},
+  {" ( "},
+  {"  IN FILENAME VARCHAR(256) CHARACTER SET UTF8) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.rm(java.lang.String)' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  READS SQL DATA "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+static const QString seabaseProcRmrexDDL[] = 
+{
+  {"  CREATE PROCEDURE IF NOT EXISTS %s.\"%s\".RMREX  "},
+  {" ( "},
+  {"  IN FILENAME VARCHAR(256) CHARACTER SET UTF8, "},
+  {"  OUT FILENAMES VARCHAR(10240) CHARACTER SET ISO88591) "},
+  {"  EXTERNAL NAME 'org.trafodion.libmgmt.FileMgmt.rmRex(java.lang.String, java.lang.String[])' "},
+  {"  EXTERNAL SECURITY DEFINER "},
+  {"  LIBRARY %s.\"%s\".%s "},
+  {"  LANGUAGE JAVA "},
+  {"  PARAMETER STYLE JAVA "},
+  {"  READS SQL DATA "},
+  {"  NO TRANSACTION REQUIRED "},
+  {" ; "}
+};
+
+struct LibmgrRoutineInfo
+{
+  // name of the procedure
+  const char * newName;
+
+  // ddl stmt corresponding to the current ddl.
+  const QString *newDDL;
+  Lng32 sizeOfnewDDL;
+};
+
+static const LibmgrRoutineInfo allLibmgrRoutineInfo[] = {
+  {SYSTEM_PROC_ADDLIB, 
+   seabaseProcAddlibDDL, 
+   sizeof(seabaseProcAddlibDDL)
+  },
+
+  {SYSTEM_PROC_ALTERLIB, 
+   seabaseProcAlterlibDDL, 
+   sizeof(seabaseProcAlterlibDDL)
+  },
+
+  {SYSTEM_PROC_DROPLIB, 
+   seabaseProcDroplibDDL, 
+   sizeof(seabaseProcDroplibDDL)
+  },
+
+  {SYSTEM_PROC_GETFILE, 
+   seabaseProcGetfileDDL, 
+   sizeof(seabaseProcGetfileDDL)
+  },
+
+  {SYSTEM_PROC_HELP, 
+   seabaseProcHelpDDL, 
+   sizeof(seabaseProcHelpDDL)
+  },
+
+  {SYSTEM_PROC_LS, 
+   seabaseProcLsDDL, 
+   sizeof(seabaseProcLsDDL)
+  },
+
+  {SYSTEM_PROC_LSALL, 
+   seabaseProcLsallDDL, 
+   sizeof(seabaseProcLsallDDL)
+  },
+
+  {SYSTEM_PROC_PUT, 
+   seabaseProcPutDDL, 
+   sizeof(seabaseProcPutDDL)
+  },
+
+  {SYSTEM_PROC_RM, 
+   seabaseProcRmDDL, 
+   sizeof(seabaseProcRmDDL)
+  },
+
+  {SYSTEM_PROC_RMREX, 
+   seabaseProcRmrexDDL, 
+   sizeof(seabaseProcRmrexDDL)
+  }
+
+};
+
+#endif