You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by sa...@apache.org on 2016/11/13 01:10:46 UTC

[1/3] incubator-trafodion git commit: Changes to transform an upsert plan into an efficient plan when indexes are present and allow for rowset type operations. The plan makes use of existing operators to eliminate duplicates at the source so we don't ne

Repository: incubator-trafodion
Updated Branches:
  refs/heads/master a20935955 -> 62f8db94c


Changes to transform an upsert plan into an efficient plan when indexes are present and allow for rowset type operations.  The plan makes use of existing operators to eliminate duplicates at the source so we don't need to perform a row at a time check and put operation. The code is currently turned off with a CD traf_upsert_to_eff_tree.


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

Branch: refs/heads/master
Commit: 851733596b226fd16aa8175573559c24fd2908bb
Parents: 12ead26
Author: Sandhya Sundaresan <sa...@apache.org>
Authored: Wed Nov 9 19:27:49 2016 +0000
Committer: Sandhya Sundaresan <sa...@apache.org>
Committed: Wed Nov 9 19:27:49 2016 +0000

----------------------------------------------------------------------
 core/sql/optimizer/BindRelExpr.cpp  | 354 ++++++++++++++++++++++++++++++-
 core/sql/optimizer/BindWA.cpp       |   1 +
 core/sql/optimizer/BindWA.h         |   4 +-
 core/sql/optimizer/Inlining.cpp     |  76 +++++--
 core/sql/optimizer/RelExpr.h        |   6 +-
 core/sql/optimizer/RelSequence.cpp  |  10 +-
 core/sql/optimizer/RelUpdate.h      |   5 +-
 core/sql/sqlcomp/DefaultConstants.h |   4 +
 core/sql/sqlcomp/nadefaults.cpp     |   1 +
 9 files changed, 428 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/optimizer/BindRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindRelExpr.cpp b/core/sql/optimizer/BindRelExpr.cpp
index 596c2c5..db324a3 100644
--- a/core/sql/optimizer/BindRelExpr.cpp
+++ b/core/sql/optimizer/BindRelExpr.cpp
@@ -2424,6 +2424,7 @@ Scan *RelExpr::getScanNode(NABoolean assertExactlyOneScanNode) const
   return (Scan *)result;
 }
 
+
 Scan *RelExpr::getLeftmostScanNode() const
 {
   RelExpr *result = (RelExpr *)this;   // cast away constness, big whoop
@@ -2436,6 +2437,32 @@ Scan *RelExpr::getLeftmostScanNode() const
   return (Scan *)result;
 }
 
+
+
+Join * RelExpr::getLeftJoinChild() const
+{
+  RelExpr *result = (RelExpr *)this;
+  
+  while(result)
+    {
+      if (result->getOperatorType() == REL_LEFT_JOIN) break;
+      result = result->child(0);
+    }
+  return (Join *)result;
+}
+
+RelSequence* RelExpr::getOlapChild() const
+{
+  RelExpr *result = (RelExpr *)this;
+  
+  while(result)
+    {
+      if (result->getOperatorType() == REL_SEQUENCE) break;
+      result = result->child(0);
+    }
+  return (RelSequence *)result;
+}
+
 // QSTUFF
 // We use this method for finding the scan node of an updatable view.
 // This may either be a base table scan or a RenameTable node inserted
@@ -9672,6 +9699,10 @@ RelExpr *Insert::bindNode(BindWA *bindWA)
       return boundExpr;
     }
   }
+
+ 
+
+
    
   bindChildren(bindWA);
   if (bindWA->errStatus()) return this;
@@ -9807,6 +9838,8 @@ RelExpr *Insert::bindNode(BindWA *bindWA)
        setRETDesc(child(0)->getRETDesc());
       } 
 
+
+
     for (i = 0; i < tgtColList.entries() && i2 < newTgtColList.entries(); i++) {
       if(tgtColList[i] != newTgtColList[i2])
         continue;
@@ -10298,10 +10331,17 @@ RelExpr *Insert::bindNode(BindWA *bindWA)
   }
 
   if (isUpsertThatNeedsMerge(isAlignedRowFormat, omittedDefaultCols, omittedCurrentDefaultClassCols)) {
-    boundExpr = xformUpsertToMerge(bindWA);
-    return boundExpr;
+    if (CmpCommon::getDefault(TRAF_UPSERT_TO_EFF_TREE) == DF_OFF) 	
+      {
+	boundExpr = xformUpsertToMerge(bindWA);  
+	return boundExpr;
+      }
+      else if( CmpCommon::getDefault(TRAF_UPSERT_TO_EFF_TREE) == DF_ON)
+	boundExpr = xformUpsertToEfficientTree(bindWA);
+    
+    
   }
-  else if (NOT (isMerge() || noIMneeded()))
+   if (NOT (isMerge() || noIMneeded()))
     boundExpr = handleInlining(bindWA, boundExpr);
 
   // turn OFF Non-atomic Inserts for ODBC if we have detected that Inlining is needed
@@ -10383,6 +10423,310 @@ NABoolean Insert::isUpsertThatNeedsMerge(NABoolean isAlignedRowFormat, NABoolean
      return FALSE;
 }
 
