http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/19c96b1e/core/sql/optimizer/NATable.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/NATable.cpp b/core/sql/optimizer/NATable.cpp
index 098e514..eb68a51 100644
--- a/core/sql/optimizer/NATable.cpp
+++ b/core/sql/optimizer/NATable.cpp
@@ -45,7 +45,6 @@
 #include "NATable.h"
 #include "Sqlcomp.h"
 #include "Const.h"
-#include "desc.h"
 #include "dfs2rec.h"
 #include "hs_read.h"
 #include "parser.h"
@@ -84,6 +83,7 @@
 #include "ComUser.h"
 #include "ComSmallDefs.h"
 #include "CmpMain.h"
+#include "TrafDDLdesc.h"
 
 #define MAX_NODE_NAME 9
 #define MAX_PRECISION_ALLOWED  18
@@ -96,7 +96,7 @@
 
 
 #include "SqlParserGlobals.h"
-extern desc_struct *generateSpecialDesc(const CorrName& corrName);
+extern TrafDesc *generateSpecialDesc(const CorrName& corrName);
 
 #include "CmpMemoryMonitor.h"
 
@@ -1871,7 +1871,7 @@ NABoolean checkColumnTypeForSupportability(const 
NAColumnArray & partColArray, c
 // -----------------------------------------------------------------------
 #pragma nowarn(1506)   // warning elimination
 static RangePartitionBoundaries * createRangePartitionBoundaries
-                                     (desc_struct * part_desc_list,
+                                     (TrafDesc * part_desc_list,
                                      Lng32 numberOfPartitions,
                                      const NAColumnArray & partColArray,
                                      NAMemory* heap)
@@ -1883,8 +1883,8 @@ static RangePartitionBoundaries * 
createRangePartitionBoundaries
   //             first partition and the last element is the descriptor
   //             for the last partition, in partitioning key sequence.
   // ---------------------------------------------------------------------
-    desc_struct * partns_desc = part_desc_list;
-  CMPASSERT(partns_desc->body.partns_desc.primarypartition);
+    TrafDesc * partns_desc = part_desc_list;
+  CMPASSERT(partns_desc->partnsDesc()->primarypartition);
 
 
   // Check all the partitioning keys.  If any of them are not
@@ -1893,7 +1893,7 @@ static RangePartitionBoundaries * 
createRangePartitionBoundaries
   // Skip past the primary partition, so that a meaningful first
   // key value can be used for the error message.
 
-  char* key = (partns_desc->header.next) ->body.partns_desc.firstkey;
+  char* key = (partns_desc->next) ->partnsDesc()->firstkey;
 
   if ( !checkColumnTypeForSupportability(partColArray, key) )
     return NULL;
@@ -1920,14 +1920,14 @@ static RangePartitionBoundaries * 
createRangePartitionBoundaries
   // string in the SMD.
   // NOTE: The RangePartitionBoundaries is 0 based.
   // ---------------------------------------------------------------------
-  partns_desc = partns_desc->header.next; // skip the primary partition
+  partns_desc = partns_desc->next; // skip the primary partition
   Lng32 counter = 1;
   char* encodedKey;
 
   while (partns_desc AND (counter < numberOfPartitions))
     {
-      encodedKey = partns_desc->body.partns_desc.encodedkey;
-      size_t encodedKeyLen = partns_desc->body.partns_desc.encodedkeylen;
+      encodedKey = partns_desc->partnsDesc()->encodedkey;
+      size_t encodedKeyLen = partns_desc->partnsDesc()->encodedkeylen;
 
       if(heap != CmpCommon::statementHeap())
       {
@@ -1938,19 +1938,19 @@ static RangePartitionBoundaries * 
createRangePartitionBoundaries
         //to make it "\0\0".
         encodedKey = new(heap) char [encodedKeyLen+2];
         encodedKey[encodedKeyLen] = encodedKey[encodedKeyLen+1] = '\0';
-        str_cpy_all(encodedKey, partns_desc->body.partns_desc.encodedkey,
+        str_cpy_all(encodedKey, partns_desc->partnsDesc()->encodedkey,
                     encodedKeyLen);
       }
 
       ItemExpr *rangePartBoundValues = NULL;
 
-      if (partns_desc->body.partns_desc.firstkey)
+      if (partns_desc->partnsDesc()->firstkey)
         // Extract and parse the partition boundary values, producing an
         // ItemExprList of the boundary values.
         //
         rangePartBoundValues = getRangePartitionBoundaryValues(
-             partns_desc->body.partns_desc.firstkey,
-             partns_desc->body.partns_desc.firstkeylen,
+             partns_desc->partnsDesc()->firstkey,
+             partns_desc->partnsDesc()->firstkeylen,
              heap);
       else
         rangePartBoundValues = getRangePartitionBoundaryValuesFromEncodedKeys(
@@ -1980,7 +1980,7 @@ static RangePartitionBoundaries * 
createRangePartitionBoundaries
         // ERROR 1123 Unable to process the partition key values...
         *CmpCommon::diags()
           << DgSqlCode(-1123)
-          << DgString0(partns_desc->body.partns_desc.firstkey)
+          << DgString0(partns_desc->partnsDesc()->firstkey)
           << DgTableName(tableName);
         delete partBounds;
         //coverity[leaked_storage]
@@ -1992,7 +1992,7 @@ static RangePartitionBoundaries * 
createRangePartitionBoundaries
            rangePartBoundValues,
            encodedKey);
 
-      partns_desc = partns_desc->header.next;
+      partns_desc = partns_desc->next;
     } // end while (partns_desc)
 
   // ---------------------------------------------------------------------
@@ -2016,7 +2016,7 @@ static RangePartitionBoundaries * 
createRangePartitionBoundaries
 // This method is used for creating a rangePartitioningFunction.
 // -----------------------------------------------------------------------
 static PartitioningFunction * createRangePartitioningFunction
-                                (desc_struct * part_desc_list,
+                                (TrafDesc * part_desc_list,
                                 const NAColumnArray & partKeyColArray,
                                  NodeMap* nodeMap,
                                 NAMemory* heap)
@@ -2024,12 +2024,12 @@ static PartitioningFunction * 
createRangePartitioningFunction
   // ---------------------------------------------------------------------
   // Compute the number of partitions.
   // ---------------------------------------------------------------------
-  desc_struct * partns_desc = part_desc_list;
+  TrafDesc * partns_desc = part_desc_list;
   Lng32 numberOfPartitions = 0;
   while (partns_desc)
     {
       numberOfPartitions++;
-      partns_desc = partns_desc->header.next;
+      partns_desc = partns_desc->next;
     }
 
   // ---------------------------------------------------------------------
@@ -2078,19 +2078,19 @@ static PartitioningFunction * 
createRangePartitioningFunction
 // -----------------------------------------------------------------------
 // LCOV_EXCL_START :cnu
 static PartitioningFunction * createRoundRobinPartitioningFunction
-                                (desc_struct * part_desc_list,
+                                (TrafDesc * part_desc_list,
                                  NodeMap* nodeMap,
                                 NAMemory* heap)
 {
   // ---------------------------------------------------------------------
   // Compute the number of partitions.
   // ---------------------------------------------------------------------
-  desc_struct * partns_desc = part_desc_list;
+  TrafDesc * partns_desc = part_desc_list;
   Lng32 numberOfPartitions = 0;
   while (partns_desc)
     {
       numberOfPartitions++;
-      partns_desc = partns_desc->header.next;
+      partns_desc = partns_desc->next;
     }
 
   // ---------------------------------------------------------------------
@@ -2115,7 +2115,7 @@ static PartitioningFunction * 
createRoundRobinPartitioningFunction
 // This method is used for creating a HashDistPartitioningFunction.
 // -----------------------------------------------------------------------
 static PartitioningFunction * createHashDistPartitioningFunction
-                                (desc_struct * part_desc_list,
+                                (TrafDesc * part_desc_list,
                                 const NAColumnArray & partKeyColArray,
                                  NodeMap* nodeMap,
                                 NAMemory* heap)
@@ -2123,12 +2123,12 @@ static PartitioningFunction * 
createHashDistPartitioningFunction
   // ---------------------------------------------------------------------
   // Compute the number of partitions.
   // ---------------------------------------------------------------------
-  desc_struct * partns_desc = part_desc_list;
+  TrafDesc * partns_desc = part_desc_list;
   Lng32 numberOfPartitions = 0;
   while (partns_desc)
     {
       numberOfPartitions++;
-      partns_desc = partns_desc->header.next;
+      partns_desc = partns_desc->next;
     }
 
   // ---------------------------------------------------------------------
@@ -2148,7 +2148,7 @@ static PartitioningFunction * 
createHashDistPartitioningFunction
 // This method is used for creating a Hash2PartitioningFunction.
 // -----------------------------------------------------------------------
 static PartitioningFunction * createHash2PartitioningFunction
-                                (desc_struct * part_desc_list,
+                                (TrafDesc * part_desc_list,
                                  const NAColumnArray & partKeyColArray,
                                  NodeMap* nodeMap,
                                  NAMemory* heap)
@@ -2156,12 +2156,12 @@ static PartitioningFunction * 
createHash2PartitioningFunction
   // ---------------------------------------------------------------------
   // Compute the number of partitions.
   // ---------------------------------------------------------------------
-  desc_struct * partns_desc = part_desc_list;
+  TrafDesc * partns_desc = part_desc_list;
   Lng32 numberOfPartitions = 0;
   while (partns_desc)
     {
       numberOfPartitions++;
-      partns_desc = partns_desc->header.next;
+      partns_desc = partns_desc->next;
     }
 
   // ---------------------------------------------------------------------
@@ -2195,16 +2195,16 @@ static PartitioningFunction * 
createHash2PartitioningFunction
 
 
 static 
-NodeMap* createNodeMapForHbase(desc_struct* desc, const NATable* table,
+NodeMap* createNodeMapForHbase(TrafDesc* desc, const NATable* table,
                                int numSaltBuckets, NAMemory* heap)
 {
    Int32 partns = 0;
    Int32 numRegions = 0;
-   desc_struct* hrk = desc;
+   TrafDesc* hrk = desc;
  
    while ( hrk ) {
      numRegions++;
-     hrk=hrk->header.next;
+     hrk=hrk->next;
    }
 
    if (numSaltBuckets <= 1)
@@ -2240,13 +2240,13 @@ NodeMap* createNodeMapForHbase(desc_struct* desc, const 
NATable* table,
 
 static 
 PartitioningFunction*
-createHash2PartitioningFunctionForHBase(desc_struct* desc,
+createHash2PartitioningFunctionForHBase(TrafDesc* desc,
                                         const NATable * table,
                                         int numSaltBuckets,
                                         NAMemory* heap)
 {
 
-   desc_struct* hrk = desc;
+   TrafDesc* hrk = desc;
  
    NodeMap* nodeMap = createNodeMapForHbase(desc, table, numSaltBuckets, heap);
 
@@ -2479,23 +2479,23 @@ createRangePartitioningFunctionForSingleRegionHBase(
    }
 
 
-   struct desc_struct* head = NULL;
-   struct desc_struct* tail = NULL;
+   struct TrafDesc* head = NULL;
+   struct TrafDesc* tail = NULL;
 
    Int32 i=0;
    for ( i=0; i<regionsToFake; i++ ) {
 
      if ( tail == NULL ) {
-        head = tail = new (heap) struct desc_struct;
+        head = tail = new (heap) TrafPartnsDesc();
 
         // to satisfy createRangePartitionBoundaries() in NATable.cpp
-        tail->body.partns_desc.primarypartition = 1;
+        tail->partnsDesc()->primarypartition = 1;
 
      } else {
-        tail->header.next = new (heap) struct desc_struct;
-        tail = tail->header.next;
+        tail->next = new (heap) TrafPartnsDesc();
+        tail = tail->next;
      }
-     tail->header.next = NULL;
+     tail->next = NULL;
 
      NAString firstkey('(');
      for ( Int32 i=0; i<keys; i++ ) {
@@ -2523,14 +2523,14 @@ createRangePartitioningFunctionForSingleRegionHBase(
 
      Int32 len = firstkey.length();
 
-     tail->body.partns_desc.firstkeylen = len;
-     tail->body.partns_desc.firstkey = new (heap) char[len];
-     memcpy(tail->body.partns_desc.firstkey, firstkey.data(), len);
+     tail->partnsDesc()->firstkeylen = len;
+     tail->partnsDesc()->firstkey = new (heap) char[len];
+     memcpy(tail->partnsDesc()->firstkey, firstkey.data(), len);
 
      // For now, assume firstkey == encodedkey
-     tail->body.partns_desc.encodedkeylen = len;
-     tail->body.partns_desc.encodedkey = new (heap) char[len];
-     memcpy(tail->body.partns_desc.encodedkey, firstkey.data(), len);
+     tail->partnsDesc()->encodedkeylen = len;
+     tail->partnsDesc()->encodedkey = new (heap) char[len];
+     memcpy(tail->partnsDesc()->encodedkey, firstkey.data(), len);
 
    }
 
@@ -2543,43 +2543,43 @@ createRangePartitioningFunctionForSingleRegionHBase(
 }
 
 void
-populatePartnDescOnEncodingKey( struct desc_struct* prevEndKey,
-                               struct desc_struct* tail, 
-                               struct desc_struct* hrk, 
+populatePartnDescOnEncodingKey( struct TrafDesc* prevEndKey,
+                               struct TrafDesc* tail, 
+                               struct TrafDesc* hrk, 
                                NAMemory* heap)
 {
      if (!prevEndKey) {
        // the start key of the first partitions has all zeroes in it
-       Int32 len = hrk->body.hbase_region_desc.endKeyLen;
+       Int32 len = hrk->hbaseRegionDesc()->endKeyLen;
 
-       tail->body.partns_desc.encodedkeylen = len;
-       tail->body.partns_desc.encodedkey = new (heap) char[len];
-       memset(tail->body.partns_desc.encodedkey, 0, len);
+       tail->partnsDesc()->encodedkeylen = len;
+       tail->partnsDesc()->encodedkey = new (heap) char[len];
+       memset(tail->partnsDesc()->encodedkey, 0, len);
      }
      else {
        // the beginning key of this partition is the end key of
        // the previous one
        // (HBase returns end keys, we need begin keys here)
-       Int32 len = prevEndKey->body.hbase_region_desc.endKeyLen;
+       Int32 len = prevEndKey->hbaseRegionDesc()->endKeyLen;
 
        // For HBase regions, we don't have the text representation
        // (value, value, ... value) of the boundary, just the encoded
        // key.
-       tail->body.partns_desc.encodedkeylen = len;
-       tail->body.partns_desc.encodedkey = new (heap) char[len];
-       memcpy(tail->body.partns_desc.encodedkey, 
-              prevEndKey->body.hbase_region_desc.endKey, len);
+       tail->partnsDesc()->encodedkeylen = len;
+       tail->partnsDesc()->encodedkey = new (heap) char[len];
+       memcpy(tail->partnsDesc()->encodedkey, 
+              prevEndKey->hbaseRegionDesc()->endKey, len);
      }
 }
 
 void
-populatePartnDescOnFirstKey( struct desc_struct* ,
-                             struct desc_struct* tail, 
-                             struct desc_struct* hrk,
+populatePartnDescOnFirstKey( struct TrafDesc* ,
+                             struct TrafDesc* tail, 
+                             struct TrafDesc* hrk,
                              NAMemory* heap)
 {
-   char* buf = hrk->body.hbase_region_desc.beginKey;
-   Int32 len = hrk->body.hbase_region_desc.beginKeyLen;
+   char* buf = hrk->hbaseRegionDesc()->beginKey;
+   Int32 len = hrk->hbaseRegionDesc()->beginKeyLen;
 
    NAString firstkey('(');
    firstkey.append('\'');
@@ -2588,51 +2588,49 @@ populatePartnDescOnFirstKey( struct desc_struct* ,
    firstkey.append(')');
 
    Int32 keyLen = firstkey.length();
-   tail->body.partns_desc.firstkeylen = keyLen;
-   tail->body.partns_desc.firstkey = new (heap) char[keyLen];
-   memcpy(tail->body.partns_desc.firstkey, firstkey.data(), keyLen);
+   tail->partnsDesc()->firstkeylen = keyLen;
+   tail->partnsDesc()->firstkey = new (heap) char[keyLen];
+   memcpy(tail->partnsDesc()->firstkey, firstkey.data(), keyLen);
 
-   tail->body.partns_desc.encodedkeylen = keyLen;
-   tail->body.partns_desc.encodedkey = new (heap) char[keyLen];
-   memcpy(tail->body.partns_desc.encodedkey, firstkey.data(), keyLen);
+   tail->partnsDesc()->encodedkeylen = keyLen;
+   tail->partnsDesc()->encodedkey = new (heap) char[keyLen];
+   memcpy(tail->partnsDesc()->encodedkey, firstkey.data(), keyLen);
 }
 
-typedef void (*populatePartnDescT)( struct desc_struct* prevEndKey,
-                                    struct desc_struct* tail, 
-                                    struct desc_struct* hrk,
+typedef void (*populatePartnDescT)( struct TrafDesc* prevEndKey,
+                                    struct TrafDesc* tail, 
+                                    struct TrafDesc* hrk,
                                     NAMemory* heap);
-static struct desc_struct*
-convertRangeDescToPartnsDesc(desc_struct* desc, populatePartnDescT funcPtr, 
NAMemory* heap)
+static struct TrafDesc*
+convertRangeDescToPartnsDesc(TrafDesc* desc, populatePartnDescT funcPtr, 
NAMemory* heap)
 {
-   desc_struct* hrk = desc;
-   desc_struct* prevEndKey = NULL;
+   TrafDesc* hrk = desc;
+   TrafDesc* prevEndKey = NULL;
  
-   struct desc_struct* head = NULL;
-   struct desc_struct* tail = NULL;
+   struct TrafDesc* head = NULL;
+   struct TrafDesc* tail = NULL;
 
    Int32 i=0;
    while ( hrk ) {
 
-     struct desc_struct *newNode = new (heap) struct desc_struct;
-     memset(&newNode->header, 0, sizeof(newNode->header));
-     memset(&newNode->body.partns_desc, 0, sizeof(tail->body.partns_desc));
-     newNode->header.nodetype = DESC_PARTNS_TYPE;
+     struct TrafDesc *newNode = 
+       TrafAllocateDDLdesc(DESC_PARTNS_TYPE, NULL);
 
      if ( tail == NULL ) {
         head = tail = newNode;
 
         // to satisfy createRangePartitionBoundaries() in NATable.cpp
-        tail->body.partns_desc.primarypartition = 1;
+        tail->partnsDesc()->primarypartition = 1;
 
      } else {
-        tail->header.next = newNode;
-        tail = tail->header.next;
+        tail->next = newNode;
+        tail = tail->next;
      }
 
      (*funcPtr)(prevEndKey, tail, hrk, heap);
 
      prevEndKey = hrk;
-     hrk     = hrk->header.next;
+     hrk     = hrk->next;
    }
 
    return head;
@@ -2642,14 +2640,14 @@ convertRangeDescToPartnsDesc(desc_struct* desc, 
populatePartnDescT funcPtr, NAMe
 static 
 PartitioningFunction*
 createRangePartitioningFunctionForMultiRegionHBase(Int32 partns,
-                                        desc_struct* desc, 
+                                        TrafDesc* desc, 
                                         const NATable* table, 
                                        const NAColumnArray & partKeyColArray,
                                         NAMemory* heap)
 {
    NodeMap* nodeMap = createNodeMapForHbase(desc, table, -1, heap);
 
-   struct desc_struct* 
+   struct TrafDesc* 
       partns_desc = ( table->isHbaseCellTable() || table->isHbaseRowTable()) ?
          convertRangeDescToPartnsDesc(desc, populatePartnDescOnFirstKey, heap)
              :
@@ -2663,13 +2661,13 @@ 
createRangePartitioningFunctionForMultiRegionHBase(Int32 partns,
                                  heap);
 }
 
-Int32 findDescEntries(desc_struct* desc)
+Int32 findDescEntries(TrafDesc* desc)
 {
    Int32 partns = 0;
-   desc_struct* hrk = desc;
+   TrafDesc* hrk = desc;
    while ( hrk ) {
      partns++;
-     hrk = hrk->header.next;
+     hrk = hrk->next;
    }
    return partns;
 }
@@ -2680,7 +2678,7 @@ Int32 findDescEntries(desc_struct* desc)
 //
 static 
 PartitioningFunction*
-createRangePartitioningFunctionForHBase(desc_struct* desc, 
+createRangePartitioningFunctionForHBase(TrafDesc* desc, 
                                        const NATable* table,
                                         const NAColumnArray & partKeyColArray,
                                         NAMemory* heap)
@@ -2723,7 +2721,7 @@ static PartitioningFunction * 
createHivePartitioningFunction
 // associated with this table or index.
 // -----------------------------------------------------------------------
 #pragma nowarn(1506)   // warning elimination
-static void createNodeMap (desc_struct* part_desc_list,
+static void createNodeMap (TrafDesc* part_desc_list,
                           NodeMap*     nodeMap,
                            NAMemory*    heap,
                            char * tableName,
@@ -2733,7 +2731,7 @@ static void createNodeMap (desc_struct* part_desc_list,
   // Loop over all partitions creating a DP2 node map entry for each
   // partition.
   // ---------------------------------------------------------------------
-  desc_struct* partns_desc      = part_desc_list;
+  TrafDesc* partns_desc      = part_desc_list;
   CollIndex    currentPartition = 0;
   if(NOT partns_desc)
   {
@@ -2744,11 +2742,11 @@ static void createNodeMap (desc_struct* part_desc_list,
   else{
     while (partns_desc)
     {
-      NodeMapEntry entry(partns_desc->body.partns_desc.partitionname,
-       partns_desc->body.partns_desc.givenname,
+      NodeMapEntry entry(partns_desc->partnsDesc()->partitionname,
+       partns_desc->partnsDesc()->givenname,
         heap,tableIdent);
       nodeMap->setNodeMapEntry(currentPartition,entry,heap);
-      partns_desc = partns_desc->header.next;
+      partns_desc = partns_desc->next;
       currentPartition++;
     }
   }
@@ -2887,7 +2885,7 @@ static void createNodeMap (hive_tbl_desc* hvt_desc,
 // on the local node.
 //-------------------------------------------------------------------------
 #pragma nowarn(262)   // warning elimination
-static NABoolean checkRemote(desc_struct* part_desc_list,
+static NABoolean checkRemote(TrafDesc* part_desc_list,
                              char * tableName)
 {
     return TRUE;
@@ -2895,19 +2893,17 @@ static NABoolean checkRemote(desc_struct* 
part_desc_list,
 #pragma warn(262)  // warning elimination
 
 
-// warning elimination (removed "inline")
 static NAString makeTableName(const NATable *table,
-                             const columns_desc_struct *column_desc)
+                             const TrafColumnsDesc *column_desc)
 {
   return NAString(
-             table ?
-               table->getTableName().getQualifiedNameAsAnsiString().data() :
-             column_desc->tablename ?
-               column_desc->tablename : "");
+       table ?
+       table->getTableName().getQualifiedNameAsAnsiString().data() : "");
 }
+
 // warning elimination (removed "inline")
 static NAString makeColumnName(const NATable *table,
-                              const columns_desc_struct *column_desc)
+                              const TrafColumnsDesc *column_desc)
 {
   NAString nam(makeTableName(table, column_desc));
   if (!nam.isNull()) nam += ".";
@@ -2916,9 +2912,9 @@ static NAString makeColumnName(const NATable *table,
 }
 
 // -----------------------------------------------------------------------
-// Method for creating NAType from desc_struct.
+// Method for creating NAType from TrafDesc.
 // -----------------------------------------------------------------------
-NABoolean createNAType(columns_desc_struct *column_desc        /*IN*/,
+NABoolean createNAType(TrafColumnsDesc *column_desc    /*IN*/,
                       const NATable *table             /*IN*/,
                       NAType *&type                    /*OUT*/,
                       NAMemory *heap                   /*IN*/,
@@ -2930,7 +2926,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
   //
   #define REC_INTERVAL REC_MIN_INTERVAL
 
-  DataType datatype = column_desc->datatype;
+  Int16 datatype = column_desc->datatype;
   if (REC_MIN_INTERVAL <= datatype && datatype <= REC_MAX_INTERVAL)
     datatype = REC_INTERVAL;
 
@@ -2938,7 +2934,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
 
   if ( DFS2REC::isAnyCharacter(column_desc->datatype) )
   {
-     if ( CharInfo::isCharSetSupported(column_desc->character_set) == FALSE ) {
+     if ( CharInfo::isCharSetSupported(column_desc->characterSet()) == FALSE ) 
{
        if (!errorCode)
        {
          *CmpCommon::diags() << DgSqlCode(-4082)
@@ -2951,7 +2947,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
        return TRUE; // error
      }
 
-     if ( CharInfo::is_NCHAR_MP(column_desc->character_set) )
+     if ( CharInfo::is_NCHAR_MP(column_desc->characterSet()) )
         charCount /= SQL_DBCHAR_SIZE;
   }
 
@@ -2960,7 +2956,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
 
     case REC_BPINT_UNSIGNED :
       type = new (heap)
-      SQLBPInt(column_desc->precision, column_desc->null_flag, FALSE, heap);
+      SQLBPInt(column_desc->precision, column_desc->isNullable(), FALSE, heap);
       break;
 
     case REC_BIN8_SIGNED:
@@ -2970,13 +2966,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   TRUE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
        SQLTiny(TRUE,
-                column_desc->null_flag,
+                column_desc->isNullable(),
                  heap
                 );
       break;
@@ -2987,13 +2983,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   FALSE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
        SQLTiny(FALSE,
-                column_desc->null_flag,
+                column_desc->isNullable(),
                  heap
                 );
       break;
@@ -3005,13 +3001,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   TRUE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
        SQLSmall(TRUE,
-                column_desc->null_flag,
+                column_desc->isNullable(),
                  heap
                 );
       break;
@@ -3022,13 +3018,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   FALSE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
        SQLSmall(FALSE,
-                column_desc->null_flag,
+                column_desc->isNullable(),
                  heap
                 );
       break;
@@ -3040,13 +3036,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   TRUE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
        SQLInt(TRUE,
-              column_desc->null_flag,
+              column_desc->isNullable(),
                heap
               );
       break;
@@ -3057,13 +3053,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   FALSE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
        SQLInt(FALSE,
-              column_desc->null_flag,
+              column_desc->isNullable(),
                heap
               );
       break;
@@ -3074,13 +3070,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   TRUE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
        SQLLargeInt(TRUE,
-                   column_desc->null_flag,
+                   column_desc->isNullable(),
                     heap
                    );
       break;
@@ -3091,13 +3087,13 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                   column_desc->precision,
                   column_desc->scale,
                   FALSE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       else
        type = new (heap)
           SQLLargeInt(FALSE,
-                   column_desc->null_flag,
+                   column_desc->isNullable(),
                     heap
                    );
       break;
@@ -3106,7 +3102,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
        SQLDecimal(column_desc->length,
                   column_desc->scale,
                   FALSE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       break;
@@ -3115,7 +3111,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
        SQLDecimal(column_desc->length,
                   column_desc->scale,
                   TRUE,
-                  column_desc->null_flag,
+                  column_desc->isNullable(),
                    heap
                   );
       break;
@@ -3125,7 +3121,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
                  column_desc->scale,
                  TRUE, // is a real bignum
                  FALSE,
-                 column_desc->null_flag,
+                 column_desc->isNullable(),
                  heap
                  );
       break;
@@ -3135,65 +3131,65 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
                  column_desc->scale,
                  TRUE, // is a real bignum
                  TRUE,
-                 column_desc->null_flag,
+                 column_desc->isNullable(),
                  heap
                  );
       break;
 
     case REC_FLOAT32:
       type = new (heap)
-       SQLReal(column_desc->null_flag, heap, column_desc->precision);
+       SQLReal(column_desc->isNullable(), heap, column_desc->precision);
       break;
 
     case REC_FLOAT64:
       type = new (heap)
-       SQLDoublePrecision(column_desc->null_flag, heap, 
column_desc->precision);
+       SQLDoublePrecision(column_desc->isNullable(), heap, 
column_desc->precision);
       break;
 
     case REC_BYTE_F_DOUBLE:
       charCount /= SQL_DBCHAR_SIZE;        // divide the storage length by 2
       type = new (heap)
        SQLChar(charCount,
-               column_desc->null_flag,
-               column_desc->upshift,
-               column_desc->caseinsensitive,
+               column_desc->isNullable(),
+               column_desc->isUpshifted(),
+               column_desc->isCaseInsensitive(),
                FALSE,
-               column_desc->character_set,
-               column_desc->collation_sequence,
+               column_desc->characterSet(),
+               column_desc->collationSequence(),
                CharInfo::IMPLICIT
                );
       break;
 
     case REC_BYTE_F_ASCII:
-      if (column_desc->character_set == CharInfo::UTF8 ||
-          (column_desc->character_set == CharInfo::SJIS &&
+      if (column_desc->characterSet() == CharInfo::UTF8 ||
+          (column_desc->characterSet() == CharInfo::SJIS &&
            column_desc->encoding_charset == CharInfo::SJIS))
       {
-        Lng32 maxBytesPerChar = 
CharInfo::maxBytesPerChar(column_desc->character_set);
+        Lng32 maxBytesPerChar = 
CharInfo::maxBytesPerChar(column_desc->characterSet());
         Lng32 sizeInChars = charCount ;  // Applies when CharLenUnit == BYTES
         if ( column_desc->precision > 0 )
            sizeInChars = column_desc->precision;
         type = new (heap)
        SQLChar(CharLenInfo(sizeInChars, charCount/*in_bytes*/),
-               column_desc->null_flag,
-               column_desc->upshift,
-               column_desc->caseinsensitive,
+               column_desc->isNullable(),
+               column_desc->isUpshifted(),
+               column_desc->isCaseInsensitive(),
                FALSE, // varLenFlag
-               column_desc->character_set,
-               column_desc->collation_sequence,
+               column_desc->characterSet(),
+               column_desc->collationSequence(),
                CharInfo::IMPLICIT, // Coercibility
-               column_desc->encoding_charset
+               column_desc->encodingCharset()
                );
       }
       else // keep the old behavior
       type = new (heap)
        SQLChar(charCount,
-               column_desc->null_flag,
-               column_desc->upshift,
-               column_desc->caseinsensitive,
+               column_desc->isNullable(),
+               column_desc->isUpshifted(),
+               column_desc->isCaseInsensitive(),
                FALSE,
-               column_desc->character_set,
-               column_desc->collation_sequence,
+               column_desc->characterSet(),
+               column_desc->collationSequence(),
                CharInfo::IMPLICIT
                );
       break;
@@ -3202,32 +3198,32 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
       charCount /= SQL_DBCHAR_SIZE;        // divide the storage length by 2
       // fall thru
     case REC_BYTE_V_ASCII:
-      if (column_desc->character_set == CharInfo::SJIS ||
-          column_desc->character_set == CharInfo::UTF8)
+      if (column_desc->characterSet() == CharInfo::SJIS ||
+          column_desc->characterSet() == CharInfo::UTF8)
       {
-        Lng32 maxBytesPerChar = 
CharInfo::maxBytesPerChar(column_desc->character_set);
+        Lng32 maxBytesPerChar = 
CharInfo::maxBytesPerChar(column_desc->characterSet());
         Lng32 sizeInChars = charCount ;  // Applies when CharLenUnit == BYTES
         if ( column_desc->precision > 0 )
            sizeInChars = column_desc->precision;
         type = new (heap)
        SQLVarChar(CharLenInfo(sizeInChars, charCount/*in_bytes*/),
-                  column_desc->null_flag,
-                  column_desc->upshift,
-                  column_desc->caseinsensitive,
-                  column_desc->character_set,
-                  column_desc->collation_sequence,
+                  column_desc->isNullable(),
+                  column_desc->isUpshifted(),
+                  column_desc->isCaseInsensitive(),
+                  column_desc->characterSet(),
+                  column_desc->collationSequence(),
                   CharInfo::IMPLICIT, // Coercibility
-                  column_desc->encoding_charset
+                  column_desc->encodingCharset()
                   );
       }
       else // keep the old behavior
       type = new (heap)
        SQLVarChar(charCount,
-                  column_desc->null_flag,
-                  column_desc->upshift,
-                  column_desc->caseinsensitive,
-                  column_desc->character_set,
-                  column_desc->collation_sequence,
+                  column_desc->isNullable(),
+                  column_desc->isUpshifted(),
+                  column_desc->isCaseInsensitive(),
+                  column_desc->characterSet(),
+                  column_desc->collationSequence(),
                   CharInfo::IMPLICIT
                   );
       break;
@@ -3236,33 +3232,33 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
       type = new (heap)
        SQLLongVarChar(charCount,
                       FALSE,
-                      column_desc->null_flag,
-                      column_desc->upshift,
-                      column_desc->caseinsensitive,
-                      column_desc->character_set,
-                      column_desc->collation_sequence,
+                      column_desc->isNullable(),
+                      column_desc->isUpshifted(),
+                      column_desc->isCaseInsensitive(),
+                      column_desc->characterSet(),
+                      column_desc->collationSequence(),
                       CharInfo::IMPLICIT
                      );
       break;
     case REC_DATETIME:
       type = DatetimeType::constructSubtype(
-                                           column_desc->null_flag,
-                                           column_desc->datetimestart,
-                                           column_desc->datetimeend,
+                                           column_desc->isNullable(),
+                                           column_desc->datetimeStart(),
+                                           column_desc->datetimeEnd(),
                                            column_desc->datetimefractprec,
                                            heap
                                            );
       CMPASSERT(type);
       if (!type->isSupportedType())
        {
-         column_desc->defaultClass = COM_NO_DEFAULT;           // can't set a 
default for these, either.
+          column_desc->setDefaultClass(COM_NO_DEFAULT);           // can't set 
a default for these, either.
          // 4030 Column is an unsupported combination of datetime fields
      if (!errorCode)
      {
          *CmpCommon::diags() << DgSqlCode(4030)
            << DgColumnName(makeColumnName(table, column_desc))
-           << DgInt0(column_desc->datetimestart)
-           << DgInt1(column_desc->datetimeend)
+           << DgInt0(column_desc->datetimeStart())
+           << DgInt1(column_desc->datetimeEnd())
            << DgInt2(column_desc->datetimefractprec);
      }
      else
@@ -3273,10 +3269,10 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
       break;
     case REC_INTERVAL:
       type = new (heap)
-         SQLInterval(column_desc->null_flag,
-                   column_desc->datetimestart,
+         SQLInterval(column_desc->isNullable(),
+                   column_desc->datetimeStart(),
                    column_desc->intervalleadingprec,
-                   column_desc->datetimeend,
+                   column_desc->datetimeEnd(),
                    column_desc->datetimefractprec,
                     heap
                    );
@@ -3285,7 +3281,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
          return TRUE;                                            // error
       if (!type->isSupportedType())
       {
-        column_desc->defaultClass = COM_NO_DEFAULT;           // can't set a 
default for these, either.
+        column_desc->setDefaultClass(COM_NO_DEFAULT);           // can't set a 
default for these, either.
         if (!errorCode)
           *CmpCommon::diags() << DgSqlCode(3044) << 
DgString0(column_desc->colname);
         else
@@ -3297,18 +3293,18 @@ NABoolean createNAType(columns_desc_struct *column_desc 
/*IN*/,
     case REC_BLOB :
       type = new (heap)
        SQLBlob(column_desc->precision,Lob_Invalid_Storage,
-               column_desc->null_flag);
+               column_desc->isNullable());
       break;
 
     case REC_CLOB :
       type = new (heap)
        SQLClob(column_desc->precision,Lob_Invalid_Storage,
-               column_desc->null_flag);
+               column_desc->isNullable());
       break;
 
     case REC_BOOLEAN :
       {
-        type = new (heap) SQLBooleanNative(column_desc->null_flag);
+        type = new (heap) SQLBooleanNative(column_desc->isNullable());
       }
       break;
 
@@ -3393,7 +3389,7 @@ NABoolean createNAType(columns_desc_struct *column_desc   
/*IN*/,
 // one for each column_desc in the list supplied as input.
 // -----------------------------------------------------------------------
 #pragma nowarn(1506)   // warning elimination
-NABoolean createNAColumns(desc_struct *column_desc_list        /*IN*/,
+NABoolean createNAColumns(TrafDesc *column_desc_list   /*IN*/,
                          NATable *table                /*IN*/,
                          NAColumnArray &colArray       /*OUT*/,
                          NAMemory *heap                /*IN*/)
@@ -3402,7 +3398,7 @@ NABoolean createNAColumns(desc_struct *column_desc_list   
/*IN*/,
   ColumnClass colClass;
   while (column_desc_list)
     {
-      columns_desc_struct * column_desc = &column_desc_list->body.columns_desc;
+      TrafColumnsDesc * column_desc = column_desc_list->columnsDesc();
       NABoolean isMvSystemAdded = FALSE;
       NABoolean hasSystemColumnAsUserColumn = FALSE;
 
@@ -3461,7 +3457,7 @@ NABoolean createNAColumns(desc_struct *column_desc_list   
/*IN*/,
          NABoolean isSaltColumn = FALSE;
          NABoolean isDivisioningColumn = FALSE;
 
-         if (column_desc->defaultClass == 
COM_ALWAYS_COMPUTE_COMPUTED_COLUMN_DEFAULT)
+         if (column_desc->defaultClass() == 
COM_ALWAYS_COMPUTE_COMPUTED_COLUMN_DEFAULT)
            {
              if (column_desc->colFlags & SEABASE_COLUMN_IS_SALT)
                isSaltColumn = TRUE;
@@ -3502,16 +3498,16 @@ NABoolean createNAColumns(desc_struct *column_desc_list 
/*IN*/,
                                heap,
                               table,
                               colClass,
-                              column_desc->defaultClass,
+                              column_desc->defaultClass(),
                               defaultValue,
                                heading,
-                              column_desc->upshift,
+                              column_desc->isUpshifted(),
                               ((column_desc->colclass == 'A') ||
                                 (column_desc->colclass == 'C')),
                                COM_UNKNOWN_DIRECTION,
                                FALSE,
                                NULL,
-                               column_desc->stored_on_disk,
+                               TRUE, // stored on disk
                                computed_column_text,
                                isSaltColumn,
                                isDivisioningColumn,
@@ -3561,7 +3557,7 @@ NABoolean createNAColumns(desc_struct *column_desc_list   
/*IN*/,
 
       colArray.insert(newColumn);
 
-      column_desc_list = column_desc_list->header.next;
+      column_desc_list = column_desc_list->next;
     } // end while
 
   return FALSE;                                                        // no 
error
@@ -3790,7 +3786,7 @@ NABoolean createNAColumns(struct hive_column_desc* 
hcolumn /*IN*/,
                              COM_NULL_DEFAULT  ,//defaultClass,
                              (char*)"", // defaultValue,
                              (char*)"", // heading,
-                             FALSE, // column_desc->upshift,
+                             FALSE, // column_desc->isUpshifted(),
                              FALSE, // added column
                              COM_UNKNOWN_DIRECTION,
                              FALSE,  // isOptional
@@ -3821,30 +3817,30 @@ NABoolean createNAColumns(struct hive_column_desc* 
hcolumn /*IN*/,
 
 
 
-NABoolean createNAKeyColumns(desc_struct *keys_desc_list       /*IN*/,
+NABoolean createNAKeyColumns(TrafDesc *keys_desc_list  /*IN*/,
                             NAColumnArray &colArray    /*IN*/,
                             NAColumnArray &keyColArray /*OUT*/,
                             CollHeap *heap             /*IN*/)
 {
-  const desc_struct *keys_desc = keys_desc_list;
+  const TrafDesc *keys_desc = keys_desc_list;
 
   while (keys_desc)
     {
-      Int32 tablecolnumber = keys_desc->body.keys_desc.tablecolnumber;
+      Int32 tablecolnumber = keys_desc->keysDesc()->tablecolnumber;
 
       NAColumn *indexColumn = colArray.getColumn(tablecolnumber);
 
       SortOrdering order = NOT_ORDERED;
 
       keyColArray.insert(indexColumn);
-      order = keys_desc->body.keys_desc.ordering ? DESCENDING : ASCENDING;
+      order = keys_desc->keysDesc()->isDescending() ? DESCENDING : ASCENDING;
       keyColArray.setAscending(keyColArray.entries()-1, order == ASCENDING);
 
       // Remember that this columns is part of the clustering
       // key and remember its key ordering (asc or desc)
       indexColumn->setClusteringKey(order);
 
-      keys_desc = keys_desc->header.next;
+      keys_desc = keys_desc->next;
     } // end while (keys_desc)
 
   return FALSE;
@@ -3893,7 +3889,7 @@ void 
processDuplicateNames(NAHashDictionaryIterator<NAString, Int32> &Iter,
 // -----------------------------------------------------------------------
 #pragma nowarn(1506)   // warning elimination
 static
-NABoolean createNAFileSets(desc_struct * table_desc       /*IN*/,
+NABoolean createNAFileSets(TrafDesc * table_desc       /*IN*/,
                            const NATable * table          /*IN*/,
                            const NAColumnArray & colArray /*IN*/,
                            NAFileSetList & indexes        /*OUT*/,
@@ -3919,17 +3915,16 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
 
   NAList<NAString *> stringList (CmpCommon::statementHeap());
 
-  desc_struct *indexes_desc = table_desc->body.table_desc.indexes_desc;
+  TrafDesc *indexes_desc = table_desc->tableDesc()->indexes_desc;
 
-  while (indexes_desc AND indexes_desc->body.indexes_desc.keytag)
-    indexes_desc = indexes_desc->header.next;
+  while (indexes_desc AND indexes_desc->indexesDesc()->keytag)
+    indexes_desc = indexes_desc->next;
 
   // must have a clustering key if not view
-  CMPASSERT((indexes_desc AND !indexes_desc->body.indexes_desc.keytag) OR
-           (table_desc->body.table_desc.views_desc));
+  CMPASSERT((indexes_desc AND !indexes_desc->indexesDesc()->keytag) OR
+           (table_desc->tableDesc()->views_desc));
 
   NABoolean isTheClusteringKey = TRUE;
-  NABoolean isVerticalPartition;
   NABoolean hasRemotePartition = FALSE;
   CollIndex numClusteringKeyColumns = 0;
   NABoolean tableAlignedRowFormat = table->isSQLMXAlignedTable();
@@ -3988,13 +3983,10 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
       NAColumnArray saveNAColumns(CmpCommon::statementHeap());// save NAColums 
of secondary index columns
       NAColumnArray partitioningKeyColumns(CmpCommon::statementHeap());// the 
partitioning key columns
       PartitioningFunction * partFunc = NULL;
-      // is this an index or is it really a VP?
-      isVerticalPartition = 
indexes_desc->body.indexes_desc.isVerticalPartition;
-      NABoolean isPacked = indexes_desc->body.indexes_desc.isPacked;
-      NABoolean indexAlignedRowFormat = 
(indexes_desc->body.indexes_desc.rowFormat == COM_ALIGNED_FORMAT_TYPE);
+      NABoolean isPacked = FALSE;
+      NABoolean indexAlignedRowFormat = 
(indexes_desc->indexesDesc()->rowFormat() == COM_ALIGNED_FORMAT_TYPE);
 
-      NABoolean isNotAvailable =
-       indexes_desc->body.indexes_desc.notAvailable;
+      NABoolean isNotAvailable = FALSE;
 
       ItemExprList hbaseSaltColumnList(CmpCommon::statementHeap());
       Int64 numOfSaltedPartitions = 0;
@@ -4002,7 +3994,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
       // ---------------------------------------------------------------------
       // loop over the clustering key columns of the index
       // ---------------------------------------------------------------------
-      const desc_struct *keys_desc = indexes_desc->body.indexes_desc.keys_desc;
+      const TrafDesc *keys_desc = indexes_desc->indexesDesc()->keys_desc;
       while (keys_desc)
        {
           // Add an index/VP key column.
@@ -4015,7 +4007,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
          // key columns used to find the base table record. This is
          // true for both SQL/MP and SQL/MX tables at this time.
          // To make these assumptions is not optimal, but the
-         // desc_structs that are used as input are a historical
+         // TrafDescs that are used as input are a historical
          // leftover from SQL/MP and therefore aren't set up very
          // well to describe index columns and index keys.  Some day
          // we might consider a direct conversion from the MX catalog
@@ -4026,17 +4018,17 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
          // of a unique alternate index (which ARE described in the
          // keys_desc) get deleted later.
           
-         Int32 tablecolnumber = keys_desc->body.keys_desc.tablecolnumber;
+         Int32 tablecolnumber = keys_desc->keysDesc()->tablecolnumber;
           indexColumn = colArray.getColumn(tablecolnumber);
           
           if ((table->isHbaseTable()) &&
-              ((indexes_desc->body.indexes_desc.keytag != 0) || 
+              ((indexes_desc->indexesDesc()->keytag != 0) || 
                 (indexAlignedRowFormat  && indexAlignedRowFormat != 
tableAlignedRowFormat)))
             {
               newIndexColumn = new(heap) NAColumn(*indexColumn);
-              
newIndexColumn->setIndexColName(keys_desc->body.keys_desc.keyname);
-              
newIndexColumn->setHbaseColFam(keys_desc->body.keys_desc.hbaseColFam);
-              
newIndexColumn->setHbaseColQual(keys_desc->body.keys_desc.hbaseColQual);
+              newIndexColumn->setIndexColName(keys_desc->keysDesc()->keyname);
+              
newIndexColumn->setHbaseColFam(keys_desc->keysDesc()->hbaseColFam);
+              
newIndexColumn->setHbaseColQual(keys_desc->keysDesc()->hbaseColQual);
               newIndexColumn->resetSerialization(); 
               saveNAColumns.insert(indexColumn);
               newColumns.insert(newIndexColumn);
@@ -4050,7 +4042,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
           // assume at first that all columns are key columns
           // and we make adjustments later
           indexKeyColumns.insert(indexColumn);
-          order = keys_desc->body.keys_desc.ordering ?
+          order = keys_desc->keysDesc()->isDescending() ?
             DESCENDING : ASCENDING;
           indexKeyColumns.setAscending(indexKeyColumns.entries() - 1,
                                        order == ASCENDING);
@@ -4109,7 +4101,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
            allColumns.insert(indexColumn);
          }
 
-         keys_desc = keys_desc->header.next;
+         keys_desc = keys_desc->next;
        } // end while (keys_desc)
 
       // ---------------------------------------------------------------------
@@ -4117,32 +4109,32 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
       // These columns get added to the list of all the columns for the index/
       // VP.  Their length also contributes to the total record length.
       // ---------------------------------------------------------------------
-      const desc_struct *non_keys_desc =
-                         indexes_desc->body.indexes_desc.non_keys_desc;
+      const TrafDesc *non_keys_desc =
+                         indexes_desc->indexesDesc()->non_keys_desc;
       while (non_keys_desc)
        {
-         Int32 tablecolnumber = non_keys_desc->body.keys_desc.tablecolnumber;
+         Int32 tablecolnumber = non_keys_desc->keysDesc()->tablecolnumber;
           indexColumn = colArray.getColumn(tablecolnumber);
 
          if ((table->isHbaseTable()) &&
-             ((indexes_desc->body.indexes_desc.keytag != 0) || 
+             ((indexes_desc->indexesDesc()->keytag != 0) || 
                (indexAlignedRowFormat  && indexAlignedRowFormat != 
tableAlignedRowFormat)))
            {
              newIndexColumn = new(heap) NAColumn(*indexColumn);
-             if (non_keys_desc->body.keys_desc.keyname)
-               
newIndexColumn->setIndexColName(non_keys_desc->body.keys_desc.keyname);
-             
newIndexColumn->setHbaseColFam(non_keys_desc->body.keys_desc.hbaseColFam);
-             
newIndexColumn->setHbaseColQual(non_keys_desc->body.keys_desc.hbaseColQual);
+             if (non_keys_desc->keysDesc()->keyname)
+               
newIndexColumn->setIndexColName(non_keys_desc->keysDesc()->keyname);
+             
newIndexColumn->setHbaseColFam(non_keys_desc->keysDesc()->hbaseColFam);
+             
newIndexColumn->setHbaseColQual(non_keys_desc->keysDesc()->hbaseColQual);
               newIndexColumn->resetSerialization(); 
              indexColumn = newIndexColumn;
               newColumns.insert(newIndexColumn);
            }
 
          allColumns.insert(indexColumn);
-         non_keys_desc = non_keys_desc->header.next;
+         non_keys_desc = non_keys_desc->next;
        } // end while (non_keys_desc)
 
-      desc_struct *files_desc;
+      TrafDesc *files_desc;
       NABoolean isSystemTable;
       if (isTheClusteringKey)
        {
@@ -4161,15 +4153,15 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
                }
            } // end for
 
-         files_desc = table_desc->body.table_desc.files_desc;
-         isSystemTable = table_desc->body.table_desc.issystemtablecode;
+         files_desc = table_desc->tableDesc()->files_desc;
+         isSystemTable = table_desc->tableDesc()->isSystemTableCode();
 
           // Record length of clustering key is the same as that of the base 
table record
-          indexes_desc->body.indexes_desc.record_length = 
table_desc->body.table_desc.record_length;
+          indexes_desc->indexesDesc()->record_length = 
table_desc->tableDesc()->record_length;
        } // endif (isTheClusteringKey)
       else
        {
-         if (indexes_desc->body.indexes_desc.unique)
+         if (indexes_desc->indexesDesc()->isUnique())
            {
              // As mentioned above, if this is a unique index,
              // the last numClusteringKeyColumns are actually not
@@ -4187,8 +4179,8 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
              //   indexKeyColumns.removeAt(indexKeyColumns.entries() - 1);
            }
 
-         files_desc = indexes_desc->body.indexes_desc.files_desc;
-         isSystemTable = indexes_desc->body.indexes_desc.issystemtablecode;
+         files_desc = indexes_desc->indexesDesc()->files_desc;
+         isSystemTable = indexes_desc->indexesDesc()->isSystemTableCode();
 
        } // endif (NOT isTheClusteringKey)
 
@@ -4204,9 +4196,10 @@ NABoolean createNAFileSets(desc_struct * table_desc      
 /*IN*/,
       // We will ASSUME here that NonStop SQL/MP or the simulator will not
       // put anything into partitioning keys desc and only SQL/MX will.  So
       // we don't have to deal with keytag columns here.
+      //
+      // Partitioning Keys Desc is not set and returned for traf tables.
       // -------------------------------------------------------------------
-      const desc_struct *partitioning_keys_desc =
-                         
indexes_desc->body.indexes_desc.partitioning_keys_desc;
+      const TrafDesc *partitioning_keys_desc = NULL;
 
       // the key columns that build the salt column for HBase table
       NAColumnArray hbaseSaltOnColumns(CmpCommon::statementHeap());
@@ -4217,16 +4210,16 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
           while (keys_desc)
            {
               Int32 tablecolnumber = keys_desc
-                                   ->body.keys_desc.tablecolnumber;
+                                   ->keysDesc()->tablecolnumber;
               indexColumn = colArray.getColumn(tablecolnumber);
               partitioningKeyColumns.insert(indexColumn);
               SortOrdering order = keys_desc
-                                    ->body.keys_desc.ordering ?
-                                      DESCENDING : ASCENDING;
+                ->keysDesc()->isDescending() ?
+                DESCENDING : ASCENDING;
              partitioningKeyColumns.setAscending
                                        (partitioningKeyColumns.entries() - 1,
                                        order == ASCENDING);
-              keys_desc = keys_desc->header.next;
+              keys_desc = keys_desc->next;
             } // end while (keys_desc)
         }
       else {
@@ -4254,29 +4247,26 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
 
       // NB: Just in case, we made a call to setupClusterInfo at the
       // beginning of this function.
-      desc_struct * partns_desc;
+      TrafDesc * partns_desc;
       Int32 indexLevels = 1;
-      Int32 blockSize = indexes_desc->body.indexes_desc.blocksize;
+      Int32 blockSize = indexes_desc->indexesDesc()->blocksize;
       if (files_desc)
       {
        if( (table->getSpecialType() != ExtendedQualName::VIRTUAL_TABLE AND
-/*
-            table->getSpecialType() != ExtendedQualName::ISP_TABLE AND
-*/
             (NOT table->isHbaseTable()))
-           OR files_desc->body.files_desc.partns_desc )
+           OR files_desc->filesDesc()->partns_desc )
          {
             nodeMap = new (heap) NodeMap(heap);
-           createNodeMap(files_desc->body.files_desc.partns_desc,
+           createNodeMap(files_desc->filesDesc()->partns_desc,
                          nodeMap,
                          heap,
-                         table_desc->body.table_desc.tablename,
+                         table_desc->tableDesc()->tablename,
                          cmpCurrentContext->getTableIdent());
            tableIdList.insert(CollIndex(cmpCurrentContext->getTableIdent()));
          }
        // Check whether the index has any remote partitions.
-       if (checkRemote(files_desc->body.files_desc.partns_desc,
-                       indexes_desc->body.indexes_desc.indexname))
+       if (checkRemote(files_desc->filesDesc()->partns_desc,
+                       indexes_desc->indexesDesc()->indexname))
          hasRemotePartition = TRUE;
        else
          hasRemotePartition = FALSE;
@@ -4285,12 +4275,12 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
        // partitioned same as the indexes, hence we used table partitioning
        // to create partitionining function. But this is not true. Hence
        // we now use the indexes partitioning function
-       switch (indexes_desc->body.indexes_desc.partitioningScheme)
+       switch (indexes_desc->indexesDesc()->partitioningScheme())
         {
        case COM_ROUND_ROBIN_PARTITIONING :
          // Round Robin partitioned table
          partFunc = createRoundRobinPartitioningFunction(
-              files_desc->body.files_desc.partns_desc,
+              files_desc->filesDesc()->partns_desc,
               nodeMap,
               heap);
          break;
@@ -4298,7 +4288,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
        case COM_HASH_V1_PARTITIONING :
          // Hash partitioned table
          partFunc = createHashDistPartitioningFunction(
-              files_desc->body.files_desc.partns_desc,
+              files_desc->filesDesc()->partns_desc,
               partitioningKeyColumns,
               nodeMap,
               heap);
@@ -4307,7 +4297,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
        case COM_HASH_V2_PARTITIONING :
          // Hash partitioned table
          partFunc = createHash2PartitioningFunction(
-              files_desc->body.files_desc.partns_desc,
+              files_desc->filesDesc()->partns_desc,
               partitioningKeyColumns,
               nodeMap,
               heap);
@@ -4325,8 +4315,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
            // values as MP Stored Text.
            //
 
-              desc_struct* hbd = 
-                   ((table_desc_struct*)table_desc)->hbase_regionkey_desc;
+              TrafDesc* hbd = table_desc->tableDesc()->hbase_regionkey_desc;
 
               // splits will be 1 for single partitioned table.
               Int32 splits = findDescEntries(hbd);
@@ -4339,7 +4328,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
               if ( hbd )
                 if ( doHash2 ) {
                   partFunc = createHash2PartitioningFunctionForHBase(
-                     ((table_desc_struct*)table_desc)->hbase_regionkey_desc,
+                        table_desc->tableDesc()->hbase_regionkey_desc,
                       table,
                       numOfSaltedPartitions,
                       heap);
@@ -4348,15 +4337,15 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
                 }
                 else
                   partFunc = createRangePartitioningFunctionForHBase(
-                     ((table_desc_struct*)table_desc)->hbase_regionkey_desc,
-                      table,
-                     partitioningKeyColumns,
-                     heap);
+                        table_desc->tableDesc()->hbase_regionkey_desc,
+                        table,
+                        partitioningKeyColumns,
+                        heap);
               else {
 
                // no region descriptor, range partitioned or single partition 
table
                partFunc = createRangePartitioningFunction(
-                  files_desc->body.files_desc.partns_desc,
+                  files_desc->filesDesc()->partns_desc,
                   partitioningKeyColumns,
                   nodeMap,
                   heap);
@@ -4392,14 +4381,14 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
         // currently we save the indexLevels in the fileset. Since there
         // is a indexLevel for each file that belongs to the fileset,
         // we get the biggest of this indexLevels and save in the fileset.
-        partns_desc = files_desc->body.files_desc.partns_desc;
+        partns_desc = files_desc->filesDesc()->partns_desc;
        if(partns_desc)
          {
            while (partns_desc)
              {
-               if ( indexLevels < partns_desc->body.partns_desc.indexlevel)
-                 indexLevels = partns_desc->body.partns_desc.indexlevel;
-               partns_desc = partns_desc->header.next;
+               if ( indexLevels < partns_desc->partnsDesc()->indexlevel)
+                 indexLevels = partns_desc->partnsDesc()->indexlevel;
+               partns_desc = partns_desc->next;
              }
 
          }
@@ -4415,21 +4404,19 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
       // Create fully qualified ANSI name from indexname, the PHYSICAL name.
       // If this descriptor was created for a sql/mp table, then the
       // indexname is a fully qualified NSK name (\sys.$vol.subvol.name).
-      QualifiedName qualIndexName(indexes_desc->body.indexes_desc.indexname,
+      QualifiedName qualIndexName(indexes_desc->indexesDesc()->indexname,
                                  1, heap, bindWA);
 
       // This ext_indexname is expected to be set up correctly as an
       // EXTERNAL-format name (i.e., dquoted if any delimited identifiers)
       // by sqlcat/read*.cpp.  The ...AsAnsiString() is just-in-case (MP?).
       NAString extIndexName(
-          indexes_desc->body.indexes_desc.ext_indexname ?
-          (NAString)indexes_desc->body.indexes_desc.ext_indexname :
           qualIndexName.getQualifiedNameAsAnsiString(),
           CmpCommon::statementHeap());
 
       QualifiedName qualExtIndexName;
 
-      //if (indexes_desc->body.indexes_desc.isVolatile)
+      //if (indexes_desc->indexesDesc()->isVolatile)
       if (table->getSpecialType() != ExtendedQualName::VIRTUAL_TABLE)
        qualExtIndexName = QualifiedName(extIndexName, 1, heap, bindWA);
       else
@@ -4437,7 +4424,7 @@ NABoolean createNAFileSets(desc_struct * table_desc       
/*IN*/,
 
       // for volatile tables, set the object part as the external name.
       // cat/sch parts are internal and should not be shown.
-      if (indexes_desc->body.indexes_desc.isVolatile)
+      if (indexes_desc->indexesDesc()->isVolatile())
        {
          ComObjectName con(extIndexName);
          extIndexName = con.getObjectNamePartAsAnsiString();
@@ -4446,12 +4433,12 @@ NABoolean createNAFileSets(desc_struct * table_desc     
  /*IN*/,
       if (partFunc)
        numberOfFiles = partFunc->getCountOfPartitions();
 
-      CMPASSERT(indexes_desc->body.indexes_desc.blocksize > 0);
+      CMPASSERT(indexes_desc->indexesDesc()->blocksize > 0);
 
       NAList<HbaseCreateOption*>* hbaseCreateOptions = NULL;
-      if ((indexes_desc->body.indexes_desc.hbaseCreateOptions) &&
+      if ((indexes_desc->indexesDesc()->hbaseCreateOptions) &&
           (CmpSeabaseDDL::genHbaseCreateOptions
-           (indexes_desc->body.indexes_desc.hbaseCreateOptions,
+           (indexes_desc->indexesDesc()->hbaseCreateOptions,
             hbaseCreateOptions,
             heap,
             NULL,
@@ -4466,833 +4453,816 @@ NABoolean createNAFileSets(desc_struct * table_desc   
    /*IN*/,
       newIndex = new (heap)
        NAFileSet(
                  qualIndexName, // QN containing "\NSK.$VOL", FUNNYSV, FUNNYNM
-                 //(indexes_desc->body.indexes_desc.isVolatile ?
                  qualExtIndexName, // :
-                 //qualIndexName),
                  extIndexName,  // string containing Ansi name CAT.SCH."indx"
-                 files_desc ? files_desc->body.files_desc.fileorganization
-                            : KEY_SEQUENCED_FILE,
+                 KEY_SEQUENCED_FILE,
                  isSystemTable,
                  numberOfFiles,
-                 MAXOF(table_desc->body.table_desc.rowcount,0),
-                  indexes_desc->body.indexes_desc.record_length,
+                 100,
+                  indexes_desc->indexesDesc()->record_length,
                   blockSize,
                  indexLevels,
                  allColumns,
                  indexKeyColumns,
                  partitioningKeyColumns,
                  partFunc,
-                 indexes_desc->body.indexes_desc.keytag,
-                 uint32ArrayToInt64(
-                      indexes_desc->body.indexes_desc.redeftime),
-                 files_desc ? files_desc->body.files_desc.audit : 0,
-                 files_desc ? files_desc->body.files_desc.auditcompress : 0,
-                 files_desc ? files_desc->body.files_desc.compressed : 0,
-                 files_desc ? 
(ComCompressionType)files_desc->body.files_desc.dcompressed : 
COM_NO_COMPRESSION,
-                 files_desc ? files_desc->body.files_desc.icompressed : 0,
-                 files_desc ? files_desc->body.files_desc.buffered: 0,
-                 files_desc ? files_desc->body.files_desc.clearOnPurge: 0,
-                 isPacked,
+                 indexes_desc->indexesDesc()->keytag,
+                  0, 
+                  files_desc ? files_desc->filesDesc()->isAudited() : 0,
+                  0,
+                  0,
+                  COM_NO_COMPRESSION,
+                  0,
+                  0,
+                  0,
+                  isPacked,
                   hasRemotePartition,
-                 ((indexes_desc->body.indexes_desc.keytag != 0) &&
-                  (indexes_desc->body.indexes_desc.unique != 0)),
-                  files_desc ? 
files_desc->body.files_desc.decoupledPartitionKeyList: 0,
-                  files_desc ? files_desc->body.files_desc.fileCode : 0,
-                 (indexes_desc->body.indexes_desc.isVolatile != 0),
-                 (indexes_desc->body.indexes_desc.isInMemoryObjectDefn != 0),
-                  indexes_desc->body.indexes_desc.indexUID,
-                  indexes_desc->body.indexes_desc.keys_desc,
+                  ((indexes_desc->indexesDesc()->keytag != 0) &&
+                   (indexes_desc->indexesDesc()->isUnique())),
+                  0,
+                  0,
+                  (indexes_desc->indexesDesc()->isVolatile()),
+                  (indexes_desc->indexesDesc()->isInMemoryObject()),
+                  indexes_desc->indexesDesc()->indexUID,
+                  indexes_desc->indexesDesc()->keys_desc,
                   NULL, // no Hive stats
-                  indexes_desc->body.indexes_desc.numSaltPartns,
+                  indexes_desc->indexesDesc()->numSaltPartns,
                   hbaseCreateOptions,
                   heap);
-
-      if (isNotAvailable)
-       newIndex->setNotAvailable(TRUE);
-
-      newIndex->setRowFormat(indexes_desc->body.indexes_desc.rowFormat);
-      // Mark each NAColumn in the list
-      indexKeyColumns.setIndexKey();
-      if ((table->isHbaseTable()) && (indexes_desc->body.indexes_desc.keytag 
!= 0))
-        saveNAColumns.setIndexKey();
-
-      if (indexes_desc->body.indexes_desc.isCreatedExplicitly)
-       newIndex->setIsCreatedExplicitly(TRUE);
-
-      //if index is unique and is on one column, then mark column as unique
-      if ((indexes_desc->body.indexes_desc.unique) &&
-         (indexKeyColumns.entries() == 1))
-        indexKeyColumns[0]->setIsUnique();
       
-      partitioningKeyColumns.setPartitioningKey();
-
-      // If it is a VP add it to the list of VPs.
-      // Otherwise, add it to the list of indices.
-      if (isVerticalPartition)
-        vertParts.insert(newIndex); // >>>> RETURN VALUE
-      else
-      {
-        indexes.insert(newIndex);
-      }
-
-      //
-      // advance to the next index
-      //
-      if (isTheClusteringKey)
-       {
-         clusteringIndex = newIndex; // >>>> RETURN VALUE
-         // switch to the alternate indexes by starting over again
-         isTheClusteringKey = FALSE;
-         indexes_desc = table_desc->body.table_desc.indexes_desc;
-       }
-      else
-       {
-         // simply advance to the next in the list
-         indexes_desc = indexes_desc->header.next;
-       }
+      if (isNotAvailable)
+         newIndex->setNotAvailable(TRUE);
 
-      // skip the clustering index, if we encounter it again
-      if (indexes_desc AND !indexes_desc->body.indexes_desc.keytag)
-       indexes_desc = indexes_desc->header.next;
-    } // end while (indexes_desc)
+       newIndex->setRowFormat(indexes_desc->indexesDesc()->rowFormat());
+       // Mark each NAColumn in the list
+       indexKeyColumns.setIndexKey();
+       if ((table->isHbaseTable()) && (indexes_desc->indexesDesc()->keytag != 
0))
+         saveNAColumns.setIndexKey();
 
-    // logic related to indexes hiding
-  return FALSE;
-} // static createNAFileSets()
-#pragma warn(1506)  // warning elimination
+       if (indexes_desc->indexesDesc()->isExplicit())
+         newIndex->setIsCreatedExplicitly(TRUE);
 
+       //if index is unique and is on one column, then mark column as unique
+       if ((indexes_desc->indexesDesc()->isUnique()) &&
+           (indexKeyColumns.entries() == 1))
+         indexKeyColumns[0]->setIsUnique();
 
-// for Hive tables
-NABoolean createNAFileSets(hive_tbl_desc* hvt_desc        /*IN*/,
-                           const NATable * table          /*IN*/,
-                           const NAColumnArray & colArray /*IN*/,
-                           NAFileSetList & indexes        /*OUT*/,
-                           NAFileSetList & vertParts      /*OUT*/,
-                           NAFileSet * & clusteringIndex  /*OUT*/,
-                          LIST(CollIndex) & tableIdList  /*OUT*/,
-                           NAMemory* heap,
-                           BindWA * bindWA,
-                          Int32 *maxIndexLevelsPtr = NULL)
-{
-  NABoolean isTheClusteringKey = TRUE;
-  NABoolean isVerticalPartition;
-  NABoolean hasRemotePartition = FALSE;
-  CollIndex numClusteringKeyColumns = 0;
+       partitioningKeyColumns.setPartitioningKey();
 
-  // Set up global cluster information.  This global information always
-  // gets put on the context heap.
-  //
-  // Note that this function call will probably not do anything, since
-  // this cluster information is set up when arkcmp is created; however,
-  // it's certainly better to have this call here, rather than in a
-  // doubly-nested loop below where it used to be ...
+       indexes.insert(newIndex);
 
-  // $$$ probably not necessary to call this even once ...
-  setUpClusterInfo(CmpCommon::contextHeap());
+       //
+       // advance to the next index
+       //
+       if (isTheClusteringKey)
+         {
+           clusteringIndex = newIndex; // >>>> RETURN VALUE
+           // switch to the alternate indexes by starting over again
+           isTheClusteringKey = FALSE;
+           indexes_desc = table_desc->tableDesc()->indexes_desc;
+         }
+       else
+         {
+           // simply advance to the next in the list
+           indexes_desc = indexes_desc->next;
+         }
 
-  // only one set of key columns to handle for hive
+       // skip the clustering index, if we encounter it again
+       if (indexes_desc AND !indexes_desc->indexesDesc()->keytag)
+         indexes_desc = indexes_desc->next;
+     } // end while (indexes_desc)
+
+     // logic related to indexes hiding
+   return FALSE;
+ } // static createNAFileSets()
+ #pragma warn(1506)  // warning elimination
+
+
+ // for Hive tables
+ NABoolean createNAFileSets(hive_tbl_desc* hvt_desc        /*IN*/,
+                            const NATable * table          /*IN*/,
+                            const NAColumnArray & colArray /*IN*/,
+                            NAFileSetList & indexes        /*OUT*/,
+                            NAFileSetList & vertParts      /*OUT*/,
+                            NAFileSet * & clusteringIndex  /*OUT*/,
+                            LIST(CollIndex) & tableIdList  /*OUT*/,
+                            NAMemory* heap,
+                            BindWA * bindWA,
+                            Int32 *maxIndexLevelsPtr = NULL)
+ {
+   NABoolean isTheClusteringKey = TRUE;
+   NABoolean hasRemotePartition = FALSE;
+   CollIndex numClusteringKeyColumns = 0;
+
+   // Set up global cluster information.  This global information always
+   // gets put on the context heap.
+   //
+   // Note that this function call will probably not do anything, since
+   // this cluster information is set up when arkcmp is created; however,
+   // it's certainly better to have this call here, rather than in a
+   // doubly-nested loop below where it used to be ...
 
-      Lng32 numberOfFiles = 1;         // always at least 1
-      //      NAColumn * indexColumn;          // an index/VP key column
-      NAFileSet * newIndex;            // a new file set
+   // $$$ probably not necessary to call this even once ...
+   setUpClusterInfo(CmpCommon::contextHeap());
 
-      // all columns that belong to an index
-      NAColumnArray allColumns(CmpCommon::statementHeap());
+   // only one set of key columns to handle for hive
 
-      // the index key columns - the SORT columns
-      NAColumnArray indexKeyColumns(CmpCommon::statementHeap());
+       Lng32 numberOfFiles = 1;                // always at least 1
+       //      NAColumn * indexColumn;         // an index/VP key column
+       NAFileSet * newIndex;           // a new file set
 
-      // the partitioning key columns - the BUCKETING columns
-      NAColumnArray partitioningKeyColumns(CmpCommon::statementHeap());
+       // all columns that belong to an index
+       NAColumnArray allColumns(CmpCommon::statementHeap());
 
-      PartitioningFunction * partFunc = NULL;
-      // is this an index or is it really a VP?
+       // the index key columns - the SORT columns
+       NAColumnArray indexKeyColumns(CmpCommon::statementHeap());
 
-      isVerticalPartition = FALSE;
-      NABoolean isPacked = FALSE;
+       // the partitioning key columns - the BUCKETING columns
+       NAColumnArray partitioningKeyColumns(CmpCommon::statementHeap());
 
-      NABoolean isNotAvailable = FALSE;
+       PartitioningFunction * partFunc = NULL;
+       // is this an index or is it really a VP?
 
-      // ---------------------------------------------------------------------
-      // loop over the clustering key columns of the index
-      // ---------------------------------------------------------------------
-      const hive_bkey_desc *hbk_desc = hvt_desc->getBucketingKeys();
+       NABoolean isPacked = FALSE;
 
-      Int32 numBucketingColumns = 0;
+       NABoolean isNotAvailable = FALSE;
 
-      while (hbk_desc)
-       {
-          NAString colName(hbk_desc->name_);
-          colName.toUpper();
+       // ---------------------------------------------------------------------
+       // loop over the clustering key columns of the index
+       // ---------------------------------------------------------------------
+       const hive_bkey_desc *hbk_desc = hvt_desc->getBucketingKeys();
 
-          NAColumn* bucketingColumn = colArray.getColumn(colName);
+       Int32 numBucketingColumns = 0;
 
-          if ( bucketingColumn ) {
-            partitioningKeyColumns.insert(bucketingColumn);
-             numBucketingColumns++;
-          }
+       while (hbk_desc)
+         {
+           NAString colName(hbk_desc->name_);
+           colName.toUpper();
 
-         hbk_desc = hbk_desc->next_;
-       } // end while (hvk_desc)
+           NAColumn* bucketingColumn = colArray.getColumn(colName);
 
-      const hive_skey_desc *hsk_desc = hvt_desc->getSortKeys();
-      if ( hsk_desc == NULL ) {
-         // assume all columns are index key columns
-          for (CollIndex i=0; i<colArray.entries(); i++ )
-            indexKeyColumns.insert(colArray[i]);
-      } else {
-        while (hsk_desc)
-        {
-          NAString colName(hsk_desc->name_);
-          colName.toUpper();
+           if ( bucketingColumn ) {
+              partitioningKeyColumns.insert(bucketingColumn);
+              numBucketingColumns++;
+           }
 
-          NAColumn* sortKeyColumn = colArray.getColumn(colName);
+           hbk_desc = hbk_desc->next_;
+         } // end while (hvk_desc)
 
-          if ( sortKeyColumn ) {
+       const hive_skey_desc *hsk_desc = hvt_desc->getSortKeys();
+       if ( hsk_desc == NULL ) {
+          // assume all columns are index key columns
+           for (CollIndex i=0; i<colArray.entries(); i++ )
+              indexKeyColumns.insert(colArray[i]);
+       } else {
+         while (hsk_desc)
+         {
+           NAString colName(hsk_desc->name_);
+           colName.toUpper();
 
-               indexKeyColumns.insert(sortKeyColumn);
-               indexKeyColumns.setAscending(indexKeyColumns.entries() - 1,
-                                            hsk_desc->orderInt_);
-          }
+           NAColumn* sortKeyColumn = colArray.getColumn(colName);
 
-          hsk_desc = hsk_desc->next_;
-        } // end while (hsk_desc)
-      }
+           if ( sortKeyColumn ) {
 
+                indexKeyColumns.insert(sortKeyColumn);
+                indexKeyColumns.setAscending(indexKeyColumns.entries() - 1,
+                                             hsk_desc->orderInt_);
+           }
 
-      // ---------------------------------------------------------------------
-      // Loop over the non key columns. 
-      // ---------------------------------------------------------------------
-      for (CollIndex i=0; i<colArray.entries(); i++)
-       {
-         allColumns.insert(colArray[i]);
-       }
+           hsk_desc = hsk_desc->next_;
+         } // end while (hsk_desc)
+       }
 
-      //increment for each table/index to create unique identifier
-      cmpCurrentContext->incrementTableIdent();
 
-      // collect file stats from HDFS for the table
-      const hive_sd_desc *sd_desc = hvt_desc->getSDs();
-      HHDFSTableStats * hiveHDFSTableStats = new(heap) HHDFSTableStats(heap);
-      hiveHDFSTableStats->
-        
setPortOverride(CmpCommon::getDefaultLong(HIVE_LIB_HDFS_PORT_OVERRIDE));
+       // ---------------------------------------------------------------------
+       // Loop over the non key columns. 
+       // ---------------------------------------------------------------------
+       for (CollIndex i=0; i<colArray.entries(); i++)
+         {
+           allColumns.insert(colArray[i]);
+         }
 
-      // create file-level statistics and estimate total row count and record 
length
-      hiveHDFSTableStats->populate(hvt_desc);
-      if (hiveHDFSTableStats->hasError())
-        {
-          *CmpCommon::diags() << DgSqlCode(-1200)
-                              << 
DgString0(hiveHDFSTableStats->getDiags().getErrMsg())
-                              << 
DgTableName(table->getTableName().getQualifiedNameAsAnsiString());
-          return TRUE;
-        }
+       //increment for each table/index to create unique identifier
+       cmpCurrentContext->incrementTableIdent();
 
-      if ((hiveHDFSTableStats->isOrcFile()) &&
-          (CmpCommon::getDefault(TRAF_ENABLE_ORC_FORMAT) == DF_OFF))
-        {
-          *CmpCommon::diags() << DgSqlCode(-3069)
-                              << 
DgTableName(table->getTableName().getQualifiedNameAsAnsiString());
-          return TRUE;
-        }
+       // collect file stats from HDFS for the table
+       const hive_sd_desc *sd_desc = hvt_desc->getSDs();
+       HHDFSTableStats * hiveHDFSTableStats = new(heap) HHDFSTableStats(heap);
+       hiveHDFSTableStats->
+         
setPortOverride(CmpCommon::getDefaultLong(HIVE_LIB_HDFS_PORT_OVERRIDE));
 
-#ifndef NDEBUG
-      NAString logFile = 
-        ActiveSchemaDB()->getDefaults().getValue(HIVE_HDFS_STATS_LOG_FILE);
-      if (logFile.length())
-        {
-          FILE *ofd = fopen(logFile, "a");
-          if (ofd)
-            {
-              hiveHDFSTableStats->print(ofd);
-              fclose(ofd);
-            }
-        }
-      // for release code, would need to sandbox the ability to write
-      // files, e.g. to a fixed log directory
-#endif
+       // create file-level statistics and estimate total row count and record 
length
+       hiveHDFSTableStats->populate(hvt_desc);
+       if (hiveHDFSTableStats->hasError())
+         {
+           *CmpCommon::diags() << DgSqlCode(-1200)
+                               << 
DgString0(hiveHDFSTableStats->getDiags().getErrMsg())
+                               << 
DgTableName(table->getTableName().getQualifiedNameAsAnsiString());
+           return TRUE;
+         }
 
+       if ((hiveHDFSTableStats->isOrcFile()) &&
+           (CmpCommon::getDefault(TRAF_ENABLE_ORC_FORMAT) == DF_OFF))
+         {
+           *CmpCommon::diags() << DgSqlCode(-3069)
+                               << 
DgTableName(table->getTableName().getQualifiedNameAsAnsiString());
+           return TRUE;
+         }
 
-      // Create a node map for partitioning function.
-      NodeMap* nodeMap = new (heap) NodeMap(heap, NodeMap::HIVE);
+ #ifndef NDEBUG
+       NAString logFile = 
+         ActiveSchemaDB()->getDefaults().getValue(HIVE_HDFS_STATS_LOG_FILE);
+       if (logFile.length())
+         {
+           FILE *ofd = fopen(logFile, "a");
+           if (ofd)
+             {
+               hiveHDFSTableStats->print(ofd);
+               fclose(ofd);
+             }
+         }
+       // for release code, would need to sandbox the ability to write
+       // files, e.g. to a fixed log directory
+ #endif
 
-      createNodeMap(hvt_desc,
-                  nodeMap,
-                  heap,
-                  (char*)(table->getTableName().getObjectName().data()),
-                  cmpCurrentContext->getTableIdent());
-      tableIdList.insert(CollIndex(cmpCurrentContext->getTableIdent()));
 
-      // For the time being, set it up as Hash2 partitioned table
-               
-      Int32 numBuckets = hvt_desc->getSDs()->buckets_;
+       // Create a node map for partitioning function.
+       NodeMap* nodeMap = new (heap) NodeMap(heap, NodeMap::HIVE);
 
-      if (numBuckets>1 && partitioningKeyColumns.entries()>0) {
-         if ( CmpCommon::getDefault(HIVE_USE_HASH2_AS_PARTFUNCION) == DF_ON )
-            partFunc = createHash2PartitioningFunction
-                          (numBuckets, partitioningKeyColumns, nodeMap, heap);
-         else
-            partFunc = createHivePartitioningFunction
-                          (numBuckets, partitioningKeyColumns, nodeMap, heap);
-      } else
-         partFunc = new (heap)
-                       SinglePartitionPartitioningFunction(nodeMap, heap);
+       createNodeMap(hvt_desc,
+                   nodeMap,
+                   heap,
+                   (char*)(table->getTableName().getObjectName().data()),
+                   cmpCurrentContext->getTableIdent());
+       tableIdList.insert(CollIndex(cmpCurrentContext->getTableIdent()));
 
-      // NB: Just in case, we made a call to setupClusterInfo at the
-      // beginning of this function.
-      //      desc_struct * partns_desc;
-      Int32 indexLevels = 1;
-      
+       // For the time being, set it up as Hash2 partitioned table
 
-      // add a new access path
-      //
-      // $$$ The estimated number of records should be available from
-      // $$$ a FILES descriptor. If it is not available, it may have
-      // $$$ to be computed by examining the EOFs of each individual
-      // $$$ file that belongs to the file set.
+       Int32 numBuckets = hvt_desc->getSDs()->buckets_;
 
-      // Create fully qualified ANSI name from indexname, the PHYSICAL name.
-      // If this descriptor was created for a sql/mp table, then the
-      // indexname is a fully qualified NSK name (\sys.$vol.subvol.name).
-      QualifiedName qualIndexName(
-                  (char*)(table->getTableName().getObjectName().data()),
-                  "HIVE", "", heap);
+       if (numBuckets>1 && partitioningKeyColumns.entries()>0) {
+          if ( CmpCommon::getDefault(HIVE_USE_HASH2_AS_PARTFUNCION) == DF_ON )
+             partFunc = createHash2PartitioningFunction
+                           (numBuckets, partitioningKeyColumns, nodeMap, heap);
+          else
+             partFunc = createHivePartitioningFunction
+                           (numBuckets, partitioningKeyColumns, nodeMap, heap);
+       } else
+          partFunc = new (heap)
+                        SinglePartitionPartitioningFunction(nodeMap, heap);
 
-      // This ext_indexname is expected to be set up correctly as an
-      // EXTERNAL-format name (i.e., dquoted if any delimited identifiers)
-      // by sqlcat/read*.cpp.  The ...AsAnsiString() is just-in-case (MP?).
-      NAString extIndexName(
-          qualIndexName.getQualifiedNameAsAnsiString(),
-          CmpCommon::statementHeap());
+       // NB: Just in case, we made a call to setupClusterInfo at the
+       // beginning of this function.
+       //      TrafDesc * partns_desc;
+       Int32 indexLevels = 1;
 
-      QualifiedName qualExtIndexName = QualifiedName(extIndexName, 1, heap, 
bindWA);
 
+       // add a new access path
+       //
+       // $$$ The estimated number of records should be available from
+       // $$$ a FILES descriptor. If it is not available, it may have
+       // $$$ to be computed by examining the EOFs of each individual
+       // $$$ file that belongs to the file set.
 
-      if (partFunc)
-       numberOfFiles = partFunc->getCountOfPartitions();
+       // Create fully qualified ANSI name from indexname, the PHYSICAL name.
+       // If this descriptor was created for a sql/mp table, then the
+       // indexname is a fully qualified NSK name (\sys.$vol.subvol.name).
+       QualifiedName qualIndexName(
+                   (char*)(table->getTableName().getObjectName().data()),
+                   "HIVE", "", heap);
 
-      Int64 estimatedRC = 0;
-      Int64 estimatedRecordLength = 0;
+       // This ext_indexname is expected to be set up correctly as an
+       // EXTERNAL-format name (i.e., dquoted if any delimited identifiers)
+       // by sqlcat/read*.cpp.  The ...AsAnsiString() is just-in-case (MP?).
+       NAString extIndexName(
+            qualIndexName.getQualifiedNameAsAnsiString(),
+            CmpCommon::statementHeap());
 
-      if ( !sd_desc->isTrulyText() ) {
-         //
-         // Poor man's estimation by assuming the record length in hive is the 
-         // same as SQ's. We can do better once we know how the binary data is
-         // stored in hdfs.
-         //
-         estimatedRecordLength = colArray.getTotalStorageSize();
-         estimatedRC = hiveHDFSTableStats->getTotalSize() / 
estimatedRecordLength;
-      } else {
-         // use the information estimated during populate() call
-         estimatedRC = hiveHDFSTableStats->getEstimatedRowCount();
-         estimatedRecordLength = 
-           Lng32(MINOF(hiveHDFSTableStats->getEstimatedRecordLength(),
-                       hiveHDFSTableStats->getEstimatedBlockSize()-100));
-      }
+       QualifiedName qualExtIndexName = QualifiedName(extIndexName, 1, heap, 
bindWA);
 
-      ((NATable*)table)-> setOriginalRowCount((double)estimatedRC);
 
-      newIndex = new (heap)
-       NAFileSet(
-                 qualIndexName, // QN containing "\NSK.$VOL", FUNNYSV, FUNNYNM
-                 //(indexes_desc->body.indexes_desc.isVolatile ?
-                 qualExtIndexName, // :
-                 //qualIndexName),
-                 extIndexName,  // string containing Ansi name CAT.SCH."indx"
+       if (partFunc)
+         numberOfFiles = partFunc->getCountOfPartitions();
 
-                  // The real orginization is a hybrid of KEY_SEQ and HASH.
-                  // Well, we just take the KEY_SEQ for now.
-                  KEY_SEQUENCED_FILE,
+       Int64 estimatedRC = 0;
+       Int64 estimatedRecordLength = 0;
 
-                 FALSE, // isSystemTable
-                 numberOfFiles,
+       if ( !sd_desc->isTrulyText() ) {
+          //
+          // Poor man's estimation by assuming the record length in hive is 
the 
+          // same as SQ's. We can do better once we know how the binary data is
+          // stored in hdfs.
+          //
+          estimatedRecordLength = colArray.getTotalStorageSize();
+          estimatedRC = hiveHDFSTableStats->getTotalSize() / 
estimatedRecordLength;
+       } else {
+          // use the information estimated during populate() call
+          estimatedRC = hiveHDFSTableStats->getEstimatedRowCount();
+          estimatedRecordLength = 
+            Lng32(MINOF(hiveHDFSTableStats->getEstimatedRecordLength(),
+                        hiveHDFSTableStats->getEstimatedBlockSize()-100));
+       }
 
-                  // HIVE-TBD
-                 Cardinality(estimatedRC),
-                  Lng32(estimatedRecordLength),
+       ((NATable*)table)-> setOriginalRowCount((double)estimatedRC);
+
+       newIndex = new (heap)
+         NAFileSet(
+                   qualIndexName, // QN containing "\NSK.$VOL", FUNNYSV, 
FUNNYNM
+                   qualExtIndexName, // :
+                   extIndexName,        // string containing Ansi name 
CAT.SCH."indx"
+
+                   // The real orginization is a hybrid of KEY_SEQ and HASH.
+                   // Well, we just take the KEY_SEQ for now.
+                   KEY_SEQUENCED_FILE,
+
+                   FALSE, // isSystemTable
+                   numberOfFiles,
+
+                   // HIVE-TBD
+                   Cardinality(estimatedRC),
+                   Lng32(estimatedRecordLength),
+
+                   //hvt_desc->getBlockSize(), 
+                   (Lng32)hiveHDFSTableStats->getEstimatedBlockSize(), 
+
+                   indexLevels, // HIVE-TBD
+                   allColumns,
+                   indexKeyColumns,
+                   partitioningKeyColumns,
+                   partFunc,
+                   0, // indexes_desc->indexesDesc()->keytag,
+
+                   hvt_desc->redeftime(), 
+
+                   1, // files_desc->filesDesc()->audit 
+                   0, // files_desc->filesDesc()->auditcompress 
+                   0, // files_desc->filesDesc()->compressed 
+                   COM_NO_COMPRESSION,
+                   0, // files_desc->filesDesc()->icompressed 
+                   0, // files_desc->filesDesc()->buffered:
+                   0, // files_desc->filesDesc()->clearOnPurge: 0,
+                   isPacked,
+                   hasRemotePartition,
+                   0, // not a unique secondary index
+                   0, // isDecoupledRangePartitioned
+                   0, // file code
+                   0, // not a volatile
+                   0, // inMemObjectDefn
+                   0,
+                   NULL, // indexes_desc->indexesDesc()->keys_desc,
+                   hiveHDFSTableStats,
+                   0, // saltPartns
+                   NULL, //hbaseCreateOptions
+                   heap);
+
+       if (isNotAvailable)
+         newIndex->setNotAvailable(TRUE);
+
+       // Mark each NAColumn in the list
+       indexKeyColumns.setIndexKey();
+
+       partitioningKeyColumns.setPartitioningKey();
+
+       // If it is a VP add it to the list of VPs.
+       // Otherwise, add it to the list of indices.
+       indexes.insert(newIndex);
+
+        clusteringIndex = newIndex;
+
+
+   return FALSE;
+ } // static createNAFileSets()
+
+ #pragma warn(1506)  // warning elimination
+
+ // -----------------------------------------------------------------------
+ // Mark columns named in PRIMARY KEY constraint (these will be different
+ // from clustering key columns when the PK is droppable), for Binder error 
4033.
+ // -----------------------------------------------------------------------
+ static void markPKCols(const TrafConstrntsDesc * constrnt /*IN*/,
+                        const NAColumnArray& columnArray       /*IN*/)
+ {
+   TrafDesc *keycols_desc = constrnt->constr_key_cols_desc;
+   while (keycols_desc)
+     {
+       TrafConstrntKeyColsDesc *key =
+         keycols_desc->constrntKeyColsDesc();
+       // Lookup by name (not position: key->position is pos *within the PK*)
+       NAColumn *nacol = columnArray.getColumn(key->colname);
+       if(nacol != NULL)
+         nacol->setPrimaryKey();
+       keycols_desc = keycols_desc->next;
+     }
+ } // static markPKCols
+
+ // -----------------------------------------------------------------------
+ // Insert MP CHECK CONSTRAINT text into NATable::checkConstraints_.
+ // -----------------------------------------------------------------------
+ static NABoolean
+ createConstraintInfo(const TrafDesc * table_desc        /*IN*/,
+                      const QualifiedName& tableQualName    /*IN*/,
+                      const NAColumnArray& columnArray      /*IN*/,
+                      CheckConstraintList& checkConstraints /*OUT*/,
+                      AbstractRIConstraintList& uniqueConstraints,
+                      AbstractRIConstraintList& refConstraints,
+                      NAMemory* heap,
+                      BindWA *bindWA)
+ {
+   TrafDesc *constrnts_desc = table_desc->tableDesc()->constrnts_desc;
+
+   while (constrnts_desc)
+     {
+       TrafConstrntsDesc *constrntHdr = constrnts_desc->constrntsDesc();
 
-                 //hvt_desc->getB

<TRUNCATED>

Reply via email to