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 2016/03/01 07:49:23 UTC

[1/3] incubator-trafodion git commit: JIRA 1844: support for 'alter table alter column datatype'

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master fabfc80d7 -> b2fe5fd7a


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
index 8f53cc9..2d97501 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp
@@ -2799,6 +2799,67 @@ short CmpSeabaseDDL::getTypeInfo(const NAType * naType,
   return 0;
 }
 
+short CmpSeabaseDDL::getNAColumnFromColDef
+(ElemDDLColDef * colNode,
+ NAColumn* &naCol)
+{
+  NAString colFamily;
+  NAString colName;
+  Lng32 datatype, length, precision, scale, dt_start, dt_end;
+  Lng32 nullable, upshifted;
+  ComColumnClass colClass;
+  ComColumnDefaultClass defaultClass;
+  NAString charset, defVal;
+  NAString heading;
+  ULng32 hbaseColFlags;
+  Int64 colFlags;
+  LobsStorage lobStorage;
+  NABoolean alignedFormat = FALSE;
+  if (getColInfo(colNode,
+                 colFamily,
+                 colName,
+                 alignedFormat,
+                 datatype, length, precision, scale, dt_start, dt_end, 
+                 upshifted, nullable,
+                 charset, colClass, defaultClass, defVal, heading, lobStorage, 
+                 hbaseColFlags, colFlags))
+    {
+     *CmpCommon::diags() << DgSqlCode(-2004);
+     return -1;
+    }
+
+  NAType * naType = colNode->getColumnDataType();
+  if (! naType)
+    {
+      *CmpCommon::diags() << DgSqlCode(-2004);
+      return -1;
+    }
+
+  char * defV = NULL;
+  if ((defaultClass != COM_NO_DEFAULT) &&
+      (! defVal.isNull()))
+    {
+      char * data = (char*) defVal.data();
+      Lng32 len = defVal.length();
+      defV = new(STMTHEAP) char[len + 2];
+      str_cpy_all((char*)defV, data, len);
+      char * c = (char*)defV;
+      c[len] = 0;
+      c[len+1] = 0;
+    }
+
+  naCol = new(STMTHEAP) NAColumn(colNode->getColumnName().data(),
+                                 -1, // position
+                                 naType, NULL, NULL,
+                                 USER_COLUMN, //colClass, 
+                                 defaultClass,
+                                 defV);
+
+  naCol->setHbaseColFlags(hbaseColFlags);
+
+  return 0;
+}
+
 short CmpSeabaseDDL::getColInfo(ElemDDLColDef * colNode, 
                                 NAString &colFamily,
                                 NAString &colName,
@@ -7801,6 +7862,95 @@ void CmpSeabaseDDL::updateVersion()
 
 }
 
