You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by sa...@apache.org on 2018/11/06 05:31:34 UTC

[2/8] trafodion git commit: Fixees to support stroring Udr libraries s Blobs.

http://git-wip-us.apache.org/repos/asf/trafodion/blob/9528f8c0/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp b/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
index 29ef68e..f6bd227 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLroutine.cpp
@@ -57,7 +57,6 @@
 #include "CmpSeabaseDDL.h"
 
 #include "ExpHbaseInterface.h"
-
 #include "ExExeUtilCli.h"
 #include "Generator.h"
 #include "ComSmallDefs.h"
@@ -73,7 +72,7 @@
 
 #include "ComCextdecs.h"
 #include <sys/stat.h>
-
+short ExExeUtilLobExtractLibrary(ExeCliInterface *cliInterface,char *libHandle, char *cachedLibName,ComDiagsArea *toDiags);
 
 // *****************************************************************************
 // *                                                                           *
@@ -389,12 +388,245 @@ void CmpSeabaseDDL::createSeabaseLibrary(
     }
  
   char * query = new(STMTHEAP) char[1000];
-  str_sprintf(query, "insert into %s.\"%s\".%s values (%ld, '%s', %d, 0)",
-	      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES,
-	      objUID,
+  
+  //We come here only if CQD says use the old style without blobs . 
+  //So insert a NULL into the blob column.
+  str_sprintf(query, "insert into %s.\"%s\".%s values (%ld, '%s',NULL, %d, 0)",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES,
+              objUID,
               libFileName.data(),
               createLibraryNode->getVersion());
+    
+ 
+  Lng32 cliRC = cliInterface.executeImmediate(query);
+
+  NADELETEBASIC(query, STMTHEAP);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      processReturn();
+      return;
+    }
+    
+ 
+
+  // hope to remove this call soon by setting thevalid flag to Y sooner
+  if (updateObjectValidDef(&cliInterface, 
+			   catalogNamePart, schemaNamePart, objectNamePart,
+			   COM_LIBRARY_OBJECT_LIT,
+			   "Y"))
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  processReturn();
+
+  return;
+}
+short CmpSeabaseDDL::isLibBlobStoreValid(ExeCliInterface *cliInterface)
+{
+  Int32 cliRC=0;
+  char buf[4000];
+  char * query = new(STMTHEAP) char[1000];
+  str_sprintf(query, "select [first 1] library_storage from %s.\"%s\".%s ",
+	      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES
+	     );
+  
+  // set pointer in diags area
+  int32_t diagsMark = CmpCommon::diags()->mark();
+  cliRC = cliInterface->fetchRowsPrologue(query, TRUE/*no exec*/);
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      CmpCommon::diags()->rewind(diagsMark);
+      NADELETEBASIC(query, STMTHEAP);
+      return -1;
+    }
+
+  cliRC = cliInterface->clearExecFetchClose(NULL, 0);
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      CmpCommon::diags()->rewind(diagsMark);
+      NADELETEBASIC(query, STMTHEAP);
+      return -1;
+    }
+
+  if (cliRC >=0)
+    {
+      //found the new column.
+      NADELETEBASIC(query, STMTHEAP);
+      CmpCommon::diags()->rewind(diagsMark);
+      return 0;
+    }
+  
+  return 0;
+  NADELETEBASIC(query, STMTHEAP);
+ 
+}
+
+void CmpSeabaseDDL::createSeabaseLibrary2(
+				      StmtDDLCreateLibrary * createLibraryNode,
+				      NAString &currCatName, 
+                                      NAString &currSchName)
+{
+  Lng32 retcode = 0;
+ 
+  ComObjectName libraryName(createLibraryNode->getLibraryName());
+  ComAnsiNamePart currCatAnsiName(currCatName);
+  ComAnsiNamePart currSchAnsiName(currSchName);
+  libraryName.applyDefaults(currCatAnsiName, currSchAnsiName);
+  const NAString catalogNamePart = 
+    libraryName.getCatalogNamePartAsAnsiString();
+  const NAString schemaNamePart = 
+    libraryName.getSchemaNamePartAsAnsiString(TRUE);
+  const NAString objectNamePart = 
+    libraryName.getObjectNamePartAsAnsiString(TRUE);
+  const NAString extLibraryName = libraryName.getExternalName(TRUE);
+  const NAString extNameForHbase = catalogNamePart + "." + schemaNamePart + 
+    "." + objectNamePart;
+  
+  // Verify that the requester has MANAGE_LIBRARY privilege.
+  if (isAuthorizationEnabled() && !ComUser::isRootUserID())
+    {
+      NAString privMgrMDLoc;
+      CONCAT_CATSCH(privMgrMDLoc, getSystemCatalog(), SEABASE_PRIVMGR_SCHEMA);
+
+      PrivMgrComponentPrivileges componentPrivileges(std::string(privMgrMDLoc.data()),CmpCommon::diags());
+
+      if (!componentPrivileges.hasSQLPriv
+            (ComUser::getCurrentUser(),SQLOperation::MANAGE_LIBRARY,true))
+      {
+         *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+         processReturn ();
+         return;
+      }
+    }
+
+  // Check to see if user has the authority to create the library
+  ExeCliInterface cliInterface(STMTHEAP, 0, NULL,
+    CmpCommon::context()->sqlSession()->getParentQid());
+  Int32 objectOwnerID = SUPER_USER;
+  Int32 schemaOwnerID = SUPER_USER;
+  ComSchemaClass schemaClass;
+
+  retcode = verifyDDLCreateOperationAuthorized(&cliInterface,
+                                               SQLOperation::CREATE_LIBRARY,
+                                               catalogNamePart,
+                                               schemaNamePart,
+                                               schemaClass,
+                                               objectOwnerID,
+                                               schemaOwnerID);
+  if (retcode != 0)
+  {
+     handleDDLCreateAuthorizationError(retcode,catalogNamePart,schemaNamePart);
+     return;
+  }
+     
+  ExpHbaseInterface * ehi = NULL;
+
+  ehi = allocEHI();
+  if (ehi == NULL)
+    {
+      processReturn();
+      return;
+    }
+
+  retcode = existsInSeabaseMDTable(&cliInterface, 
+				   catalogNamePart, schemaNamePart, 
+                                   objectNamePart, COM_LIBRARY_OBJECT, 
+                                   TRUE, FALSE);
+  if (retcode < 0)
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  if (retcode == 1) // already exists
+    {
+      *CmpCommon::diags() << DgSqlCode(-1390)
+			  << DgString0(extLibraryName);
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  NAString libFileName = createLibraryNode->getFilename() ;
+  // strip blank spaces
+  libFileName = libFileName.strip(NAString::both, ' ');
+
+  //Source file needs to exist on local node for LOB function 
+  //filetolob to succeed
+   if (validateLibraryFileExists(libFileName, FALSE))
+     {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+  size_t lastSlash = libFileName.last('/');
+  NAString libNameNoPath;
+  if (lastSlash != NA_NPOS)
+    libNameNoPath = libFileName(lastSlash+1, libFileName.length()-lastSlash-1);
+  else
+    {
+      *CmpCommon::diags() << DgSqlCode(-1382)
+                        << DgString0(libFileName);
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+      
+    }
+  ComTdbVirtTableTableInfo * tableInfo = new(STMTHEAP) ComTdbVirtTableTableInfo[1];
+  tableInfo->tableName = NULL,
+  tableInfo->createTime = 0;
+  tableInfo->redefTime = 0;
+  tableInfo->objUID = 0;
+  tableInfo->objOwnerID = objectOwnerID;
+  tableInfo->schemaOwnerID = schemaOwnerID;
+  tableInfo->isAudited = 1;
+  tableInfo->validDef = 1;
+  tableInfo->hbaseCreateOptions = NULL;
+  tableInfo->numSaltPartns = 0;
+  tableInfo->rowFormat = COM_UNKNOWN_FORMAT_TYPE;
+  tableInfo->objectFlags = 0;
   
+  Int64 objUID = -1;
+  if (updateSeabaseMDTable(&cliInterface, 
+			   catalogNamePart, schemaNamePart, objectNamePart,
+			   COM_LIBRARY_OBJECT,
+			   "N",
+			   tableInfo,
+			   0,
+			   NULL,
+			   0,			       
+			   NULL,
+			   0, NULL,
+                           objUID))
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  if (objUID == -1)
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+ 
+  char * query = new(STMTHEAP) char[1000];
+ 
+  str_sprintf(query, "insert into %s.\"%s\".%s values (%ld, '%s',filetolob('%s'), %d, 0)",
+                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES,
+                  objUID,
+              libNameNoPath.data(),
+              libFileName.data(),
+                  createLibraryNode->getVersion());
+    
   Lng32 cliRC = cliInterface.executeImmediate(query);
 
   NADELETEBASIC(query, STMTHEAP);
@@ -421,7 +653,7 @@ void CmpSeabaseDDL::createSeabaseLibrary(
   return;
 }
 
-void CmpSeabaseDDL::dropSeabaseLibrary(StmtDDLDropLibrary * dropLibraryNode,
+void CmpSeabaseDDL::dropSeabaseLibrary2(StmtDDLDropLibrary * dropLibraryNode,
                                        NAString &currCatName, 
                                        NAString &currSchName)
 {
@@ -563,32 +795,332 @@ void CmpSeabaseDDL::dropSeabaseLibrary(StmtDDLDropLibrary * dropLibraryNode,
   processReturn();
   return;
 }
-
-void  CmpSeabaseDDL::alterSeabaseLibrary(StmtDDLAlterLibrary  *alterLibraryNode,
-					 NAString &currCatName, 
-					 NAString &currSchName)
+void CmpSeabaseDDL::dropSeabaseLibrary(StmtDDLDropLibrary * dropLibraryNode,
+                                       NAString &currCatName, 
+                                       NAString &currSchName)
 {
-  Lng32 cliRC;
-  Lng32 retcode;
-  
-  NAString libraryName = alterLibraryNode->getLibraryName();
-  NAString libFileName = alterLibraryNode->getFilename();
-  
-  ComObjectName libName(libraryName, COM_TABLE_NAME);
+  Lng32 cliRC = 0;
+  Lng32 retcode = 0;
+
+  BindWA bindWA(ActiveSchemaDB(), CmpCommon::context(), FALSE/*inDDL*/);
+  NARoutineDB *pRoutineDBCache  = ActiveSchemaDB()->getNARoutineDB();
+  const NAString &objName = dropLibraryNode->getLibraryName();
+
+  ComObjectName libraryName(objName);
   ComAnsiNamePart currCatAnsiName(currCatName);
   ComAnsiNamePart currSchAnsiName(currSchName);
-  libName.applyDefaults(currCatAnsiName, currSchAnsiName);
-  
-  NAString catalogNamePart = libName.getCatalogNamePartAsAnsiString();
-  NAString schemaNamePart = libName.getSchemaNamePartAsAnsiString(TRUE);
-  NAString libNamePart = libName.getObjectNamePartAsAnsiString(TRUE);
-  const NAString extLibName = libName.getExternalName(TRUE);
-  
-  ExeCliInterface cliInterface(STMTHEAP, 0, NULL,
-			       CmpCommon::context()->sqlSession()->getParentQid());
-  
+  libraryName.applyDefaults(currCatAnsiName, currSchAnsiName);
+
+  const NAString catalogNamePart = libraryName.
+    getCatalogNamePartAsAnsiString();
+  const NAString schemaNamePart = libraryName.
+    getSchemaNamePartAsAnsiString(TRUE);
+  const NAString objectNamePart = libraryName.
+    getObjectNamePartAsAnsiString(TRUE);
+  const NAString extLibraryName = libraryName.getExternalName(TRUE);
+
+  ExeCliInterface cliInterface(STMTHEAP, 0, NULL, 
+    CmpCommon::context()->sqlSession()->getParentQid());
+
+  ExpHbaseInterface * ehi = allocEHI();
+  if (ehi == NULL)
+    return;
+
   retcode = existsInSeabaseMDTable(&cliInterface, 
-				   catalogNamePart, schemaNamePart, libNamePart,
+				   catalogNamePart, schemaNamePart, 
+                                   objectNamePart,
+				   COM_LIBRARY_OBJECT, TRUE, FALSE);
+  if (retcode < 0)
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  if (retcode == 0) // does not exist
+    {
+      *CmpCommon::diags() << DgSqlCode(-1389)
+			  << DgString0(extLibraryName);
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  Int32 objectOwnerID = 0;
+  Int32 schemaOwnerID = 0;
+  Int64 objectFlags = 0;
+  Int64 objUID = getObjectInfo(&cliInterface,
+			      catalogNamePart.data(), schemaNamePart.data(), 
+			      objectNamePart.data(), COM_LIBRARY_OBJECT,
+                              objectOwnerID,schemaOwnerID,objectFlags);
+  if (objUID < 0 || objectOwnerID == 0 || schemaOwnerID == 0)
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  if (!isDDLOperationAuthorized(SQLOperation::DROP_LIBRARY,
+                                objectOwnerID,
+                                schemaOwnerID))
+  {
+     *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+     processReturn ();
+     return;
+  }
+  
+  Queue * usingRoutinesQueue = NULL;
+  cliRC = getUsingRoutines(&cliInterface, objUID, usingRoutinesQueue);
+  if (cliRC < 0)
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+  // If RESTRICT and the library is being used, return an error
+  if (cliRC != 100 && dropLibraryNode->getDropBehavior() == COM_RESTRICT_DROP_BEHAVIOR) 
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_DEPENDENT_ROUTINES_EXIST);
+
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+    
+  usingRoutinesQueue->position();
+  for (size_t i = 0; i < usingRoutinesQueue->numEntries(); i++)
+  { 
+     OutputInfo * rou = (OutputInfo*)usingRoutinesQueue->getNext(); 
+     
+     char * routineName = rou->get(0);
+     ComObjectType objectType = PrivMgr::ObjectLitToEnum(rou->get(1));
+
+     if (dropSeabaseObject(ehi, routineName,
+                           currCatName, currSchName, objectType,
+                           dropLibraryNode->ddlXns(),
+                           TRUE, FALSE))
+     {
+       deallocEHI(ehi); 
+       processReturn();
+       return;
+     }
+
+     // Remove routine from DBRoutinCache
+     ComObjectName objectName(routineName);
+     QualifiedName qualRoutineName(objectName, STMTHEAP);
+     NARoutineDBKey key(qualRoutineName, STMTHEAP);
+     NARoutine *cachedNARoutine = pRoutineDBCache->get(&bindWA, &key);
+
+     if (cachedNARoutine)
+     {
+       Int64 routineUID = *(Int64*)rou->get(2);
+       pRoutineDBCache->removeNARoutine(qualRoutineName,
+                                        ComQiScope::REMOVE_FROM_ALL_USERS,
+                                        routineUID,
+                                        dropLibraryNode->ddlXns(), FALSE);
+     }
+
+   }
+ 
+  // can get a slight perf. gain if we pass in objUID
+  if (dropSeabaseObject(ehi, objName,
+                        currCatName, currSchName, COM_LIBRARY_OBJECT,
+                        dropLibraryNode->ddlXns(),
+                        TRUE, FALSE))
+    {
+      deallocEHI(ehi); 
+      processReturn();
+      return;
+    }
+
+  deallocEHI(ehi);      
+  processReturn();
+  return;
+}
+
+void  CmpSeabaseDDL::alterSeabaseLibrary2(StmtDDLAlterLibrary  *alterLibraryNode,
+					 NAString &currCatName, 
+					 NAString &currSchName)
+{
+  Lng32 cliRC;
+  Lng32 retcode;
+  
+  NAString libraryName = alterLibraryNode->getLibraryName();
+  NAString libFileName = alterLibraryNode->getFilename();
+  
+  ComObjectName libName(libraryName, COM_TABLE_NAME);
+  ComAnsiNamePart currCatAnsiName(currCatName);
+  ComAnsiNamePart currSchAnsiName(currSchName);
+  libName.applyDefaults(currCatAnsiName, currSchAnsiName);
+  
+  NAString catalogNamePart = libName.getCatalogNamePartAsAnsiString();
+  NAString schemaNamePart = libName.getSchemaNamePartAsAnsiString(TRUE);
+  NAString libNamePart = libName.getObjectNamePartAsAnsiString(TRUE);
+  const NAString extLibName = libName.getExternalName(TRUE);
+  
+  ExeCliInterface cliInterface(STMTHEAP, 0, NULL,
+			       CmpCommon::context()->sqlSession()->getParentQid());
+  
+  retcode = existsInSeabaseMDTable(&cliInterface, 
+				   catalogNamePart, schemaNamePart, libNamePart,
+				   COM_LIBRARY_OBJECT, TRUE, FALSE);
+  if (retcode < 0)
+    {
+      processReturn();
+      return;
+    }
+  
+  if (retcode == 0) // does not exist
+    {
+      CmpCommon::diags()->clear();
+      *CmpCommon::diags() << DgSqlCode(-1389)
+			  << DgString0(extLibName);
+      processReturn();
+      return;
+    }
+  
+  // strip blank spaces
+  libFileName = libFileName.strip(NAString::both, ' ');
+  if (validateLibraryFileExists(libFileName, FALSE))
+    {
+      processReturn();
+      return;
+    }
+  
+  Int32 objectOwnerID = 0;
+  Int32 schemaOwnerID = 0;
+  Int64 objectFlags = 0;
+  Int64 libUID = getObjectInfo(&cliInterface,
+			       catalogNamePart.data(), schemaNamePart.data(),
+			       libNamePart.data(), COM_LIBRARY_OBJECT,
+			       objectOwnerID,schemaOwnerID,objectFlags);
+  
+  // Check for error getting metadata information
+  if (libUID == -1 || objectOwnerID == 0)
+    {
+      if (CmpCommon::diags()->getNumber(DgSqlCode::ERROR_) == 0)
+	SEABASEDDL_INTERNAL_ERROR("getting object UID and owner for alter library");
+      processReturn();
+      return;
+    }
+  
+  // Verify that the current user has authority to perform operation
+  if (!isDDLOperationAuthorized(SQLOperation::ALTER_LIBRARY,
+				objectOwnerID,schemaOwnerID))
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      processReturn();
+      return;
+    }
+  
+  Int64 redefTime = NA_JulianTimestamp();
+    size_t lastSlash = libFileName.last('/');
+  NAString libNameNoPath;
+  if (lastSlash != NA_NPOS)
+    libNameNoPath = libFileName(lastSlash+1, libFileName.length()-lastSlash-1);
+  else
+    {
+      *CmpCommon::diags() << DgSqlCode(-1382)
+                        << DgString0(libFileName);
+      processReturn();
+      return;
+      
+    }
+  char buf[2048]; // filename max length is 512. Additional bytes for long
+  // library names.
+  str_sprintf(buf, "update %s.\"%s\".%s set library_filename = '%s' , library_storage = filetolob('%s') where library_uid = %ld",
+	      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES,
+              libNameNoPath.data(),
+	      libFileName.data(),
+	      libUID);
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      return;
+    }
+  
+  if (updateObjectRedefTime(&cliInterface,
+			    catalogNamePart, schemaNamePart, libNamePart,
+			    COM_LIBRARY_OBJECT_LIT,
+			    redefTime))
+    {
+      processReturn();
+      return;
+   }
+  SQL_QIKEY qiKey;
+
+  
+  qiKey.ddlObjectUID = libUID;
+  qiKey.operation[0] = 'O';
+  qiKey.operation[1] = 'R';
+
+  cliRC = SQL_EXEC_SetSecInvalidKeys(1, &qiKey);
+  if (cliRC < 0)
+    {
+      processReturn();
+      return;
+    }
+  BindWA bindWA(ActiveSchemaDB(), CmpCommon::context(), FALSE/*inDDL*/);
+  NARoutineDB *pRoutineDBCache  = ActiveSchemaDB()->getNARoutineDB();
+  Queue * usingRoutinesQueue = NULL;
+  cliRC = getUsingRoutines(&cliInterface, libUID, usingRoutinesQueue);
+  if (cliRC < 0)
+    {
+      processReturn();
+      return;
+    }
+  usingRoutinesQueue->position();
+  for (size_t i = 0; i < usingRoutinesQueue->numEntries(); i++)
+    { 
+      OutputInfo * rou = (OutputInfo*)usingRoutinesQueue->getNext();    
+      char * routineName = rou->get(0);
+      ComObjectType objectType = PrivMgr::ObjectLitToEnum(rou->get(1));
+      // Remove routine from DBRoutinCache
+      ComObjectName objectName(routineName);
+      QualifiedName qualRoutineName(objectName, STMTHEAP);
+      NARoutineDBKey key(qualRoutineName, STMTHEAP);
+      NARoutine *cachedNARoutine = pRoutineDBCache->get(&bindWA, &key);
+      if (cachedNARoutine)
+	{
+	  Int64 routineUID = *(Int64*)rou->get(2);
+	  pRoutineDBCache->removeNARoutine(qualRoutineName,
+					   ComQiScope::REMOVE_FROM_ALL_USERS,
+					   routineUID,
+					   alterLibraryNode->ddlXns(), FALSE);
+	}
+    }
+  
+  return;
+}
+
+
+void  CmpSeabaseDDL::alterSeabaseLibrary(StmtDDLAlterLibrary  *alterLibraryNode,
+					 NAString &currCatName, 
+					 NAString &currSchName)
+{
+  Lng32 cliRC;
+  Lng32 retcode;
+  
+  NAString libraryName = alterLibraryNode->getLibraryName();
+  NAString libFileName = alterLibraryNode->getFilename();
+  
+  ComObjectName libName(libraryName, COM_TABLE_NAME);
+  ComAnsiNamePart currCatAnsiName(currCatName);
+  ComAnsiNamePart currSchAnsiName(currSchName);
+  libName.applyDefaults(currCatAnsiName, currSchAnsiName);
+  
+  NAString catalogNamePart = libName.getCatalogNamePartAsAnsiString();
+  NAString schemaNamePart = libName.getSchemaNamePartAsAnsiString(TRUE);
+  NAString libNamePart = libName.getObjectNamePartAsAnsiString(TRUE);
+  const NAString extLibName = libName.getExternalName(TRUE);
+  
+  ExeCliInterface cliInterface(STMTHEAP, 0, NULL,
+			       CmpCommon::context()->sqlSession()->getParentQid());
+  
+  retcode = existsInSeabaseMDTable(&cliInterface, 
+				   catalogNamePart, schemaNamePart, libNamePart,
 				   COM_LIBRARY_OBJECT, TRUE, FALSE);
   if (retcode < 0)
     {
@@ -697,6 +1229,26 @@ void  CmpSeabaseDDL::alterSeabaseLibrary(StmtDDLAlterLibrary  *alterLibraryNode,
   return;
 }
 
+short CmpSeabaseDDL::extractLibrary(ExeCliInterface *cliInterface,  char *libHandle, char *cachedLibName)
+{
+  struct stat statbuf;
+  Int64 libUID = 0;
+  short retcode = 0;
+      if (stat(cachedLibName, &statbuf) != 0)
+        {
+          retcode =  ExExeUtilLobExtractLibrary(cliInterface, libHandle, cachedLibName, 
+                                                CmpCommon::diags());
+          if (retcode < 0)
+            {
+              *CmpCommon::diags() <<  DgSqlCode(-4316)
+                                  << DgString0(cachedLibName);
+              processReturn();
+            }
+        }
+      
+  return retcode;
+}
+
 void CmpSeabaseDDL::createSeabaseRoutine(
 				      StmtDDLCreateRoutine * createRoutineNode,
 				      NAString &currCatName, 
@@ -727,7 +1279,7 @@ void CmpSeabaseDDL::createSeabaseRoutine(
   Int32 objectOwnerID = SUPER_USER;
   Int32 schemaOwnerID = SUPER_USER;
   ComSchemaClass schemaClass;
-
+  NAString libSuffix, libPrefix;
   retcode = verifyDDLCreateOperationAuthorized(&cliInterface,
                                                SQLOperation::CREATE_ROUTINE,
                                                catalogNamePart,
@@ -780,14 +1332,20 @@ void CmpSeabaseDDL::createSeabaseRoutine(
   NAString libObjNamePart = libName.getObjectNamePartAsAnsiString(TRUE);
   const NAString extLibraryName = libName.getExternalName(TRUE);
   char externalPath[512] ;
+  char libBlobHandle[LOB_HANDLE_LEN];
   Lng32 cliRC = 0;
-	
+  Int64 redefTime =0;	
   // this call needs to change
-  Int64 libUID = getObjectUID(&cliInterface, 
-                              libCatNamePart, 
-                              libSchNamePart, 
-                              libObjNamePart,
-                              COM_LIBRARY_OBJECT_LIT);
+  Int64 libUID = 0;
+  
+  Int32 dummy32;
+  Int64 dummy64;
+       
+  libUID = getObjectInfo(&cliInterface,
+                         libCatNamePart, libSchNamePart, 
+                         libObjNamePart, COM_LIBRARY_OBJECT,
+                         dummy32,dummy32,dummy64,FALSE,FALSE, &dummy64, &redefTime);
+     
 
   if (libUID < 0)
     {
@@ -804,13 +1362,15 @@ void CmpSeabaseDDL::createSeabaseRoutine(
       return;
     }
 
-  // read the library path name from the LIBRARIES metadata table
-
+  
+  // read the library  name from the LIBRARIES metadata table
   char * buf = new(STMTHEAP) char[200];
-  str_sprintf(buf, "select library_filename from %s.\"%s\".%s"
+ 
+  str_sprintf(buf, "select library_filename, library_storage from %s.\"%s\".%s"
               " where library_uid = %ld for read uncommitted access",
               getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES, libUID);
-
+    
+   
   cliRC = cliInterface.fetchRowsPrologue(buf, TRUE/*no exec*/);
   if (cliRC < 0)
     {
@@ -839,21 +1399,27 @@ void CmpSeabaseDDL::createSeabaseRoutine(
 
   char * ptr = NULL;
   Lng32 len = 0;
+  
   cliInterface.getPtrAndLen(1, ptr, len);
   str_cpy_all(externalPath, ptr, len);
   externalPath[len] = '\0'; 
-
+  
+  cliInterface.getPtrAndLen(2, ptr, len);
+  str_cpy_all(libBlobHandle, ptr, len);
+  libBlobHandle[len] = '\0'; 
+    
+  
+  NAString extPath(externalPath);
+  size_t lastDot = extPath.last('.');
+     
+  
+  if (lastDot != NA_NPOS)  
+    libSuffix = extPath(lastDot,extPath.length()-lastDot);
+       
   // determine language and parameter styles based on the library
   // type, unless already specified
   if (!createRoutineNode->isLanguageTypeSpecified())
-    {
-      NAString extPath(externalPath);
-      size_t lastDot = extPath.last('.');
-      NAString libSuffix;
-
-      if (lastDot != NA_NPOS)
-        libSuffix = extPath(lastDot,extPath.length()-lastDot);
-
+    {   
       libSuffix.toUpper();
 
       if (libSuffix == ".JAR")
@@ -995,180 +1561,293 @@ void CmpSeabaseDDL::createSeabaseRoutine(
   sigBuf[0] = '\0';
 
   if (style == COM_STYLE_JAVA_CALL) 
-  {
-     // validate routine for Java call based on signature
-     Lng32 numJavaParam = 0;
-     ComFSDataType *paramType = new ComFSDataType[numParams];
-     ComUInt32     *subType   = new ComUInt32    [numParams];
-     ComColumnDirection *direction = new ComColumnDirection[numParams];
-     NAType *genericType;
+    {
+      // validate routine for Java call based on signature
+      Lng32 numJavaParam = 0;
+      ComFSDataType *paramType = new ComFSDataType[numParams];
+      ComUInt32     *subType   = new ComUInt32    [numParams];
+      ComColumnDirection *direction = new ComColumnDirection[numParams];
+      NAType *genericType;
 
-     // Gather the param attributes for LM from the paramDefArray previously
-     // populated and from the routineparamList generated from paramDefArray.
+      // Gather the param attributes for LM from the paramDefArray previously
+      // populated and from the routineparamList generated from paramDefArray.
 
-     for (CollIndex i = 0; (Int32)i < numParams; i++)
-     {
-       paramType[i] = (ComFSDataType)routineParamArray[i]->getParamDataType()->getFSDatatype();
-       subType[i] = 0;  // default
-       // Set subType for special cases detected by LM
-       switch ( paramType[i] )
-       {
-         case COM_SIGNED_BIN8_FSDT :
-         case COM_UNSIGNED_BIN8_FSDT :
-         case COM_SIGNED_BIN16_FSDT :
-         case COM_SIGNED_BIN32_FSDT :
-         case COM_SIGNED_BIN64_FSDT :
-         case COM_UNSIGNED_BIN16_FSDT :
-         case COM_UNSIGNED_BIN32_FSDT :
-         case COM_UNSIGNED_BPINT_FSDT :
-         {
-           genericType = routineParamArray[i]->getParamDataType() ;
-           if (genericType->getTypeName() == LiteralNumeric)
-             subType[i] = genericType->getPrecision();
-           else
-             subType[i] = 0 ;
-
-           break;
-         }
-
-         case COM_DATETIME_FSDT :
-         {
-           genericType = routineParamArray[i]->getParamDataType() ;
-           DatetimeType & datetimeType = (DatetimeType &) *genericType;
-            if (datetimeType.getSimpleTypeName() EQU "DATE")
-              subType[i] = 1 ;
-            else if (datetimeType.getSimpleTypeName() EQU "TIME")
-              subType[i] = 2;
-            else if (datetimeType.getSimpleTypeName() EQU "TIMESTAMP")
-              subType[i] = 3;
-         }
-       } // end switch paramType[i]
-
-       direction[i] = (ComColumnDirection) routineParamArray[i]->getParamDirection();
-     }
+      for (CollIndex i = 0; (Int32)i < numParams; i++)
+        {
+          paramType[i] = (ComFSDataType)routineParamArray[i]->getParamDataType()->getFSDatatype();
+          subType[i] = 0;  // default
+          // Set subType for special cases detected by LM
+          switch ( paramType[i] )
+            {
+            case COM_SIGNED_BIN8_FSDT :
+            case COM_UNSIGNED_BIN8_FSDT :
+            case COM_SIGNED_BIN16_FSDT :
+            case COM_SIGNED_BIN32_FSDT :
+            case COM_SIGNED_BIN64_FSDT :
+            case COM_UNSIGNED_BIN16_FSDT :
+            case COM_UNSIGNED_BIN32_FSDT :
+            case COM_UNSIGNED_BPINT_FSDT :
+              {
+                genericType = routineParamArray[i]->getParamDataType() ;
+                if (genericType->getTypeName() == LiteralNumeric)
+                  subType[i] = genericType->getPrecision();
+                else
+                  subType[i] = 0 ;
+
+                break;
+              }
+
+            case COM_DATETIME_FSDT :
+              {
+                genericType = routineParamArray[i]->getParamDataType() ;
+                DatetimeType & datetimeType = (DatetimeType &) *genericType;
+                if (datetimeType.getSimpleTypeName() EQU "DATE")
+                  subType[i] = 1 ;
+                else if (datetimeType.getSimpleTypeName() EQU "TIME")
+                  subType[i] = 2;
+                else if (datetimeType.getSimpleTypeName() EQU "TIMESTAMP")
+                  subType[i] = 3;
+              }
+            } // end switch paramType[i]
+
+          direction[i] = (ComColumnDirection) routineParamArray[i]->getParamDirection();
+        }
     
-     // If the syntax specified a signature, pass that to LanguageManager.
-     NAString specifiedSig( createRoutineNode->getJavaSignature() );
-     char* optionalSig;
-     if ( specifiedSig.length() == 0 )
-       optionalSig = NULL;
-     else
-       optionalSig = (char *)specifiedSig.data();
+      // If the syntax specified a signature, pass that to LanguageManager.
+      NAString specifiedSig( createRoutineNode->getJavaSignature() );
+      char* optionalSig;
+      if ( specifiedSig.length() == 0 )
+        optionalSig = NULL;
+      else
+        optionalSig = (char *)specifiedSig.data();
      
-     ComBoolean isJavaMain =
-       ((str_cmp_ne(createRoutineNode->getJavaMethodName(), "main") == 0) ? TRUE : FALSE);
-
-     LmResult createSigResult;
-     LmJavaSignature *lmSignature =  new (STMTHEAP) LmJavaSignature(NULL,
-                                                                    STMTHEAP);
-     createSigResult = lmSignature->createSig(paramType, subType, direction,
-                                              numParams, COM_UNKNOWN_FSDT, 0,
-                                              createRoutineNode->getMaxResults(), optionalSig, isJavaMain, sigBuf,
-                                              MAX_SIGNATURE_LENGTH,
-                                              CmpCommon::diags());
-     NADELETE(lmSignature, LmJavaSignature, STMTHEAP);
-     delete [] paramType;
-     delete [] subType;
-     delete [] direction;
-
-     // Lm returned error. Lm fills diags area, so no need to worry about diags.
-     if (createSigResult == LM_ERR)
-     {
-       *CmpCommon::diags() << DgSqlCode(-1231)
-			  << DgString0(extRoutineName);
-       deallocEHI(ehi); 
-       processReturn();
-       return;
-     }
+      ComBoolean isJavaMain =
+        ((str_cmp_ne(createRoutineNode->getJavaMethodName(), "main") == 0) ? TRUE : FALSE);
+
+      LmResult createSigResult;
+      LmJavaSignature *lmSignature =  new (STMTHEAP) LmJavaSignature(NULL,
+                                                                     STMTHEAP);
+      createSigResult = lmSignature->createSig(paramType, subType, direction,
+                                               numParams, COM_UNKNOWN_FSDT, 0,
+                                               createRoutineNode->getMaxResults(), optionalSig, isJavaMain, sigBuf,
+                                               MAX_SIGNATURE_LENGTH,
+                                               CmpCommon::diags());
+      NADELETE(lmSignature, LmJavaSignature, STMTHEAP);
+      delete [] paramType;
+      delete [] subType;
+      delete [] direction;
+
+      // Lm returned error. Lm fills diags area, so no need to worry about diags.
+      if (createSigResult == LM_ERR)
+        {
+          *CmpCommon::diags() << DgSqlCode(-1231)
+                              << DgString0(extRoutineName);
+          deallocEHI(ehi); 
+          processReturn();
+          return;
+        }
 
-     numJavaParam = (isJavaMain ? 1 : numParams);
+      numJavaParam = (isJavaMain ? 1 : numParams);
+
+      if( libBlobHandle[0] != '\0' )
+        {
+          NAString dummyUser;
+          NAString cachedLibName, cachedLibPath;
+          
+          if(ComGenerateUdrCachedLibName(extPath,redefTime,libSchNamePart,dummyUser, cachedLibName, cachedLibPath))
+            {
+              *CmpCommon::diags() << DgSqlCode(-1231)
+                                  << DgString0(extRoutineName);
+              deallocEHI(ehi); 
+              processReturn();
+              return;
+            }
+         
+          NAString cachedFullName = cachedLibPath+"/"+cachedLibName;
+          
+          if (extractLibrary(&cliInterface,libBlobHandle, (char *)cachedFullName.data()))
+            {
+              *CmpCommon::diags() << DgSqlCode(-1231)
+                                  << DgString0(extRoutineName);
+              deallocEHI(ehi); 
+              processReturn();
+              return;
+            }
+                       
+          if (validateRoutine(&cliInterface, 
+                              createRoutineNode->getJavaClassName(),
+                              createRoutineNode->getJavaMethodName(),
+                              cachedFullName,
+                              sigBuf,
+                              numJavaParam,
+                              createRoutineNode->getMaxResults(),
+                              optionalSig))
+            {
+              *CmpCommon::diags() << DgSqlCode(-1231)
+                                  << DgString0(extRoutineName);
+              deallocEHI(ehi); 
+              processReturn();
+              return;
+            }
+        }
+    
+      else
+        {                      
+          if (validateRoutine(&cliInterface, 
+                          createRoutineNode->getJavaClassName(),
+                          createRoutineNode->getJavaMethodName(),
+                          externalPath,
+                          sigBuf,
+                          numJavaParam,
+                          createRoutineNode->getMaxResults(),
+                          optionalSig))
+           
+            {
+              *CmpCommon::diags() << DgSqlCode(-1231)
+                              << DgString0(extRoutineName);
+              deallocEHI(ehi); 
+              processReturn();
+              return;
+            }
+        }
+    }
+          
 
-     if (validateRoutine(&cliInterface, 
-                         createRoutineNode->getJavaClassName(),
-                         createRoutineNode->getJavaMethodName(),
-                         externalPath,
-                         sigBuf,
-                         numJavaParam,
-                         createRoutineNode->getMaxResults(),
-                         optionalSig))
-     {
-       *CmpCommon::diags() << DgSqlCode(-1231)
-                           << DgString0(extRoutineName);
-       deallocEHI(ehi); 
-       processReturn();
-       return;
-     }
-  }
   else if (style == COM_STYLE_JAVA_OBJ ||
            style == COM_STYLE_CPP_OBJ)
-  {
-    // validate existence of the C++ or Java class in the library
-    Int32 routineHandle = NullCliRoutineHandle;
-    NAString externalPrefix(externalPath);
-    NAString externalNameForValidation(externalName);
-    NAString containerName;
+    {
+      // validate existence of the C++ or Java class in the library
+      Int32 routineHandle = NullCliRoutineHandle;
+      NAString externalPrefix(externalPath);
+      NAString externalNameForValidation(externalName);
+      NAString containerName;
 
-    if (language == COM_LANGUAGE_C || language == COM_LANGUAGE_CPP)
-      {
-        // separate the actual DLL name from the prefix
-        char separator = '/';
-        size_t separatorPos = externalPrefix.last(separator);
+      if (language == COM_LANGUAGE_C || language == COM_LANGUAGE_CPP)
+        {
+          if( libBlobHandle[0] != '\0' )
+            {
+              NAString dummyUser;
+              NAString cachedLibName, cachedLibPath;
+              
+              if (ComGenerateUdrCachedLibName(externalPrefix,redefTime,libSchNamePart,dummyUser, cachedLibName, cachedLibPath))
+                {
+                  *CmpCommon::diags() << DgSqlCode(-1231)
+                                      << DgString0(extRoutineName);
+                  deallocEHI(ehi); 
+                  processReturn();
+                  return;
+                }
+         
+              NAString cachedFullName = cachedLibPath+"/"+cachedLibName;
+          
+              if (extractLibrary(&cliInterface,libBlobHandle, (char *)cachedFullName.data()))
+                {
+                  *CmpCommon::diags() << DgSqlCode(-1231)
+                                      << DgString0(extRoutineName);
+                  deallocEHI(ehi); 
+                  processReturn();
+                  return;
+                }
+              externalPrefix = cachedLibPath;
+              containerName = cachedLibName;
+              
+            }
+          else
+            {
+              // separate the actual DLL name from the prefix
+              char separator = '/';
+              size_t separatorPos = externalPrefix.last(separator);
+
+              if (separatorPos != NA_NPOS)
+                {
+                  containerName = externalPrefix(separatorPos+1,
+                                                 externalPrefix.length()-separatorPos-1);
+                  externalPrefix.remove(separatorPos,
+                                        externalPrefix.length()-separatorPos);
+                }
+              else
+                {
+                  // assume the entire string is a local name
+                  containerName = externalPrefix;
+                  externalPrefix = ".";
+                }
+            }
+        }
+      else
+        {
+          // For Java, the way the language manager works is that the
+          // external path is the fully qualified name of the jar and
+          // the container is the class name (external name).  We load
+          // the container (the class) by searching in the path (the
+          // jar). The external name is the method name, which in this
+          // case is the constructor of the class, <init>.
 
-        if (separatorPos != NA_NPOS)
-          {
-            containerName = externalPrefix(separatorPos+1,
-                                           externalPrefix.length()-separatorPos-1);
-            externalPrefix.remove(separatorPos,
-                                  externalPrefix.length()-separatorPos);
-          }
-        else
-          {
-            // assume the entire string is a local name
-            containerName = externalPrefix;
-            externalPrefix = ".";
-          }
-      }
-    else
-      {
-        // For Java, the way the language manager works is that the
-        // external path is the fully qualified name of the jar and
-        // the container is the class name (external name).  We load
-        // the container (the class) by searching in the path (the
-        // jar). The external name is the method name, which in this
-        // case is the constructor of the class, <init>.
-
-        // leave externalPrevix unchanged, fully qualified jar file
-        containerName = externalName;
-        externalNameForValidation = "<init>";
-      }
+          // leave externalPrevix unchanged, fully qualified jar file
 
-    // use a CLI call to validate that the library contains the routine
-    if (cliInterface.getRoutine(
-             NULL, // No InvocationInfo specified in this step
-             0,
-             NULL,
-             0,
-             (Int32) language,
-             (Int32) style,
-             externalNameForValidation.data(),
-             containerName.data(),
-             externalPrefix.data(),
-             extLibraryName.data(),
-             &routineHandle,
-             CmpCommon::diags()) != LME_ROUTINE_VALIDATED)
-      {
-        if (routineHandle != NullCliRoutineHandle)
-          cliInterface.putRoutine(routineHandle,
-                                  CmpCommon::diags());
 
-        CMPASSERT(CmpCommon::diags()->mainSQLCODE() < 0);
-        processReturn();
-        return;
-      }
 
-    cliInterface.putRoutine(routineHandle,
-                            CmpCommon::diags());
-  }
+          if( libBlobHandle[0] != '\0' )
+            {
+              NAString dummyUser;
+              NAString cachedLibName, cachedLibPath;
+              NAString libSchema(libSchNamePart);
+              if(ComGenerateUdrCachedLibName(extPath,redefTime,libSchNamePart,dummyUser, cachedLibName, cachedLibPath))
+                {
+                  *CmpCommon::diags() << DgSqlCode(-1231)
+                                      << DgString0(extRoutineName);
+                  deallocEHI(ehi); 
+                  processReturn();
+                  return;
+                }
+         
+              NAString cachedFullName = cachedLibPath+"/"+cachedLibName;
+          
+              if (extractLibrary(&cliInterface,libBlobHandle, (char *)cachedFullName.data()))
+                {
+                  *CmpCommon::diags() << DgSqlCode(-1231)
+                                      << DgString0(extRoutineName);
+                  deallocEHI(ehi); 
+                  processReturn();
+                  return;
+                }
+              externalPrefix = cachedFullName;
+              containerName = externalName;
+              externalNameForValidation="<init>";
+            }
+          else
+            {
+              containerName = externalName;
+              externalNameForValidation = "<init>";
+            }
+        }
+
+      // use a CLI call to validate that the library contains the routine
+      if (cliInterface.getRoutine(
+               NULL, // No InvocationInfo specified in this step
+               0,
+               NULL,
+               0,
+               (Int32) language,
+               (Int32) style,
+               externalNameForValidation.data(),
+               containerName.data(),
+               externalPrefix.data(),
+               extLibraryName.data(),
+               &routineHandle,
+               CmpCommon::diags()) != LME_ROUTINE_VALIDATED)
+        {
+          if (routineHandle != NullCliRoutineHandle)
+            cliInterface.putRoutine(routineHandle,
+                                    CmpCommon::diags());
+
+          CMPASSERT(CmpCommon::diags()->mainSQLCODE() < 0);
+          processReturn();
+          return;
+        }
+
+      cliInterface.putRoutine(routineHandle,
+                              CmpCommon::diags());
+    }
 
   ComTdbVirtTableColumnInfo * colInfoArray = (ComTdbVirtTableColumnInfo*)
     new(STMTHEAP) ComTdbVirtTableColumnInfo[numParams];
@@ -1502,6 +2181,7 @@ short CmpSeabaseDDL::validateRoutine(ExeCliInterface *cliInterface,
               numSqlParam, maxResultSets, optionalSig ? 1 : 0); 
              
   Lng32 cliRC = cliInterface->fetchRowsPrologue(query, TRUE/*no exec*/);
+  
   if (cliRC < 0)
   {
      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
@@ -1602,9 +2282,11 @@ short CmpSeabaseDDL::createSeabaseLibmgr(ExeCliInterface * cliInterface)
       *CmpCommon::diags() << DgSqlCode(-TRAF_NOT_INITIALIZED);
       return -1;
     }
-
+  
   NAString jarLocation(getenv("TRAF_HOME"));
   jarLocation += "/export/lib/lib_mgmt.jar";
+   
+   
   char queryBuf[strlen(getSystemCatalog()) + strlen(SEABASE_LIBMGR_SCHEMA) +
                 strlen(SEABASE_LIBMGR_LIBRARY) + strlen(DB__LIBMGRROLE) + 
                 jarLocation.length() + 100];
@@ -1749,7 +2431,124 @@ short CmpSeabaseDDL::grantLibmgrPrivs(ExeCliInterface *cliInterface)
   return 0;
 }
 
-short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
+short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
+{
+  if (!ComUser::isRootUserID())
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      return -1;
+    }
+
+  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
+    {
+      // give an error if the Java library does not exist, since that is
+      // an indication that we never ran
+      // INITIALIZE TRAFODION, CREATE LIBRARY MANAGEMENT
+      NAString libraryName(getSystemCatalog());
+      libraryName + ".\"" + SEABASE_LIBMGR_SCHEMA + "\"" + SEABASE_LIBMGR_LIBRARY;
+      *CmpCommon::diags() << DgSqlCode(-1389)
+                          << DgString0(libraryName.data());
+      return -1;
+    }
+
+  // Update the jar locations for system procedures and functions.  This should 
+  // be done before adding any new jar's since we use a system procedure to add
+  // procedures.
+  NAString jarLocation(getenv("TRAF_HOME"));
+  jarLocation += "/export/lib";
+
+  char queryBuf[1000];
+
+  // trafodion-sql_currversion.jar 
+  Int32 stmtSize = snprintf(queryBuf, sizeof(queryBuf), "update %s.\"%s\".%s  "
+           "set library_filename = '%s/trafodion-sql-currversion.jar' "
+           "where library_uid = "
+           "(select object_uid from %s.\"%s\".%s "
+           " where object_name = '%s'  and object_type = 'LB')",
+           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_LIBRARIES, jarLocation.data(),
+           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_OBJECTS, SEABASE_VALIDATE_LIBRARY);
+  CMPASSERT(stmtSize < sizeof(queryBuf));
+
+  cliRC = cliInterface->executeImmediate(queryBuf);
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+
+  // lib_mgmt.jar
+  stmtSize = snprintf(queryBuf, sizeof(queryBuf), "update %s.\"%s\".%s  "
+           "set library_filename = '%s/lib_mgmt.jar' "
+           "where library_uid = "
+           "(select object_uid from %s.\"%s\".%s "
+           " where object_name = '%s'  and object_type = 'LB')",
+           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_LIBRARIES, jarLocation.data(),
+           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_OBJECTS, SEABASE_LIBMGR_LIBRARY);
+  CMPASSERT(stmtSize < sizeof(queryBuf));
+
+  cliRC = cliInterface->executeImmediate(queryBuf);
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+
+  // libudr_predef.so
+  NAString dllLocation(getenv("TRAF_HOME"));
+  dllLocation += "/export/lib64";
+  if (strcmp(getenv("SQ_MBTYPE"), "64d") == 0)
+    dllLocation += "d";
+
+  stmtSize = snprintf(queryBuf, sizeof(queryBuf), "update %s.\"%s\".%s  "
+           "set library_filename = '%s/libudr_predef.so' "
+           "where library_uid = "
+           "(select object_uid from %s.\"%s\".%s "
+           " where object_name = '%s'  and object_type = 'LB')",
+           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_LIBRARIES, dllLocation.data(),
+           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_OBJECTS, SEABASE_LIBMGR_LIBRARY_CPP);
+  CMPASSERT(stmtSize < sizeof(queryBuf));
+
+  cliRC = cliInterface->executeImmediate(queryBuf);
+  if (cliRC < 0)
+    {
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+
+
+  // now check for the C++ library, which was added in Trafodion 2.3
+  cliRC = existsInSeabaseMDTable(cliInterface,
+                                 getSystemCatalog(), SEABASE_LIBMGR_SCHEMA,
+                                 SEABASE_LIBMGR_LIBRARY_CPP,
+                                 COM_LIBRARY_OBJECT, TRUE, FALSE);
+  if (cliRC < 0)
+    return -1;
+
+  if (cliRC == 0)
+    {
+      // The Java library exists, but the C++ library does not yet
+      // exist. This means that we last created or upgraded the
+      // library management subsystem in Trafodion 2.2 or earlier.
+      // Create the C++ library, as it is needed for Trafodion 2.3
+      // and higher.
+      if (createSeabaseLibmgrCPPLib(cliInterface) < 0)
+        return -1;
+    }
+
+  return (createLibmgrProcs(cliInterface));
+}
+
+
+short CmpSeabaseDDL::upgradeSeabaseLibmgr2(ExeCliInterface * cliInterface)
 {
   if (!ComUser::isRootUserID())
     {
@@ -1778,6 +2577,8 @@ short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
       return -1;
     }
 
+  
+  
   // Update the jar locations for system procedures and functions.  This should 
   // be done before adding any new jar's since we use a system procedure to add
   // procedures.
@@ -1788,11 +2589,12 @@ short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
 
   // trafodion-sql_currversion.jar 
   Int32 stmtSize = snprintf(queryBuf, sizeof(queryBuf), "update %s.\"%s\".%s  "
-           "set library_filename = '%s/trafodion-sql-currversion.jar' "
+           "set library_filename = 'trafodion-sql-currversion.jar', "
+           "library_storage =   empty_blob() "              
            "where library_uid = "
            "(select object_uid from %s.\"%s\".%s "
            " where object_name = '%s'  and object_type = 'LB')",
-           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_LIBRARIES, jarLocation.data(),
+           getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_LIBRARIES,
            getSystemCatalog(),SEABASE_MD_SCHEMA, SEABASE_OBJECTS, SEABASE_VALIDATE_LIBRARY);
   CMPASSERT(stmtSize < sizeof(queryBuf));
 
@@ -1805,7 +2607,8 @@ short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
 
   // lib_mgmt.jar
   stmtSize = snprintf(queryBuf, sizeof(queryBuf), "update %s.\"%s\".%s  "
-           "set library_filename = '%s/lib_mgmt.jar' "
+           "set library_filename = 'lib_mgmt.jar', "
+           "library_storage = filetolob('%s/lib_mgmt.jar') "
            "where library_uid = "
            "(select object_uid from %s.\"%s\".%s "
            " where object_name = '%s'  and object_type = 'LB')",
@@ -1827,7 +2630,8 @@ short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
     dllLocation += "d";
 
   stmtSize = snprintf(queryBuf, sizeof(queryBuf), "update %s.\"%s\".%s  "
-           "set library_filename = '%s/libudr_predef.so' "
+           "set library_filename = 'libudr_predef.so', "
+           "library_storage = filetolob('%s/libudr_predef.so') "
            "where library_uid = "
            "(select object_uid from %s.\"%s\".%s "
            " where object_name = '%s'  and object_type = 'LB')",
@@ -1865,6 +2669,7 @@ short CmpSeabaseDDL::upgradeSeabaseLibmgr(ExeCliInterface * cliInterface)
   return (createLibmgrProcs(cliInterface));
 }
 
+
 short CmpSeabaseDDL::dropSeabaseLibmgr(ExeCliInterface *cliInterface)
 {
     if (!ComUser::isRootUserID())
@@ -1925,3 +2730,568 @@ short CmpSeabaseDDL::createSeabaseLibmgrCPPLib(ExeCliInterface * cliInterface)
     }
   return 0;
 }
+
+short CmpSeabaseDDL::upgradeLibraries(ExeCliInterface * cliInterface,
+                                  CmpDDLwithStatusInfo *mdui)
+{
+Lng32 cliRC = 0;
+
+  while (1) // exit via return stmt in switch
+    {
+      switch (mdui->subStep())
+        {
+        case 0:
+          {
+            mdui->setMsg("Upgrade Libraries: Started");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+      
+        case 1:
+          {
+            mdui->setMsg("  Start: Drop Old Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 2:
+          {
+            // drop old libraries
+            if (dropLibraries(cliInterface, TRUE/*old */))
+              return -3;  // error, but no recovery needed 
+        
+            mdui->setMsg("  End:   Drop Old Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 3:
+          {
+            mdui->setMsg("  Start: Rename Current Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+        
+        case 4:
+          {
+            // rename current libraries tables to *_OLD_LIBRARIES
+            if (alterRenameLibraries(cliInterface, TRUE))
+              return -2;  // error, need to undo the rename only
+
+            mdui->setMsg("  End:   Rename Current Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 5:
+          {
+            mdui->setMsg("  Start: Create New Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+         
+        case 6:
+          {
+            // create new libraries
+            if (createLibraries(cliInterface))
+              return -1;  // error, need to drop new libraies then undo rename
+        
+            mdui->setMsg("  End:   Create New Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+  
+            return 0;
+          }
+          break;
+
+        case 7:
+          {
+            mdui->setMsg("  Start: Copy Old Libraries Contents ");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 8:
+          {
+            // copy old contents into new 
+           
+            if (copyOldLibrariesToNew(cliInterface))
+              {
+                mdui->setMsg(" Copy Old Libraries failed ! Drop  and recreate the following :   ");
+                //return -1;  // error, need to drop new libraries then undo rename
+              }
+        
+            mdui->setMsg("  End:   Copy Old Libraries Contents ");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+         
+            return 0;
+          }
+          break;
+
+        case 9:
+          {
+            mdui->setMsg("Upgrade Libraries: Done except for cleaning up");
+            mdui->setSubstep(0);
+            mdui->setEndStep(TRUE);
+        
+            return 0;
+          }
+          break;
+
+        default:
+          return -1;
+        }
+    } // while
+
+  return 0;
+}
+
+short CmpSeabaseDDL::upgradeLibrariesComplete(ExeCliInterface * cliInterface,
+                                              CmpDDLwithStatusInfo *mdui)
+{
+  switch (mdui->subStep())
+    {
+    case 0:
+      {
+        mdui->setMsg("Upgrade Libraries: Drop old libraries");
+        mdui->subStep()++;
+        mdui->setEndStep(FALSE);
+        
+        return 0;
+      }
+      break;
+    case 1:
+      {
+        // drop old libraries; ignore errors
+        dropLibraries(cliInterface, TRUE/*old repos*/, FALSE/*no schema drop*/);
+        
+        mdui->setMsg("Upgrade Libraries: Drop Old Libraries done");
+        mdui->setEndStep(TRUE);
+        mdui->setSubstep(0);
+         
+        return 0;
+      }
+      break;
+
+    default:
+      return -1;
+    }
+
+return 0;
+}
+
+
+short CmpSeabaseDDL::upgradeLibrariesUndo(ExeCliInterface * cliInterface,
+                                  CmpDDLwithStatusInfo *mdui)
+{
+  Lng32 cliRC = 0;
+
+  while (1) // exit via return stmt in switch
+    {
+      switch (mdui->subStep())
+        {
+        // error return codes from upgradeLibraries can be mapped to
+        // the right recovery substep by this formula: substep = -(retcode + 1)
+        case 0: // corresponds to -1 return code from upgradeRepos (or
+                // to full recovery after some error after upgradeRepos)
+        case 1: // corresponds to -2 return code from upgradeRepos
+        case 2: // corresponds to -3 return code from upgradeRepos
+          {
+            mdui->setMsg("Upgrade Libraries: Restoring Old Libraries");
+            mdui->setSubstep(2*mdui->subStep()+3); // go to appropriate case
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 3:
+          {
+            mdui->setMsg(" Start: Drop New Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 4:
+          {
+            // drop new Libraries; ignore errors
+            dropLibraries(cliInterface, FALSE/*new repos*/, 
+                          TRUE /* don't drop new tables that haven't been upgraded */);
+            cliInterface->clearGlobalDiags();
+            mdui->setMsg(" End: Drop New Libraries");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 5:
+          {
+            mdui->setMsg(" Start: Rename Old Libraries back to New");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+ 
+        case 6:
+          {
+            // rename old Libraries to current; ignore errors
+            alterRenameLibraries(cliInterface, FALSE);
+            cliInterface->clearGlobalDiags();
+            mdui->setMsg(" End: Rename Old Libraries back to New");
+            mdui->subStep()++;
+            mdui->setEndStep(FALSE);
+        
+            return 0;
+          }
+          break;
+
+        case 7:
+          {
+            mdui->setMsg("Upgrade Libraries: Restore done");
+            mdui->setSubstep(0);
+            mdui->setEndStep(TRUE);
+        
+            return 0;
+          }
+          break;
+
+        default:
+          return -1;
+        }
+    } // while
+
+  return 0;
+
+}
+
+short CmpSeabaseDDL::createLibraries(ExeCliInterface * cliInterface)
+{
+ Lng32 cliRC = 0;
+
+  char queryBuf[20000];
+
+  NABoolean xnWasStartedHere = FALSE;
+
+
+  for (Int32 i = 0; i < sizeof(allLibrariesUpgradeInfo)/sizeof(MDUpgradeInfo); i++)
+    {
+      const MDUpgradeInfo &lti = allLibrariesUpgradeInfo[i];
+
+      if (! lti.newName)
+        continue;
+
+      for (Int32 j = 0; j < NUM_MAX_PARAMS; j++)
+	{
+	  param_[j] = NULL;
+	}
+
+      const QString * qs = NULL;
+      Int32 sizeOfqs = 0;
+
+      qs = lti.newDDL;
+      sizeOfqs = lti.sizeOfnewDDL; 
+
+      Int32 qryArraySize = sizeOfqs / sizeof(QString);
+      char * gluedQuery;
+      Lng32 gluedQuerySize;
+      glueQueryFragments(qryArraySize,  qs,
+			 gluedQuery, gluedQuerySize);
+
+ 
+      param_[0] = getSystemCatalog();
+      param_[1] = SEABASE_MD_SCHEMA;
+
+      str_sprintf(queryBuf, gluedQuery, param_[0], param_[1]);
+      NADELETEBASICARRAY(gluedQuery, STMTHEAP);
+
+      if (beginXnIfNotInProgress(cliInterface, xnWasStartedHere))
+        goto label_error;
+      
+      cliRC = cliInterface->executeImmediate(queryBuf);
+      if (cliRC == -1390)  // table already exists
+	{
+	  // ignore error.
+          cliRC = 0;
+	}
+      else if (cliRC < 0)
+	{
+	  cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+	}
+
+      if (endXnIfStartedHere(cliInterface, xnWasStartedHere, cliRC) < 0)
+        goto label_error;
+      
+    } // for
+  
+  
+  return 0;
+
+  label_error:
+   
+   return -1;
+}
+
+short CmpSeabaseDDL::dropLibraries(ExeCliInterface * cliInterface,
+                               NABoolean oldLibrary,
+                               NABoolean inRecovery)
+{
+ Lng32 cliRC = 0;
+  NABoolean xnWasStartedHere = FALSE;
+  char queryBuf[1000];
+
+  for (Int32 i = 0; i < sizeof(allLibrariesUpgradeInfo)/sizeof(MDUpgradeInfo); i++)
+    {
+      const MDUpgradeInfo &lti = allLibrariesUpgradeInfo[i];
+
+      // If we are dropping the new repository as part of a recovery action,
+      // and there is no "old" table (because the table didn't change in this
+      // upgrade), then don't drop the new table. (If we did, we would be 
+      // dropping the existing data.)
+      if (!oldLibrary && inRecovery && !lti.oldName)
+        continue;
+
+      if ((oldLibrary  && !lti.oldName) || (NOT oldLibrary && ! lti.newName))
+        continue;
+
+      str_sprintf(queryBuf, "drop table %s.\"%s\".%s cascade; ",
+                  getSystemCatalog(), SEABASE_MD_SCHEMA,
+                  (oldLibrary ? lti.oldName : lti.newName));
+    
+      if (beginXnIfNotInProgress(cliInterface, xnWasStartedHere))
+        {
+          cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+          return -1;
+        }    
+
+      cliRC = cliInterface->executeImmediate(queryBuf);
+      if (cliRC == -1389)  // table doesn't exist
+	{
+	  // ignore the error.
+          cliRC = 0;
+	}
+      else if (cliRC < 0)
+        {
+          cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+        }
+ 
+      if (endXnIfStartedHere(cliInterface, xnWasStartedHere, cliRC) < 0)
+        {
+          cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+          return -1;
+        }
+ 
+      if (cliRC < 0)
+        {
+          return -1;  
+        }
+
+    }
+
+
+  return 0;
+}
+
+short CmpSeabaseMDupgrade::dropLibrariesTables(ExpHbaseInterface *ehi,
+                                           NABoolean oldLibraries)
+{
+  Lng32 retcode = 0;
+  Lng32 errcode = 0;
+
+  for (Int32 i = 0; i < sizeof(allLibrariesUpgradeInfo)/sizeof(MDUpgradeInfo); i++)
+    {
+      const MDUpgradeInfo &lti = allLibrariesUpgradeInfo[i];
+
+      if ((NOT oldLibraries) && (!lti.newName))
+	continue;
+
+      HbaseStr hbaseTable;
+      NAString extNameForHbase = TRAFODION_SYSCAT_LIT;
+      extNameForHbase += ".";
+      extNameForHbase += SEABASE_MD_SCHEMA;
+      extNameForHbase +=  ".";
+
+      if (oldLibraries)
+	{
+          if (!lti.oldName)
+            continue;
+          
+          extNameForHbase += lti.oldName;
+	}
+      else
+	extNameForHbase += lti.newName;
+      
+      hbaseTable.val = (char*)extNameForHbase.data();
+      hbaseTable.len = extNameForHbase.length();
+      
+      retcode = dropHbaseTable(ehi, &hbaseTable, FALSE, FALSE);
+      if (retcode < 0)
+	{
+	  errcode = -1;
+	}
+      
+    } // for
+  
+  return errcode;
+}
+
+
+short CmpSeabaseDDL::alterRenameLibraries(ExeCliInterface * cliInterface,
+                                          NABoolean newToOld)
+{
+ Lng32 cliRC = 0;
+
+  char queryBuf[10000];
+
+  NABoolean xnWasStartedHere = FALSE;
+
+  // alter table rename cannot run inside of a transaction.
+  // return an error if a xn is in progress
+  if (xnInProgress(cliInterface))
+    {
+      *CmpCommon::diags() << DgSqlCode(-20123);
+      return -1;
+    }
+
+  for (Int32 i = 0; i < sizeof(allLibrariesUpgradeInfo)/sizeof(MDUpgradeInfo); i++)
+    {
+      const MDUpgradeInfo &lti = allLibrariesUpgradeInfo[i];
+
+      if ((! lti.newName) || (! lti.oldName) || (NOT lti.upgradeNeeded))
+        continue;
+
+      if (newToOld)
+        str_sprintf(queryBuf, "alter table %s.\"%s\".%s rename to %s ; ",
+                    getSystemCatalog(), SEABASE_MD_SCHEMA, lti.newName, lti.oldName);
+      else
+        str_sprintf(queryBuf, "alter table %s.\"%s\".%s rename to %s ; ",
+                    getSystemCatalog(), SEABASE_MD_SCHEMA, lti.oldName, lti.newName);
+        
+      cliRC = cliInterface->executeImmediate(queryBuf);
+      if (cliRC == -1389 || cliRC == -1390)
+        {
+          // ignore.
+          cliRC = 0;
+        }
+      else if (cliRC < 0)
+	{
+	  cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+	}
+    }
+
+  return 0;
+}
+
+short CmpSeabaseDDL::copyOldLibrariesToNew(ExeCliInterface * cliInterface)
+{
+  Lng32 cliRC = 0;
+  NAString failedLibraries;
+
+  char queryBuf[10000];
+  for (Int32 i = 0; i < sizeof(allLibrariesUpgradeInfo)/sizeof(MDUpgradeInfo); i++)
+    {
+      const MDUpgradeInfo lti = allLibrariesUpgradeInfo[i];
+
+      if ((! lti.newName) || (! lti.oldName) || (NOT lti.upgradeNeeded))
+        continue;
+      // Update all existing libraries  so the blob contains the library
+      char * sbuf = new(STMTHEAP) char[200];
+      char * ubuf = new(STMTHEAP) char[500];
+      NABoolean libsToUpgrade = TRUE;
+      Queue *userLibsQ = NULL;
+      str_sprintf(sbuf, "select library_filename,library_uid from %s.\"%s\".%s"
+                  " for read uncommitted access",
+                  getSystemCatalog(), SEABASE_MD_SCHEMA, lti.oldName); 
+      cliRC = cliInterface->fetchAllRows(userLibsQ,sbuf, 0, FALSE,FALSE,TRUE/*no exec*/);
+      if (cliRC < 0)
+        {
+          cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+          return -1;
+        }
+
+    
+     
+      str_sprintf(queryBuf, "upsert using load into %s.\"%s\".%s %s%s%s select %s from %s.\"%s\".%s SRC %s;",
+                  TRAFODION_SYSCAT_LIT,
+                  SEABASE_MD_SCHEMA,
+                  lti.newName, 
+                  (lti.insertedCols ? "(" : ""),
+                  (lti.insertedCols ? lti.selectedCols : ""), // insert only the original column values
+                  (lti.insertedCols ? ")" : ""),
+                  (lti.selectedCols ? lti.selectedCols : "*"),
+                  TRAFODION_SYSCAT_LIT,
+                  SEABASE_MD_SCHEMA,
+                  lti.oldName,
+                  (lti.wherePred ? lti.wherePred : ""));
+
+      cliRC = cliInterface->executeImmediate(queryBuf);
+      if (cliRC < 0)
+        {
+          cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+          return -1;
+        }
+
+      //update contents into  the new library table so the libname doesn't contain the path and the 
+      //lib contents get loaded into the blob column.
+      userLibsQ->position();
+      for (size_t i = 0; i < userLibsQ->numEntries(); i++)
+        {
+          OutputInfo *userLibRow = (OutputInfo *)userLibsQ->getNext();
+          char *libName = userLibRow->get(0);
+          NAString libFileName(libName);
+          Int64 libuid = *(Int64 *)userLibRow->get(1);
+
+          size_t lastSlash = libFileName.last('/');
+          NAString libNameNoPath;
+          if (lastSlash != NA_NPOS)
+            libNameNoPath = libFileName(lastSlash+1, libFileName.length()-lastSlash-1);
+          str_sprintf(ubuf," update %s.\"%s\".%s set library_filename = '%s', library_storage = filetolob('%s') where library_uid = %ld",
+                      getSystemCatalog(),SEABASE_MD_SCHEMA, lti.newName,
+                      libNameNoPath.data(),libName,libuid
+                      );
+          cliRC = cliInterface->executeImmediate(ubuf);
+          if (cliRC < 0)
+            {
+              if (failedLibraries.length() ==0)
+                failedLibraries += "Libraries Upgrade failed for :";
+              failedLibraries += libFileName;
+              failedLibraries += ";";
+              //cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
+              //return -1;
+            }
+        } //end for 
+    
+    }//end for
+
+  if (failedLibraries.length())
+    SQLMXLoggingArea::logSQLMXPredefinedEvent(failedLibraries, LL_WARN);
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafodion/blob/9528f8c0/core/sql/sqlcomp/CmpSeabaseDDLroutine.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLroutine.h b/core/sql/sqlcomp/CmpSeabaseDDLroutine.h
index 92adf18..e98c80b 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLroutine.h
+++ b/core/sql/sqlcomp/CmpSeabaseDDLroutine.h
@@ -23,6 +23,7 @@
 #define _CMP_SEABASE_PROCEDURES_H_
 
 #include "CmpSeabaseDDL.h"
+#include "CmpSeabaseDDLupgrade.h"
 
 // To add a new procedure:
 //   update export/lib/lib_mgmt.jar to include code for the new procedure 
@@ -407,4 +408,80 @@ static const LibmgrRoutineInfo allLibmgrRoutineInfo[] = {
 
 };
 
+/////////////////////////////////////////////////////////////////////
+//
+// Information about changed old metadata tables from which upgrade
+// is being done to the current version.
+// These definitions have changed in the current version of code.
+// 
+// Old definitions have the form (for ex for METRIC_QUERY_TABLE table):
+//            createOldTrafv??MetricQueryTable[]
+// v?? is the old version.
+//
+// When definitions change, make new entries between
+// START_OLD_MD_v?? and END_OLD_MD_v??.
+// Do not remove older entries. We want to keep them around for
+// historical purpose.
+//
+// Change entries in allReposUpgradeInfo[] struct in this file
+// to reflect the 'old' repository tables.
+//
+//////////////////////////////////////////////////////////////////////
+//----------------------------------------------------------------
+//-- LIBRARIES
+//----------------------------------------------------------------
+static const QString createLibrariesTable[] =
+  {
+    {" create table %s.\"%s\"." SEABASE_LIBRARIES" "},
+    {" ( "},
+    {"  library_uid largeint not null not serialized, "},
+    {"  library_filename varchar(512) character set iso88591 not null not serialized, "},
+    {"  library_storage  blob,"},
+    {"  version int not null not serialized, "},
+    {"  flags largeint not null not serialized "},
+    {" ) "},
+    {" primary key (library_uid) "},
+    {" attribute hbase format "},
+    {" ; "}
+  };
+
+#define SEABASE_LIBRARIES_OLD SEABASE_LIBRARIES"_OLD"
+static const QString createOldTrafv210LibrariesTable[] =
+{
+{" create table %s.\"%s\"." SEABASE_LIBRARIES" "},
+    {" ( "},
+    {"  library_uid largeint not null not serialized, "},
+    {"  library_filename varchar(512) character set iso88591 not null not serialized, "},
+    {"  version int not null not serialized, "},
+    {"  flags largeint not null not serialized "},
+    {" ) "},
+    {" primary key (library_uid) "},
+    {" attribute hbase format "},
+    {" ; "}
+  };
+
+static const MDUpgradeInfo allLibrariesUpgradeInfo[] = {
+  // LIBRARIES
+  {
+    SEABASE_LIBRARIES,  SEABASE_LIBRARIES_OLD,
+    createLibrariesTable,  sizeof(createLibrariesTable),
+    createOldTrafv210LibrariesTable,  sizeof(createOldTrafv210LibrariesTable),
+    NULL, 0,
+    TRUE,
+    //new table columns
+    "library_uid,"
+    "library_filename,"
+    "library_storage,"
+    "version,"
+    "flags",
+    //old table columns
+    "library_uid,"
+    "library_filename,"
+    "version,"
+    "flags",
+    NULL, TRUE, FALSE, FALSE, FALSE, FALSE, FALSE}
+};
+
+
+  
 #endif

http://git-wip-us.apache.org/repos/asf/trafodion/blob/9528f8c0/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp b/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
index a620624..3569b90 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
@@ -5475,7 +5475,7 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
       if (pColDef->getDefaultClauseStatus() != ElemDDLColDef::DEFAULT_CLAUSE_SPEC)
         {
           *CmpCommon::diags() << DgSqlCode(-CAT_DEFAULT_REQUIRED);
-
+          deallocEHI(ehi);
           processReturn();
 
           return;
@@ -5492,7 +5492,7 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
           if (pDefVal->isNull()) 
             {
               *CmpCommon::diags() << DgSqlCode(-CAT_CANNOT_BE_DEFAULT_NULL_AND_NOT_NULL);
-
+              deallocEHI(ehi);
               processReturn();
 
               return;
@@ -5504,7 +5504,7 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
   if (pColDef->getDefaultClauseStatus() == ElemDDLColDef::NO_DEFAULT_CLAUSE_SPEC)
     {
       *CmpCommon::diags() << DgSqlCode(-CAT_DEFAULT_REQUIRED);
-
+      deallocEHI(ehi);
       processReturn();
 
       return;
@@ -5513,7 +5513,7 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
   if (pColDef->getSGOptions())
     {
       *CmpCommon::diags() << DgSqlCode(-1514);
-
+      deallocEHI(ehi);
       processReturn();
 
       return;
@@ -5606,11 +5606,12 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
           *CmpCommon::diags() << DgSqlCode(-CAT_DUPLICATE_COLUMNS)
                               << DgColumnName(colName);
         }
-      
+      deallocEHI(ehi);
       processReturn();
 
       return;
     }
+  /*
   // If column is a LOB column , error
    if ((datatype == REC_BLOB) || (datatype == REC_CLOB))
      {
@@ -5619,8 +5620,80 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
       processReturn();
       return;
      }
-  char * col_name = new(STMTHEAP) char[colName.length() + 1];
+  */
+  char *col_name = new(STMTHEAP) char[colName.length() + 1];
   strcpy(col_name, (char*)colName.data());
+ if ((datatype == REC_BLOB) ||
+          (datatype == REC_CLOB))
+   {
+ 
+     short *lobNumList = new (STMTHEAP) short;
+     short *lobTypList = new (STMTHEAP) short;
+     char  **lobLocList = new (STMTHEAP) char*[1];
+     char **lobColNameList = new (STMTHEAP) char*[1];
+    
+     Int64 lobMaxSize =  CmpCommon::getDefaultNumeric(LOB_MAX_SIZE)*1024*1024;
+    
+    
+
+     lobNumList[0] = nacolArr.entries();
+     lobTypList[0] = (short)(pColDef->getLobStorage());
+     char * loc = new (STMTHEAP) char[1024];
+	  
+     const char* f = ActiveSchemaDB()->getDefaults().
+       getValue(LOB_STORAGE_FILE_DIR);
+	  
+     strcpy(loc, f);
+	  
+     lobLocList[0] = loc;
+     lobColNameList[0] = col_name;
+     char  lobHdfsServer[256] ; // max length determined by dfs.namenode.fs-limits.max-component-length(255)
+     memset(lobHdfsServer,0,256);
+     strncpy(lobHdfsServer,CmpCommon::getDefaultString(LOB_HDFS_SERVER),sizeof(lobHdfsServer)-1);
+     Int32 lobHdfsPort = (Lng32)CmpCommon::getDefaultNumeric(LOB_HDFS_PORT);
+      Int32 rc = sendAllControls(FALSE, FALSE, TRUE);
+      
+      Int64 objUID = getObjectUID(&cliInterface,
+                                  catalogNamePart.data(), schemaNamePart.data(), 
+                                  objectNamePart.data(),
+                                  COM_BASE_TABLE_OBJECT_LIT);
+      
+      ComString newSchName = "\"";
+      newSchName += catalogNamePart;
+      newSchName.append("\".\"");
+      newSchName.append(schemaNamePart);
+      newSchName += "\"";
+      NABoolean lobTrace=FALSE;
+      if (getenv("TRACE_LOB_ACTIONS"))
+        lobTrace=TRUE;
+      Int32 numLobs = 1;
+      
+      rc = SQL_EXEC_LOBddlInterface((char*)newSchName.data(),
+                                          newSchName.length(),
+                                          objUID,
+                                          numLobs,
+                                          LOB_CLI_CREATE,
+                                          lobNumList,
+                                          lobTypList,
+                                          lobLocList,
+                                          lobColNameList,
+                                          lobHdfsServer,
+                                          lobHdfsPort,
+                                          lobMaxSize,
+                                          lobTrace);
+      if (rc < 0)
+        {
+          // retrieve the cli diags here.
+          CmpCommon::diags()->mergeAfter(*(GetCliGlobals()->currContext()->getDiagsArea()));
+         
+          deallocEHI(ehi); 	   
+          processReturn();
+	   
+          return ;
+        }
+   }
+
+
 
   ULng32 maxColQual = nacolArr.getMaxTrafHbaseColQualifier();
 
@@ -13758,7 +13831,8 @@ TrafDesc *CmpSeabaseDDL::getSeabaseRoutineDesc(const NAString &catName,
                                       const NAString &objName)
 {
    TrafDesc *result = NULL;
-
+   NABoolean useLibBlobStore = FALSE;
+   
    if (switchCompiler(CmpContextInfo::CMPCONTEXT_TYPE_META))
      return NULL;
 
@@ -13771,12 +13845,13 @@ TrafDesc *CmpSeabaseDDL::getSeabaseRoutineDesc(const NAString &catName,
 
 
 TrafDesc *CmpSeabaseDDL::getSeabaseRoutineDescInternal(const NAString &catName,
-                                      const NAString &schName,
-                                      const NAString &objName)
+                                                       const NAString &schName,
+                                                       const NAString &objName
+                                                       )
 {
   Lng32 retcode = 0;
   Lng32 cliRC = 0;
-
+  
   TrafDesc *result;
   char query[4000];
   char buf[4000];
@@ -13803,20 +13878,23 @@ TrafDesc *CmpSeabaseDDL::getSeabaseRoutineDescInternal(const NAString &catName,
       return NULL;
     }
 
+ 
+    
+    
   str_sprintf(buf, "select udr_type, language_type, deterministic_bool,"
-  " sql_access, call_on_null, isolate_bool, param_style,"
-  " transaction_attributes, max_results, state_area_size, external_name,"
-  " parallelism, user_version, external_security, execution_mode,"
-  " library_filename, version, signature,  catalog_name, schema_name,"
-  " object_name"
-  " from %s.\"%s\".%s r, %s.\"%s\".%s l, %s.\"%s\".%s o "
-  " where r.udr_uid = %ld and r.library_uid = l.library_uid "
-  " and l.library_uid = o.object_uid for read committed access",
-       getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_ROUTINES,
-       getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES,
-       getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_OBJECTS,
-       objectUID);
-
+                  " sql_access, call_on_null, isolate_bool, param_style,"
+                  " transaction_attributes, max_results, state_area_size, external_name,"
+                  " parallelism, user_version, external_security, execution_mode,"
+                  " library_filename, version, signature,  catalog_name, schema_name,"
+                  " object_name, redef_time,library_storage, l.library_uid"
+                  " from %s.\"%s\".%s r, %s.\"%s\".%s l, %s.\"%s\".%s o "
+                  " where r.udr_uid = %ld and r.library_uid = l.library_uid "
+                  " and l.library_uid = o.object_uid for read committed access",
+                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_ROUTINES,
+                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_LIBRARIES,
+                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_OBJECTS,
+                  objectUID);
+    
 
   cliRC = cliInterface.fetchRowsPrologue(buf, TRUE/*no exec*/);
   if (cliRC < 0)
@@ -13912,6 +13990,8 @@ TrafDesc *CmpSeabaseDDL::getSeabaseRoutineDescInternal(const NAString &catName,
   cliInterface.getPtrAndLen(20, ptr, len);
   char *libSch = new (STMTHEAP) char[len+1];    
   str_cpy_and_null(libSch, ptr, len, '\0', ' ', TRUE);
+  routineInfo->lib_sch_name = new (STMTHEAP) char[len+1];
+  str_cpy_and_null(routineInfo->lib_sch_name, ptr, len, '\0', ' ', TRUE);
   cliInterface.getPtrAndLen(21, ptr, len);
   char *libObj = new (STMTHEAP) char[len+1];    
   str_cpy_and_null(libObj, ptr, len, '\0', ' ', TRUE);
@@ -13925,6 +14005,29 @@ TrafDesc *CmpSeabaseDDL::getSeabaseRoutineDescInternal(const NAString &catName,
                    libSQLExtName.data(),
                    libSQLExtName.length(),
                    '\0', ' ', TRUE);
+  NAString naLibSch(libSch);
+ 
+  
+  
+  cliInterface.getPtrAndLen(22,ptr,len);
+  routineInfo->lib_redef_time = *(Int64 *)ptr;
+
+
+  cliInterface.getPtrAndLen(23, ptr, len);
+  routineInfo->lib_blob_handle = new (STMTHEAP) char[len+1];    
+  str_cpy_and_null(routineInfo->lib_blob_handle, ptr, len, '\0', ' ', TRUE);
+
+  cliInterface.getPtrAndLen(24,ptr,len);
+  routineInfo->lib_obj_uid= *(Int64 *)ptr;
+  
+  
+    
+  if ((routineInfo->lib_blob_handle[0] == '\0')|| (naLibSch  == NAString(SEABASE_MD_SCHEMA)))
+    {
+      routineInfo->lib_redef_time = -1;
+      routineInfo->lib_blob_handle=NULL;
+      routineInfo->lib_obj_uid = 0;
+    }  
   
   ComTdbVirtTableColumnInfo *paramsArray;
   Lng32 numParams;