+#ifdef __ignore
+// take an insert(src) node and transform it into
+// a tuple_flow with old/new rows flowing to the IM tree.
+// with a newly created input_scan
+RelExpr* Insert::xformUpsertToEfficientTreeNoDup(BindWA *bindWA) 
+{
+  NATable *naTable = bindWA->getNATable(getTableName());
+  if (bindWA->errStatus())
+    return NULL;
+  if ((naTable->getViewText() != NULL) && (naTable->getViewCheck()))		
+    {		
+      *CmpCommon::diags() << DgSqlCode(-3241) 		
+			  << DgString0(" View with check option not allowed.");	    		
+      bindWA->setErrStatus();		
+      return NULL;		
+    }
+
+  RelExpr *topNode = this;
+  // Create a new BindScope, to encompass the new nodes 
+  // upsert(left_join(input_scan, tuple))
+  // and any inlining nodes that will be created. Any values the upsert
+  // and children will need from src will be marked as outer references in that
+  // new BindScope. We assume that "src" is already bound.
+  ValueIdSet currOuterRefs = bindWA->getCurrentScope()->getOuterRefs();
+
+  CMPASSERT(child(0)->nodeIsBound());
+
+  BindScope *upsertScope = bindWA->getCurrentScope();
+
+  // columns of the target table
+  const ValueIdList &tableCols = updateToSelectMap().getTopValues();
+  const ValueIdList &sourceVals = updateToSelectMap().getBottomValues();
+
+  // create a Join node - left join of the base table columns with the columns to be upserted.
+  // columns of the target table
+  CMPASSERT(child(0)->nodeIsBound());
+  
+  Scan * targetTableScan =
+    new (bindWA->wHeap())
+    Scan(CorrName(getTableDesc()->getCorrNameObj(), bindWA->wHeap()));
+
+ 
+  //join predicate between source columns and target table.
+  ItemExpr * keyPred = NULL;
+  ItemExpr * keyPredPrev = NULL;
+  BaseColumn* baseCol;
+  ColReference * targetColRef;
+  int predCount = 0;
+  ValueIdSet newOuterRefs;
+  ItemExpr * pkeyValPrev;
+  ItemExpr * pkeyVals;
+  for (CollIndex i = 0; i < tableCols.entries(); i++)
+    {
+      baseCol = (BaseColumn *)(tableCols[i].getItemExpr()) ;
+      if (baseCol->getNAColumn()->isSystemColumn())
+	continue;
+
+      targetColRef = new(bindWA->wHeap()) ColReference(
+						       new(bindWA->wHeap()) ColRefName(
+										       baseCol->getNAColumn()->getFullColRefName(), bindWA->wHeap()));
+    
+
+      if (baseCol->getNAColumn()->isClusteringKey())
+	{
+	  // create a join/key predicate between source and target table,
+	  // on the clustering key columns of the target table, making
+	  // ColReference nodes for the target table, so that we can bind
+	  // those to the new scan
+	  keyPredPrev = keyPred;
+	  keyPred = new (bindWA->wHeap())
+	    BiRelat(ITM_EQUAL, targetColRef, 
+		    sourceVals[i].getItemExpr(),
+		    baseCol->getType().supportsSQLnull());
+	  predCount++;
+	  if (predCount > 1) 
+	    {
+	      keyPred = new(bindWA->wHeap()) BiLogic(ITM_AND,
+						     keyPredPrev,
+						     keyPred);  
+	    }
+	  pkeyValPrev = pkeyVals;
+    
+	  pkeyVals = tableCols[i].getItemExpr();
+	  if (i > 0) 
+	    {
+	      pkeyVals = new(bindWA->wHeap()) ItemList(pkeyVals,pkeyValPrev);
+      
+	    }
+	}
+     
+    }
+ 
+  // Map the table's primary key values to the source lists key values
+  ValueIdList tablePKeyVals = NULL;
+  ValueIdList sourcePKeyVals = NULL;
+  
+  pkeyVals->convertToValueIdList(tablePKeyVals,bindWA,ITM_ITEM_LIST);
+  updateToSelectMap().mapValueIdListDown(tablePKeyVals,sourcePKeyVals);
+  
+  Join *lj = new(bindWA->wHeap()) Join(child(0),targetTableScan,REL_LEFT_JOIN,keyPred);
+  lj->doNotTransformToTSJ();	  
+  lj->setTSJForWrite(TRUE);
+   bindWA->getCurrentScope()->xtnmStack()->createXTNM();
+  RelExpr *boundLJ = lj->bindNode(bindWA);
+  if (bindWA->errStatus())
+    return NULL;
+  bindWA->getCurrentScope()->xtnmStack()->removeXTNM();
+  setChild(0,boundLJ);
+  topNode = handleInlining(bindWA,topNode);
+
+
+  return topNode; 
+}
+#endif
+// take an insert(src) node and transform it into
+// a tuple_flow with old/new rows flowing to the IM tree.
+// with a newly created sequence node used to eliminate duplicates.
+/*
+               NJ
+            /      \
+         Sequence   NJ
+        /            \  
+     Left Join        IM Tree 
+     /        \
+    /          \
+Input Tuplelist  Target Table Scan
+or select list
+*/
+         
+RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA) 
+{
+  NATable *naTable = bindWA->getNATable(getTableName());
+  if (bindWA->errStatus())
+    return NULL;
+  if ((naTable->getViewText() != NULL) && (naTable->getViewCheck()))		
+    {		
+      *CmpCommon::diags() << DgSqlCode(-3241) 		
+			  << DgString0(" View with check option not allowed.");	    		
+      bindWA->setErrStatus();		
+      return NULL;		
+    }
+
+  RelExpr *topNode = this;
+ 
+  CMPASSERT(child(0)->nodeIsBound());
+
+  BindScope *upsertScope = bindWA->getCurrentScope();
+  // Create a new BindScope, to encompass the new nodes 
+  // upsert(left_join(input_scan, tuple))
+  // and any inlining nodes that will be created. Any values the upsert
+  // and children will need from src will be marked as outer references in that
+  // new BindScope. We assume that "src" is already bound.
+  ValueIdSet currOuterRefs = bindWA->getCurrentScope()->getOuterRefs();
+  // Save the current RETDesc.
+  RETDesc *prevRETDesc = bindWA->getCurrentScope()->getRETDesc();
+
+  // columns of the target table
+  const ValueIdList &tableCols = updateToSelectMap().getTopValues();
+  const ValueIdList &sourceVals = updateToSelectMap().getBottomValues();
+
+  // create a Join node - left join of the base table columns with the columns to be upserted.
+  // columns of the target table
+  CMPASSERT(child(0)->nodeIsBound());
+  
+  Scan * targetTableScan =
+    new (bindWA->wHeap())
+    Scan(CorrName(getTableDesc()->getCorrNameObj(), bindWA->wHeap()));
+
+ 
+  //join predicate between source columns and target table.
+  ItemExpr * keyPred = NULL;
+  ItemExpr * keyPredPrev = NULL;
+  BaseColumn* baseCol;
+  ColReference * targetColRef;
+  int predCount = 0;
+  ValueIdSet newOuterRefs;
+  ItemExpr * pkeyValPrev;
+  ItemExpr * pkeyVals = NULL;
+  for (CollIndex i = 0; i < tableCols.entries(); i++)
+    {
+      baseCol = (BaseColumn *)(tableCols[i].getItemExpr()) ;
+      if (baseCol->getNAColumn()->isSystemColumn())
+	continue;
+
+      targetColRef = new(bindWA->wHeap()) ColReference(
+						       new(bindWA->wHeap()) ColRefName(
+										       baseCol->getNAColumn()->getFullColRefName(), bindWA->wHeap()));
+    
+
+      if (baseCol->getNAColumn()->isClusteringKey())
+	{
+	  // create a join/key predicate between source and target table,
+	  // on the clustering key columns of the target table, making
+	  // ColReference nodes for the target table, so that we can bind
+	  // those to the new scan
+	  keyPredPrev = keyPred;
+	  keyPred = new (bindWA->wHeap())
+	    BiRelat(ITM_EQUAL, targetColRef, 
+		    sourceVals[i].getItemExpr(),
+		    baseCol->getType().supportsSQLnull());
+	  predCount++;
+	  if (predCount > 1) 
+	    {
+	      keyPred = new(bindWA->wHeap()) BiLogic(ITM_AND,
+						     keyPredPrev,
+						     keyPred);  
+	    }
+	  pkeyValPrev = pkeyVals;
+    
+	  pkeyVals = tableCols[i].getItemExpr();
+	  
+	  if (i > 0) 
+	    {
+	      pkeyVals = new(bindWA->wHeap()) ItemList(pkeyVals,pkeyValPrev);
+      
+	    }
+	}
+     
+    }
+ 
+  // Map the table's primary key values to the source lists key values
+  ValueIdList tablePKeyVals = NULL;
+  ValueIdList sourcePKeyVals = NULL;
+  
+  pkeyVals->convertToValueIdList(tablePKeyVals,bindWA,ITM_ITEM_LIST);
+  updateToSelectMap().mapValueIdListDown(tablePKeyVals,sourcePKeyVals);
+  
+
+
+  Join *lj = new(bindWA->wHeap()) Join(child(0),targetTableScan,REL_LEFT_JOIN,keyPred);
+  lj->doNotTransformToTSJ();	  
+  lj->setTSJForWrite(TRUE);
+  bindWA->getCurrentScope()->xtnmStack()->createXTNM();
+
+  
+  RelExpr *boundLJ = lj->bindNode(bindWA);
+  if (bindWA->errStatus())
+    return NULL;
+  bindWA->getCurrentScope()->xtnmStack()->removeXTNM();
+ 
+ 
+  ValueIdSet sequenceFunction = NULL;		
+ 
+  ItemExpr *constOne = new (bindWA->wHeap()) ConstValue(1);
+ 
+  //Retrieve all the system and user columns of the left join output
+  ValueIdList  ljOutCols = NULL;
+  boundLJ->getRETDesc()->getValueIdList(ljOutCols);
+  //Retrieve the null instantiated part of the LJ output
+  ValueIdList ljNullInstColumns = lj->nullInstantiatedOutput();
+  
+  //Create the olap node and use the primary key of the table as the 
+  //"partition by" columns for the olap node.
+  RelSequence *seqNode = new(bindWA->wHeap()) RelSequence(boundLJ, sourcePKeyVals.rebuildExprTree(ITM_ITEM_LIST),  (ItemExpr *)NULL);
+ 
+
+  // Create a LEAD Item Expr for a random value 999. 
+  // Use this to eliminate rows which have a NULL for this LEAD value within 
+  // a particular partition range.
+  ItemExpr *leadItem, *boundLeadItem = NULL;
+  ItemExpr *constLead999 = new (bindWA->wHeap()) ConstValue( 999);
+
+  leadItem = new(bindWA->wHeap()) ItmLeadOlapFunction(constLead999,1);
+  ((ItmLeadOlapFunction *)leadItem)->setIsOLAP(TRUE);
+  boundLeadItem = leadItem->bindNode(bindWA);
+  if (bindWA->errStatus()) return this;
+  boundLeadItem->convertToValueIdSet(sequenceFunction);
+  seqNode->setSequenceFunctions(sequenceFunction);
+  
+  // Add a selection predicate (post predicate) to check if the LEAD item is NULL
+  ItemExpr *selPredOnLead = NULL;
+  selPredOnLead = new (bindWA->wHeap()) UnLogic(ITM_IS_NULL,leadItem);
+  selPredOnLead->bindNode(bindWA);
+  if (bindWA->errStatus()) return this;
+  seqNode->selectionPred() += selPredOnLead->getValueId();
+  
+  RelExpr *boundSeqNode = seqNode->bindNode(bindWA);  
+  boundSeqNode->setChild(0,boundLJ);
+
+  
+  setChild(0,boundSeqNode);
+
+  // Fixup the newRecExpr() and newRecExprArray() to refer to the new 
+  // valueIds of the new child - i.e RelSequence. Use the saved off valueIdMap
+  // from the current bindScope for this.
+  ValueIdSet newNewRecExpr;
+  ValueIdMap notCoveredMap = bindWA->getCurrentScope()->getNcToOldMap();
+  notCoveredMap.rewriteValueIdSetDown(newRecExpr(),newNewRecExpr);
+  newRecExpr() = newNewRecExpr;
+  
+  ValueIdList oldRecArrList(newRecExprArray());
+  ValueIdList newRecArrList;
+  notCoveredMap.rewriteValueIdListDown(oldRecArrList,newRecArrList);
+  ValueIdArray newNewRecArray(newRecArrList.entries());
+  
+  for (CollIndex i = 0; i < newRecArrList.entries(); i++)
+    {
+      newNewRecArray.insertAt(i,newRecArrList.at(i));
+    }
+  newRecExprArray() = newNewRecArray;
+  return topNode; 
+}
+
+
 // take an insert(src) node and transform it into
 // tsj_flow(src, merge_update(input_scan))
 // with a newly created input_scan