+// this method truncates an hbase table by dropping it and then recreating
+// it. Options that were used for the original hbase table are stored in
+// traf metadata and are passed in to hbase during table create.
+// When hbase truncate api is available (HBAse 1.0 and later), then this
+// method will call it instead of drop/recreate.
+short CmpSeabaseDDL::truncateHbaseTable(const NAString &catalogNamePart, 
+                                        const NAString &schemaNamePart, 
+                                        const NAString &objectNamePart,
+                                        NATable * naTable,
+                                        ExpHbaseInterface * ehi)
+{
+  Lng32 retcode = 0;
+
+  const NAString extNameForHbase = 
+    catalogNamePart + "." + schemaNamePart + "." + objectNamePart;
+
+  HbaseStr hbaseTable;
+  hbaseTable.val = (char*)extNameForHbase.data();
+  hbaseTable.len = extNameForHbase.length();
+
+  // drop this table from hbase
+  retcode = dropHbaseTable(ehi, &hbaseTable, FALSE);
+  if (retcode)
+    {
+      deallocEHI(ehi); 
+      
+      processReturn();
+      
+      return -1;
+    }
+
+  // and recreate it.
+  NAFileSet * naf = naTable->getClusteringIndex();
+
+  NAList<HbaseCreateOption*> * hbaseCreateOptions = 
+    naTable->hbaseCreateOptions();
+  Lng32 numSaltPartns = naf->numSaltPartns();
+  Lng32 numSaltSplits = numSaltPartns - 1;
+  Lng32 numSplits = 0;
+  const Lng32 numKeys = naf->getIndexKeyColumns().entries();
+  Lng32 keyLength = naf->getKeyLength();
+  char ** encodedKeysBuffer = NULL;
+
+  const desc_struct * tableDesc = naTable->getTableDesc();
+  desc_struct * colDescs = tableDesc->body.table_desc.columns_desc; 
+  desc_struct * keyDescs = (desc_struct*)naf->getKeysDesc();
+
+  if (createEncodedKeysBuffer(encodedKeysBuffer/*out*/,
+                              numSplits/*out*/,
+                              colDescs, keyDescs,
+                              numSaltPartns,
+                              numSaltSplits,
+                              NULL,
+                              numKeys, 
+                              keyLength,
+                              FALSE))
+    {
+      deallocEHI(ehi); 
+
+      processReturn();
+      
+      return -1;
+    }
+  
+  std::vector<NAString> userColFamVec;
+  std::vector<NAString> trafColFamVec;
+  NAString outColFam;
+  for (int i = 0; i < naTable->allColFams().entries(); i++)
+    {
+      processColFamily(naTable->allColFams()[i], outColFam,
+                       &userColFamVec, &trafColFamVec);
+    } // for
+  
+  retcode = createHbaseTable(ehi, &hbaseTable, trafColFamVec,
+                             hbaseCreateOptions,
+                             numSplits, keyLength, 
+                             encodedKeysBuffer);
+  if (retcode == -1)
+    {
+      deallocEHI(ehi); 
+
+      processReturn();
+
+      return -1;
+    }
+
+  return 0;
+}
+
 void CmpSeabaseDDL::purgedataHbaseTable(DDLExpr * ddlExpr,
                                        NAString &currCatName, NAString &currSchName)
 {
@@ -7966,14 +8116,9 @@ void CmpSeabaseDDL::purgedataHbaseTable(DDLExpr * ddlExpr,
     }
                                  
   NABoolean asyncDrop = (CmpCommon::getDefault(HBASE_ASYNC_DROP_TABLE) == DF_ON);
-
-  HbaseStr hbaseTable;
-  hbaseTable.val = (char*)extNameForHbase.data();
-  hbaseTable.len = extNameForHbase.length();
-
-  // drop this table from hbase
-  retcode = dropHbaseTable(ehi, &hbaseTable, FALSE);
-  if (retcode)
+  
+  if (truncateHbaseTable(catalogNamePart, schemaNamePart, objectNamePart,
+                         naTable, ehi))
     {
       deallocEHI(ehi); 
       
@@ -7982,60 +8127,6 @@ void CmpSeabaseDDL::purgedataHbaseTable(DDLExpr * ddlExpr,
       return;
     }
 
-  // and recreate it.
-  NAFileSet * naf = naTable->getClusteringIndex();
-
-  NAList<HbaseCreateOption*> * hbaseCreateOptions = naTable->hbaseCreateOptions();
-  Lng32 numSaltPartns = naf->numSaltPartns();
-  Lng32 numSaltSplits = numSaltPartns - 1;
-  Lng32 numSplits = 0;
-  Lng32 numKeys = naf->getIndexKeyColumns().entries();
-  Lng32 keyLength = naf->getKeyLength();
-  char ** encodedKeysBuffer = NULL;
-
-  const desc_struct * tableDesc = naTable->getTableDesc();
-  desc_struct * colDescs = tableDesc->body.table_desc.columns_desc; 
-  desc_struct * keyDescs = (desc_struct*)naf->getKeysDesc();
-
-  if (createEncodedKeysBuffer(encodedKeysBuffer/*out*/,
-                              numSplits/*out*/,
-                              colDescs, keyDescs,
-                              numSaltPartns,
-                              numSaltSplits,
-                              NULL,
-                              numKeys, 
-                              keyLength,
-                              FALSE))
-    {
-      deallocEHI(ehi); 
-
-      processReturn();
-      
-      return;
-    }
-  
-  std::vector<NAString> userColFamVec;
-  std::vector<NAString> trafColFamVec;
-  NAString outColFam;
-  for (int i = 0; i < naTable->allColFams().entries(); i++)
-    {
-      processColFamily(naTable->allColFams()[i], outColFam,
-                       &userColFamVec, &trafColFamVec);
-    } // for
-  
-  retcode = createHbaseTable(ehi, &hbaseTable, trafColFamVec,
-                             hbaseCreateOptions,
-                             numSplits, keyLength, 
-                             encodedKeysBuffer);
-  if (retcode == -1)
-    {
-      deallocEHI(ehi); 
-
-      processReturn();
-
-      return;
-    }
-
   if (naTable->hasSecondaryIndexes()) // user indexes
     {
       const NAFileSetList &indexList = naTable->getIndexList();
@@ -8190,6 +8281,8 @@ short CmpSeabaseDDL::executeSeabaseDDL(DDLExpr * ddlExpr, ExprNode * ddlNode,
         (ddlNode->getOperatorType() == DDL_POPULATE_INDEX) ||
         (ddlNode->getOperatorType() == DDL_CREATE_TABLE) ||
         (ddlNode->getOperatorType() == DDL_ALTER_TABLE_DROP_COLUMN) ||
+        (ddlNode->getOperatorType() == DDL_ALTER_TABLE_ALTER_COLUMN_DATATYPE) ||
+
         (ddlNode->getOperatorType() == DDL_DROP_TABLE))))
     {
       // transaction will be started and commited in called methods.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp b/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
index 0e0bf64..5b3c8c3 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLschema.cpp
@@ -100,6 +100,8 @@ static bool transferObjectPrivs(
 // *                                                                           *
 // *   0: Schema was added                                                      *
 // *  -1: Schema was not added.  A CLI error is put into the diags area.       *
+// *   1: Schema already exists and ignoreIfExists is specified.               *
+// *      No error is added to the diags area.                                 *
 // *                                                                           *
 // *****************************************************************************
 int CmpSeabaseDDL::addSchemaObject(
@@ -136,7 +138,7 @@ Lng32 retcode = existsInSeabaseMDTable(&cliInterface,catalogName,schemaNamePart,
    if (retcode == 1 ) // already exists
    {
       if (ignoreIfExists)
-        return 0;
+        return 1;
       else
         *CmpCommon::diags() << DgSqlCode(-CAT_SCHEMA_ALREADY_EXISTS)
                             << DgSchemaName(schemaName.getExternalName().data());

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp b/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
index 7a0498d..9ebf15e 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
+++ b/core/sql/sqlcomp/CmpSeabaseDDLtable.cpp
@@ -66,11 +66,34 @@ extern short CmpDescribeSeabaseTable (
                              NABoolean withoutDivisioning = FALSE,
                              NABoolean noTrailingSemi = FALSE,
 
-                             // used to add or remove column definition from col list.
-                             // valid for 'createLike' mode. Used for 'alter add/drop col'.
+                             // used to add,rem,alter column definition from col list.
+                             // valid for 'createLike' mode. 
+                             // Used for 'alter add/drop/alter col'.
                              char * colName = NULL,
-                             NABoolean isAdd = FALSE,
-                             const NAColumn * nacol = NULL);
+                             short ada = 0, // 0,add. 1,drop. 2,alter
+                             const NAColumn * nacol = NULL,
+                             const NAType * natype = NULL);
+
+// type:  1, invoke. 2, showddl. 3, create_like
+extern short cmpDisplayColumn(const NAColumn *nac,
+                              char * inColName,
+                              const NAType *inNAT,
+                              short displayType,
+                              Space *inSpace,
+                              char * buf,
+                              Lng32 &ii,
+                              NABoolean namesOnly,
+                              NABoolean &identityCol,
+                              NABoolean isExternalTable,
+                              NABoolean isAlignedRowFormat);
+
+extern short cmpDisplayPrimaryKey(const NAColumnArray & naColArr,
+                                  Lng32 numKeys,
+                                  NABoolean displaySystemCols,
+                                  Space &space, char * buf, 
+                                  NABoolean displayCompact,
+                                  NABoolean displayAscDesc,
+                                  NABoolean displayParens);
                              
 static bool checkSpecifiedPrivs(
    ElemDDLPrivActArray & privActsArray,  
@@ -232,8 +255,6 @@ desc_struct * CmpSeabaseDDL::convertVirtTableKeyInfoArrayToDescStructs(
   return first_key_desc;
 }
 
-
-
 void CmpSeabaseDDL::createSeabaseTableLike(
                                            StmtDDLCreateTable * createTableNode,
                                            NAString &currCatName, NAString &currSchName)
@@ -286,16 +307,19 @@ void CmpSeabaseDDL::createSeabaseTableLike(
         }
 
       keyClause += ")";
+    }
+
+  // Check for other common options that are currently not supported
+  // with CREATE TABLE LIKE. Those could all be passed into
+  // CmpDescribeSeabaseTable as strings if we wanted to support them.
 
-      // NOTE: This is not currently supported
+  if (NOT keyClause.isNull())
+    {
       *CmpCommon::diags() << DgSqlCode(-3111)
                           << DgString0("PRIMARY KEY/STORE BY");
       return;
     }
 
-  // Check for other common options that are currently not supported
-  // with CREATE TABLE LIKE. Those could all be passed into
-  // CmpDescribeSeabaseTable as strings if we wanted to support them.
   if (createTableNode->isPartitionSpecified() ||
       createTableNode->isPartitionBySpecified())
     {
@@ -4053,154 +4077,26 @@ void CmpSeabaseDDL::alterSeabaseTableHBaseOptions(
   return;
 }
 
-/////////////////////////////////////////////////////////////////////
-// currTab:          table on which column is being added to or dropped from
-// newTempTab:  temporary table with new definition
-// currTempTab:  name of table that currTab that will be renamed to 
-//
-// Steps:
-//   create newTempTab based on currTab and added/dropped column
-//   insert data into newTempTab from currTab
-//   rename currTab to currTempTab
-//   rename newTempTab to currTab
-//   drop currTempTab
-//
-/////////////////////////////////////////////////////////////////////
-short CmpSeabaseDDL::alignedFormatTableAddDropColumn
-(
- Int64 objUID,
- NABoolean isAdd,
- const NAString &catalogNamePart,
- const NAString &schemaNamePart,
- const NAString &objectNamePart,
- char * colName, const NAColumn * nacol)
+short CmpSeabaseDDL::createSeabaseTableLike2(
+     CorrName &cn,
+     const NAString &likeTableName,
+     NABoolean withPartns,
+     NABoolean withoutSalt,
+     NABoolean withoutDivision)
 {
-  Lng32 cliRC = 0;
   Lng32 retcode = 0;
 
-  ExeCliInterface cliInterface(STMTHEAP, NULL, NULL, 
-                               CmpCommon::context()->sqlSession()->getParentQid());
-
-  NABoolean xnWasStartedHere = FALSE;
-  Queue * usingViewsQueue = NULL;
-  cliRC = getUsingViews(&cliInterface, objUID, usingViewsQueue);
-  if (cliRC < 0)
-    {
-      processReturn();
-      
-      return -1;
-    }
-
-  if (usingViewsQueue->numEntries() > 0)
-    {
-      if (beginXnIfNotInProgress(&cliInterface, xnWasStartedHere))
-        return -1;
-    }
-
-  NAList<NAString> viewNameList(STMTHEAP, usingViewsQueue->numEntries());
-  NAList<NAString> viewDefnList(STMTHEAP, usingViewsQueue->numEntries());
-
-  // create temp table based on the current table DDL and added/dropped column.
-  // add/drop col name is colName.
-  // Added col defn is contained in nacol.
-  NAString newTempTab; 
-  ComDeriveRandomInternalName ( ComGetNameInterfaceCharSet(),
-                                objectNamePart, newTempTab, STMTHEAP);
-
-  char newTempTabStr[1000];
-  str_sprintf(newTempTabStr, "%s.\"%s\".%s", 
-              catalogNamePart.data(), schemaNamePart.data(), newTempTab.data());
-
-  CorrName newTempTabCN(newTempTab,
-                        STMTHEAP, schemaNamePart, catalogNamePart);
-
-  // current table and curr temp table
-  char currTabStr[1000];
-  str_sprintf(currTabStr, "%s.\"%s\".%s", 
-              catalogNamePart.data(), schemaNamePart.data(), objectNamePart.data());
-
-  CorrName currTabCN(objectNamePart, STMTHEAP, 
-                     schemaNamePart, catalogNamePart);
-
-  NAString currTempTab; 
-  ComDeriveRandomInternalName ( ComGetNameInterfaceCharSet(),
-                                objectNamePart, currTempTab, STMTHEAP);
-  
-  char currTempTabStr[1000];
-  str_sprintf(currTempTabStr, "%s.\"%s\".%s", 
-              catalogNamePart.data(), schemaNamePart.data(), currTempTab.data());
-
-
-  // create DDL for newTempTab  
   char * buf = NULL;
   ULng32 buflen = 0;
-  retcode = CmpDescribeSeabaseTable(currTabCN, 3/*createlike*/, buf, buflen, 
-                                    STMTHEAP,
+  retcode = CmpDescribeSeabaseTable(cn, 3/*createlike*/, buf, buflen, STMTHEAP,
                                     NULL,
-                                    FALSE, FALSE, FALSE,
-                                    TRUE,
-                                    colName, isAdd, nacol);
+                                    withPartns, withoutSalt, withoutDivision,
+                                    TRUE);
   if (retcode)
     return -1;
 
-  // find out any views on this table.
-  // save their definition and drop them.
-  // they will be recreated before return.
-  usingViewsQueue->position();
-  for (int idx = 0; idx < usingViewsQueue->numEntries(); idx++)
-    {
-      OutputInfo * vi = (OutputInfo*)usingViewsQueue->getNext(); 
-      char * viewName = vi->get(0);
-      
-      viewNameList.insert(viewName);
-
-      ComObjectName viewCO(viewName, COM_TABLE_NAME);
-
-      const NAString catName = viewCO.getCatalogNamePartAsAnsiString();
-      const NAString schName = viewCO.getSchemaNamePartAsAnsiString(TRUE);
-      const NAString objName = viewCO.getObjectNamePartAsAnsiString(TRUE);
-
-      Int64 viewUID = getObjectUID(&cliInterface,
-                                   catName.data(), schName.data(), objName.data(), 
-                                   COM_VIEW_OBJECT_LIT);
-      if (viewUID < 0 )
-        {
-          endXnIfStartedHere(&cliInterface, xnWasStartedHere, -1);
-          
-          return -1;
-        }
-
-      NAString viewText;
-      if (getTextFromMD(&cliInterface, viewUID, COM_VIEW_TEXT, 0, viewText))
-        {
-          endXnIfStartedHere(&cliInterface, xnWasStartedHere, -1);
-          
-          return -1;
-        }
-
-      viewDefnList.insert(viewText);
-
-      if (dropOneTableorView(cliInterface,viewName,COM_VIEW_OBJECT,false))
-        {
-          endXnIfStartedHere(&cliInterface, xnWasStartedHere, -1);
-  
-          processReturn();
-          
-          return -1;
-        }
-    }
-
-  endXnIfStartedHere(&cliInterface, xnWasStartedHere, 0);
-  
-  BindWA bindWA(ActiveSchemaDB(), CmpCommon::context(), FALSE);
-  NATable * naTable = NULL;
-
-  NAString colNames;
-
-  char queryBuf[1000];
-
   NAString query = "create table ";
-  query += newTempTabStr;
+  query += likeTableName;
   query += " ";
 
   NABoolean done = FALSE;
@@ -4218,157 +4114,108 @@ short CmpSeabaseDDL::alignedFormatTableAddDropColumn
         done = TRUE;
     }
 
-  cliRC = cliInterface.executeImmediate((char*)query.data());
-  if (cliRC < 0)
-    {
-      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-      goto label_error;
-    }
-
-  //  naTable = bindWA.getNATable((isAdd ? currTabCN : newTempTabCN));
-  naTable = bindWA.getNATable(newTempTabCN);
-  if (! naTable)
-    {
-      goto label_error;
-    }
+  query += ";";
 
-  // update metadata to change column type to 'A'(added)
-  if (isAdd)
-    {
-      str_sprintf(queryBuf, "update %s.\"%s\".%s set column_class = 'A' where object_uid = %Ld and column_name = '%s' ",
-                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
-                  naTable->objectUid().castToInt64(), colName);
-      
-      cliRC = cliInterface.executeImmediate(queryBuf);
-      if (cliRC < 0)
-        {
-          cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-          goto label_error;
-        }
-      
-    }
+  // send any user CQDs down 
+  Lng32 retCode = sendAllControls(FALSE, FALSE, TRUE);
 
-  // insert data from current table into new temp table.
-  //  query = "upsert using load into ";
-  query = "insert into ";
-  query += newTempTabStr;
-  query += " ";
+  ExeCliInterface cliInterface(STMTHEAP, NULL, NULL, 
+  CmpCommon::context()->sqlSession()->getParentQid());
 
-  for (Lng32 i = 0; i < naTable->getNAColumnArray().entries(); i++)
+  Lng32 cliRC = 0;
+  cliRC = cliInterface.executeImmediate((char*)query.data());
+  if (cliRC < 0)
     {
-      const NAColumn *nac = naTable->getNAColumnArray()[i];
-
-      if (nac->isSystemColumn())
-        continue;
- 
-      if ((isAdd) && (colName == nac->getColName()))
-        continue;
-
-      colNames += nac->getColName();
-
-      colNames += ",";
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
     }
 
-  // remove last comma
-  colNames = colNames.strip(NAString::trailing, ',');
-
-  query += "(" + colNames + ")";
+  return 0;
+}
 
-  query += " select ";
-  query += colNames;
+short CmpSeabaseDDL::cloneHbaseTable(
+     const NAString &srcTable, const NAString &clonedTable,
+     ExpHbaseInterface * inEHI)
+{
+  HbaseStr hbaseTable;
+  hbaseTable.val = (char*)srcTable.data();
+  hbaseTable.len = srcTable.length();
 
-  query += " from ";
-  query += currTabStr;
+  HbaseStr clonedHbaseTable;
+  clonedHbaseTable.val = (char*)clonedTable.data();
+  clonedHbaseTable.len = clonedTable.length();
 
-  query += ";";
+  ExpHbaseInterface * ehi = (inEHI ? inEHI : allocEHI());
 
-  cliRC = cliInterface.executeImmediate(query.data());
-  if (cliRC < 0)
+  // copy hbaseTable as clonedHbaseTable
+  if (ehi->copy(hbaseTable, clonedHbaseTable, TRUE))
     {
-      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      if (! inEHI)
+        deallocEHI(ehi); 
       
       processReturn();
-
-      goto label_error;
-    }
-  
-  // rename current table to temp
-  str_sprintf(queryBuf, "alter table \"%s\".\"%s\".\"%s\" rename to \"%s\" ",
-              catalogNamePart.data(), schemaNamePart.data(), objectNamePart.data(),
-              currTempTab.data());
-  
-  cliRC = cliInterface.executeImmediate(queryBuf);
-  if (cliRC < 0)
-    {
-      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
       
-      processReturn();
-
-      goto label_error;
+      return -1;
     }
 
-  // rename new temp table to current table
-  str_sprintf(queryBuf, "alter table \"%s\".\"%s\".\"%s\" rename to \"%s\" ",
-              catalogNamePart.data(), schemaNamePart.data(), newTempTab.data(),
-              objectNamePart.data());
-  
-  cliRC = cliInterface.executeImmediate(queryBuf);
-  if (cliRC < 0)
-    {
-      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-      
-      processReturn();
+  if (! inEHI)
+    deallocEHI(ehi); 
+ 
+  return 0;
+}
 
-      goto label_error1;
-    }
+short CmpSeabaseDDL::cloneSeabaseTable(
+     CorrName &cn,
+     const NATable * naTable,
+     const NAString &clonedTableName,
+     ExpHbaseInterface * inEHI,
+     ExeCliInterface * cliInterface)
+{
+  Lng32 cliRC = 0;
+  Lng32 retcode = 0;
 
-  // drop curr temp table
-  str_sprintf(queryBuf, "drop table \"%s\".\"%s\".\"%s\" ",
-              catalogNamePart.data(), schemaNamePart.data(), currTempTab.data(),
-              objectNamePart.data());
-  
-  cliRC = cliInterface.executeImmediate(queryBuf);
+  retcode = createSeabaseTableLike2(cn, clonedTableName);
+  if (retcode)
+    return -1;
+
+  ComObjectName tableName(clonedTableName, COM_TABLE_NAME);
+  const NAString catalogNamePart = tableName.getCatalogNamePartAsAnsiString();
+  const NAString schemaNamePart = tableName.getSchemaNamePartAsAnsiString(TRUE);
+  const NAString objectNamePart = tableName.getObjectNamePartAsAnsiString(TRUE);
+  Int64 clonedTableUID = 
+    getObjectUID
+    (cliInterface,
+     catalogNamePart.data(), schemaNamePart.data(), objectNamePart.data(),
+     COM_BASE_TABLE_OBJECT_LIT);
+ 
+  char buf[2000];
+  str_sprintf(buf, "merge into %s.\"%s\".%s using (select column_name, column_class from %s.\"%s\".%s where object_uid = %Ld) x on (object_uid = %Ld and column_name = x.column_name) when matched then update set column_class = x.column_class;",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              naTable->objectUid().castToInt64(),
+              clonedTableUID);
+  cliRC = cliInterface->executeImmediate(buf);
   if (cliRC < 0)
     {
-      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-      
+      cliInterface->retrieveSQLDiagnostics(CmpCommon::diags());
       processReturn();
 
-      goto label_error0;
+      return -1;
     }
+ 
+  const NAString extNameForHbase = 
+    cn.getQualifiedNameObj().getCatalogName() + "." + 
+    cn.getQualifiedNameObj().getSchemaName() + "." + 
+    cn.getQualifiedNameObj().getObjectName();
 
-   if (recreateViews(cliInterface, viewNameList, viewDefnList))
+  if (cloneHbaseTable(extNameForHbase, clonedTableName, inEHI))
     {
+      processReturn();
+      
       return -1;
     }
-
-  return 0;
-
- label_error1:
-  // rename current temp table to current
-  str_sprintf(queryBuf, "alter table \"%s\".\"%s\".\"%s\" rename to \"%s\" ",
-              catalogNamePart.data(), schemaNamePart.data(), currTempTab.data(),
-              objectNamePart.data());
   
-  cliRC = cliInterface.executeImmediate(queryBuf);
-
- label_error:
-  cleanupObjectAfterError(cliInterface,
-                          catalogNamePart, schemaNamePart, newTempTab,
-                          COM_BASE_TABLE_OBJECT);
-
-  recreateViews(cliInterface, viewNameList, viewDefnList);
-
-  return -1;
-
- label_error0:
-  cleanupObjectAfterError(cliInterface,
-                          catalogNamePart, schemaNamePart, currTempTab,
-                          COM_BASE_TABLE_OBJECT);
-
-  recreateViews(cliInterface, viewNameList, viewDefnList);
-
-  return -1;
+  return 0;
 }
 
 short CmpSeabaseDDL::recreateViews(ExeCliInterface &cliInterface,
@@ -4626,11 +4473,97 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
     }
 
   Int64 objUID = naTable->objectUid().castToInt64();
+  Int32 newColNum = naTable->getColumnCount();
+  for (Int32 cc = nacolArr.entries()-1; cc >= 0; cc--)
+    {
+      const NAColumn *nac = nacolArr[cc];
+
+      if ((NOT naTable->isSQLMXAlignedTable()) &&
+          (nac->isComputedColumn()))
+        {
+          str_sprintf(query, "update %s.\"%s\".%s set column_number = column_number + 1 where object_uid = %Ld and column_number = %d",
+                      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+                      objUID,
+                      nac->getPosition());
+          
+          cliRC = cliInterface.executeImmediate(query);
+          if (cliRC < 0)
+            {
+              cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+              
+              return;
+            }
+
+          str_sprintf(query, "update %s.\"%s\".%s set column_number = column_number + 1 where object_uid = %Ld and column_number = %d",
+                      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_KEYS,
+                      objUID,
+                      nac->getPosition());
+          
+          cliRC = cliInterface.executeImmediate(query);
+          if (cliRC < 0)
+            {
+              cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+              
+              return;
+            }
+
+          str_sprintf(query, "update %s.\"%s\".%s set sub_id = sub_id + 1 where text_uid = %Ld and text_type = %d and sub_id = %d",
+                      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_TEXT,
+                      objUID,
+                      COM_COMPUTED_COL_TEXT,
+                      nac->getPosition());
+          
+          cliRC = cliInterface.executeImmediate(query);
+          if (cliRC < 0)
+            {
+              cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+              
+              return;
+            }
+
+          // keys for indexes refer to base table column number.
+          // modify it so they now refer to new column numbers.
+          if (naTable->hasSecondaryIndexes())
+            {
+              const NAFileSetList &naFsList = naTable->getIndexList();
+              
+              for (Lng32 i = 0; i < naFsList.entries(); i++)
+                {
+                  const NAFileSet * naFS = naFsList[i];
+                  
+                  // skip clustering index
+                  if (naFS->getKeytag() == 0)
+                    continue;
+                  
+                  const QualifiedName &indexName = naFS->getFileSetName();
+                  
+                  str_sprintf(query, "update %s.\"%s\".%s set column_number = column_number + 1  where column_number = %d and object_uid = (select object_uid from %s.\"%s\".%s where catalog_name = '%s' and schema_name = '%s' and object_name = '%s' and object_type = 'IX') ",
+                              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_KEYS,
+                              nac->getPosition(),
+                              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_OBJECTS,
+                              indexName.getCatalogName().data(),
+                              indexName.getSchemaName().data(),
+                              indexName.getObjectName().data());
+                  cliRC = cliInterface.executeImmediate(query);
+                  if (cliRC < 0)
+                    {
+                      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+                      
+                      goto label_return;
+                    }
+                  
+                } // for
+            } // secondary indexes present
+          
+          newColNum--;
+        }
+    }
+
   str_sprintf(query, "insert into %s.\"%s\".%s values (%Ld, '%s', %d, '%s', %d, '%s', %d, %d, %d, %d, %d, '%s', %d, %d, '%s', %d, '%s', '%s', '%s', '%u', '%s', '%s', %Ld )",
               getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
               objUID,
               col_name,
-              naTable->getColumnCount(), 
+              newColNum, //naTable->getColumnCount(), 
               COM_ADDED_USER_COLUMN_LIT,
               datatype,
               getAnsiTypeStrFromFSType(datatype),
@@ -4756,36 +4689,144 @@ void CmpSeabaseDDL::alterSeabaseTableAddColumn(
   return;
 }
 
-void CmpSeabaseDDL::alterSeabaseTableDropColumn(
-                                                StmtDDLAlterTableDropColumn * alterDropColNode,
-                                                NAString &currCatName, NAString &currSchName)
+short CmpSeabaseDDL::updateMDforDropCol(ExeCliInterface &cliInterface,
+                                        const NATable * naTable,
+                                        Lng32 dropColNum)
 {
   Lng32 cliRC = 0;
-  Lng32 retcode = 0;
-
-  const NAString &tabName = alterDropColNode->getTableName();
-
-  ComObjectName tableName(tabName, COM_TABLE_NAME);
-  ComAnsiNamePart currCatAnsiName(currCatName);
-  ComAnsiNamePart currSchAnsiName(currSchName);
-  tableName.applyDefaults(currCatAnsiName, currSchAnsiName);
-
-  const NAString catalogNamePart = tableName.getCatalogNamePartAsAnsiString();
-  const NAString schemaNamePart = tableName.getSchemaNamePartAsAnsiString(TRUE);
-  const NAString objectNamePart = tableName.getObjectNamePartAsAnsiString(TRUE);
-  const NAString extTableName = tableName.getExternalName(TRUE);
-  const NAString extNameForHbase = catalogNamePart + "." + schemaNamePart + "." + objectNamePart;
 
-  ExeCliInterface cliInterface(STMTHEAP, NULL, NULL, 
-  CmpCommon::context()->sqlSession()->getParentQid());
+  Int64 objUID = naTable->objectUid().castToInt64();
 
-  if ((isSeabaseReservedSchema(tableName)) &&
-      (!Get_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL)))
+  char buf[4000];
+  str_sprintf(buf, "delete from %s.\"%s\".%s where object_uid = %Ld and column_number = %d",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              objUID,
+              dropColNum);
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
     {
-      *CmpCommon::diags() << DgSqlCode(-CAT_CANNOT_ALTER_DEFINITION_METADATA_SCHEMA);
-      processReturn();
-      return;
-    }
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+  
+  str_sprintf(buf, "update %s.\"%s\".%s set column_number = column_number - 1 where object_uid = %Ld and column_number >= %d",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              objUID,
+              dropColNum);
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+  
+  str_sprintf(buf, "update %s.\"%s\".%s set column_number = column_number - 1 where object_uid = %Ld and column_number >= %d",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_KEYS,
+              objUID,
+              dropColNum);
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+  
+  str_sprintf(buf, "update %s.\"%s\".%s set sub_id = sub_id - 1 where text_uid = %Ld and text_type = %d and sub_id > %d",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_TEXT,
+              objUID,
+              COM_COMPUTED_COL_TEXT,
+              dropColNum);
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+  
+  // keys for pkey constraint refer to base table column number.
+  // modify it so they now refer to new column numbers.
+  str_sprintf(buf, "update %s.\"%s\".%s K set column_number = column_number - 1  where K.column_number >= %d and K.object_uid = (select C.constraint_uid from %s.\"%s\".%s C where C.table_uid = %Ld and C.constraint_type = 'P')",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_KEYS,
+              dropColNum,
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_TABLE_CONSTRAINTS,
+              objUID);
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      return -1;
+    }
+  
+  // keys for indexes refer to base table column number.
+  // modify it so they now refer to new column numbers.
+  if (naTable->hasSecondaryIndexes())
+    {
+      const NAFileSetList &naFsList = naTable->getIndexList();
+      
+      for (Lng32 i = 0; i < naFsList.entries(); i++)
+        {
+          const NAFileSet * naFS = naFsList[i];
+          
+          // skip clustering index
+          if (naFS->getKeytag() == 0)
+            continue;
+          
+          const QualifiedName &indexName = naFS->getFileSetName();
+          
+          str_sprintf(buf, "update %s.\"%s\".%s set column_number = column_number - 1  where column_number >=  %d and object_uid = (select object_uid from %s.\"%s\".%s where catalog_name = '%s' and schema_name = '%s' and object_name = '%s' and object_type = 'IX') ",
+                      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_KEYS,
+                      dropColNum,
+                      getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_OBJECTS,
+                      indexName.getCatalogName().data(),
+                      indexName.getSchemaName().data(),
+                      indexName.getObjectName().data());
+          cliRC = cliInterface.executeImmediate(buf);
+          if (cliRC < 0)
+            {
+              cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+              return -1;
+            }
+          
+        } // for
+    } // secondary indexes present
+
+  return 0;
+}
+
+void CmpSeabaseDDL::alterSeabaseTableDropColumn(
+                                                StmtDDLAlterTableDropColumn * alterDropColNode,
+                                                NAString &currCatName, NAString &currSchName)
+{
+  Lng32 cliRC = 0;
+  Lng32 retcode = 0;
+
+  const NAString &tabName = alterDropColNode->getTableName();
+
+  ComObjectName tableName(tabName, COM_TABLE_NAME);
+  ComAnsiNamePart currCatAnsiName(currCatName);
+  ComAnsiNamePart currSchAnsiName(currSchName);
+  tableName.applyDefaults(currCatAnsiName, currSchAnsiName);
+
+  const NAString catalogNamePart = tableName.getCatalogNamePartAsAnsiString();
+  const NAString schemaNamePart = tableName.getSchemaNamePartAsAnsiString(TRUE);
+  const NAString objectNamePart = tableName.getObjectNamePartAsAnsiString(TRUE);
+  const NAString extTableName = tableName.getExternalName(TRUE);
+  const NAString extNameForHbase = catalogNamePart + "." + schemaNamePart + "." + objectNamePart;
+
+  ExeCliInterface cliInterface(STMTHEAP, NULL, NULL, 
+  CmpCommon::context()->sqlSession()->getParentQid());
+
+  if ((isSeabaseReservedSchema(tableName)) &&
+      (!Get_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL)))
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_CANNOT_ALTER_DEFINITION_METADATA_SCHEMA);
+      processReturn();
+      return;
+    }
 
   ExpHbaseInterface * ehi = allocEHI();
   if (ehi == NULL)
@@ -4816,7 +4857,7 @@ void CmpSeabaseDDL::alterSeabaseTableDropColumn(
               tableName.getSchemaNamePart().getInternalName(),
               tableName.getCatalogNamePart().getInternalName());
 
-  NATable *naTable = bindWA.getNATable(cn); 
+  const NATable *naTable = bindWA.getNATable(cn); 
   if (naTable == NULL || bindWA.errStatus())
     {
       *CmpCommon::diags()
@@ -4933,10 +4974,12 @@ void CmpSeabaseDDL::alterSeabaseTableDropColumn(
   char *col = NULL;
   if (naTable->isSQLMXAlignedTable())
     {
-      if (alignedFormatTableAddDropColumn(naTable->objectUid().castToInt64(),
-                                          FALSE, // drop col
-                                          catalogNamePart, schemaNamePart, objectNamePart,
-                                          (char*)colName.data(), NULL))
+      if (alignedFormatTableDropColumn
+          (
+           catalogNamePart, schemaNamePart, objectNamePart,
+           naTable, 
+           alterDropColNode->getColName(),
+           NULL))
         {
           processReturn();
           return;
@@ -4946,80 +4989,12 @@ void CmpSeabaseDDL::alterSeabaseTableDropColumn(
     {
       if (beginXnIfNotInProgress(&cliInterface, xnWasStartedHere))
         return;
-      
-      char buf[4000];
-      str_sprintf(buf, "delete from %s.\"%s\".%s where object_uid = %Ld and column_number = %d",
-                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
-                  objUID,
-                  colNumber);
-      
-      cliRC = cliInterface.executeImmediate(buf);
-      if (cliRC < 0)
-        {
-          cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-          goto label_return;
-        }
-      
-      str_sprintf(buf, "update %s.\"%s\".%s set column_number = column_number - 1 where object_uid = %Ld and column_number >= %d",
-                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
-                  objUID,
-                  colNumber);
-      
-      cliRC = cliInterface.executeImmediate(buf);
-      if (cliRC < 0)
-        {
-          cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-          
-          goto label_return;
-        }
-      
-      str_sprintf(buf, "update %s.\"%s\".%s set column_number = column_number - 1 where object_uid = %Ld and column_number >= %d",
-                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_KEYS,
-                  objUID,
-                  colNumber);
-      
-      cliRC = cliInterface.executeImmediate(buf);
-      if (cliRC < 0)
+
+      if (updateMDforDropCol(cliInterface, naTable, colNumber))
         {
-          cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-          
           goto label_return;
         }
 
-      // keys for indexes refer to base table column number.
-      // modify it so they now refer to new column numbers.
-      if (naTable->hasSecondaryIndexes())
-        {
-          const NAFileSetList &naFsList = naTable->getIndexList();
-          
-          for (Lng32 i = 0; i < naFsList.entries(); i++)
-            {
-              naFS = naFsList[i];
-              
-              // skip clustering index
-              if (naFS->getKeytag() == 0)
-                continue;
-              
-              const QualifiedName &indexName = naFS->getFileSetName();
-
-              str_sprintf(buf, "update %s.\"%s\".%s set column_number = column_number - 1  where column_number >=  %d and object_uid = (select object_uid from %s.\"%s\".%s where catalog_name = '%s' and schema_name = '%s' and object_name = '%s' and object_type = 'IX') ",
-                          getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_KEYS,
-                          colNumber,
-                          getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_OBJECTS,
-                          indexName.getCatalogName().data(),
-                          indexName.getSchemaName().data(),
-                          indexName.getObjectName().data());
-              cliRC = cliInterface.executeImmediate(buf);
-              if (cliRC < 0)
-                {
-                  cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
-                  
-                  goto label_return;
-                }
-
-            } // for
-        } // secondary indexes present
-      
       // remove column from all rows of the base table
       HbaseStr hbaseTable;
       hbaseTable.val = (char*)extNameForHbase.data();
@@ -5261,166 +5236,627 @@ void CmpSeabaseDDL::alterSeabaseTableAlterIdentityColumn(
         }
     }
 
-  //  CorrName cn(objectNamePart, STMTHEAP, schemaNamePart, catalogNamePart);
   ActiveSchemaDB()->getNATableDB()->removeNATable(cn,
     NATableDB::REMOVE_FROM_ALL_USERS, COM_BASE_TABLE_OBJECT);
 
   return;
 }
 
-void CmpSeabaseDDL::alterSeabaseTableAlterColumnDatatype(
-     StmtDDLAlterTableAlterColumnDatatype * alterColNode,
-     NAString &currCatName, NAString &currSchName)
+///////////////////////////////////////////////////////////////////////
+//
+// An aligned table constains all columns in one hbase cell.
+// To drop a column, we need to read each row, create a
+// new row with the removed column and insert into the original table.
+//
+// Steps to drop a column from an aligned table:
+//
+// -- make a copy of the source aligned table using hbase copy
+// -- truncate the source table
+// -- Update metadata and remove the dropped column.
+// -- bulk load data from copied table into the source table
+// -- drop the copied temp table
+//
+// If an error happens after the source table has been truncated, then
+// it will be restored from the copied table.
+//
+///////////////////////////////////////////////////////////////////////
+short CmpSeabaseDDL::alignedFormatTableDropColumn
+(
+ const NAString &catalogNamePart,
+ const NAString &schemaNamePart,
+ const NAString &objectNamePart,
+ const NATable * naTable,
+ const NAString &altColName,
+ ElemDDLColDef *pColDef)
 {
   Lng32 cliRC = 0;
-  Lng32 retcode = 0;
 
-  const NAString &tabName = alterColNode->getTableName();
+  const NAFileSet * naf = naTable->getClusteringIndex();
+  
+  CorrName cn(objectNamePart, STMTHEAP, schemaNamePart, catalogNamePart);
 
-  ComObjectName tableName(tabName, COM_TABLE_NAME);
-  ComAnsiNamePart currCatAnsiName(currCatName);
-  ComAnsiNamePart currSchAnsiName(currSchName);
-  tableName.applyDefaults(currCatAnsiName, currSchAnsiName);
+  ComUID comUID;
+  comUID.make_UID();
+  Int64 objUID = comUID.get_value();
+  
+  char objUIDbuf[100];
 
-  const NAString catalogNamePart = tableName.getCatalogNamePartAsAnsiString();
-  const NAString schemaNamePart = tableName.getSchemaNamePartAsAnsiString(TRUE);
-  const NAString objectNamePart = tableName.getObjectNamePartAsAnsiString(TRUE);
-  const NAString extTableName = tableName.getExternalName(TRUE);
-  const NAString extNameForHbase = catalogNamePart + "." + schemaNamePart + "." + objectNamePart;
+  NAString tempTable(naTable->getTableName().getQualifiedNameAsAnsiString());
+  tempTable += "_";
+  tempTable += str_ltoa(objUID, objUIDbuf);
 
-  ExeCliInterface cliInterface(STMTHEAP, NULL, NULL, 
-  CmpCommon::context()->sqlSession()->getParentQid());
+  ExpHbaseInterface * ehi = allocEHI();
+  ExeCliInterface cliInterface
+    (STMTHEAP, NULL, NULL, 
+     CmpCommon::context()->sqlSession()->getParentQid());
 
-  if ((isSeabaseReservedSchema(tableName)) &&
-      (!Get_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL)))
+  Int64 tableUID = naTable->objectUid().castToInt64();
+  const NAColumnArray &naColArr = naTable->getNAColumnArray();
+  const NAColumn * altNaCol = naColArr.getColumn(altColName);
+  Lng32 altColNum = altNaCol->getPosition();
+
+  NAString tgtCols;
+  NAString srcCols;
+
+ NABoolean xnWasStartedHere = FALSE;
+
+  char buf[4000];
+
+  if (cloneSeabaseTable(cn, naTable, tempTable, ehi, &cliInterface))
     {
-      *CmpCommon::diags() << DgSqlCode(-CAT_CANNOT_ALTER_DEFINITION_METADATA_SCHEMA);
-      processReturn();
-      return;
+      cliRC = -1;
+      goto label_drop;
     }
-
-  ExpHbaseInterface * ehi = allocEHI();
-  if (ehi == NULL)
+  
+  if (truncateHbaseTable(catalogNamePart, schemaNamePart, objectNamePart,
+                         (NATable*)naTable, ehi))
     {
-      processReturn();
-      
-      return;
+      cliRC = -1;
+      goto label_restore;
     }
 
-  retcode = existsInSeabaseMDTable(&cliInterface, 
-                                   catalogNamePart, schemaNamePart, objectNamePart,
-                                   COM_BASE_TABLE_OBJECT,
-                                   (Get_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL) 
-                                    ? FALSE : TRUE),
-                                   TRUE, TRUE);
-  if (retcode < 0)
+  if (beginXnIfNotInProgress(&cliInterface, xnWasStartedHere))
     {
-      processReturn();
-
-      return;
+      cliRC = -1;
+      goto label_restore;
     }
 
-  ActiveSchemaDB()->getNATableDB()->useCache();
-
-  BindWA bindWA(ActiveSchemaDB(), CmpCommon::context(), FALSE/*inDDL*/);
-  CorrName cn(tableName.getObjectNamePart().getInternalName(),
-              STMTHEAP,
-              tableName.getSchemaNamePart().getInternalName(),
-              tableName.getCatalogNamePart().getInternalName());
+  if (updateMDforDropCol(cliInterface, naTable, altColNum))
+    {
+      cliRC = -1;
+      goto label_restore;
+    }
+  
+  ActiveSchemaDB()->getNATableDB()->removeNATable
+    (cn,
+     NATableDB::REMOVE_FROM_ALL_USERS, 
+     COM_BASE_TABLE_OBJECT);
 
-  NATable *naTable = bindWA.getNATable(cn); 
-  if (naTable == NULL || bindWA.errStatus())
+  for (Int32 c = 0; c < naColArr.entries(); c++)
     {
-      *CmpCommon::diags()
-        << DgSqlCode(-4082)
-        << DgTableName(cn.getExposedNameAsAnsiString());
-    
-      processReturn();
+      const NAColumn * nac = naColArr[c];
+      if (nac->getColName() == altColName)
+        continue;
 
-      return;
-    }
+      if (nac->isComputedColumn())
+        continue;
 
-  // Make sure user has the privilege to perform the alter column
-  if (!isDDLOperationAuthorized(SQLOperation::ALTER_TABLE,
-                                naTable->getOwner(),naTable->getSchemaOwner()))
-  {
-     *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+      if (nac->isSystemColumn())
+        continue;
 
-     processReturn ();
+      tgtCols += nac->getColName();
+      tgtCols += ",";
+    } // for
 
-     return;
-  }
+  tgtCols = tgtCols.strip(NAString::trailing, ',');
 
-  // return an error if trying to alter a column from a volatile table
-  if (naTable->isVolatileTable())
+  str_sprintf(buf, "upsert using load into %s(%s) select %s from %s",
+              naTable->getTableName().getQualifiedNameAsAnsiString().data(),
+              tgtCols.data(),
+              tgtCols.data(),
+              tempTable.data());
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
     {
-      *CmpCommon::diags() << DgSqlCode(-CAT_REGULAR_OPERATION_ON_VOLATILE_OBJECT);
-     
-      processReturn ();
-
-      return;
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      goto label_restore;
     }
 
-  const NAColumnArray &nacolArr = naTable->getNAColumnArray();
-  const NAString &colName = alterColNode->getColumnName();
+  endXnIfStartedHere(&cliInterface, xnWasStartedHere, 0);
 
-  const NAColumn * nacol = nacolArr.getColumn(colName);
-  if (! nacol)
+  str_sprintf(buf, "drop table %s", tempTable.data());
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
     {
-      // column doesnt exist. Error.
-      *CmpCommon::diags() << DgSqlCode(-CAT_COLUMN_DOES_NOT_EXIST_ERROR)
-                          << DgColumnName(colName);
-
-      processReturn();
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      goto label_restore;
+    }
+  
+  deallocEHI(ehi); 
+  
+  return 0;
 
-      return;
+ label_restore:
+  endXnIfStartedHere(&cliInterface, xnWasStartedHere, -1);
+
+  cloneHbaseTable(tempTable, 
+                  naTable->getTableName().getQualifiedNameAsAnsiString(),
+                  ehi);
+ 
+ label_drop:  
+  str_sprintf(buf, "drop table %s", tempTable.data());
+  Lng32 cliRC2 = cliInterface.executeImmediate(buf);
+  
+  deallocEHI(ehi); 
+  
+  return (cliRC < 0 ? -1 : 0);  
+}
+
+///////////////////////////////////////////////////////////////////////
+//
+// An aligned table constains all columns in one hbase cell.
+// To alter a column, we need to read each row, create a
+// new row with the altered column and insert into the original table.
+//
+// Steps to alter a column from an aligned table:
+//
+// -- make a copy of the source aligned table using hbase copy
+// -- truncate the source table
+// -- Update metadata column definition with the new definition
+// -- bulk load data from copied table into the source table
+// -- drop the copied temp table
+//
+// If an error happens after the source table has been truncated, then
+// it will be restored from the copied table.
+//
+///////////////////////////////////////////////////////////////////////
+short CmpSeabaseDDL::alignedFormatTableAlterColumn
+(
+ const NAString &catalogNamePart,
+ const NAString &schemaNamePart,
+ const NAString &objectNamePart,
+ const NATable * naTable,
+ const NAString &altColName,
+ ElemDDLColDef *pColDef)
+{
+  Lng32 cliRC = 0;
+
+  const NAFileSet * naf = naTable->getClusteringIndex();
+  
+  CorrName cn(objectNamePart, STMTHEAP, schemaNamePart, catalogNamePart);
+
+  ComUID comUID;
+  comUID.make_UID();
+  Int64 objUID = comUID.get_value();
+  
+  char objUIDbuf[100];
+
+  NAString tempTable(naTable->getTableName().getQualifiedNameAsAnsiString());
+  tempTable += "_";
+  tempTable += str_ltoa(objUID, objUIDbuf);
+
+  ExpHbaseInterface * ehi = allocEHI();
+  ExeCliInterface cliInterface
+    (STMTHEAP, NULL, NULL, 
+     CmpCommon::context()->sqlSession()->getParentQid());
+
+  Int64 tableUID = naTable->objectUid().castToInt64();
+  const NAColumnArray &naColArr = naTable->getNAColumnArray();
+  const NAColumn * altNaCol = naColArr.getColumn(altColName);
+  Lng32 altColNum = altNaCol->getPosition();
+
+  char buf[4000];
+  NAString colFamily;
+  NAString colName;
+  Lng32 datatype, length, precision, scale, dt_start, dt_end, 
+    nullable, upshifted;
+  ComColumnClass colClass;
+  ComColumnDefaultClass defaultClass;
+  NAString charset, defVal;
+  NAString heading;
+  ULng32 hbaseColFlags;
+  Int64 colFlags;
+  LobsStorage lobStorage;
+  NAString quotedDefVal;
+
+  NABoolean xnWasStartedHere = FALSE;
+
+  if (cloneSeabaseTable(cn, naTable, tempTable, ehi, &cliInterface))
+    {
+      cliRC = -1;
+      goto label_drop;
+    }
+  
+  if (truncateHbaseTable(catalogNamePart, schemaNamePart, objectNamePart,
+                         (NATable*)naTable, ehi))
+    {
+      cliRC = -1;
+      goto label_restore;
     }
 
-  const NAType * currType = nacol->getType();
-  NAType * newType = alterColNode->getType();
 
-  // Column that can be altered must meet these conditions:
-  //   -- old and new column datatype must be VARCHAR
-  //   -- new col length must be greater than or equal to old length
-  //   -- old and new character sets must be the same
-  NABoolean canAlter = FALSE;
-  if ((DFS2REC::isSQLVarChar(currType->getFSDatatype())) &&
-      (DFS2REC::isSQLVarChar(newType->getFSDatatype())) &&
-      (currType->getFSDatatype() == newType->getFSDatatype()) &&
-      (currType->getNominalSize() <= newType->getNominalSize()) &&
-      (((CharType*)currType)->getCharSet() == ((CharType*)newType)->getCharSet()))
-    canAlter = TRUE;
+  if (beginXnIfNotInProgress(&cliInterface, xnWasStartedHere))
+    goto label_restore;
 
-  if (NOT canAlter)
+  if (getColInfo(pColDef,
+                 colFamily,
+                 colName, 
+                 naTable->isSQLMXAlignedTable(),
+                 datatype, length, precision, scale, dt_start, dt_end, 
+                 upshifted, nullable,
+                 charset, colClass, defaultClass, defVal, heading, lobStorage,
+                 hbaseColFlags, colFlags))
     {
+      cliRC = -1;
+      processReturn();
+      
+      goto label_restore;
+    }
+  
+  if (NOT defVal.isNull())
+    {
+      ToQuotedString(quotedDefVal, defVal, FALSE);
+    }
+  
+  str_sprintf(buf, "update %s.\"%s\".%s set (column_class, fs_data_type, sql_data_type, column_size, column_precision, column_scale, datetime_start_field, datetime_end_field, is_upshifted, nullable, character_set, default_class, default_value) = ('%s', %d, '%s', %d, %d, %d, %d, %d, '%s', %d, '%s', %d, '%s') where object_uid = %Ld and column_number = %d",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              COM_ALTERED_USER_COLUMN_LIT,
+              datatype,
+              getAnsiTypeStrFromFSType(datatype),
+              length,
+              precision,
+              scale,
+              dt_start,
+              dt_end,
+              (upshifted ? "Y" : "N"),
+              nullable,
+              (char*)charset.data(),
+              (Lng32)defaultClass,
+              (quotedDefVal.isNull() ? "" : quotedDefVal.data()),
+              tableUID,
+              altColNum);
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      goto label_restore;
+    }
+  
+  ActiveSchemaDB()->getNATableDB()->removeNATable
+    (cn,
+     NATableDB::REMOVE_FROM_ALL_USERS, 
+     COM_BASE_TABLE_OBJECT);
+  
+  str_sprintf(buf, "upsert using load into %s select * from %s",
+              naTable->getTableName().getQualifiedNameAsAnsiString().data(),
+              tempTable.data());
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+
       NAString reason;
-      if (NOT ((DFS2REC::isSQLVarChar(currType->getFSDatatype())) &&
-               (DFS2REC::isSQLVarChar(newType->getFSDatatype()))))
-        reason = "Old and New datatypes must be VARCHAR.";
-      else if (currType->getFSDatatype() != newType->getFSDatatype())
-        reason = "Old and New datatypes must be the same.";
-      else if (((CharType*)currType)->getCharSet() != ((CharType*)newType)->getCharSet())
-        reason = "Old and New character sets must be the same.";
-      else if (currType->getNominalSize() > newType->getNominalSize())
-        reason = "New length must be greater than or equal to old length.";
+      reason = "Old data could not be updated using the altered column definition.";
+      
+      // column cannot be altered
+      *CmpCommon::diags() << DgSqlCode(-1404)
+                          << DgColumnName(altColName)
+                          << DgString0(reason);
 
-      // key column cannot be altered
+      goto label_restore;
+    }
+
+  deallocEHI(ehi); 
+  
+  endXnIfStartedHere(&cliInterface, xnWasStartedHere, 0);
+  
+  return 0;
+
+ label_restore:
+  cloneHbaseTable(tempTable, 
+                  naTable->getTableName().getQualifiedNameAsAnsiString(),
+                  ehi);
+ 
+ label_drop:  
+  str_sprintf(buf, "drop table %s", tempTable.data());
+  Lng32 cliRC2 = cliInterface.executeImmediate(buf);
+  
+  deallocEHI(ehi); 
+  
+  endXnIfStartedHere(&cliInterface, xnWasStartedHere, -1);
+
+  return (cliRC < 0 ? -1 : 0);
+}
+
+short CmpSeabaseDDL::alterColumnAttr(
+     const NAString &catalogNamePart, const NAString &schemaNamePart,
+     const NAString &objectNamePart,
+     const NATable * naTable, const NAColumn * naCol, NAType * newType,
+     StmtDDLAlterTableAlterColumnDatatype * alterColNode)
+{
+  if (naTable->isSQLMXAlignedTable())
+    {
+      ElemDDLColDef *pColDef = 
+        alterColNode->getColToAlter()->castToElemDDLColDef();
+
+      if (alignedFormatTableAlterColumn
+          (
+           catalogNamePart, schemaNamePart, objectNamePart,
+           naTable,
+           naCol->getColName(), 
+           pColDef))
+        {
+          processReturn();
+          return -1;
+        }
+
+      return 0;
+    }
+
+  CorrName cn(objectNamePart, STMTHEAP, schemaNamePart,catalogNamePart);
+
+  Lng32 cliRC = 0;
+  Lng32 retcode = 0;
+
+  ComUID comUID;
+  comUID.make_UID();
+  Int64 objUID = comUID.get_value();
+  
+  char objUIDbuf[100];
+
+  NAString tempCol(naCol->getColName());
+  tempCol += "_";
+  tempCol += str_ltoa(objUID, objUIDbuf);
+
+  char dispBuf[1000];
+  Lng32 ii = 0;
+  NABoolean identityCol;
+  ElemDDLColDef *pColDef = alterColNode->getColToAlter()->castToElemDDLColDef();
+  NAColumn *nac = NULL;
+  if (getNAColumnFromColDef(pColDef, nac))
+    return -1;
+
+  dispBuf[0] = 0;
+  if (cmpDisplayColumn(nac, (char*)tempCol.data(), newType, 3, NULL, dispBuf, 
+                       ii, FALSE, identityCol, 
+                       FALSE, FALSE))
+    return -1;
+  
+  ExeCliInterface cliInterface
+    (STMTHEAP, NULL, NULL, 
+     CmpCommon::context()->sqlSession()->getParentQid());
+
+  Int64 tableUID = naTable->objectUid().castToInt64();
+  const NAColumnArray &nacolArr = naTable->getNAColumnArray();
+  const NAString &altColName = naCol->getColName();
+  const NAColumn * altNaCol = nacolArr.getColumn(altColName);
+  Lng32 altColNum = altNaCol->getPosition();
+
+  char buf[4000];
+  str_sprintf(buf, "alter table %s add column %s",
+              naTable->getTableName().getQualifiedNameAsAnsiString().data(), 
+              dispBuf);
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+
+      processReturn();
+      return -1;
+    }
+
+  str_sprintf(buf, "update %s set %s = %s",
+              naTable->getTableName().getQualifiedNameAsAnsiString().data(), 
+              tempCol.data(),
+              naCol->getColName().data());
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+
+      goto label_error1;
+    }
+   
+  str_sprintf(buf, "delete from %s.\"%s\".%s where object_uid = %Ld and column_number = %d",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              tableUID,
+              altColNum);
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      goto label_error1;
+    }
+  
+  str_sprintf(buf, "insert into %s.\"%s\".%s select object_uid, '%s', %d, '%s', fs_data_type, sql_data_type, column_size, column_precision, column_scale, datetime_start_field, datetime_end_field, is_upshifted, column_flags, nullable, character_set, default_class, default_value, column_heading, '%s', '%s', direction, is_optional, flags from %s.\"%s\".%s where object_uid = %Ld and column_number = (select column_number from %s.\"%s\".%s where object_uid = %Ld and column_name = '%s')",
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              naCol->getColName().data(),              
+              altColNum,
+              COM_ALTERED_USER_COLUMN_LIT,
+              altNaCol->getHbaseColFam().data(),
+              altNaCol->getHbaseColQual().data(),
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              tableUID,
+              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+              tableUID,
+              tempCol.data());
+  
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      goto label_error1;
+    }
+
+  ActiveSchemaDB()->getNATableDB()->removeNATable
+    (cn,
+     NATableDB::REMOVE_FROM_ALL_USERS, 
+     COM_BASE_TABLE_OBJECT);
+  str_sprintf(buf, "update %s set %s = %s",
+              naTable->getTableName().getQualifiedNameAsAnsiString().data(), 
+              naCol->getColName().data(),
+              tempCol.data());
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+
+      NAString reason;
+      reason = "Old data could not be updated into the new column definition.";
+      
+      // column cannot be altered
       *CmpCommon::diags() << DgSqlCode(-1404)
-                          << DgColumnName(colName)
+                          << DgColumnName(naCol->getColName())
                           << DgString0(reason);
 
       processReturn();
+      goto label_error1;
+    }
+   
+  str_sprintf(buf, "alter table %s drop column %s",
+              naTable->getTableName().getQualifiedNameAsAnsiString().data(), 
+              tempCol.data());
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
       
+      processReturn();
+      return -1;
+    }
+
+  return 0;
+
+ label_error1:
+  str_sprintf(buf, "alter table %s drop column %s",
+              naTable->getTableName().getQualifiedNameAsAnsiString().data(), 
+              tempCol.data());
+  cliRC = cliInterface.executeImmediate(buf);
+  if (cliRC < 0)
+    {
+      processReturn();
+      return -1;
+    } 
+ 
+  return -1;
+}
+
+void CmpSeabaseDDL::alterSeabaseTableAlterColumnDatatype(
+     StmtDDLAlterTableAlterColumnDatatype * alterColNode,
+     NAString &currCatName, NAString &currSchName)
+{
+  Lng32 cliRC = 0;
+  Lng32 retcode = 0;
+
+  const NAString &tabName = alterColNode->getTableName();
+
+  ComObjectName tableName(tabName, COM_TABLE_NAME);
+  ComAnsiNamePart currCatAnsiName(currCatName);
+  ComAnsiNamePart currSchAnsiName(currSchName);
+  tableName.applyDefaults(currCatAnsiName, currSchAnsiName);
+
+  const NAString catalogNamePart = tableName.getCatalogNamePartAsAnsiString();
+  const NAString schemaNamePart = tableName.getSchemaNamePartAsAnsiString(TRUE);
+  const NAString objectNamePart = tableName.getObjectNamePartAsAnsiString(TRUE);
+  const NAString extTableName = tableName.getExternalName(TRUE);
+  const NAString extNameForHbase = catalogNamePart + "." + schemaNamePart + "." + objectNamePart;
+
+  ExeCliInterface cliInterface(STMTHEAP, NULL, NULL, 
+  CmpCommon::context()->sqlSession()->getParentQid());
+
+  if ((isSeabaseReservedSchema(tableName)) &&
+      (!Get_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL)))
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_CANNOT_ALTER_DEFINITION_METADATA_SCHEMA);
+      processReturn();
       return;
     }