@@ -10413,7 +10757,6 @@ RelExpr* Insert::xformUpsertToMerge(BindWA *bindWA)
 
   CMPASSERT(child(0)->nodeIsBound());
   bindWA->initNewScope();
-
   BindScope *mergeScope = bindWA->getCurrentScope();
 
   // create a new scan of the target table, to be used in the merge
@@ -10463,6 +10806,7 @@ RelExpr* Insert::xformUpsertToMerge(BindWA *bindWA)
                                                 keyPredPrev,
                                                 keyPred);  
       }
+
     }
     if (sourceVals[i].getItemExpr()->getOperatorType() != ITM_CONSTANT)
       {
@@ -10871,7 +11215,7 @@ RelExpr *MergeUpdate::bindNode(BindWA *bindWA)
   if (needsBindScope_)
     bindWA->initNewScope();
 
-  // For an xformaed upsert any UDF or subquery is guaranteed to be 
+  // For an xformed upsert any UDF or subquery is guaranteed to be 
   // in the using clause. Upsert will not generate a merge without using 
   // clause. ON clause, when matched SET clause and when not matched INSERT
   // clauses all use expressions from the using clause. (same vid).

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/optimizer/BindWA.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindWA.cpp b/core/sql/optimizer/BindWA.cpp
index 3b27d8e..113b525 100644
--- a/core/sql/optimizer/BindWA.cpp
+++ b/core/sql/optimizer/BindWA.cpp
@@ -75,6 +75,7 @@ BindScope::BindScope(BindWA* bindWA) :
   xtnm_(bindWA ? bindWA->wHeap() : NULL),
   RETDesc_(NULL),
   sequenceNode_(NULL),
+  ncToOldMap_(NULL),
   OlapPartitionChange_ (NULL),
   HasOlapSeqFunctions_ ( BindScope::OLAPUNKNOWN_ ),
   isFirstOlapWindowSpec_( TRUE ),

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/optimizer/BindWA.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindWA.h b/core/sql/optimizer/BindWA.h
index ac3bfa0..4089291 100644
--- a/core/sql/optimizer/BindWA.h
+++ b/core/sql/optimizer/BindWA.h
@@ -685,6 +685,8 @@ public:
   // --------------------------------------------------------------------
   RelExpr *&getSequenceNode() { return sequenceNode_; };
 
+  ValueIdMap &getNcToOldMap() { return ncToOldMap_;}
+
   ValueIdList getOlapPartition() const {return OlapPartition_; };
 
   ValueIdList getOlapOrder() const {return OlapOrder_; };
@@ -786,7 +788,7 @@ private:
   // node, an error is issued.
   //
   RelExpr *sequenceNode_;
-
+  ValueIdMap ncToOldMap_;
   // --------------------------------------------------------------------
   // Context info for this scope.
   // --------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/optimizer/Inlining.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/Inlining.cpp b/core/sql/optimizer/Inlining.cpp
index ceebc02..ad534e7 100644
--- a/core/sql/optimizer/Inlining.cpp
+++ b/core/sql/optimizer/Inlining.cpp
@@ -56,6 +56,7 @@
 #include "MjvBuilder.h"
 #include "ItmFlowControlFunction.h"
 #include <CmpMain.h>
+#include "RelSequence.h"
 
 #ifdef NA_DEBUG_GUI
 	#include "ComSqlcmpdbg.h"
@@ -618,25 +619,58 @@ RETDesc *GenericUpdate::createOldAndNewCorrelationNames(BindWA *bindWA, NABoolea
   }
 
   if ((getOperatorType() != REL_UNARY_INSERT) || 
-	getUpdateCKorUniqueIndexKey())
+      getUpdateCKorUniqueIndexKey() ||
+      ((getOperatorType() == REL_UNARY_INSERT) &&((Insert *)this)->isMerge()) ||
+      ((getOperatorType() == REL_UNARY_INSERT) && ((Insert *)this)->isUpsert() && (CmpCommon::getDefault(TRAF_UPSERT_TO_EFF_TREE) == DF_ON )))  
   {
     // DELETE or UPDATE --
     // Now merge the old/target/before valueid's (the Scan child RETDesc)
     // into this RETDesc such that these cols are all named "OLD@.<col>"
-    //
+    //f
     Scan *scan ;
     if (getOperatorType() != REL_UNARY_INSERT)
       scan = getScanNode();
     else 
       scan = getLeftmostScanNode();
+    if ((getOperatorType() == REL_UNARY_INSERT) && ((Insert *)this)->isUpsert() && (CmpCommon::getDefault(TRAF_UPSERT_TO_EFF_TREE) == DF_ON ))
+      {
+	RelSequence *olapChild = getOlapChild();
+	CorrName corrName(getTableDesc()->getCorrNameObj().getQualifiedNameObj(), 
+			  bindWA->wHeap(),
+			  OLDCorr);
+	
+        //	ColumnDescList *colList = (olapChild->getRETDesc())->getColumnList();
+	for (short i = 0; i< olapChild->getRETDesc()->getDegree();i++)
+	  {
+	    // we remembered if the original columns was from the right side of
+	    // this olap node so add those to the RetDesc since those are the 
+	    //ones we want to delete from the dependent indexes.
+	    if ((olapChild->getRETDesc()->getValueId(i)).getItemExpr()->origOpType() == ITM_INSTANTIATE_NULL)
+	      {		    
+		rd->addColumn(bindWA, 
+			      ColRefName(olapChild->getRETDesc()->getColRefNameObj(i).getColName(), corrName),
+			      olapChild->getRETDesc()->getValueId(i),
+			      USER_COLUMN,
+			      olapChild->getRETDesc()->getHeading(i));
+		    
+	      }	    
+	  }
+	rd->addColumns(bindWA, *olapChild->getRETDesc()->getSystemColumnList(),  SYSTEM_COLUMN,&corrName);
+	
 
-    CMPASSERT(scan);
-    CorrName corrName(scan->getTableDesc()->getCorrNameObj().getQualifiedNameObj(), 
-	bindWA->wHeap(),
-	OLDCorr);
+      }	
+    
+    else
+      {
+	CMPASSERT(scan);
+	CorrName corrName(scan->getTableDesc()->getCorrNameObj().getQualifiedNameObj(), 
+			  bindWA->wHeap(),
+			  OLDCorr);
 
-    rd->addColumns(bindWA, *scan->getRETDesc(), &corrName);
+	rd->addColumns(bindWA, *scan->getRETDesc(), &corrName);
+      }
   }