- // If column is a LOB column , error
+
+  retcode = existsInSeabaseMDTable(&cliInterface, 
+                                   catalogNamePart, schemaNamePart, objectNamePart,
+                                   COM_BASE_TABLE_OBJECT,
+                                   (Get_SqlParser_Flags(INTERNAL_QUERY_FROM_EXEUTIL) 
+                                    ? FALSE : TRUE),
+                                   TRUE, TRUE);
+  if (retcode < 0)
+    {
+      processReturn();
+
+      return;
+    }
+
+  ActiveSchemaDB()->getNATableDB()->useCache();
+
+  BindWA bindWA(ActiveSchemaDB(), CmpCommon::context(), FALSE/*inDDL*/);
+  CorrName cn(tableName.getObjectNamePart().getInternalName(),
+              STMTHEAP,
+              tableName.getSchemaNamePart().getInternalName(),
+              tableName.getCatalogNamePart().getInternalName());
+
+  NATable *naTable = bindWA.getNATable(cn); 
+  if (naTable == NULL || bindWA.errStatus())
+    {
+      *CmpCommon::diags()
+        << DgSqlCode(-4082)
+        << DgTableName(cn.getExposedNameAsAnsiString());
+    
+      processReturn();
+
+      return;
+    }
+
+  // Make sure user has the privilege to perform the alter column
+  if (!isDDLOperationAuthorized(SQLOperation::ALTER_TABLE,
+                                naTable->getOwner(),naTable->getSchemaOwner()))
+  {
+     *CmpCommon::diags() << DgSqlCode(-CAT_NOT_AUTHORIZED);
+
+     processReturn ();
+
+     return;
+  }
+
+  // return an error if trying to alter a column from a volatile table
+  if (naTable->isVolatileTable())
+    {
+      *CmpCommon::diags() << DgSqlCode(-CAT_REGULAR_OPERATION_ON_VOLATILE_OBJECT);
+     
+      processReturn ();
+
+      return;
+    }
+
+  ElemDDLColDef *pColDef = alterColNode->getColToAlter()->castToElemDDLColDef();
+  
+  const NAColumnArray &nacolArr = naTable->getNAColumnArray();
+  const NAString &colName = pColDef->getColumnName();
+
+  const NAColumn * nacol = nacolArr.getColumn(colName);
+  if (! nacol)
+    {
+      // column doesnt exist. Error.
+      *CmpCommon::diags() << DgSqlCode(-CAT_COLUMN_DOES_NOT_EXIST_ERROR)
+                          << DgColumnName(colName);
+
+      processReturn();
+
+      return;
+    }
+
+  const NAType * currType = nacol->getType();
+  NAType * newType = pColDef->getColumnDataType();
+
+  // If column is a LOB column , error
   if ((currType->getFSDatatype() == REC_BLOB) || (currType->getFSDatatype() == REC_CLOB))
-     {
+    {
       *CmpCommon::diags() << DgSqlCode(-CAT_LOB_COLUMN_ALTER)
-                              << DgColumnName(colName);
+                          << DgColumnName(colName);
       processReturn();
       return;
      }
+
   const NAFileSet * naFS = naTable->getClusteringIndex();
   const NAColumnArray &naKeyColArr = naFS->getIndexKeyColumns();
   if (naKeyColArr.getColumn(colName))
@@ -5461,32 +5897,104 @@ void CmpSeabaseDDL::alterSeabaseTableAlterColumnDatatype(
         } // for
     } // secondary indexes present
 
-  Int64 objUID = naTable->objectUid().castToInt64();
+  if ((NOT currType->isCompatible(*newType)) &&
+      (NOT ((currType->getTypeQualifier() == NA_CHARACTER_TYPE) &&
+            (newType->getTypeQualifier() == NA_CHARACTER_TYPE))))
+    {
+      NAString reason = "Old and New datatypes must be compatible.";
 
-  Lng32 colNumber = nacol->getPosition();
-  char *col = NULL;
+      // column cannot be altered
+      *CmpCommon::diags() << DgSqlCode(-1404)
+                          << DgColumnName(colName)
+                          << DgString0(reason);
+      
+      processReturn();
+      
+      return;
+    }
 
-  char buf[4000];
-  str_sprintf(buf, "update %s.\"%s\".%s set column_size = %d where object_uid = %Ld and column_number = %d",
-              getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
-              newType->getNominalSize(),
-              objUID,
-              colNumber);
-  
-  cliRC = cliInterface.executeImmediate(buf);
-  if (cliRC < 0)
+  // Column that can be altered by updating metadata only
+  // must meet these conditions:
+  //   -- old and new column datatype must be VARCHAR
+  //   -- old and new datatype must have the same nullable attr
+  //   -- new col length must be greater than or equal to old length
+  //   -- old and new character sets must be the same
+  NABoolean mdAlterOnly = FALSE;
+  if ((DFS2REC::isSQLVarChar(currType->getFSDatatype())) &&
+      (DFS2REC::isSQLVarChar(newType->getFSDatatype())) &&
+      (currType->getFSDatatype() == newType->getFSDatatype()) &&
+      (currType->supportsSQLnull() == newType->supportsSQLnull()) &&
+      (currType->getNominalSize() <= newType->getNominalSize()) &&
+      (((CharType*)currType)->getCharSet() == ((CharType*)newType)->getCharSet()))
+    mdAlterOnly = TRUE;
+
+  if ((NOT mdAlterOnly) &&
+      (CmpCommon::getDefault(TRAF_ALTER_COL_ATTRS) == DF_OFF))
     {
-      cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+      NAString reason;
+      if (NOT ((DFS2REC::isSQLVarChar(currType->getFSDatatype())) &&
+               (DFS2REC::isSQLVarChar(newType->getFSDatatype()))))
+        reason = "Old and New datatypes must be VARCHAR.";
+      else if (currType->getFSDatatype() != newType->getFSDatatype())
+        reason = "Old and New datatypes must be the same.";
+      else if (((CharType*)currType)->getCharSet() != ((CharType*)newType)->getCharSet())
+        reason = "Old and New character sets must be the same.";
+      else if (currType->getNominalSize() > newType->getNominalSize())
+        reason = "New length must be greater than or equal to old length.";
+      else if (currType->supportsSQLnull() != newType->supportsSQLnull())
+        reason = "Old and New nullability must be the same.";
+
+      // column cannot be altered
+      *CmpCommon::diags() << DgSqlCode(-1404)
+                          << DgColumnName(colName)
+                          << DgString0(reason);
 
       processReturn();
+      
       return;
     }
+
+  if (mdAlterOnly)
+    {
+      Int64 objUID = naTable->objectUid().castToInt64();
+      
+      Lng32 colNumber = nacol->getPosition();
+      
+      char buf[4000];
+      str_sprintf(buf, "update %s.\"%s\".%s set column_size = %d, column_class = '%s' where object_uid = %Ld and column_number = %d",
+                  getSystemCatalog(), SEABASE_MD_SCHEMA, SEABASE_COLUMNS,
+                  newType->getNominalSize(),
+                  COM_ALTERED_USER_COLUMN_LIT,
+                  objUID,
+                  colNumber);
+      
+      cliRC = cliInterface.executeImmediate(buf);
+      if (cliRC < 0)
+        {
+          cliInterface.retrieveSQLDiagnostics(CmpCommon::diags());
+          
+          processReturn();
+          return;
+        }
+    }
+  else
+    {
+      if (alterColumnAttr(catalogNamePart, schemaNamePart, objectNamePart,
+                          naTable, nacol, newType, alterColNode))
+        return;
+    }
   
-  deallocEHI(ehi); 
-  heap_->deallocateMemory(col);
-  
-  ActiveSchemaDB()->getNATableDB()->removeNATable(cn,
-                                                  NATableDB::REMOVE_FROM_ALL_USERS, COM_BASE_TABLE_OBJECT);
+  cliRC = updateObjectRedefTime(&cliInterface,
+                                catalogNamePart, schemaNamePart, objectNamePart,
+                                COM_BASE_TABLE_OBJECT_LIT);
+  if (cliRC < 0)
+    {
+      return;
+    }
+
+  ActiveSchemaDB()->getNATableDB()->removeNATable
+    (cn,
+     NATableDB::REMOVE_FROM_ALL_USERS, COM_BASE_TABLE_OBJECT);
   
   processReturn();
   