+   
 
   Set_SqlParser_Flags(ALLOW_FUNNY_IDENTIFIER);	// allow "@" processing
   Set_SqlParser_Flags(DELAYED_RESET); // allow multiple parser calls.
@@ -1857,7 +1891,8 @@ static RelExpr *createIMNode(BindWA *bindWA,
 			     NABoolean isIMInsert,
 			     NABoolean useInternalSyskey,
                              NABoolean isForUpdate,
-                             NABoolean isForMerge)
+                             NABoolean isForMerge,
+			     NABoolean isEffUpsert)
 {
    
  // See createOldAndNewCorrelationNames() for info on OLDCorr/NEWCorr
@@ -1874,13 +1909,13 @@ static RelExpr *createIMNode(BindWA *bindWA,
   // that correspond to the base table. Hence we introduce 
   // robustDelete below. This flag could also be called 
   // isIMOnAUniqueIndexForMerge
-  NABoolean robustDelete = isForMerge && index->isUniqueIndex();
+  NABoolean robustDelete = (isForMerge && index->isUniqueIndex()) || (isEffUpsert && index->isUniqueIndex());
 
-  tableCorrName.setCorrName((isIMInsert)?  NEWCorr : OLDCorr);
+  tableCorrName.setCorrName(isIMInsert ?  NEWCorr : OLDCorr);
   
   ItemExprList *colRefList = new(bindWA->wHeap()) ItemExprList(bindWA->wHeap());
   
-  const ValueIdList &indexColVids = ((isIMInsert || robustDelete)? 
+  const ValueIdList &indexColVids = ((isIMInsert || robustDelete )? 
 				     index->getIndexColumns() : 
 				     index->getIndexKey());
   ItemExpr *preCond = NULL; // pre-condition for delete, insert if any
@@ -1906,7 +1941,7 @@ static RelExpr *createIMNode(BindWA *bindWA,
   // Index Type/IM operation->  Delete  | Insert
   // Non-unique Index           Yes        No 
   // Unique Index               Yes        Yes
-  if ((!isIMInsert && isForUpdate)||robustDelete)
+  if ((!isIMInsert && isForUpdate)||robustDelete )
     {
       // For delete nodes that are part of an update, generate a
       // comparison expression between old and new index column values
@@ -1996,7 +2031,7 @@ static RelExpr *createIMNode(BindWA *bindWA,
       imNode = new (bindWA->wHeap()) LeafDelete(indexCorrName,
                                                 NULL,
                                                 colRefList,
-                                                (robustDelete)?TRUE:FALSE,
+                                                (robustDelete )?TRUE:FALSE,
                                                 REL_LEAF_DELETE,
                                                 preCond,
                                                 bindWA->wHeap());
@@ -2041,6 +2076,7 @@ RelExpr *GenericUpdate::createIMNodes(BindWA *bindWA,
   RelExpr *indexInsert = NULL, *indexDelete = NULL, *indexOp = NULL;
   NABoolean isForUpdate = (getOperatorType() == REL_UNARY_UPDATE ||
                            isMergeUpdate());
+  NABoolean isEffUpsert = ((CmpCommon::getDefault(TRAF_UPSERT_TO_EFF_TREE) == DF_ON ) && (getOperatorType() == REL_UNARY_INSERT && ((Insert*)this)->isUpsert()));
 
   if (indexCorrName.getUgivenName().isNull())
     {
@@ -2051,7 +2087,7 @@ RelExpr *GenericUpdate::createIMNodes(BindWA *bindWA,
   // ALL the index columns as AFTER/NEW columns.
   //
   if (getOperatorType() == REL_UNARY_INSERT ||
-      getOperatorType() == REL_UNARY_UPDATE)
+      getOperatorType() == REL_UNARY_UPDATE || isEffUpsert)
     indexInsert = indexOp = createIMNode(bindWA, 
 					 tableCorrName, 
 					 indexCorrName,
@@ -2059,22 +2095,26 @@ RelExpr *GenericUpdate::createIMNodes(BindWA *bindWA,
 					 TRUE, 
 					 useInternalSyskey,
                                          isForUpdate,
-                                         isMerge());
+                                         isMerge(),
+					 isEffUpsert);
 
   // Create a list of base columns ColReferences for
   // ONLY the index KEY columns as BEFORE/OLD columns.
   //
   if (getOperatorType() == REL_UNARY_DELETE ||
-      getOperatorType() == REL_UNARY_UPDATE)
+      getOperatorType() == REL_UNARY_UPDATE ||
+      isEffUpsert)
+    
     indexDelete = indexOp = createIMNode(bindWA, 
 					 tableCorrName, indexCorrName,
 					 index, 
 					 FALSE,
     			       		 useInternalSyskey,
                                          isForUpdate,
-                                         isMerge());
+                                         isMerge(),
+					 isEffUpsert);
 
-  if (getOperatorType() == REL_UNARY_UPDATE) {
+  if ((getOperatorType() == REL_UNARY_UPDATE) || isEffUpsert){
     indexOp = new (bindWA->wHeap()) Union(indexDelete, indexInsert, 
                                           NULL, NULL, REL_UNION, 
                                           CmpCommon::statementHeap(),TRUE,TRUE);

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/optimizer/RelExpr.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelExpr.h b/core/sql/optimizer/RelExpr.h
index 545ee2c..d2d6c20 100644
--- a/core/sql/optimizer/RelExpr.h
+++ b/core/sql/optimizer/RelExpr.h
@@ -96,8 +96,10 @@ class MVInfoForDDL;
 class PlanPriority;
 class Hint;
 class TableMappingUDF;
+class RelSequence;
 class CSEInfo;
 
+
 ////////////////////
 class CANodeIdSet;
 class QueryAnalysis;
@@ -680,9 +682,9 @@ public:
 
   Scan *getScanNode(NABoolean assertExactlyOneScanNode = TRUE) const;
   Scan *getLeftmostScanNode() const;
-
   Scan *getAnyScanNode() const;
-
+  Join *getLeftJoinChild() const;
+  RelSequence *getOlapChild() const;
   void getAllTableDescs(TableDescList &tableDescs);
 
   TableDesc *findFirstTableDescAndValueIdMap(RelExpr *currentNode,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/optimizer/RelSequence.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelSequence.cpp b/core/sql/optimizer/RelSequence.cpp
index 0047812..e7b67b7 100644
--- a/core/sql/optimizer/RelSequence.cpp
+++ b/core/sql/optimizer/RelSequence.cpp
@@ -1179,20 +1179,21 @@ RelExpr *RelSequence::bindNode(BindWA *bindWA)
 
     sequencedColumns() += newColumn->getValueId();
   }
-  
+  ValueIdMap ncToOldMap;
   for(i = 0; i < colList->entries(); i++) 
   {
     ValueId columnValueId = colList->at(i)->getValueId();
     ItemExpr *newColumn = new (bindWA->wHeap()) 
       NotCovered (columnValueId.getItemExpr());
     newColumn->synthTypeAndValueId();
+    newColumn->setOrigOpType(columnValueId.getItemExpr()->origOpType());
     
     resultTable->addColumn(bindWA,
       colList->at(i)->getColRefNameObj(),
       newColumn->getValueId(),
       USER_COLUMN,
       colList->at(i)->getHeading());
-
+    ncToOldMap.addMapEntry(columnValueId,newColumn->getValueId());
     if(colList->at(i)->isGrouped()) {
       ColumnNameMap *cnm =
         resultTable->findColumn(colList->at(i)->getColRefNameObj());
@@ -1201,7 +1202,7 @@ RelExpr *RelSequence::bindNode(BindWA *bindWA)
 
     sequencedColumns() += newColumn->getValueId();
   }
-
+  
   // Set the return descriptor
   //
   setRETDesc(resultTable);
@@ -1260,7 +1261,8 @@ RelExpr *RelSequence::bindNode(BindWA *bindWA)
   //CMPASSERT(!bindWA->getCurrentScope()->getSequenceNode());
 
   bindWA->getCurrentScope()->getSequenceNode() = boundExpr;
-
+  // save the ncToOldmap in the current scope. It will be used in Insert::bindnode for a special case.
+  bindWA->getCurrentScope()->getNcToOldMap() = ncToOldMap;
   return boundExpr;
 
 } // RelSequence::bindNode()

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/optimizer/RelUpdate.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelUpdate.h b/core/sql/optimizer/RelUpdate.h
index c1f872b..36569e3 100644
--- a/core/sql/optimizer/RelUpdate.h
+++ b/core/sql/optimizer/RelUpdate.h
@@ -1155,10 +1155,9 @@ public:
     baseColRefs_ = val;
   }
 
-  NABoolean isUpsertThatNeedsMerge(NABoolean isAlignedRowFormat, NABoolean omittedDefaultCols,
-                                   NABoolean omittedCurrentDefaultCols) const;
+  NABoolean isUpsertThatNeedsMerge(NABoolean isAlignedRowFormat, NABoolean omittedDefaultCols,NABoolean omittedCurrentDefaultCols) const;
   RelExpr* xformUpsertToMerge(BindWA *bindWA) ;
-
+  RelExpr* xformUpsertToEfficientTree(BindWA *bindWA) ;
 protected:
 
   InsertType       insertType_;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/sqlcomp/DefaultConstants.h
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/DefaultConstants.h b/core/sql/sqlcomp/DefaultConstants.h
index 51f8d55..1e08107 100644
--- a/core/sql/sqlcomp/DefaultConstants.h
+++ b/core/sql/sqlcomp/DefaultConstants.h
@@ -3834,6 +3834,9 @@ enum DefaultConstants
   // if FALSE, create unsigned literal for +ve and signed literal for -ve nums.
   TRAF_CREATE_SIGNED_NUMERIC_LITERAL,
 
+
+  TRAF_UPSERT_TO_EFF_TREE,
+
   // if TRUE, create tinyint literal insteadl of smallint.
   TRAF_CREATE_TINYINT_LITERAL,
 
@@ -3867,6 +3870,7 @@ enum DefaultConstants
   CSE_CLEANUP_HIVE_TABLES,
   CSE_CACHE_TEMP_QUERIES,
 
+
   // This enum constant must be the LAST one in the list; it's a count,
   // not an Attribute (it's not IN DefaultDefaults; it's the SIZE of it)!
   __NUM_DEFAULT_ATTRIBUTES

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/85173359/core/sql/sqlcomp/nadefaults.cpp
----------------------------------------------------------------------
diff --git a/core/sql/sqlcomp/nadefaults.cpp b/core/sql/sqlcomp/nadefaults.cpp
index e8d7cfa..149f097 100644
--- a/core/sql/sqlcomp/nadefaults.cpp
+++ b/core/sql/sqlcomp/nadefaults.cpp
@@ -3409,6 +3409,7 @@ XDDkwd__(SUBQUERY_UNNESTING,			"ON"),
   DDkwd__(TRAF_UNLOAD_SKIP_WRITING_TO_FILES,           "OFF"),
   DDkwd__(TRAF_UPSERT_ADJUST_PARAMS,                   "OFF"),
   DDkwd__(TRAF_UPSERT_MODE,                            "MERGE"),
+  DDkwd__(TRAF_UPSERT_TO_EFF_TREE,                     "OFF"),
   DDint__(TRAF_UPSERT_WB_SIZE,                         "2097152"),
   DDkwd__(TRAF_UPSERT_WRITE_TO_WAL,                    "OFF"),
 


[2/3] incubator-trafodion git commit: Added regression test and fixed all the review comments.

Posted by sa...@apache.org.
Added regression test and fixed all the review comments.


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

Branch: refs/heads/master
Commit: f67784b047d4e815bd0b18b3ecc7642101c97cd7
Parents: 8517335
Author: Sandhya Sundaresan <sa...@apache.org>
Authored: Sat Nov 12 17:28:01 2016 +0000
Committer: Sandhya Sundaresan <sa...@apache.org>
Committed: Sat Nov 12 17:28:01 2016 +0000

----------------------------------------------------------------------
 core/sql/optimizer/BindRelExpr.cpp       |  61 ++++++++-------
 core/sql/optimizer/BindWA.h              |   3 +-
 core/sql/optimizer/Inlining.cpp          |   2 +-
 core/sql/optimizer/RelSequence.cpp       |   4 +-
 core/sql/regress/executor/EXPECTED015.SB | 108 ++++++++++++++++++++++++++
 core/sql/regress/executor/TEST015        |  25 ++++++
 6 files changed, 171 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f67784b0/core/sql/optimizer/BindRelExpr.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindRelExpr.cpp b/core/sql/optimizer/BindRelExpr.cpp
index db324a3..f63ddd2 100644
--- a/core/sql/optimizer/BindRelExpr.cpp
+++ b/core/sql/optimizer/BindRelExpr.cpp
@@ -2445,7 +2445,8 @@ Join * RelExpr::getLeftJoinChild() const
   
   while(result)
     {
-      if (result->getOperatorType() == REL_LEFT_JOIN) break;
+      if (result->getOperatorType() == REL_LEFT_JOIN) 
+        break;
       result = result->child(0);
     }
   return (Join *)result;
@@ -2457,7 +2458,8 @@ RelSequence* RelExpr::getOlapChild() const
   
   while(result)
     {
-      if (result->getOperatorType() == REL_SEQUENCE) break;
+      if (result->getOperatorType() == REL_SEQUENCE) 
+        break;
       result = result->child(0);
     }
   return (RelSequence *)result;
@@ -10322,12 +10324,12 @@ RelExpr *Insert::bindNode(BindWA *bindWA)
     const NodeMap* np;
     Lng32 partns = 1;
     if ( pf && (np = pf->getNodeMap()) )
-    {
-       partns = np->getNumEntries();
-       if(partns > 1  && CmpCommon::getDefault(ATTEMPT_ESP_PARALLELISM) == DF_OFF)
-         // 4490 - BULK LOAD into a salted table is not supported if ESP parallelism is turned off
-         *CmpCommon::diags() << DgSqlCode(-4490);
-    }
+      {
+        partns = np->getNumEntries();
+        if(partns > 1  && CmpCommon::getDefault(ATTEMPT_ESP_PARALLELISM) == DF_OFF)
+          // 4490 - BULK LOAD into a salted table is not supported if ESP parallelism is turned off
+          *CmpCommon::diags() << DgSqlCode(-4490);
+      }
   }
 
   if (isUpsertThatNeedsMerge(isAlignedRowFormat, omittedDefaultCols, omittedCurrentDefaultClassCols)) {
@@ -10336,12 +10338,12 @@ RelExpr *Insert::bindNode(BindWA *bindWA)
 	boundExpr = xformUpsertToMerge(bindWA);  
 	return boundExpr;
       }
-      else if( CmpCommon::getDefault(TRAF_UPSERT_TO_EFF_TREE) == DF_ON)
-	boundExpr = xformUpsertToEfficientTree(bindWA);
+    else 
+      boundExpr = xformUpsertToEfficientTree(bindWA);
     
     
   }
-   if (NOT (isMerge() || noIMneeded()))
+  if (NOT (isMerge() || noIMneeded()))
     boundExpr = handleInlining(bindWA, boundExpr);
 
   // turn OFF Non-atomic Inserts for ODBC if we have detected that Inlining is needed
@@ -10423,7 +10425,8 @@ NABoolean Insert::isUpsertThatNeedsMerge(NABoolean isAlignedRowFormat, NABoolean
      return FALSE;
 }
 
-#ifdef __ignore
+/** commenting the following method out for future work. This may be enabled as a further performance improvement if we can eliminate the sort node that gets geenrated as part of the Sequence Node. In case of no duplicates we won't need the Sequence node at all. 
+
 // take an insert(src) node and transform it into
 // a tuple_flow with old/new rows flowing to the IM tree.
 // with a newly created input_scan
@@ -10536,7 +10539,8 @@ RelExpr* Insert::xformUpsertToEfficientTreeNoDup(BindWA *bindWA)
 
   return topNode; 
 }
-#endif
+*/
+
 // take an insert(src) node and transform it into
 // a tuple_flow with old/new rows flowing to the IM tree.
 // with a newly created sequence node used to eliminate duplicates.
@@ -10599,7 +10603,7 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
   ColReference * targetColRef;
   int predCount = 0;
   ValueIdSet newOuterRefs;
-  ItemExpr * pkeyValPrev;
+  ItemExpr * pkeyValPrev = NULL;
   ItemExpr * pkeyVals = NULL;
   for (CollIndex i = 0; i < tableCols.entries(); i++)
     {
@@ -10630,11 +10634,12 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
 						     keyPredPrev,
 						     keyPred);  
 	    }
-	  pkeyValPrev = pkeyVals;
+          
+          pkeyValPrev = pkeyVals;
     
 	  pkeyVals = tableCols[i].getItemExpr();
 	  
-	  if (i > 0) 
+	  if (pkeyValPrev != NULL ) 
 	    {
 	      pkeyVals = new(bindWA->wHeap()) ItemList(pkeyVals,pkeyValPrev);
       
@@ -10644,8 +10649,8 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
     }
  
   // Map the table's primary key values to the source lists key values
-  ValueIdList tablePKeyVals = NULL;
-  ValueIdList sourcePKeyVals = NULL;
+  ValueIdList tablePKeyVals ;
+  ValueIdList sourcePKeyVals ;
   
   pkeyVals->convertToValueIdList(tablePKeyVals,bindWA,ITM_ITEM_LIST);
   updateToSelectMap().mapValueIdListDown(tablePKeyVals,sourcePKeyVals);
@@ -10653,8 +10658,7 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
 
 
   Join *lj = new(bindWA->wHeap()) Join(child(0),targetTableScan,REL_LEFT_JOIN,keyPred);
-  lj->doNotTransformToTSJ();	  
-  lj->setTSJForWrite(TRUE);
+  
   bindWA->getCurrentScope()->xtnmStack()->createXTNM();
 
   
@@ -10664,7 +10668,7 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
   bindWA->getCurrentScope()->xtnmStack()->removeXTNM();
  
  
-  ValueIdSet sequenceFunction = NULL;		
+  ValueIdSet sequenceFunction ;		
  
   ItemExpr *constOne = new (bindWA->wHeap()) ConstValue(1);
  
@@ -10676,6 +10680,7 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
   
   //Create the olap node and use the primary key of the table as the 
   //"partition by" columns for the olap node.
+  CMPASSERT(!bindWA->getCurrentScope()->getSequenceNode());
   RelSequence *seqNode = new(bindWA->wHeap()) RelSequence(boundLJ, sourcePKeyVals.rebuildExprTree(ITM_ITEM_LIST),  (ItemExpr *)NULL);
  
 
@@ -10695,14 +10700,14 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
   // Add a selection predicate (post predicate) to check if the LEAD item is NULL
   ItemExpr *selPredOnLead = NULL;
   selPredOnLead = new (bindWA->wHeap()) UnLogic(ITM_IS_NULL,leadItem);
-  selPredOnLead->bindNode(bindWA);
+  selPredOnLead = selPredOnLead->bindNode(bindWA);
   if (bindWA->errStatus()) return this;
   seqNode->selectionPred() += selPredOnLead->getValueId();
-  
-  RelExpr *boundSeqNode = seqNode->bindNode(bindWA);  
-  boundSeqNode->setChild(0,boundLJ);
+  seqNode->setChild(0,boundLJ);
 
-  
+ 
+  RelExpr *boundSeqNode = seqNode->bindNode(bindWA);  
+   
   setChild(0,boundSeqNode);
 
   // Fixup the newRecExpr() and newRecExprArray() to refer to the new 
@@ -10710,12 +10715,12 @@ RelExpr* Insert::xformUpsertToEfficientTree(BindWA *bindWA)
   // from the current bindScope for this.
   ValueIdSet newNewRecExpr;
   ValueIdMap notCoveredMap = bindWA->getCurrentScope()->getNcToOldMap();
-  notCoveredMap.rewriteValueIdSetDown(newRecExpr(),newNewRecExpr);
+  notCoveredMap.rewriteValueIdSetUp(newNewRecExpr, newRecExpr());
   newRecExpr() = newNewRecExpr;
   
   ValueIdList oldRecArrList(newRecExprArray());
   ValueIdList newRecArrList;
-  notCoveredMap.rewriteValueIdListDown(oldRecArrList,newRecArrList);
+  notCoveredMap.rewriteValueIdListUp(newRecArrList, oldRecArrList);
   ValueIdArray newNewRecArray(newRecArrList.entries());
   
   for (CollIndex i = 0; i < newRecArrList.entries(); i++)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f67784b0/core/sql/optimizer/BindWA.h
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/BindWA.h b/core/sql/optimizer/BindWA.h
index 4089291..c074814 100644
--- a/core/sql/optimizer/BindWA.h
+++ b/core/sql/optimizer/BindWA.h
@@ -685,7 +685,8 @@ public:
   // --------------------------------------------------------------------
   RelExpr *&getSequenceNode() { return sequenceNode_; };
 
-  ValueIdMap &getNcToOldMap() { return ncToOldMap_;}
+  const ValueIdMap &getNcToOldMap() { return ncToOldMap_;}
+  void setNCToOldMap(ValueIdMap vmap) {ncToOldMap_ = vmap; }
 
   ValueIdList getOlapPartition() const {return OlapPartition_; };
 

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f67784b0/core/sql/optimizer/Inlining.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/Inlining.cpp b/core/sql/optimizer/Inlining.cpp
index ad534e7..c98bf29 100644
--- a/core/sql/optimizer/Inlining.cpp
+++ b/core/sql/optimizer/Inlining.cpp
@@ -626,7 +626,7 @@ RETDesc *GenericUpdate::createOldAndNewCorrelationNames(BindWA *bindWA, NABoolea
     // DELETE or UPDATE --
     // Now merge the old/target/before valueid's (the Scan child RETDesc)
     // into this RETDesc such that these cols are all named "OLD@.<col>"
-    //f
+    //
     Scan *scan ;
     if (getOperatorType() != REL_UNARY_INSERT)
       scan = getScanNode();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f67784b0/core/sql/optimizer/RelSequence.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/RelSequence.cpp b/core/sql/optimizer/RelSequence.cpp
index e7b67b7..626f58b 100644
--- a/core/sql/optimizer/RelSequence.cpp
+++ b/core/sql/optimizer/RelSequence.cpp
@@ -1193,7 +1193,7 @@ RelExpr *RelSequence::bindNode(BindWA *bindWA)
       newColumn->getValueId(),
       USER_COLUMN,
       colList->at(i)->getHeading());
-    ncToOldMap.addMapEntry(columnValueId,newColumn->getValueId());
+    ncToOldMap.addMapEntry(newColumn->getValueId(),columnValueId);
     if(colList->at(i)->isGrouped()) {
       ColumnNameMap *cnm =
         resultTable->findColumn(colList->at(i)->getColRefNameObj());
@@ -1262,7 +1262,7 @@ RelExpr *RelSequence::bindNode(BindWA *bindWA)
 
   bindWA->getCurrentScope()->getSequenceNode() = boundExpr;
   // save the ncToOldmap in the current scope. It will be used in Insert::bindnode for a special case.
-  bindWA->getCurrentScope()->getNcToOldMap() = ncToOldMap;
+  bindWA->getCurrentScope()->setNCToOldMap( ncToOldMap);
   return boundExpr;
 
 } // RelSequence::bindNode()

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f67784b0/core/sql/regress/executor/EXPECTED015.SB
----------------------------------------------------------------------
diff --git a/core/sql/regress/executor/EXPECTED015.SB b/core/sql/regress/executor/EXPECTED015.SB
index a80f143..15e3d87 100755
--- a/core/sql/regress/executor/EXPECTED015.SB
+++ b/core/sql/regress/executor/EXPECTED015.SB
@@ -163,6 +163,17 @@
 
 --- SQL operation complete.
 >>
+>>create table t015t16 (a largeint  not null, b int not null, 
++>c int, d int, e int, primary key(a,b)) ;
+
+--- SQL operation complete.
+>>create index t015t16i1 on t015t16(c) ;
+
+--- SQL operation complete.
+>>create unique index t015t16i2 on t015t16(d) ;
+
+--- SQL operation complete.
+>>
 >>prepare explainIt from
 +>select substring(operator,1,16) operator
 +>from table (explain(NULL,'XX')) t
@@ -1959,4 +1970,101 @@ TRAFODION_MERGE
 
 --- 1 row(s) selected.
 >>
+>>--upsert transformation to eff tree tests
+>>cqd traf_upsert_to_eff_tree 'ON';
+
+--- SQL operation complete.
+>>prepare xx from upsert into t015t16 values (1,2,3,4,5),(6,7,8,9,10);
+
+--- SQL command prepared.
+>>explain options 'f' xx;
+
+LC   RC   OP   OPERATOR              OPT       DESCRIPTION           CARD
+---- ---- ---- --------------------  --------  --------------------  ---------
+
+15   .    16   root                            x                     4.00E+000
+7    14   15   nested_join                                           4.00E+000
+10   13   14   merge_union                                           4.00E+000
+11   12   13   blocked_union                                         2.00E+000
+.    .    12   trafodion_insert                T015T16I2             1.00E+000
+.    .    11   trafodion_vsbb_delet            T015T16I2             1.00E+000
+8    9    10   blocked_union                                         2.00E+000
+.    .    9    trafodion_insert                T015T16I1             1.00E+000
+.    .    8    trafodion_vsbb_delet            T015T16I1             1.00E+000
+5    6    7    nested_join                                           1.00E+000
+.    .    6    trafodion_upsert                T015T16               1.00E+000
+4    .    5    sequence                                              1.00E+000
+3    .    4    sort                                                  2.00E+000
+2    1    3    left_hybrid_hash_joi                                  2.00E+000
+.    .    2    tuplelist                                             2.00E+000
+.    .    1    trafodion_scan                  T015T16               1.00E+002
+
+--- SQL operation complete.
+>>execute xx;
+
+--- 2 row(s) inserted.
+>>select * from t015t16 order by a;
+
+A                     B            C            D            E          
+--------------------  -----------  -----------  -----------  -----------
+
+                   1            2            3            4            5
+                   6            7            8            9           10
+
+--- 2 row(s) selected.
+>>prepare xx2 from upsert into t015t16 values (10,11,12,13,14);
+
+--- SQL command prepared.
+>>execute xx2;
+
+--- 1 row(s) inserted.
+>>select * from t015t16 order by a;
+
+A                     B            C            D            E          
+--------------------  -----------  -----------  -----------  -----------
+
+                   1            2            3            4            5
+                   6            7            8            9           10
+                  10           11           12           13           14
+
+--- 3 row(s) selected.
+>>
+>>prepare xx2 from upsert into t015t16 values (1,2,30,40,50);
+
+--- SQL command prepared.
+>>execute xx2;
+
+--- 1 row(s) inserted.
+>>select * from t015t16 order by a;
+
+A                     B            C            D            E          
+--------------------  -----------  -----------  -----------  -----------
+
+                   1            2           30           40           50
+                   6            7            8            9           10
+                  10           11           12           13           14
+
+--- 3 row(s) selected.
+>>
+>>prepare xx2 from upsert into t015t16 values (3,4,1,1,1),(6,7,80,90,100),(6,7,81,91,101);
+
+--- SQL command prepared.
+>>execute xx2;
+
+--- 2 row(s) inserted.
+>>select * from t015t16 order by a;
+
+A                     B            C            D            E          
+--------------------  -----------  -----------  -----------  -----------
+
+                   1            2           30           40           50
+                   3            4            1            1            1
+                   6            7           81           91          101
+                  10           11           12           13           14
+
+--- 4 row(s) selected.
+>>
+>>cqd traf_upsert_to_eff_tree 'OFF';
+
+--- SQL operation complete.
 >>log;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/f67784b0/core/sql/regress/executor/TEST015
----------------------------------------------------------------------
diff --git a/core/sql/regress/executor/TEST015 b/core/sql/regress/executor/TEST015
index bcf05e4..1f5915e 100755
--- a/core/sql/regress/executor/TEST015
+++ b/core/sql/regress/executor/TEST015
@@ -48,6 +48,7 @@ drop table ot;
 drop table odt cascade;
 drop table de cascade ;
 drop table dec1 cascade;
+drop table t015t16 cascade;
 
 --control query default POS 'OFF';
 
@@ -260,6 +261,11 @@ PRIMARY KEY (ID)
 ALTER TABLE OT 
 ADD CONSTRAINT UI1_OT UNIQUE (objectCode, version);
 
+create table t015t16 (a largeint  not null, b int not null, 
+c int, d int, e int, primary key(a,b)) ;
+create index t015t16i1 on t015t16(c) ;
+create unique index t015t16i2 on t015t16(d) ;
+
 prepare explainIt from
 select substring(operator,1,16) operator
 from table (explain(NULL,'XX')) t
@@ -865,5 +871,24 @@ prepare XX from UPSERT INTO DE (ID, dataElementConceptID, valueDomainID)
 VALUES  (1,  (select d.id from DEC1 d where d.codeValue = 'aa'),  3 ) ;
 execute explainIt;
 
+--upsert transformation to eff tree tests
+cqd traf_upsert_to_eff_tree 'ON';
+prepare xx from upsert into t015t16 values (1,2,3,4,5),(6,7,8,9,10);
+explain options 'f' xx;
+execute xx;
+select * from t015t16 order by a;
+prepare xx2 from upsert into t015t16 values (10,11,12,13,14);
+execute xx2;
+select * from t015t16 order by a;
+
+prepare xx2 from upsert into t015t16 values (1,2,30,40,50);
+execute xx2;
+select * from t015t16 order by a;
+
+prepare xx2 from upsert into t015t16 values (3,4,1,1,1),(6,7,80,90,100),(6,7,81,91,101);
+execute xx2;
+select * from t015t16 order by a;
+
+cqd traf_upsert_to_eff_tree 'OFF';
 log;
 


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

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


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

Branch: refs/heads/master
Commit: 62f8db94c76ce19a1a699c558ef11cfb4a793721
Parents: a209359 f67784b
Author: Sandhya Sundaresan <sa...@apache.org>
Authored: Sun Nov 13 01:10:15 2016 +0000
Committer: Sandhya Sundaresan <sa...@apache.org>
Committed: Sun Nov 13 01:10:15 2016 +0000

----------------------------------------------------------------------
 core/sql/optimizer/BindRelExpr.cpp       | 371 +++++++++++++++++++++++++-
 core/sql/optimizer/BindWA.cpp            |   1 +
 core/sql/optimizer/BindWA.h              |   5 +-
 core/sql/optimizer/Inlining.cpp          |  74 +++--
 core/sql/optimizer/RelExpr.h             |   6 +-
 core/sql/optimizer/RelSequence.cpp       |  10 +-
 core/sql/optimizer/RelUpdate.h           |   5 +-
 core/sql/regress/executor/EXPECTED015.SB | 108 ++++++++
 core/sql/regress/executor/TEST015        |  25 ++
 core/sql/sqlcomp/DefaultConstants.h      |   4 +
 core/sql/sqlcomp/nadefaults.cpp          |   1 +
 11 files changed, 572 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/62f8db94/core/sql/optimizer/RelSequence.cpp
----------------------------------------------------------------------