@@ -8204,6 +8712,8 @@ Lng32 CmpSeabaseDDL::getSeabaseColumnInfo(ExeCliInterface *cliInterface,
         colInfo.columnClass = COM_SYSTEM_COLUMN;
       else if (strcmp(colClass,COM_ADDED_USER_COLUMN_LIT) == 0)
         colInfo.columnClass = COM_ADDED_USER_COLUMN;
+      else if (strcmp(colClass,COM_ALTERED_USER_COLUMN_LIT) == 0)
+        colInfo.columnClass = COM_ALTERED_USER_COLUMN;
       else if (strcmp(colClass,COM_MV_SYSTEM_ADDED_COLUMN_LIT) == 0)
         colInfo.columnClass = COM_MV_SYSTEM_ADDED_COLUMN;
       else

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/sqlcomp/DefaultConstants.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/DefaultConstants.h b/core/sql/sqlcomp/DefaultConstants.h
index d4acd8d..3a14da0 100644
--- a/core/sql/sqlcomp/DefaultConstants.h
+++ b/core/sql/sqlcomp/DefaultConstants.h
@@ -3796,6 +3796,10 @@ enum DefaultConstants
   // real charset in the HIVE table
   HIVE_FILE_CHARSET,
 
+  // By default only alter of varchar col length is supported.
+  // If this cqd is on, then other alters (name, datatype) are also supported.
+  TRAF_ALTER_COL_ATTRS,
+
   // This enum constant must be the LAST one in the list; it's a count,
   // not an Attribute (it's not IN DefaultDefaults; it's the SIZE of it)!
   __NUM_DEFAULT_ATTRIBUTES

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/sqlcomp/nadefaults.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/nadefaults.cpp b/core/sql/sqlcomp/nadefaults.cpp
index b556100..ff75603 100644
--- a/core/sql/sqlcomp/nadefaults.cpp
+++ b/core/sql/sqlcomp/nadefaults.cpp
@@ -3296,9 +3296,9 @@ XDDkwd__(SUBQUERY_UNNESTING,			"ON"),
   DDint__(TEST_PASS_TWO_ASSERT_TASK_NUMBER,	"-1"),
 
  XDDintN2(TIMEOUT,				"6000"),
-
-  DDflt0_(TMUDF_CARDINALITY_FACTOR, "1"),
-  DDflt0_(TMUDF_LEAF_CARDINALITY, "1"),
+ 
+ DDflt0_(TMUDF_CARDINALITY_FACTOR, "1"),
+ DDflt0_(TMUDF_LEAF_CARDINALITY, "1"),
 
   DDkwd__(TOTAL_RESOURCE_COSTING,               "ON"),
 
@@ -3310,6 +3310,8 @@ XDDkwd__(SUBQUERY_UNNESTING,			"ON"),
 
  DDkwd__(TRAF_ALLOW_SELF_REF_CONSTR,                 "ON"),   
 
+ DDkwd__(TRAF_ALTER_COL_ATTRS,                 "ON"),   
+
  DDkwd__(TRAF_BLOB_AS_VARCHAR,                 "ON"), //set to OFF to enable Lobs support  
 
  DDkwd__(TRAF_BOOTSTRAP_MD_MODE,                            "OFF"),   

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
----------------------------------------------------------------------
diff --git a/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java b/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
index 61f8081..01e15a3 100644
--- a/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
+++ b/core/sql/src/main/java/org/trafodion/sql/HBaseClient.java
@@ -794,12 +794,12 @@ public class HBaseClient {
             return regionInfo;
     }
 
-    public boolean copy(String currTblName, String oldTblName)
+    public boolean copy(String srcTblName, String tgtTblName, boolean force)
 	throws MasterNotRunningException, IOException, SnapshotCreationException, InterruptedException {
-            if (logger.isDebugEnabled()) logger.debug("HBaseClient.copy(" + currTblName + oldTblName + ") called.");
+            if (logger.isDebugEnabled()) logger.debug("HBaseClient.copy(" + srcTblName + tgtTblName + ") called.");
             HBaseAdmin admin = new HBaseAdmin(config);
 	    
-	    String snapshotName = currTblName + "_SNAPSHOT";
+	    String snapshotName = srcTblName + "_SNAPSHOT";
 	    
 	    List<SnapshotDescription> l = new ArrayList<SnapshotDescription>(); 
 	    //	    l = admin.listSnapshots(snapshotName);
@@ -807,28 +807,25 @@ public class HBaseClient {
 	    if (! l.isEmpty())
 		{
 		    for (SnapshotDescription sd : l) {
-			//			System.out.println("here 1");
-			//			System.out.println(snapshotName);
-			//			System.out.println(sd.getName());
 			if (sd.getName().compareTo(snapshotName) == 0)
 			    {
-				//				System.out.println("here 2");
-				//			    admin.enableTable(snapshotName);
-				//				System.out.println("here 3");
 				admin.deleteSnapshot(snapshotName);
-				//				System.out.println("here 4");
 			    }
 		    }
 		}
-	    //	    System.out.println(snapshotName);
-	    if (! admin.isTableDisabled(currTblName))
-		admin.disableTable(currTblName);
-	    //	    System.out.println("here 5");
-	    admin.snapshot(snapshotName, currTblName);
-	    admin.cloneSnapshot(snapshotName, oldTblName);
+
+            if ((force == true) &&
+                (admin.tableExists(tgtTblName))) {
+                admin.disableTable(tgtTblName);
+                admin.deleteTable(tgtTblName);
+            }
+                
+	    if (! admin.isTableDisabled(srcTblName))
+		admin.disableTable(srcTblName);
+	    admin.snapshot(snapshotName, srcTblName);
+	    admin.cloneSnapshot(snapshotName, tgtTblName);
 	    admin.deleteSnapshot(snapshotName);
-	    //	    System.out.println("here 6");
-	    admin.enableTable(currTblName);
+	    admin.enableTable(srcTblName);
             admin.close();
             return true;
     }


[3/3] incubator-trafodion git commit: Merge remote branch 'origin/pr/344/head' into merge_trafodion344

Posted by sa...@apache.org.
Merge remote branch 'origin/pr/344/head' into merge_trafodion344


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

Branch: refs/heads/master
Commit: b2fe5fd7a7a030bcd14adfdb3f45e9a80e5b07d4
Parents: fabfc80 7b99938
Author: Sandhya Sundaresan <sa...@apache.org>
Authored: Tue Mar 1 06:48:58 2016 +0000
Committer: Sandhya Sundaresan <sa...@apache.org>
Committed: Tue Mar 1 06:48:58 2016 +0000

----------------------------------------------------------------------
 core/sql/common/ComSmallDefs.h                  |    2 +
 core/sql/executor/HBaseClient_JNI.cpp           |   25 +-
 core/sql/executor/HBaseClient_JNI.h             |    3 +-
 core/sql/exp/ExpHbaseInterface.cpp              |    8 +-
 core/sql/exp/ExpHbaseInterface.h                |   12 +-
 core/sql/generator/Generator.cpp                |    5 +
 core/sql/optimizer/NAColumn.h                   |   43 +-
 core/sql/optimizer/NATable.cpp                  |    7 +-
 .../sql/parser/StmtDDLAlterTableAlterColumn.cpp |   20 +-
 core/sql/parser/StmtDDLAlterTableAlterColumn.h  |   17 +-
 core/sql/parser/sqlparser.y                     |    6 +-
 core/sql/regress/compGeneral/EXPECTED071        |    8 +-
 core/sql/regress/core/EXPECTED056.SB            |   52 +-
 core/sql/regress/executor/EXPECTED013.SB        |   38 +-
 core/sql/regress/privs2/EXPECTED138             |    2 +-
 core/sql/regress/seabase/EXPECTED011            |  101 +-
 core/sql/regress/seabase/EXPECTED027            |  433 ++++-
 core/sql/regress/seabase/TEST011                |   16 -
 core/sql/regress/seabase/TEST027                |   58 +
 core/sql/sqlcomp/CmpDescribe.cpp                |  180 ++-
 core/sql/sqlcomp/CmpSeabaseDDL.h                |   68 +-
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp        |  217 ++-
 core/sql/sqlcomp/CmpSeabaseDDLschema.cpp        |    4 +-
 core/sql/sqlcomp/CmpSeabaseDDLtable.cpp         | 1504 ++++++++++++------
 core/sql/sqlcomp/DefaultConstants.h             |    4 +
 core/sql/sqlcomp/nadefaults.cpp                 |    8 +-
 .../java/org/trafodion/sql/HBaseClient.java     |   33 +-
 27 files changed, 1974 insertions(+), 900 deletions(-)
----------------------------------------------------------------------



[2/3] incubator-trafodion git commit: JIRA 1844: support for 'alter table alter column datatype'

Posted by sa...@apache.org.
JIRA 1844:  support for 'alter table alter column datatype'


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

Branch: refs/heads/master
Commit: 7b99938cea8ba68f2c90ed655e8f4d85cc24f06f
Parents: fa430a0
Author: Cloud User <ce...@ansharma-3.novalocal>
Authored: Fri Feb 26 23:09:18 2016 +0000
Committer: Cloud User <ce...@ansharma-3.novalocal>
Committed: Fri Feb 26 23:09:18 2016 +0000

----------------------------------------------------------------------
 core/sql/common/ComSmallDefs.h                  |    2 +
 core/sql/executor/HBaseClient_JNI.cpp           |   25 +-
 core/sql/executor/HBaseClient_JNI.h             |    3 +-
 core/sql/exp/ExpHbaseInterface.cpp              |    8 +-
 core/sql/exp/ExpHbaseInterface.h                |   12 +-
 core/sql/generator/Generator.cpp                |    5 +
 core/sql/optimizer/NAColumn.h                   |   43 +-
 core/sql/optimizer/NATable.cpp                  |    7 +-
 .../sql/parser/StmtDDLAlterTableAlterColumn.cpp |   20 +-
 core/sql/parser/StmtDDLAlterTableAlterColumn.h  |   17 +-
 core/sql/parser/sqlparser.y                     |    6 +-
 core/sql/regress/compGeneral/EXPECTED071        |    8 +-
 core/sql/regress/core/EXPECTED056.SB            |   52 +-
 core/sql/regress/executor/EXPECTED013.SB        |   38 +-
 core/sql/regress/privs2/EXPECTED138             |    2 +-
 core/sql/regress/seabase/EXPECTED011            |  101 +-
 core/sql/regress/seabase/EXPECTED027            |  433 ++++-
 core/sql/regress/seabase/TEST011                |   16 -
 core/sql/regress/seabase/TEST027                |   58 +
 core/sql/sqlcomp/CmpDescribe.cpp                |  180 ++-
 core/sql/sqlcomp/CmpSeabaseDDL.h                |   68 +-
 core/sql/sqlcomp/CmpSeabaseDDLcommon.cpp        |  217 ++-
 core/sql/sqlcomp/CmpSeabaseDDLschema.cpp        |    4 +-
 core/sql/sqlcomp/CmpSeabaseDDLtable.cpp         | 1504 ++++++++++++------
 core/sql/sqlcomp/DefaultConstants.h             |    4 +
 core/sql/sqlcomp/nadefaults.cpp                 |    8 +-
 .../java/org/trafodion/sql/HBaseClient.java     |   33 +-
 27 files changed, 1974 insertions(+), 900 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/common/ComSmallDefs.h
----------------------------------------------------------------------
diff --git a/core/sql/common/ComSmallDefs.h b/core/sql/common/ComSmallDefs.h
index 1c888fb..3252eac 100644
--- a/core/sql/common/ComSmallDefs.h
+++ b/core/sql/common/ComSmallDefs.h
@@ -560,6 +560,7 @@ enum ComColumnClass { COM_UNKNOWN_CLASS
                     , COM_USER_COLUMN
                     , COM_ADDED_USER_COLUMN
                     , COM_MV_SYSTEM_ADDED_COLUMN
+                    , COM_ALTERED_USER_COLUMN
                     };
 
 #define COM_UNKNOWN_CLASS_LIT               "  "
@@ -567,6 +568,7 @@ enum ComColumnClass { COM_UNKNOWN_CLASS
 #define COM_USER_COLUMN_LIT                 "U "
 #define COM_ADDED_USER_COLUMN_LIT           "A "
 #define COM_MV_SYSTEM_ADDED_COLUMN_LIT      "M "
+#define COM_ALTERED_USER_COLUMN_LIT         "C "
 
 enum ComColumnDefaultClass { COM_CURRENT_DEFAULT
                            , COM_NO_DEFAULT

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/executor/HBaseClient_JNI.cpp
----------------------------------------------------------------------
diff --git a/core/sql/executor/HBaseClient_JNI.cpp b/core/sql/executor/HBaseClient_JNI.cpp
index 3ff3e1f..34ab08f 100644
--- a/core/sql/executor/HBaseClient_JNI.cpp
+++ b/core/sql/executor/HBaseClient_JNI.cpp
@@ -452,7 +452,7 @@ HBC_RetCode HBaseClient_JNI::init()
     JavaMethods_[JM_GET_REGION_STATS       ].jm_name      = "getRegionStats";
     JavaMethods_[JM_GET_REGION_STATS       ].jm_signature = "(Ljava/lang/String;)Lorg/trafodion/sql/ByteArrayList;";
     JavaMethods_[JM_COPY       ].jm_name      = "copy";
-    JavaMethods_[JM_COPY       ].jm_signature = "(Ljava/lang/String;Ljava/lang/String;)Z";
+    JavaMethods_[JM_COPY       ].jm_signature = "(Ljava/lang/String;Ljava/lang/String;Z)Z";
     JavaMethods_[JM_EXISTS     ].jm_name      = "exists";
     JavaMethods_[JM_EXISTS     ].jm_signature = "(Ljava/lang/String;)Z";
     JavaMethods_[JM_GRANT      ].jm_name      = "grant";
@@ -1523,9 +1523,11 @@ ByteArrayList* HBaseClient_JNI::getRegionStats(const char* tblName)
 //////////////////////////////////////////////////////////////////////////////
 // 
 //////////////////////////////////////////////////////////////////////////////
-HBC_RetCode HBaseClient_JNI::copy(const char* currTblName, const char* oldTblName)
+HBC_RetCode HBaseClient_JNI::copy(const char* srcTblName, 
+                                  const char* tgtTblName,
+                                  NABoolean force)
 {
-  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HBaseClient_JNI::copy(%s,%s) called.", currTblName, oldTblName);
+  QRLogger::log(CAT_SQL_HBASE, LL_DEBUG, "HBaseClient_JNI::copy(%s,%s) called.", srcTblName, tgtTblName);
   if (jenv_ == NULL)
      if (initJVM() != JOI_OK)
          return HBC_ERROR_INIT_PARAM;
@@ -1534,28 +1536,31 @@ HBC_RetCode HBaseClient_JNI::copy(const char* currTblName, const char* oldTblNam
      getExceptionDetails();
      return HBC_ERROR_DROP_EXCEPTION;
   }
-  jstring js_currTblName = jenv_->NewStringUTF(currTblName);
-  if (js_currTblName == NULL) 
+  jstring js_srcTblName = jenv_->NewStringUTF(srcTblName);
+  if (js_srcTblName == NULL) 
   {
     GetCliGlobals()->setJniErrorStr(getErrorText(HBC_ERROR_DROP_PARAM));
     jenv_->PopLocalFrame(NULL);
     return HBC_ERROR_DROP_PARAM;
   }
 
-  jstring js_oldTblName = jenv_->NewStringUTF(oldTblName);
-  if (js_oldTblName == NULL) 
+  jstring js_tgtTblName = jenv_->NewStringUTF(tgtTblName);
+  if (js_tgtTblName == NULL) 
   {
     GetCliGlobals()->setJniErrorStr(getErrorText(HBC_ERROR_DROP_PARAM));
     jenv_->PopLocalFrame(NULL);
     return HBC_ERROR_DROP_PARAM;
   }
 
+  jboolean j_force = force;
   tsRecentJMFromJNI = JavaMethods_[JM_COPY].jm_full_name;
-  jboolean jresult = jenv_->CallBooleanMethod(javaObj_, JavaMethods_[JM_COPY].methodID, js_currTblName, js_oldTblName);
+  jboolean jresult = jenv_->CallBooleanMethod(
+       javaObj_, JavaMethods_[JM_COPY].methodID, 
+       js_srcTblName, js_tgtTblName, j_force);
 
-  jenv_->DeleteLocalRef(js_currTblName);  
+  jenv_->DeleteLocalRef(js_srcTblName);  
 
-  jenv_->DeleteLocalRef(js_oldTblName);  
+  jenv_->DeleteLocalRef(js_tgtTblName);  
 
   if (jenv_->ExceptionCheck())
   {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/executor/HBaseClient_JNI.h
----------------------------------------------------------------------
diff --git a/core/sql/executor/HBaseClient_JNI.h b/core/sql/executor/HBaseClient_JNI.h
index d015ec2..1ecd0e2 100644
--- a/core/sql/executor/HBaseClient_JNI.h
+++ b/core/sql/executor/HBaseClient_JNI.h
@@ -511,7 +511,8 @@ public:
   HBC_RetCode drop(const char* fileName, bool async, Int64 transID);
   HBC_RetCode drop(const char* fileName, JNIEnv* jenv, Int64 transID); // thread specific
   HBC_RetCode dropAll(const char* pattern, bool async);
-  HBC_RetCode copy(const char* currTblName, const char* oldTblName);
+  HBC_RetCode copy(const char* srcTblName, const char* tgtTblName,
+                   NABoolean force);
   ByteArrayList* listAll(const char* pattern);
   ByteArrayList* getRegionStats(const char* tblName);
   static HBC_RetCode flushAllTablesStatic();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/exp/ExpHbaseInterface.cpp
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpHbaseInterface.cpp b/core/sql/exp/ExpHbaseInterface.cpp
index b58383c..3ca8d8c 100644
--- a/core/sql/exp/ExpHbaseInterface.cpp
+++ b/core/sql/exp/ExpHbaseInterface.cpp
@@ -249,7 +249,8 @@ Lng32  ExpHbaseInterface::fetchAllRows(
   return retcode;
 }
 
-Lng32 ExpHbaseInterface::copy(HbaseStr &currTblName, HbaseStr &oldTblName)
+Lng32 ExpHbaseInterface::copy(HbaseStr &srcTblName, HbaseStr &tgtTblName,
+                              NABoolean force)
 {
   return -HBASE_COPY_ERROR;
 }
@@ -582,7 +583,8 @@ ByteArrayList* ExpHbaseInterface_JNI::listAll(const char * pattern)
 }
 
 //----------------------------------------------------------------------------
-Lng32 ExpHbaseInterface_JNI::copy(HbaseStr &currTblName, HbaseStr &oldTblName)
+Lng32 ExpHbaseInterface_JNI::copy(HbaseStr &srcTblName, HbaseStr &tgtTblName,
+                                  NABoolean force)
 {
   if (client_ == NULL)
   {
@@ -590,7 +592,7 @@ Lng32 ExpHbaseInterface_JNI::copy(HbaseStr &currTblName, HbaseStr &oldTblName)
       return -HBASE_ACCESS_ERROR;
   }
     
-  retCode_ = client_->copy(currTblName.val, oldTblName.val);
+  retCode_ = client_->copy(srcTblName.val, tgtTblName.val, force);
 
   if (retCode_ == HBC_OK)
     return HBASE_ACCESS_SUCCESS;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/exp/ExpHbaseInterface.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpHbaseInterface.h b/core/sql/exp/ExpHbaseInterface.h
index 474503c..3a61e22 100644
--- a/core/sql/exp/ExpHbaseInterface.h
+++ b/core/sql/exp/ExpHbaseInterface.h
@@ -135,8 +135,10 @@ class ExpHbaseInterface : public NABasicObject
   // retrieve all objects from hbase that match the pattern
   virtual ByteArrayList* listAll(const char * pattern) = 0;
 
-  // make a copy of currTableName as oldTblName.
-  virtual Lng32 copy(HbaseStr &currTblName, HbaseStr &oldTblName);
+  // make a copy of srcTblName as tgtTblName
+  // if force is true, remove target before copying.
+  virtual Lng32 copy(HbaseStr &srcTblName, HbaseStr &tgtTblName,
+                     NABoolean force = FALSE);
 
   virtual Lng32 exists(HbaseStr &tblName) = 0;
 
@@ -448,8 +450,10 @@ class ExpHbaseInterface_JNI : public ExpHbaseInterface
 
   virtual ByteArrayList* listAll(const char * pattern);
 
-  // make a copy of currTableName as oldTblName.
-  virtual Lng32 copy(HbaseStr &currTblName, HbaseStr &oldTblName);
+  // make a copy of srcTblName as tgtTblName
+  // if force is true, remove target before copying.
+  virtual Lng32 copy(HbaseStr &srcTblName, HbaseStr &tgtTblName,
+                     NABoolean force = FALSE);
 
   // -1, if table exists. 0, if doesn't. -ve num, error.
   virtual Lng32 exists(HbaseStr &tblName);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/generator/Generator.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/Generator.cpp b/core/sql/generator/Generator.cpp
index 9c8cee3..2486755 100644
--- a/core/sql/generator/Generator.cpp
+++ b/core/sql/generator/Generator.cpp
@@ -1594,6 +1594,11 @@ desc_struct* Generator::createColDescs(
 	  col_desc->body.columns_desc.colclass = 'A';
 	  col_desc->body.columns_desc.addedColumn = 1;
 	}
+      else if (info->columnClass == COM_ALTERED_USER_COLUMN)
+	{
+	  col_desc->body.columns_desc.colclass = 'C';
+	  col_desc->body.columns_desc.addedColumn = 1;
+	}
 
       if (info->colHeading)
 	col_desc->body.columns_desc.heading = (char*)info->colHeading;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/optimizer/NAColumn.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NAColumn.h b/core/sql/optimizer/NAColumn.h
index 6aeac49..ec58fe9 100644
--- a/core/sql/optimizer/NAColumn.h
+++ b/core/sql/optimizer/NAColumn.h
@@ -88,23 +88,24 @@ public:
   // ---------------------------------------------------------------------
 
   NAColumn(const char* colName,
-    Lng32 position,
-    NAType *type,
-    CollHeap *h,
-    const NATable* table = NULL,
-    ColumnClass columnClass = USER_COLUMN,
-    const ComColumnDefaultClass defaultClass = COM_NO_DEFAULT,
-    char* defaultValue = NULL,
-    char* heading = NULL,
-    NABoolean upshift = FALSE,
-    NABoolean addedColumn = FALSE,
-    ComColumnDirection colDirection = COM_UNKNOWN_DIRECTION,
-    NABoolean isOptional = FALSE,
-    char *routineParamType = NULL,
-    NABoolean storedOnDisk = TRUE,
-    char *computedColExpr = NULL,
-    NABoolean isSaltColumn = FALSE,
-    NABoolean isDivisioningColumn = FALSE)
+           Lng32 position,
+           NAType *type,
+           CollHeap *h,
+           const NATable* table = NULL,
+           ColumnClass columnClass = USER_COLUMN,
+           const ComColumnDefaultClass defaultClass = COM_NO_DEFAULT,
+           char* defaultValue = NULL,
+           char* heading = NULL,
+           NABoolean upshift = FALSE,
+           NABoolean addedColumn = FALSE,
+           ComColumnDirection colDirection = COM_UNKNOWN_DIRECTION,
+           NABoolean isOptional = FALSE,
+           char *routineParamType = NULL,
+           NABoolean storedOnDisk = TRUE,
+           char *computedColExpr = NULL,
+           NABoolean isSaltColumn = FALSE,
+           NABoolean isDivisioningColumn = FALSE,
+           NABoolean isAlteredColumn = FALSE)
   : heap_(h),
     colName_(colName, h),
     position_(position),
@@ -116,6 +117,7 @@ public:
     heading_(heading),
     upshift_(upshift),
     addedColumn_(addedColumn),
+    alteredColumn_(isAlteredColumn),
     keyKind_(NON_KEY),
     clusteringKeyOrdering_(NOT_ORDERED),
     isNotNullNondroppable_(NULL),
@@ -154,6 +156,7 @@ public:
     heading_(nac.heading_),
     upshift_(nac.upshift_),
     addedColumn_(nac.addedColumn_),
+    alteredColumn_(nac.alteredColumn_),
     keyKind_(nac.keyKind_),
     clusteringKeyOrdering_(nac.clusteringKeyOrdering_),
     isNotNullNondroppable_(nac.isNotNullNondroppable_),
@@ -242,6 +245,7 @@ public:
   inline const char* getComputedColumnExprString() const { return computedColumnExpression_; }
   inline NABoolean isStoredOnDisk() const       { return storedOnDisk_; }
   inline NABoolean isAddedColumn() const { return addedColumn_; }
+  inline NABoolean isAlteredColumn() const { return alteredColumn_; }
   inline NABoolean isSaltColumn() const        { return isSaltColumn_;}
   inline NABoolean isDivisioningColumn() const { return isDivisioningColumn_; }
 
@@ -518,6 +522,11 @@ private:
   NABoolean addedColumn_;
 
   // ----------------------------------------------------
+  // Set to TRUE if this column was altered by datatype change
+  // ----------------------------------------------------
+  NABoolean alteredColumn_;
+
+  // ----------------------------------------------------
   // Set to TRUE if there is a join predicate on this column.
   // ----------------------------------------------------
   NABoolean hasJoinPred_;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/optimizer/NATable.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NATable.cpp b/core/sql/optimizer/NATable.cpp
index 88036c6..0e59dcf 100644
--- a/core/sql/optimizer/NATable.cpp
+++ b/core/sql/optimizer/NATable.cpp
@@ -3379,6 +3379,7 @@ NABoolean createNAColumns(desc_struct *column_desc_list	/*IN*/,
 	  colClass = USER_COLUMN;
 	  break;
         case 'A':
+        case 'C':
 	  colClass = USER_COLUMN;
 	  break;
         case 'M':  // MVs --
@@ -3454,14 +3455,16 @@ NABoolean createNAColumns(desc_struct *column_desc_list	/*IN*/,
 			       defaultValue,
                                heading,
 			       column_desc->upshift,
-			       (column_desc->colclass == 'A'),
+			       ((column_desc->colclass == 'A') ||
+                                (column_desc->colclass == 'C')),
                                COM_UNKNOWN_DIRECTION,
                                FALSE,
                                NULL,
                                column_desc->stored_on_disk,
                                computed_column_text,
                                isSaltColumn,
-                               isDivisioningColumn);
+                               isDivisioningColumn,
+                               (column_desc->colclass == 'C'));
 	}
       else
         {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/parser/StmtDDLAlterTableAlterColumn.cpp
----------------------------------------------------------------------
diff --git a/core/sql/parser/StmtDDLAlterTableAlterColumn.cpp b/core/sql/parser/StmtDDLAlterTableAlterColumn.cpp
index cf994d2..b2c1f9c 100644
--- a/core/sql/parser/StmtDDLAlterTableAlterColumn.cpp
+++ b/core/sql/parser/StmtDDLAlterTableAlterColumn.cpp
@@ -55,16 +55,22 @@ StmtDDLAlterTableAlterColumn::getText() const
 //----------------------------------------------------------------------------
 // CLASS StmtDDLAlterTableAlterColumnDatatype
 //----------------------------------------------------------------------------
-StmtDDLAlterTableAlterColumnDatatype::StmtDDLAlterTableAlterColumnDatatype( 
-     const NAString &columnName 
-     , NAType * natype
-     , CollHeap *heap)
+StmtDDLAlterTableAlterColumnDatatype::StmtDDLAlterTableAlterColumnDatatype(
+     ElemDDLNode * pColumnToAlter
+     ,CollHeap    * heap)
      : StmtDDLAlterTableAlterColumn(DDL_ALTER_TABLE_ALTER_COLUMN_DATATYPE,
-                                    columnName,
+                                    NAString(""),
                                     NULL,
-                                    heap)
+                                    heap),
+       pColumnToAlter_(pColumnToAlter)
 {
-  natype_ = natype->newCopy(heap);
+  ElemDDLColDef *pColDef = pColumnToAlter->castToElemDDLColDef();
+  if (pColDef NEQ NULL)
+    {
+      getColDefArray().insert(pColDef);
+    }
+  else
+    *SqlParser_Diags << DgSqlCode(-1001);
 }
 
 //

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/parser/StmtDDLAlterTableAlterColumn.h
----------------------------------------------------------------------
diff --git a/core/sql/parser/StmtDDLAlterTableAlterColumn.h b/core/sql/parser/StmtDDLAlterTableAlterColumn.h
index 92fe23f..7ad20de 100644
--- a/core/sql/parser/StmtDDLAlterTableAlterColumn.h
+++ b/core/sql/parser/StmtDDLAlterTableAlterColumn.h
@@ -144,9 +144,8 @@ class StmtDDLAlterTableAlterColumnDatatype : public StmtDDLAlterTableAlterColumn
 public:
 
   // constructor
-  StmtDDLAlterTableAlterColumnDatatype( const NAString &columnName
-                                        , NAType * natype
-                                          , CollHeap    *heap = PARSERHEAP());
+  StmtDDLAlterTableAlterColumnDatatype( ElemDDLNode * pColumnToAlter
+                                        ,CollHeap    * heap = PARSERHEAP());
 
     // virtual destructor
   virtual ~StmtDDLAlterTableAlterColumnDatatype();
@@ -157,12 +156,17 @@ public:
   // method for tracing
   virtual const NAString getText() const;
 
-  const NAType * getDatatype() const { return natype_; }
-   NAType * getType() { return natype_; }
+  inline ElemDDLNode * getColToAlter() { return pColumnToAlter_; };
+  inline ElemDDLColDefArray & getColDefArray() { return columnDefArray_; };
 
 private: 
 
-  NAType * natype_;
+  // column definition
+  ElemDDLNode * pColumnToAlter_;
+
+  // list of (only one) column definition
+  ElemDDLColDefArray columnDefArray_;
+
   //
   // please do not use the following methods
   //
@@ -174,5 +178,4 @@ private:
 
 }; // class StmtDDLAlterTableAlterColumnDatatype
 
-
 #endif //STMTDDLALTERTABLEALTERCOLUMN_H

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/parser/sqlparser.y
----------------------------------------------------------------------
diff --git a/core/sql/parser/sqlparser.y b/core/sql/parser/sqlparser.y
index 4e8033d..e565f14 100755
--- a/core/sql/parser/sqlparser.y
+++ b/core/sql/parser/sqlparser.y
@@ -31229,13 +31229,11 @@ alter_table_column_clause : TOK_COLUMN identifier heading
                                 }
 
 // type pStmtDDL
-alter_table_alter_column_datatype : TOK_ALTER TOK_COLUMN column_name predefined_type
+alter_table_alter_column_datatype : TOK_ALTER TOK_COLUMN column_definition
 				{
                                   $$ = new (PARSERHEAP())
                                     StmtDDLAlterTableAlterColumnDatatype(
-                                         *$3, // column name
-                                         $4 );
-                                  delete $3;
+                                         $3 /* column definition */);
                                   restoreInferCharsetState();
 				}
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/compGeneral/EXPECTED071
----------------------------------------------------------------------
diff --git a/core/sql/regress/compGeneral/EXPECTED071 b/core/sql/regress/compGeneral/EXPECTED071
index a5a4d39..8c1813c 100644
--- a/core/sql/regress/compGeneral/EXPECTED071
+++ b/core/sql/regress/compGeneral/EXPECTED071
@@ -162,8 +162,8 @@ MTDTRIGTARGET2D  BT        0  STORE_ID                        0       4  U
 MTDTRIGTARGET2D  BT        1  ITEM_ID                         0       4  U                   1  3         none                                                                                                                                                                                                                                            
 MTDTRIGTARGET2D  BT        2  SALE_DATE                       0       4  U                   3  4         none                                                                                                                                                                                                                                            
 MTDTRIGTARGET2D  BT        3  SALE_AMT                        ?       8  U                   2  none      none                                                                                                                                                                                                                                            
-MTDTRIGTARGET2D  BT        4  _DIVISION_1_                    0       4  S                   7  1         DATE_PART('YEARMONTH',SALE_DATE)                                                                                                                                                                                                                
-MTDTRIGTARGET2D  BT        5  SALE_AMT_BEFORE                 ?       8  A                   3  none      none                                                                                                                                                                                                                                            
+MTDTRIGTARGET2D  BT        4  SALE_AMT_BEFORE                 ?       8  A                   3  none      none                                                                                                                                                                                                                                            
+MTDTRIGTARGET2D  BT        5  _DIVISION_1_                    0       4  S                   7  1         DATE_PART('YEARMONTH',SALE_DATE)                                                                                                                                                                                                                
 
 --- 49 row(s) selected.
 >>
@@ -1414,8 +1414,8 @@ MTDTRIGTARGET2D  BT        0  STORE_ID                        0       4  U
 MTDTRIGTARGET2D  BT        1  ITEM_ID                         0       4  U                   1  3         none                                                                                                                                                                                                                                            
 MTDTRIGTARGET2D  BT        2  SALE_DATE                       0       4  U                   3  4         none                                                                                                                                                                                                                                            
 MTDTRIGTARGET2D  BT        3  SALE_AMT                        ?       8  U                   2  none      none                                                                                                                                                                                                                                            
-MTDTRIGTARGET2D  BT        4  _DIVISION_1_                    0       4  S                   7  1         DATE_PART('YEARMONTH',SALE_DATE)                                                                                                                                                                                                                
-MTDTRIGTARGET2D  BT        5  SALE_AMT_BEFORE                 ?       8  A                   3  none      none                                                                                                                                                                                                                                            
+MTDTRIGTARGET2D  BT        4  SALE_AMT_BEFORE                 ?       8  A                   3  none      none                                                                                                                                                                                                                                            
+MTDTRIGTARGET2D  BT        5  _DIVISION_1_                    0       4  S                   7  1         DATE_PART('YEARMONTH',SALE_DATE)                                                                                                                                                                                                                
 
 --- 70 row(s) selected.
 >>prepare smdviewquery from

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/core/EXPECTED056.SB
----------------------------------------------------------------------
diff --git a/core/sql/regress/core/EXPECTED056.SB b/core/sql/regress/core/EXPECTED056.SB
index 4ddfcd4..f1ebf88 100755
--- a/core/sql/regress/core/EXPECTED056.SB
+++ b/core/sql/regress/core/EXPECTED056.SB
@@ -1,14 +1,14 @@
 >>obey TEST056(tests1);
 >>showddl T056t10;
 
-CREATE TABLE SEABASE.SCH.T056T10
+CREATE TABLE TRAFODION.SCH.T056T10
   (
     SWALLOW                          INT DEFAULT 121 NOT NULL NOT DROPPABLE
   , BARNOWL                          INT DEFAULT 1000 NOT NULL NOT DROPPABLE
   )
 ;
 
-CREATE UNIQUE INDEX T056IN5 ON SEABASE.SCH.T056T10
+CREATE UNIQUE INDEX T056IN5 ON TRAFODION.SCH.T056T10
   (
     BARNOWL ASC
   )
@@ -22,7 +22,7 @@ CREATE UNIQUE INDEX T056IN5 ON SEABASE.SCH.T056T10
 >>--showlabel index T056in5,detail;
 >>showddl T056t11;
 
-CREATE TABLE SEABASE.SCH.T056T11
+CREATE TABLE TRAFODION.SCH.T056T11
   (
     WREN                             INT NO DEFAULT NOT NULL NOT DROPPABLE
   , JUNCO                            INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -31,7 +31,7 @@ CREATE TABLE SEABASE.SCH.T056T11
   )
 ;
 
-CREATE UNIQUE INDEX T056IN6 ON SEABASE.SCH.T056T11
+CREATE UNIQUE INDEX T056IN6 ON TRAFODION.SCH.T056T11
   (
     BLACKBIRD ASC
   )
@@ -102,14 +102,14 @@ CREATE UNIQUE INDEX T056IN6 ON SEABASE.SCH.T056T11
 >>
 >>showddl T056t10;
 
-CREATE TABLE SEABASE.SCH.T056T10
+CREATE TABLE TRAFODION.SCH.T056T10
   (
     SWALLOW                          INT DEFAULT 121 NOT NULL NOT DROPPABLE
   , BARNOWL                          INT DEFAULT 1000 NOT NULL NOT DROPPABLE
-  , DUCK                             INT DEFAULT 20 NOT NULL NOT DROPPABLE /*
-      added col */
-  , GOSLING                          INT DEFAULT 700 HEADING 'goose' /* added
-      col */
+  , DUCK                             INT DEFAULT 20 NOT NULL NOT DROPPABLE
+      /*added_col*/
+  , GOSLING                          INT DEFAULT 700 HEADING 'goose'
+      /*added_col*/
   )
 ;
 
@@ -135,17 +135,17 @@ ALTER TABLE TRAFODION.SCH.T056T10 ADD CONSTRAINT TRAFODION.SCH.DUCK_PK UNIQUE
 --- SQL operation complete.
 >>showddl T056t11;
 
-CREATE TABLE SEABASE.SCH.T056T11
+CREATE TABLE TRAFODION.SCH.T056T11
   (
     WREN                             INT NO DEFAULT NOT NULL NOT DROPPABLE
   , JUNCO                            INT NO DEFAULT NOT NULL NOT DROPPABLE
   , BLACKBIRD                        INT DEFAULT NULL
-  , BLUEJAY                          INT DEFAULT 203 NOT NULL NOT DROPPABLE /*
-      added col */
-  , RAVEN                            INT DEFAULT NULL /* added col */
-  , SEAGULL                          INT DEFAULT 400 NOT NULL NOT DROPPABLE /*
-      added col */
-  , ROBIN                            INT DEFAULT 410 /* added col */
+  , BLUEJAY                          INT DEFAULT 203 NOT NULL NOT DROPPABLE
+      /*added_col*/
+  , RAVEN                            INT DEFAULT NULL /*added_col*/
+  , SEAGULL                          INT DEFAULT 400 NOT NULL NOT DROPPABLE
+      /*added_col*/
+  , ROBIN                            INT DEFAULT 410 /*added_col*/
   , PRIMARY KEY (WREN ASC, JUNCO ASC)
   )
 ;
@@ -185,10 +185,10 @@ ALTER TABLE TRAFODION.SCH.T056T11 ADD CONSTRAINT TRAFODION.SCH.T056T11_CK
 >>#ifMX
 >>showddl T056vw12;
 
-CREATE VIEW SEABASE.SCH.T056VW12 AS
-  SELECT SEABASE.SCH.T056T10.DUCK, SEABASE.SCH.T056T11.RAVEN,
-    SEABASE.SCH.T056T10.SWALLOW, SEABASE.SCH.T056T11.WREN FROM
-    SEABASE.SCH.T056T10, SEABASE.SCH.T056T11 ;
+CREATE VIEW TRAFODION.SCH.T056VW12 AS
+  SELECT TRAFODION.SCH.T056T10.DUCK, TRAFODION.SCH.T056T11.RAVEN,
+    TRAFODION.SCH.T056T10.SWALLOW, TRAFODION.SCH.T056T11.WREN FROM
+    TRAFODION.SCH.T056T10, TRAFODION.SCH.T056T11 ;
 
 --- SQL operation complete.
 >>#ifMX
@@ -2030,7 +2030,7 @@ T056T56_COL            T056T56_COL2  T056T56_COL3  T056T56_COL4
 --- SQL operation complete.
 >>showddl t056t57;
 
-CREATE TABLE SEABASE.SCH.T056T57
+CREATE TABLE TRAFODION.SCH.T056T57
   (
     A1                               NUMERIC(2, 2) DEFAULT 0 NOT NULL NOT
       DROPPABLE
@@ -2055,7 +2055,7 @@ A1
 --- SQL operation complete.
 >>showddl t056t58;
 
-CREATE TABLE SEABASE.SCH.T056T58
+CREATE TABLE TRAFODION.SCH.T056T58
   (
     A1                               DECIMAL(4, 4) DEFAULT -0.0000 NOT NULL NOT
       DROPPABLE
@@ -2080,7 +2080,7 @@ A1
 --- SQL operation complete.
 >>showddl t056t59;
 
-CREATE TABLE SEABASE.SCH.T056T59
+CREATE TABLE TRAFODION.SCH.T056T59
   (
     A1                               DECIMAL(3, 3) DEFAULT -0000.12 NOT NULL
       NOT DROPPABLE
@@ -2105,7 +2105,7 @@ A1
 --- SQL operation complete.
 >>showddl t056t60;
 
-CREATE TABLE SEABASE.SCH.T056T60
+CREATE TABLE TRAFODION.SCH.T056T60
   (
     A1                               NUMERIC(2, 2) UNSIGNED DEFAULT 0.00 NOT
       NULL NOT DROPPABLE
@@ -2130,7 +2130,7 @@ A1
 --- SQL operation complete.
 >>showddl t056t61;
 
-CREATE TABLE SEABASE.SCH.T056T61
+CREATE TABLE TRAFODION.SCH.T056T61
   (
     A1                               NUMERIC(2, 2) DEFAULT -0.0 NOT NULL NOT
       DROPPABLE
@@ -2918,7 +2918,7 @@ I                     VCH1  VCH2  VCH3
 >>obey test056(test6NullVarchar);
 >>drop table t056t64;
 
-*** ERROR[1389] Object SEABASE.SCH.T056T64 does not exist in SeaBase.
+*** ERROR[1389] Object TRAFODION.SCH.T056T64 does not exist in Trafodion.
 
 --- SQL operation failed with errors.
 >>create table t056t64

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/executor/EXPECTED013.SB
----------------------------------------------------------------------
diff --git a/core/sql/regress/executor/EXPECTED013.SB b/core/sql/regress/executor/EXPECTED013.SB
index e4faaec..0b31ff5 100644
--- a/core/sql/regress/executor/EXPECTED013.SB
+++ b/core/sql/regress/executor/EXPECTED013.SB
@@ -21,7 +21,7 @@
 >>invoke t013t1;
 
 -- Definition of Trafodion table TRAFODION.T013_SCH.T013T1
--- Definition current  Sat Jan 23 02:44:37 2016
+-- Definition current  Sun Feb 21 07:35:15 2016
 
   (
     A                                INT DEFAULT NULL
@@ -49,7 +49,7 @@ A
 >>invoke t013t1;
 
 -- Definition of Trafodion volatile table T013T1
--- Definition current  Sat Jan 23 02:44:54 2016
+-- Definition current  Sun Feb 21 07:35:27 2016
 
   (
     A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -111,7 +111,7 @@ A
 >>invoke t013t1;
 
 -- Definition of Trafodion table TRAFODION.T013_SCH.T013T1
--- Definition current  Sat Jan 23 02:45:57 2016
+-- Definition current  Sun Feb 21 07:36:26 2016
 
   (
     A                                INT DEFAULT NULL
@@ -136,7 +136,7 @@ A
 >>invoke t013t1;
 
 -- Definition of Trafodion volatile table T013T1
--- Definition current  Sat Jan 23 02:46:10 2016
+-- Definition current  Sun Feb 21 07:36:38 2016
 
   (
     A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -175,7 +175,7 @@ A            B            C
 >>invoke t013t1;
 
 -- Definition of Trafodion table TRAFODION.T013_SCH.T013T1
--- Definition current  Sat Jan 23 02:46:25 2016
+-- Definition current  Sun Feb 21 07:36:52 2016
 
   (
     A                                INT DEFAULT NULL
@@ -197,7 +197,7 @@ A
 >>invoke t013t1;
 
 -- Definition of Trafodion volatile table T013T1
--- Definition current  Sat Jan 23 02:46:32 2016
+-- Definition current  Sun Feb 21 07:36:58 2016
 
   (
     A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -448,12 +448,12 @@ CONTROL QUERY DEFAULT
 >>invoke t013t1;
 
 -- Definition of Trafodion table TRAFODION.T013_SCH.T013T1
--- Definition current  Sat Jan 23 02:47:38 2016
+-- Definition current  Sun Feb 21 07:37:55 2016
 
   (
     A                                INT DEFAULT NULL
-  , Z                                INT DEFAULT 0 NOT NULL NOT DROPPABLE /*
-      added col */
+  , Z                                INT DEFAULT 0 NOT NULL NOT DROPPABLE
+      /*added_col*/
   )
 
 --- SQL operation complete.
@@ -468,12 +468,12 @@ CONTROL QUERY DEFAULT
 >>invoke t013t1;
 
 -- Definition of Trafodion table TRAFODION.T013_SCH.T013T1
--- Definition current  Sat Jan 23 02:47:43 2016
+-- Definition current  Sun Feb 21 07:38:00 2016
 
   (
     A                                INT DEFAULT NULL
-  , Z                                INT DEFAULT 0 NOT NULL NOT DROPPABLE /*
-      added col */
+  , Z                                INT DEFAULT 0 NOT NULL NOT DROPPABLE
+      /*added_col*/
   )
 
 --- SQL operation complete.
@@ -559,7 +559,7 @@ control query shape nested_join(anything,anything);
 >>invoke t013t3;
 
 -- Definition of Trafodion table TRAFODION.T013SCH1.T013T3
--- Definition current  Sat Jan 23 02:48:28 2016
+-- Definition current  Sun Feb 21 07:38:45 2016
 
   (
     A                                INT DEFAULT NULL
@@ -576,7 +576,7 @@ control query shape nested_join(anything,anything);
 >>invoke t013t3;
 
 -- Definition of Trafodion volatile table T013T3
--- Definition current  Sat Jan 23 02:48:35 2016
+-- Definition current  Sun Feb 21 07:38:50 2016
 
   (
     A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -591,7 +591,7 @@ control query shape nested_join(anything,anything);
 >>invoke t013t3;
 
 -- Definition of Trafodion table TRAFODION.T013SCH1.T013T3
--- Definition current  Sat Jan 23 02:48:47 2016
+-- Definition current  Sun Feb 21 07:39:03 2016
 
   (
     A                                INT DEFAULT NULL
@@ -656,7 +656,7 @@ CREATE VOLATILE TABLE T013T3
 >>invoke t013t3;
 
 -- Definition of Trafodion table TRAFODION.T013SCH1.T013T3
--- Definition current  Sat Jan 23 02:49:31 2016
+-- Definition current  Sun Feb 21 07:39:46 2016
 
   (
     A                                INT DEFAULT NULL
@@ -1277,12 +1277,12 @@ CREATE VOLATILE TABLE T013T1
 >>invoke t013t1;
 
 -- Definition of Trafodion table TRAFODION.T013_SCH.T013T1
--- Definition current  Sat Jan 23 02:55:57 2016
+-- Definition current  Sun Feb 21 07:46:01 2016
 
   (
     A                                INT DEFAULT NULL
-  , Z                                INT DEFAULT 0 NOT NULL NOT DROPPABLE /*
-      added col */
+  , Z                                INT DEFAULT 0 NOT NULL NOT DROPPABLE
+      /*added_col*/
   )
 
 --- SQL operation complete.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/privs2/EXPECTED138
----------------------------------------------------------------------
diff --git a/core/sql/regress/privs2/EXPECTED138 b/core/sql/regress/privs2/EXPECTED138
index a67e390..92caae5 100644
--- a/core/sql/regress/privs2/EXPECTED138
+++ b/core/sql/regress/privs2/EXPECTED138
@@ -830,7 +830,7 @@ CREATE TABLE TRAFODION.T138SCH.USER1_T1
   (
     C1                               INT NO DEFAULT NOT NULL NOT DROPPABLE
   , C2                               INT DEFAULT NULL
-  , C3                               INT DEFAULT 0 /* added col */
+  , C3                               INT DEFAULT 0 /*added_col*/
   , PRIMARY KEY (C1 ASC)
   )
 ;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/seabase/EXPECTED011
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED011 b/core/sql/regress/seabase/EXPECTED011
index 012f55f..61031c9 100644
--- a/core/sql/regress/seabase/EXPECTED011
+++ b/core/sql/regress/seabase/EXPECTED011
@@ -7,7 +7,7 @@
 >>invoke T011T1;
 
 -- Definition of Trafodion table TRAFODION.SCH.T011T1
--- Definition current  Sat Jan 23 04:06:15 2016
+-- Definition current  Wed Feb 24 09:06:39 2016
 
   (
     A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -142,7 +142,7 @@ A            B
 ------------------------------------------------------------------ PLAN SUMMARY
 MODULE_NAME .............. DYNAMICALLY COMPILED
 STATEMENT_NAME ........... NOT NAMED
-PLAN_ID .................. 212320281977039065
+PLAN_ID .................. 212323064800054288
 ROWS_OUT ................. 7
 EST_TOTAL_COST ........... 0.01
 STATEMENT ................ select * from t011t1 where a >= 3 and a < 5;
@@ -181,7 +181,7 @@ DESCRIPTION
   SCHEMA ................. TRAFODION.SCH
   QUERY_CACHE ............ 0
   GENERATE_EXPLAIN ....... ON
-  ObjectUIDs ............. 3130652840489752733
+  ObjectUIDs ............. 2452863879398277316
   select_list ............ TRAFODION.SCH.T011T1.A, TRAFODION.SCH.T011T1.B
 
 
@@ -1268,7 +1268,7 @@ METRIC_TEXT_TABLE
 >>invoke trafodion."_REPOS_".metric_query_table;
 
 -- Definition of Trafodion table TRAFODION."_REPOS_".METRIC_QUERY_TABLE
--- Definition current  Sat Jan 23 04:08:38 2016
+-- Definition current  Wed Feb 24 09:08:40 2016
 
   (
     INSTANCE_ID                      INT UNSIGNED NO DEFAULT NOT NULL NOT
@@ -1450,7 +1450,7 @@ METRIC_TEXT_TABLE
 >>-- get qid for the prepared stmt
 >>get qid for statement explstmt;
 
-MXID11000005561212320281953925061000000000206U3333300_1924_EXPLSTMT
+MXID11000020741212323064781124001000000000206U3333300_1924_EXPLSTMT
 
 --- SQL operation complete.
 >>
@@ -1502,7 +1502,7 @@ SEQ_NUM      OPERATOR
 --- SQL command prepared.
 >>get qid for statement explstmt2;
 
-MXID11000005561212320281953925061000000000206U3333300_1935_EXPLSTMT2
+MXID11000020741212323064781124001000000000206U3333300_1935_EXPLSTMT2
 
 --- SQL operation complete.
 >>set qid MXID123456 for explstmt2;
@@ -2067,91 +2067,6 @@ LC   RC   OP   OPERATOR              OPT       DESCRIPTION           CARD
 
 --- SQL operation complete.
 >>
->>-- tests for alter column
->>create table t011t7(a int not null, b varchar(2), 
-+>   c varchar(4) character set utf8 not null);
-
---- SQL operation complete.
->>invoke t011t7;
-
--- Definition of Trafodion table TRAFODION.SCH.T011T7
--- Definition current  Sat Jan 23 04:09:41 2016
-
-  (
-    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
-  , B                                VARCHAR(2) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-
---- SQL operation complete.
->>alter table t011t7 alter column b varchar(3);
-
---- SQL operation complete.
->>invoke t011t7;
-
--- Definition of Trafodion table TRAFODION.SCH.T011T7
--- Definition current  Sat Jan 23 04:09:45 2016
-
-  (
-    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
-  , B                                VARCHAR(3) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-
---- SQL operation complete.
->>alter table t011t7 alter column c varchar(5) character set utf8;
-
---- SQL operation complete.
->>invoke t011t7;
-
--- Definition of Trafodion table TRAFODION.SCH.T011T7
--- Definition current  Sat Jan 23 04:09:49 2016
-
-  (
-    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
-  , B                                VARCHAR(3) CHARACTER SET ISO88591 COLLATE
-      DEFAULT DEFAULT NULL
-  , C                                VARCHAR(20 BYTES) CHARACTER SET UTF8
-      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
-  )
-
---- SQL operation complete.
->>
->>-- negative tests for alter column
->>alter table t011t7 alter column a largeint;
-
-*** ERROR[1404] Column A cannot be altered. Reason: Old and New datatypes must be VARCHAR.
-
---- SQL operation failed with errors.
->>alter table t011t7 alter column b int;
-
-*** ERROR[1404] Column B cannot be altered. Reason: Old and New datatypes must be VARCHAR.
-
---- SQL operation failed with errors.
->>alter table t011t7 alter column b varchar(4) character set utf8;
-
-*** ERROR[1404] Column B cannot be altered. Reason: Old and New character sets must be the same.
-
---- SQL operation failed with errors.
->>alter table t011t7 alter column c varchar(6);
-
-*** ERROR[1404] Column C cannot be altered. Reason: Old and New character sets must be the same.
-
---- SQL operation failed with errors.
->>alter table t011t7 alter column b varchar(2);
-
-*** ERROR[1404] Column B cannot be altered. Reason: New length must be greater than or equal to old length.
-
---- SQL operation failed with errors.
->>alter table t011t7 alter column c varchar(4) character set utf8;
-
-*** ERROR[1404] Column C cannot be altered. Reason: New length must be greater than or equal to old length.
-
---- SQL operation failed with errors.
 >>
 >>
 >>
@@ -2191,7 +2106,9 @@ LC   RC   OP   OPERATOR              OPT       DESCRIPTION           CARD
 >>
 >>drop table t011t7;
 
---- SQL operation complete.
+*** ERROR[1389] Object TRAFODION.SCH.T011T7 does not exist in Trafodion.
+
+--- SQL operation failed with errors.
 >>
 >>-- delete explain for statement explstmt from repository
 >>set parserflags 131072;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/seabase/EXPECTED027
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/EXPECTED027 b/core/sql/regress/seabase/EXPECTED027
index 5071189..8c48546 100644
--- a/core/sql/regress/seabase/EXPECTED027
+++ b/core/sql/regress/seabase/EXPECTED027
@@ -12,7 +12,7 @@
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:18:48 2015
+-- Definition current  Fri Feb 26 20:36:16 2016
 
   (
     A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -39,7 +39,7 @@ CREATE TABLE TRAFODION.SCH027.T027T01
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:18:55 2015
+-- Definition current  Fri Feb 26 20:36:25 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -67,7 +67,7 @@ CREATE TABLE TRAFODION.SCH027.T027T01
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:19:04 2015
+-- Definition current  Fri Feb 26 20:36:30 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -98,7 +98,7 @@ CREATE TABLE TRAFODION.SCH027.T027T01
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:19:14 2015
+-- Definition current  Fri Feb 26 20:36:39 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -127,7 +127,7 @@ CREATE TABLE TRAFODION.SCH027.T027T01
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:19:19 2015
+-- Definition current  Fri Feb 26 20:36:40 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -191,13 +191,13 @@ A            B            C
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:19:22 2015
+-- Definition current  Fri Feb 26 20:36:46 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf2".D                          INT DEFAULT NULL /* added col */
+  , "cf2".D                          INT DEFAULT NULL /*added_col*/
   )
 
 --- SQL operation complete.
@@ -208,7 +208,7 @@ CREATE TABLE TRAFODION.SCH027.T027T01
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf2".D                          INT DEFAULT NULL /* added col */
+  , "cf2".D                          INT DEFAULT NULL /*added_col*/
   , PRIMARY KEY (A ASC)
   )
  ATTRIBUTES DEFAULT COLUMN FAMILY 'cf'
@@ -228,14 +228,14 @@ CREATE INDEX T027T01I1 ON TRAFODION.SCH027.T027T01
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:19:29 2015
+-- Definition current  Fri Feb 26 20:36:49 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf2".D                          INT DEFAULT NULL /* added col */
-  , "cf4".E                          INT DEFAULT NULL /* added col */
+  , "cf2".D                          INT DEFAULT NULL /*added_col*/
+  , "cf4".E                          INT DEFAULT NULL /*added_col*/
   )
 
 --- SQL operation complete.
@@ -246,8 +246,8 @@ CREATE TABLE TRAFODION.SCH027.T027T01
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf2".D                          INT DEFAULT NULL /* added col */
-  , "cf4".E                          INT DEFAULT NULL /* added col */
+  , "cf2".D                          INT DEFAULT NULL /*added_col*/
+  , "cf4".E                          INT DEFAULT NULL /*added_col*/
   , PRIMARY KEY (A ASC)
   )
  ATTRIBUTES DEFAULT COLUMN FAMILY 'cf'
@@ -267,13 +267,13 @@ CREATE INDEX T027T01I1 ON TRAFODION.SCH027.T027T01
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:19:31 2015
+-- Definition current  Fri Feb 26 20:36:54 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf4".E                          INT DEFAULT NULL /* added col */
+  , "cf4".E                          INT DEFAULT NULL /*added_col*/
   )
 
 --- SQL operation complete.
@@ -284,7 +284,7 @@ CREATE TABLE TRAFODION.SCH027.T027T01
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf4".E                          INT DEFAULT NULL /* added col */
+  , "cf4".E                          INT DEFAULT NULL /*added_col*/
   , PRIMARY KEY (A ASC)
   )
  ATTRIBUTES DEFAULT COLUMN FAMILY 'cf'
@@ -304,14 +304,14 @@ CREATE INDEX T027T01I1 ON TRAFODION.SCH027.T027T01
 >>invoke t027t01;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T01
--- Definition current  Tue Nov 10 19:19:36 2015
+-- Definition current  Fri Feb 26 20:36:55 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf4".E                          INT DEFAULT NULL /* added col */
-  , "cf2".D                          INT DEFAULT NULL /* added col */
+  , "cf4".E                          INT DEFAULT NULL /*added_col*/
+  , "cf2".D                          INT DEFAULT NULL /*added_col*/
   )
 
 --- SQL operation complete.
@@ -322,8 +322,8 @@ CREATE TABLE TRAFODION.SCH027.T027T01
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
   , "cf2".B                          INT DEFAULT NULL
   , CF3.C                            INT DEFAULT NULL
-  , "cf4".E                          INT DEFAULT NULL /* added col */
-  , "cf2".D                          INT DEFAULT NULL /* added col */
+  , "cf4".E                          INT DEFAULT NULL /*added_col*/
+  , "cf2".D                          INT DEFAULT NULL /*added_col*/
   , PRIMARY KEY (A ASC)
   )
  ATTRIBUTES DEFAULT COLUMN FAMILY 'cf'
@@ -343,7 +343,7 @@ CREATE INDEX T027T01I1 ON TRAFODION.SCH027.T027T01
 >>invoke t027t011;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T011
--- Definition current  Tue Nov 10 19:19:47 2015
+-- Definition current  Fri Feb 26 20:37:02 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -387,7 +387,7 @@ A            B            C            E            D
 >>invoke t027t011;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T011
--- Definition current  Tue Nov 10 19:19:53 2015
+-- Definition current  Fri Feb 26 20:37:06 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -426,7 +426,7 @@ CREATE TABLE TRAFODION.SCH027.T027T011
 >>invoke t027t011;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T011
--- Definition current  Tue Nov 10 19:20:03 2015
+-- Definition current  Fri Feb 26 20:37:15 2016
 
   (
     A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -460,7 +460,7 @@ CREATE TABLE TRAFODION.SCH027.T027T011
 >>invoke t027t011;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T011
--- Definition current  Tue Nov 10 19:20:07 2015
+-- Definition current  Fri Feb 26 20:37:19 2016
 
   (
     "cf".A                           INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -491,7 +491,7 @@ CREATE TABLE TRAFODION.SCH027.T027T011
 >>invoke t027t03;
 
 -- Definition of Trafodion volatile table T027T03
--- Definition current  Tue Nov 10 19:20:14 2015
+-- Definition current  Fri Feb 26 20:37:25 2016
 
   (
     "cf1".A                          INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -522,7 +522,7 @@ CREATE VOLATILE TABLE T027T03
 >>invoke t027t03;
 
 -- Definition of Trafodion volatile table T027T03
--- Definition current  Tue Nov 10 19:20:36 2015
+-- Definition current  Fri Feb 26 20:37:45 2016
 
   (
     "cf1".A                          INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -570,7 +570,7 @@ CREATE VOLATILE TABLE T027T03
 >>invoke t027t02;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T02
--- Definition current  Tue Nov 10 19:20:58 2015
+-- Definition current  Fri Feb 26 20:38:04 2016
 
   (
     "cf".A0A0                        INT DEFAULT NULL
@@ -621,7 +621,7 @@ CREATE VOLATILE TABLE T027T03
 >>invoke t027t03;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T03
--- Definition current  Tue Nov 10 19:21:19 2015
+-- Definition current  Fri Feb 26 20:38:26 2016
 
   (
     "cf1".A                          INT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -657,7 +657,7 @@ A            B            C            D
 >>invoke t027t02;
 
 -- Definition of Trafodion table TRAFODION.SCH027.T027T02
--- Definition current  Tue Nov 10 19:21:35 2015
+-- Definition current  Fri Feb 26 20:38:40 2016
 
   (
     OBJECT_UID                       LARGEINT NO DEFAULT NOT NULL NOT DROPPABLE
@@ -756,8 +756,381 @@ create index t027t01i2 on t027t01("cf2".b);
 
 --- SQL operation failed with errors.
 >>
+>>-- tests for alter varchar column 
+>>drop table if exists t027t7;
+
+--- SQL operation complete.
+>>create table t027t7(a int not null, b varchar(2), 
++>   c varchar(4) character set utf8 not null, z int not null primary key)
++>salt using 2 partitions;
+
+--- SQL operation complete.
+>>
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:39:06 2016
+
+  (
+    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  , B                                VARCHAR(2) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>insert into t027t7 values (1, 'ab', 'cd', 10);
+
+--- 1 row(s) inserted.
+>>select * from t027t7;
+
+A            B   C                 Z          
+-----------  --  ----------------  -----------
+
+          1  ab  cd                         10
+
+--- 1 row(s) selected.
+>>alter table t027t7 alter column b varchar(3);
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:39:13 2016
+
+  (
+    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  , B                                VARCHAR(3) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>alter table t027t7 alter column c varchar(5) character set utf8;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:39:35 2016
+
+  (
+    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  , B                                VARCHAR(3) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(5 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>alter table t027t7 alter column c varchar(4) character set utf8;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:39:50 2016
+
+  (
+    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  , B                                VARCHAR(3) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>
+>>alter table t027t7 alter column a largeint;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:40:08 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , B                                VARCHAR(3) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>alter table t027t7 alter column b varchar(4) character set utf8;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:40:26 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , B                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>alter table t027t7 alter column c varchar(6);
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:40:43 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , B                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(6) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>alter table t027t7 alter column b varchar(2);
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:40:58 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , B                                VARCHAR(2) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(6) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>alter table t027t7 alter column a int;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:41:12 2016
+
+  (
+    A                                INT DEFAULT NULL /*altered_col*/
+  , B                                VARCHAR(2) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(6) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>alter table t027t7 alter column a smallint default 0 not null;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:41:30 2016
+
+  (
+    A                                SMALLINT DEFAULT 0 NOT NULL NOT DROPPABLE
+      /*altered_col*/
+  , B                                VARCHAR(2) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(6) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL /*altered_col*/
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>select * from t027t7;
+
+A       B   C       Z          
+------  --  ------  -----------
+
+     1  ab  cd               10
+
+--- 1 row(s) selected.
+>>
+>>-- aligned format
+>>drop table if exists t027t7;
+
+--- SQL operation complete.
+>>create table t027t7(a int not null, b varchar(2), 
++>   c varchar(4) character set utf8 not null, z int not null primary key)
++>salt using 2 partitions attribute aligned format;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:41:52 2016
+
+  (
+    A                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  , B                                VARCHAR(2) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>insert into t027t7 values (1, 'ab', 'cd', 10);
+
+--- 1 row(s) inserted.
+>>select * from t027t7;
+
+A            B   C                 Z          
+-----------  --  ----------------  -----------
+
+          1  ab  cd                         10
+
+--- 1 row(s) selected.
+>>alter table t027t7 alter column a largeint;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:42:09 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , B                                VARCHAR(2) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT NULL
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>select * from t027t7;
+
+A                     B   C                 Z          
+--------------------  --  ----------------  -----------
+
+                   1  ab  cd                         10
+
+--- 1 row(s) selected.
+>>alter table t027t7 drop column b;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:42:35 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>select * from t027t7;
+
+A                     C                 Z          
+--------------------  ----------------  -----------
+
+                   1  cd                         10
+
+--- 1 row(s) selected.
+>>alter table t027t7 add column b char(10) default 'abc' not null;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:42:38 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  , B                                CHAR(10) CHARACTER SET ISO88591 COLLATE
+      DEFAULT DEFAULT _ISO88591'abc' NOT NULL NOT DROPPABLE /*added_col*/
+  )
+
+--- SQL operation complete.
+>>select * from t027t7;
+
+A                     C                 Z            B         
+--------------------  ----------------  -----------  ----------
+
+                   1  cd                         10  abc       
+
+--- 1 row(s) selected.
+>>alter table t027t7 drop column b;
+
+--- SQL operation complete.
+>>invoke t027t7;
+
+-- Definition of Trafodion table TRAFODION.SCH027.T027T7
+-- Definition current  Fri Feb 26 20:43:02 2016
+
+  (
+    A                                LARGEINT DEFAULT NULL /*altered_col*/
+  , C                                VARCHAR(4 CHARS) CHARACTER SET UTF8
+      COLLATE DEFAULT NO DEFAULT NOT NULL NOT DROPPABLE
+  , Z                                INT NO DEFAULT NOT NULL NOT DROPPABLE
+  )
+
+--- SQL operation complete.
+>>select * from t027t7;
+
+A                     C                 Z          
+--------------------  ----------------  -----------
+
+                   1  cd                         10
+
+--- 1 row(s) selected.
+>>
+>>-- negative tests for alter/drop column
+>>alter table t027t7 alter column b int;
+
+*** ERROR[1009] Column B does not exist in the specified table.
+
+--- SQL operation failed with errors.
+>>alter table t027t7 alter column a char(10);
+
+*** ERROR[1404] Column A cannot be altered. Reason: Old and New datatypes must be compatible.
+
+--- SQL operation failed with errors.
+>>alter table t027t7 alter column c char(1);
+
+*** ERROR[8402] A string overflow occurred during the evaluation of a character expression. Conversion of Source Type:VARCHAR(REC_BYTE_V_ASCII,2 BYTES,ISO88591) Source Value:cd to Target Type:CHAR(REC_BYTE_F_ASCII,1 BYTES,ISO88591).
+
+*** ERROR[8839] Transaction was aborted.
+
+*** ERROR[1404] Column C cannot be altered. Reason: Old data could not be updated using the altered column definition.
+
+--- SQL operation failed with errors.
+>>alter table t027t7 drop column e;
+
+*** ERROR[1009] Column E does not exist in the specified table.
+
+--- SQL operation failed with errors.
 >>
 >>-- cleanup
+>>drop table if exists t027t7;
+
+--- SQL operation complete.
 >>drop table if exists t027t01;
 
 --- SQL operation complete.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/seabase/TEST011
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/TEST011 b/core/sql/regress/seabase/TEST011
index ed3b15d..d33bee0 100644
--- a/core/sql/regress/seabase/TEST011
+++ b/core/sql/regress/seabase/TEST011
@@ -557,22 +557,6 @@ select count(*) from t011t6b;
 drop table t011t6a ;
 drop table t011t6b ;
 
--- tests for alter column
-create table t011t7(a int not null, b varchar(2), 
-   c varchar(4) character set utf8 not null);
-invoke t011t7;
-alter table t011t7 alter column b varchar(3);
-invoke t011t7;
-alter table t011t7 alter column c varchar(5) character set utf8;
-invoke t011t7;
-
--- negative tests for alter column
-alter table t011t7 alter column a largeint;
-alter table t011t7 alter column b int;
-alter table t011t7 alter column b varchar(4) character set utf8;
-alter table t011t7 alter column c varchar(6);
-alter table t011t7 alter column b varchar(2);
-alter table t011t7 alter column c varchar(4) character set utf8;
 
 
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/regress/seabase/TEST027
----------------------------------------------------------------------
diff --git a/core/sql/regress/seabase/TEST027 b/core/sql/regress/seabase/TEST027
index 82b8b82..d828640 100644
--- a/core/sql/regress/seabase/TEST027
+++ b/core/sql/regress/seabase/TEST027
@@ -20,6 +20,7 @@
 -- @@@ END COPYRIGHT @@@
 --
 -- tests for support of multiple column families
+-- tests for alter table alter column
 
 create schema trafodion.sch027;
 set schema trafodion.sch027;
@@ -178,8 +179,65 @@ create table t027t02 (
      a20.a20 int, a21.a21 int, a22.a22 int, a23.a23 int, a24.a24 int, a25.a25 int, a26.a26 int, a27.a27 int, a28.a28 int, a29.a29 int,
     a30.a30 int, a31.a31 int) attribute default column family 'cf';
 
+-- tests for alter varchar column 
+drop table if exists t027t7;
+create table t027t7(a int not null, b varchar(2), 
+   c varchar(4) character set utf8 not null, z int not null primary key)
+salt using 2 partitions;
+
+invoke t027t7;
+insert into t027t7 values (1, 'ab', 'cd', 10);
+select * from t027t7;
+alter table t027t7 alter column b varchar(3);
+invoke t027t7;
+alter table t027t7 alter column c varchar(5) character set utf8;
+invoke t027t7;
+alter table t027t7 alter column c varchar(4) character set utf8;
+invoke t027t7;
+
+alter table t027t7 alter column a largeint;
+invoke t027t7;
+alter table t027t7 alter column b varchar(4) character set utf8;
+invoke t027t7;
+alter table t027t7 alter column c varchar(6);
+invoke t027t7;
+alter table t027t7 alter column b varchar(2);
+invoke t027t7;
+alter table t027t7 alter column a int;
+invoke t027t7;
+alter table t027t7 alter column a smallint default 0 not null;
+invoke t027t7;
+select * from t027t7;
+
+-- aligned format
+drop table if exists t027t7;
+create table t027t7(a int not null, b varchar(2), 
+   c varchar(4) character set utf8 not null, z int not null primary key)
+salt using 2 partitions attribute aligned format;
+invoke t027t7;
+insert into t027t7 values (1, 'ab', 'cd', 10);
+select * from t027t7;
+alter table t027t7 alter column a largeint;
+invoke t027t7;
+select * from t027t7;
+alter table t027t7 drop column b;
+invoke t027t7;
+select * from t027t7;
+alter table t027t7 add column b char(10) default 'abc' not null;
+invoke t027t7;
+select * from t027t7;
+alter table t027t7 drop column b;
+invoke t027t7;
+select * from t027t7;
+
+-- negative tests for alter/drop column
+alter table t027t7 alter column b int;
+alter table t027t7 alter column a char(10);
+alter table t027t7 alter column c char(1);
+alter table t027t7 drop column e;
 
 -- cleanup
+drop table if exists t027t7;
 drop table if exists t027t01;
 drop table if exists t027t02;
 drop table if exists t027t011;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/sqlcomp/CmpDescribe.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpDescribe.cpp b/core/sql/sqlcomp/CmpDescribe.cpp
index 480853b..8b692aa 100644
--- a/core/sql/sqlcomp/CmpDescribe.cpp
+++ b/core/sql/sqlcomp/CmpDescribe.cpp
@@ -200,11 +200,13 @@ short CmpDescribeSeabaseTable (
                              NABoolean withoutDivisioning = FALSE,
                              NABoolean noTrailingSemi = FALSE,
 
-                             // used to add or remove column definition from col list.
-                             // valid for 'createLike' mode. Used for 'alter add/drop col'.
+                             // used to add,rem,alter column definition from col list.
+                             // valid for 'createLike' mode. 
+                             // Used for 'alter add/drop/alter col'.
                              char * colName = NULL,
-                             NABoolean isAdd = FALSE,
-                             const NAColumn * nacol = NULL);
+                             short ada = 0, // 0,add. 1,drop. 2,alter
+                             const NAColumn * nacol = NULL,
+                             const NAType * natype = NULL);
 
 short CmpDescribeSequence ( 
                              const CorrName  &dtName,
@@ -2304,26 +2306,37 @@ short CmpDescribeHiveTable (
 }
 
 // type:  1, invoke. 2, showddl. 3, create_like
-static short cmpDisplayColumn(const NAColumn *nac,
-                              short type,
-                              Space &space, char * buf,
-                              Lng32 &ii,
-                              NABoolean namesOnly,
-                              NABoolean &identityCol,
-                              NABoolean isExternalTable,
-                              NABoolean isAlignedRowFormat)
+short cmpDisplayColumn(const NAColumn *nac,
+                       char * inColName,
+                       const NAType *inNAT,
+                       short displayType,
+                       Space *inSpace,
+                       char * buf,
+                       Lng32 &ii,
+                       NABoolean namesOnly,
+                       NABoolean &identityCol,
+                       NABoolean isExternalTable,
+                       NABoolean isAlignedRowFormat)
 {
+  Space lSpace;
+  
+  Space * space;
+  if (inSpace)
+    space = inSpace;
+  else
+    space = &lSpace;
+
   identityCol = FALSE;
   
-  const NAString &colName = nac->getColName();
+  NAString colName(inColName ? inColName : nac->getColName());
   NATable * naTable = (NATable*)nac->getNATable();
 
   NAString colFam;
-  if ((nac->getNATable()->isSQLMXAlignedTable()) || 
+  if ((nac->getNATable() && nac->getNATable()->isSQLMXAlignedTable()) || 
       (nac->getHbaseColFam() == SEABASE_DEFAULT_COL_FAMILY) ||
       isExternalTable)
     colFam = "";
-  else if (nac->getNATable()->isSeabaseTable())
+  else if (nac->getNATable() && nac->getNATable()->isSeabaseTable())
     {
       int index = 0;
       CmpSeabaseDDL::extractTrafColFam(nac->getHbaseColFam(), index);
@@ -2336,7 +2349,7 @@ static short cmpDisplayColumn(const NAColumn *nac,
       colFam += ".";
     }
 
-  if (type == 3)
+  if (displayType == 3)
     {
       NAString quotedColName = "\"";
       quotedColName += colName.data(); 
@@ -2358,12 +2371,12 @@ static short cmpDisplayColumn(const NAColumn *nac,
     {
       NAString colString(buf);
       Int32 j = ii;
-      outputColumnLine(space, colString, j);
+      outputColumnLine(*space, colString, j);
       
       return 0;
     }
   
-  const NAType * nat = nac->getType();
+  const NAType * nat = (inNAT ? inNAT : nac->getType());
   
   NAString nas;
   ((NAType*)nat)->getMyTypeAsText(&nas, FALSE);
@@ -2399,7 +2412,7 @@ static short cmpDisplayColumn(const NAColumn *nac,
         defVal = "GENERATED ALWAYS AS IDENTITY";
       
       NAString idOptions;
-      if ((type != 1) && (nac->getNATable()->getSGAttributes()))
+      if ((displayType != 1) && (nac->getNATable()->getSGAttributes()))
         nac->getNATable()->getSGAttributes()->display(NULL, &idOptions, TRUE);
       
       if (NOT idOptions.isNull())
@@ -2434,35 +2447,43 @@ static short cmpDisplayColumn(const NAColumn *nac,
   
   char * sqlmxRegr = getenv("SQLMX_REGRESS");
   if ((! sqlmxRegr) ||
-      (type == 3))
+      (displayType == 3))
     {
       if ((NOT isAlignedRowFormat) &&
             CmpSeabaseDDL::isSerialized(nac->getHbaseColFlags()))
         attrStr += " SERIALIZED";
       else if ((CmpCommon::getDefault(HBASE_SERIALIZATION) == DF_ON) ||
-                (type == 3))
+                (displayType == 3))
         attrStr += " NOT SERIALIZED";
     }
   
-  if (nac->isAddedColumn())
+  if (displayType != 3)
     {
-      attrStr += " /* added col */ ";
+      if (nac->isAlteredColumn())
+        {
+          attrStr += " /*altered_col*/ ";
+        }
+      else if (nac->isAddedColumn())
+        {
+          attrStr += " /*added_col*/ ";
+        }
     }
-  
+
   sprintf(&buf[strlen(buf)], "%s %s", 
           nas.data(), 
           attrStr.data());
   
   NAString colString(buf);
   Int32 j = ii;
-  outputColumnLine(space, colString, j);
+  if (inSpace)
+    outputColumnLine(*space, colString, j);
 
   return 0;
 }
 
 // type:  1, invoke. 2, showddl. 3, create_like
 short cmpDisplayColumns(const NAColumnArray & naColArr,
-                        short type,
+                        short displayType,
                         Space &space, char * buf, 
                         NABoolean displaySystemCols,
                         NABoolean namesOnly,
@@ -2470,8 +2491,9 @@ short cmpDisplayColumns(const NAColumnArray & naColArr,
                         NABoolean isExternalTable,
 			NABoolean isAlignedRowFormat,
                         char * inColName = NULL,
-                        NABoolean isAdd = FALSE,
-                        const NAColumn * nacol = NULL)
+                        short ada = 0, // 0,add. 1,drop. 2,alter
+                        const NAColumn * nacol = NULL,
+                        const NAType * natype = NULL)
 {
   Lng32 ii = 0;
   identityColPos = -1;
@@ -2488,14 +2510,25 @@ short cmpDisplayColumns(const NAColumnArray & naColArr,
 
       const NAString &colName = nac->getColName();
 
-      if ((inColName) && (NOT isAdd))
+      if ((inColName) && (ada == 1))
         {
           if (colName == inColName) // remove this column
             continue;
         }
       
-       if (cmpDisplayColumn(nac, type, space, buf, ii, namesOnly, identityCol, isExternalTable, isAlignedRowFormat))
-        return -1;
+      if ((inColName) && (colName == inColName) &&
+          (ada == 2) && (nacol) && (natype))
+        {
+          if (cmpDisplayColumn(nac, NULL, natype, displayType, &space, buf, ii, namesOnly, identityCol, 
+                               isExternalTable, isAlignedRowFormat))
+            return -1;
+        }
+      else
+        {
+          if (cmpDisplayColumn(nac, NULL, NULL, displayType, &space, buf, ii, namesOnly, identityCol, 
+                               isExternalTable, isAlignedRowFormat))
+            return -1;
+        }
 
       if (identityCol)
         identityColPos = i;
@@ -2503,31 +2536,36 @@ short cmpDisplayColumns(const NAColumnArray & naColArr,
       ii++;
     }
 
-  if ((inColName) && (isAdd) && (nacol))
+  if ((inColName) && (ada == 0) && (nacol))
     {
-      if (cmpDisplayColumn(nacol, type, space, buf, ii, namesOnly, identityCol, isExternalTable, isAlignedRowFormat))
+      if (cmpDisplayColumn(nacol, NULL, NULL, displayType, &space, buf, ii, namesOnly, identityCol, 
+                           isExternalTable, isAlignedRowFormat))
         return -1;
     }
 
   return 0;
 }
 
-static short cmpDisplayPrimaryKey(const NAColumnArray & naColArr,
-                                  Lng32 numKeys,
-                                  NABoolean displaySystemCols,
-                                  Space &space, char * buf, 
-                                  NABoolean displayCompact,
-                                  NABoolean displayAscDesc)
+short cmpDisplayPrimaryKey(const NAColumnArray & naColArr,
+                           Lng32 numKeys,
+                           NABoolean displaySystemCols,
+                           Space &space, char * buf, 
+                           NABoolean displayCompact,
+                           NABoolean displayAscDesc,
+                           NABoolean displayParens)
 {
   if (numKeys > 0)
     {
-      if (displayCompact)
-        sprintf(&buf[strlen(buf)],  "(");
-      else
+      if (displayParens)
         {
-          outputShortLine(space, "  ( ");
+          if (displayCompact)
+            sprintf(&buf[strlen(buf)],  "(");
+          else
+            {
+              outputShortLine(space, "  ( ");
+            }
         }
-      
+
       NABoolean isFirst = TRUE;
       Int32 j = -1;
       for (Int32 jj = 0; jj < numKeys; jj++)
@@ -2544,12 +2582,14 @@ static short cmpDisplayPrimaryKey(const NAColumnArray & naColArr,
           
           const NAString &keyName = nac->getColName();
           if (displayCompact)
-            sprintf(&buf[strlen(buf)], "%s%s%s", 
-                    (NOT isFirst ? ", " : ""),
-                    ANSI_ID(keyName.data()),
-                    (displayAscDesc ?
-                     (! naColArr.isAscending(jj) ? " DESC" : " ASC") :
-                     " "));
+            {
+              sprintf(&buf[strlen(buf)], "%s%s%s", 
+                      (NOT isFirst ? ", " : ""),
+                      ANSI_ID(keyName.data()),
+                      (displayAscDesc ?
+                       (! naColArr.isAscending(jj) ? " DESC" : " ASC") :
+                       " "));
+            }
           else
             {
               sprintf(buf, "%s%s", 
@@ -2564,15 +2604,18 @@ static short cmpDisplayPrimaryKey(const NAColumnArray & naColArr,
 
           isFirst = FALSE;
         } // for
-      
-      if (displayCompact)
-        {
-          sprintf(&buf[strlen(buf)],  ")");
-          outputLine(space, buf, 2);
-        }
-      else
+
+      if (displayParens)
         {
-          outputShortLine(space, "  )");
+          if (displayCompact)
+            {
+              sprintf(&buf[strlen(buf)],  ")");
+              outputLine(space, buf, 2);
+            }
+          else
+            {
+              outputShortLine(space, "  )");
+            }
         }
     } // if
   
@@ -2593,8 +2636,9 @@ short CmpDescribeSeabaseTable (
                                NABoolean withoutDivisioning,
                                NABoolean noTrailingSemi,
                                char * colName,
-                               NABoolean isAdd,
-                               const NAColumn * nacol)
+                               short ada,
+                               const NAColumn * nacol,
+                               const NAType * natype)
 {
   const NAString& tableName =
     dtName.getQualifiedNameObj().getQualifiedNameAsAnsiString(TRUE);
@@ -2646,6 +2690,7 @@ short CmpDescribeSeabaseTable (
     displayPrivilegeGrants = FALSE;
  
   // display syscols for invoke if not running regrs
+  //
   NABoolean displaySystemCols = ((!sqlmxRegr) && (type == 1));
 
   NABoolean isView = (naTable->getViewText() ? TRUE : FALSE);
@@ -2782,7 +2827,7 @@ short CmpDescribeSeabaseTable (
 		    FALSE,
                     identityColPos,
                     isExternalTable, naTable->isSQLMXAlignedTable(),
-                    colName, isAdd, nacol);
+                    colName, ada, nacol, natype);
 
   Int32 nonSystemKeyCols = 0;
   NABoolean isStoreBy = FALSE;
@@ -2861,7 +2906,7 @@ short CmpDescribeSeabaseTable (
           cmpDisplayPrimaryKey(naf->getIndexKeyColumns(), 
                                naf->getIndexKeyColumns().entries(),
                                displaySystemCols,
-                               space, buf, TRUE, TRUE);
+                               space, buf, TRUE, TRUE, TRUE);
         } // if
     }
 
@@ -2876,7 +2921,7 @@ short CmpDescribeSeabaseTable (
           cmpDisplayPrimaryKey(naf->getIndexKeyColumns(), 
                                naf->getIndexKeyColumns().entries(),
                                displaySystemCols,
-                               space, buf, TRUE, TRUE);
+                               space, buf, TRUE, TRUE, TRUE);
         }
       
       if ((isSalted) && !withoutSalt)
@@ -2985,6 +3030,7 @@ short CmpDescribeSeabaseTable (
 
       NABoolean attributesSet = FALSE;
       if (((NOT sqlmxRegr) && ((NOT isAudited) || (isAligned))) ||
+          ((sqlmxRegr) && (type == 3) && ((NOT isAudited) || (isAligned))) ||
           ((NOT naTable->defaultColFam().isNull()) && 
            (naTable->defaultColFam() != SEABASE_DEFAULT_COL_FAMILY)))
         {
@@ -3132,7 +3178,7 @@ short CmpDescribeSeabaseTable (
 
 	  cmpDisplayPrimaryKey(naf->getIndexKeyColumns(), numIndexCols, 
 			       displaySystemCols,
-			       space, buf, FALSE, TRUE);
+			       space, buf, FALSE, TRUE, TRUE);
 
           if ((NOT sqlmxRegr) && isAligned)
           {
@@ -3207,7 +3253,7 @@ short CmpDescribeSeabaseTable (
               cmpDisplayPrimaryKey(nacarr, 
                                    uniqConstr->keyColumns().entries(),
                                    FALSE,
-                                   space, &buf[strlen(buf)], FALSE, FALSE);
+                                   space, &buf[strlen(buf)], FALSE, FALSE, TRUE);
 
               outputShortLine(space, ";");
             } // for
@@ -3252,7 +3298,7 @@ short CmpDescribeSeabaseTable (
               cmpDisplayPrimaryKey(nacarr, 
                                    refConstr->keyColumns().entries(),
                                    FALSE,
-                                   space, &buf[strlen(buf)], FALSE, FALSE);
+                                   space, &buf[strlen(buf)], FALSE, FALSE, TRUE);
 
               const NAString& ansiOtherTableName = 
                 uniqueConstraintReferencedByMe.getTableName().getQualifiedNameAsAnsiString(TRUE);             
@@ -3272,7 +3318,7 @@ short CmpDescribeSeabaseTable (
               cmpDisplayPrimaryKey(nacarr2, 
                                    otherConstr->keyColumns().entries(),
                                    FALSE,
-                                   space, &buf[strlen(buf)], FALSE, FALSE);
+                                   space, &buf[strlen(buf)], FALSE, FALSE, TRUE);
 
               if (NOT refConstr->getIsEnforced())
                 {

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/7b99938c/core/sql/sqlcomp/CmpSeabaseDDL.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/CmpSeabaseDDL.h b/core/sql/sqlcomp/CmpSeabaseDDL.h
index 5ab446e..e616f85 100644
--- a/core/sql/sqlcomp/CmpSeabaseDDL.h
+++ b/core/sql/sqlcomp/CmpSeabaseDDL.h
@@ -563,6 +563,9 @@ class CmpSeabaseDDL
 		   LobsStorage &lobStorage,
 		   ULng32 &hbaseColFlags,
                    Int64 &colFlags);
+
+  short getNAColumnFromColDef(ElemDDLColDef * colNode,
+                              NAColumn* &naCol);
   
   short createRowId(NAString &key,
 		    NAString &part1, Lng32 part1MaxLen,
@@ -905,7 +908,7 @@ class CmpSeabaseDDL
                             NABoolean isCompound = FALSE);
   
   void createSeabaseTable(
-			  StmtDDLCreateTable                  * createTableNode,
+			  StmtDDLCreateTable * createTableNode,
 			  NAString &currCatName, NAString &currSchName,
                           NABoolean isCompound = FALSE);
  
@@ -913,6 +916,13 @@ class CmpSeabaseDDL
 			  StmtDDLCreateTable                  * createTableNode,
 			  NAString &currCatName, NAString &currSchName);
  
+  short createSeabaseTableLike2(
+       CorrName &cn,
+       const NAString &likeTabName,
+       NABoolean withPartns = FALSE,
+       NABoolean withoutSalt = FALSE,
+       NABoolean withoutDivision = FALSE);
+
   void createSeabaseTableLike(
 			      StmtDDLCreateTable                  * createTableNode,
 			      NAString &currCatName, NAString &currSchName);
@@ -923,6 +933,19 @@ class CmpSeabaseDDL
                                    const ComObjectName &tgtTableName,
                                    const ComObjectName &srcTableName);
 
+  // makes a copy of underlying hbase table
+  short cloneHbaseTable(
+       const NAString &srcTable, const NAString &clonedTable,
+       ExpHbaseInterface * inEHI);
+
+  // makes a copy of traf metadata and underlying hbase table
+  short cloneSeabaseTable(
+       CorrName &cn,
+       const NATable * naTable,
+       const NAString &clonedTabName,
+       ExpHbaseInterface * ehi = NULL,
+       ExeCliInterface * cilInterface = NULL);
+
   short dropSeabaseTable2(
                           ExeCliInterface *cliInterface,
                           StmtDDLDropTable * dropTableNode,
@@ -969,17 +992,23 @@ class CmpSeabaseDDL
        StmtDDLAlterTableAddColumn * alterAddColNode,
        NAString &currCatName, NAString &currSchName);
   
-  void alterSeabaseTableDropColumn(
-       StmtDDLAlterTableDropColumn * alterDropColNode,
-       NAString &currCatName, NAString &currSchName);
-  
-  short alignedFormatTableAddDropColumn(
-       Int64 objUID,
-       NABoolean isAdd,
+
+  short updateMDforDropCol(ExeCliInterface &cliInterface,
+                           const NATable * naTable,
+                           Lng32 dropColNum);
+
+  short alignedFormatTableDropColumn
+  (
        const NAString &catalogNamePart,
        const NAString &schemaNamePart,
        const NAString &objectNamePart,
-       char * colName, const NAColumn * nacol);
+       const NATable * naTable,
+       const NAString &altColName,
+       ElemDDLColDef *pColDef);
+ 
+  void alterSeabaseTableDropColumn(
+       StmtDDLAlterTableDropColumn * alterDropColNode,
+       NAString &currCatName, NAString &currSchName);
   
   short recreateViews(ExeCliInterface &cliInterface,
                       NAList<NAString> &viewNameList,
@@ -989,6 +1018,21 @@ class CmpSeabaseDDL
        StmtDDLAlterTableAlterColumnSetSGOption * alterIdentityColNode,
        NAString &currCatName, NAString &currSchName);
   
+  short alignedFormatTableAlterColumn
+  (
+       const NAString &catalogNamePart,
+       const NAString &schemaNamePart,
+       const NAString &objectNamePart,
+       const NATable * naTable,
+       const NAString &altColName,
+       ElemDDLColDef *pColDef);
+  
+  short alterColumnAttr(
+       const NAString &catalogNamePart, const NAString &schemaNamePart,
+       const NAString &objectNamePart,
+       const NATable * naTable, const NAColumn * naCol, NAType * newType,
+       StmtDDLAlterTableAlterColumnDatatype * alterColNode);
+  
   void alterSeabaseTableAlterColumnDatatype(
        StmtDDLAlterTableAlterColumnDatatype * alterColumnDatatype,
        NAString &currCatName, NAString &currSchName);
@@ -1142,6 +1186,12 @@ class CmpSeabaseDDL
   short dropSeabaseObjectsFromHbase(const char * pattern);
   short updateSeabaseAuths(ExeCliInterface * cliInterface, const char * sysCat);
 
+  short truncateHbaseTable(const NAString &catalogNamePart, 
+                           const NAString &schemaNamePart, 
+                           const NAString &objectNamePart,
+                           NATable * naTable,
+                           ExpHbaseInterface * ehi);
+
   void purgedataHbaseTable(DDLExpr * ddlExpr,
 			   NAString &currCatName, NAString &currSchName);